diff --git a/LICENSE b/LICENSE old mode 100644 new mode 100755 diff --git a/README.md b/README.md old mode 100644 new mode 100755 diff --git a/bin/check_jstorm_Supervisor.sh b/bin/check_jstorm_Supervisor.sh new file mode 100644 index 000000000..1ec43941a --- /dev/null +++ b/bin/check_jstorm_Supervisor.sh @@ -0,0 +1,13 @@ +#!/bin/bash +JAVA_HOME=/opt/taobao/java +export PATH=$PATH:$JAVA_HOME/bin + +LOG=/home/admin/logs/check.log +SP=`ps -ef |grep com.alibaba.jstorm.daemon.supervisor.Supervisor |grep -v grep |wc -l` +if [ $SP -lt 1 ];then + mkdir -p /home/admin/logs + echo -e "`date` [ERROR] no process and restart Jstorm Suppervisor" >>$LOG + cd /home/admin/bin; nohup /home/admin/jstorm/bin/jstorm supervisor >/dev/null 2>&1 & +else + echo -e "`date` [INFO:] return $SP Jstorm Supervisor ok " >>$LOG +fi diff --git a/jstorm-server/bin/jstorm.py b/bin/jstorm.py old mode 100644 new mode 100755 similarity index 81% rename from jstorm-server/bin/jstorm.py rename to bin/jstorm.py index e3c15dfef..c4b3fe7fd --- a/jstorm-server/bin/jstorm.py +++ b/bin/jstorm.py @@ -1,3 +1,21 @@ +#!/usr/bin/env python +# +# 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. + #!/usr/bin/python import os @@ -24,8 +42,10 @@ def cygpath(x): CLIENT_CONF_FILE = "" JSTORM_DIR = "/".join(os.path.realpath( __file__ ).split("/")[:-2]) JSTORM_CONF_DIR = os.getenv("JSTORM_CONF_DIR", JSTORM_DIR + "/conf" ) -LOG4J_CONF = JSTORM_CONF_DIR + "/jstorm.log4j.properties" +LOGBACK_CONF = JSTORM_CONF_DIR + "/jstorm.logback.xml" CONFIG_OPTS = [] +EXCLUDE_JARS = [] +INCLUDE_JARS = [] STATUS = 0 @@ -41,13 +61,15 @@ def get_config_opts(): return "-Dstorm.options=" + (','.join(CONFIG_OPTS)).replace(' ', "%%%%") def get_client_childopts(): - ret = (" -Dstorm.root.logger=INFO,stdout -Dlog4j.configuration=File:%s/conf/aloha_log4j.properties " %JSTORM_DIR) + ret = (" -Dstorm.root.logger=INFO,stdout -Dlogback.configurationFile=" + JSTORM_DIR + + "/conf/client_logback.xml -Dlog4j.configuration=File:" + JSTORM_DIR + + "/conf/client_log4j.properties") if CLIENT_CONF_FILE != "": ret += (" -Dstorm.conf.file=" + CLIENT_CONF_FILE) return ret def get_server_childopts(log_name): - ret = (" -Dlogfile.name=%s -Dlog4j.configuration=File:%s" %(log_name, LOG4J_CONF)) + ret = (" -Dlogfile.name=%s -Dlogback.configurationFile=%s" %(log_name, LOGBACK_CONF)) return ret if not os.path.exists(JSTORM_DIR + "/RELEASE"): @@ -61,15 +83,27 @@ def get_jars_full(adir): files = os.listdir(adir) ret = [] for f in files: - if f.endswith(".jar"): + if f.endswith(".jar") == False: + continue + filter = False + for exclude_jar in EXCLUDE_JARS: + if f.find(exclude_jar) >= 0: + filter = True + break + + if filter == True: + print "Don't add " + f + " to classpath" + else: ret.append(adir + "/" + f) return ret def get_classpath(extrajars): ret = [] + ret.extend(extrajars) ret.extend(get_jars_full(JSTORM_DIR)) ret.extend(get_jars_full(JSTORM_DIR + "/lib")) - ret.extend(extrajars) + ret.extend(INCLUDE_JARS) + return normclasspath(":".join(ret)) def confvalue(name, extrapaths): @@ -119,7 +153,7 @@ def jar(jarfile, klass, *args): """Syntax: [jstorm jar topology-jar-path class ...] Runs the main method of class with the specified arguments. - The jstorm jars and configs in ~/.jstorm are put on the classpath. + The jstorm jars and configs in $JSTORM_CONF_DIR/storm.yaml are put on the classpath. The process is configured so that StormSubmitter (https://github.com/alibaba/jstorm/wiki/JStorm-Chinese-Documentation) will upload the jar at topology-jar-path when the topology is submitted. @@ -229,6 +263,17 @@ def restart(*args): extrajars=[JSTORM_CONF_DIR, JSTORM_DIR + "/bin", CLIENT_CONF_FILE], childopts=childopts) +def update_config(*args): + """Syntax: [jstorm restart topology-name [conf]] + """ + childopts = get_client_childopts() + exec_storm_class( + "backtype.storm.command.update_config", + args=args, + jvmtype="-client -Xms256m -Xmx256m", + extrajars=[JSTORM_CONF_DIR, JSTORM_DIR + "/bin", CLIENT_CONF_FILE], + childopts=childopts) + def nimbus(): """Syntax: [jstorm nimbus] @@ -291,9 +336,11 @@ def print_classpath(): def print_commands(): """Print all client commands and link to documentation""" - print "jstorm command [--config client_storm.yaml] [command parameter]" + print "jstorm command [--config client_storm.yaml] [--exclude-jars exclude1.jar,exclude2.jar] [-c key1=value1,key2=value2][command parameter]" print "Commands:\n\t", "\n\t".join(sorted(COMMANDS.keys())) - print "\n\t[--config client_storm.yaml]\t\t optional, setting client's storm.yaml" + print "\n\t[--config client_storm.yaml]\t\t\t optional, setting client's storm.yaml" + print "\n\t[--exclude-jars exclude1.jar,exclude2.jar]\t optional, exclude jars, avoid jar conflict" + print "\n\t[-c key1=value1,key2=value2]\t\t\t optional, add key=value pair to configuration" print "\nHelp:", "\n\thelp", "\n\thelp " print "\nDocumentation for the jstorm client can be found at https://github.com/alibaba/jstorm/wiki/JStorm-Chinese-Documentation\n" @@ -341,7 +388,7 @@ def list(*args): "drpc": drpc, "supervisor": supervisor, "localconfvalue": print_localconfvalue, "remoteconfvalue": print_remoteconfvalue, "classpath": print_classpath, "activate": activate, "deactivate": deactivate, "rebalance": rebalance, "help": print_usage, - "metricsMonitor": metrics_Monitor, "list": list, "restart": restart} + "metricsMonitor": metrics_Monitor, "list": list, "restart": restart, "update_config": update_config} def parse_config(config_list): global CONFIG_OPTS @@ -349,6 +396,18 @@ def parse_config(config_list): for config in config_list: CONFIG_OPTS.append(config) +def parse_exclude_jars(jars): + global EXCLUDE_JARS + EXCLUDE_JARS = jars.split(",") + print " Excludes jars:" + print EXCLUDE_JARS + +def parse_include_jars(jars): + global INCLUDE_JARS + INCLUDE_JARS = jars.split(",") + print " Include jars:" + print INCLUDE_JARS + def parse_config_opts(args): curr = args[:] curr.reverse() @@ -362,6 +421,10 @@ def parse_config_opts(args): elif token == "--config": global CLIENT_CONF_FILE CLIENT_CONF_FILE = curr.pop() + elif token == "--exclude-jars": + parse_exclude_jars(curr.pop()) + elif token == "--include-jars": + parse_include_jars(curr.pop()) else: args_list.append(token) diff --git a/bin/start.sh b/bin/start.sh new file mode 100644 index 000000000..01f81e0f0 --- /dev/null +++ b/bin/start.sh @@ -0,0 +1,78 @@ +#!/bin/sh + +if [ -e ~/.bashrc ] +then + source ~/.bashrc +fi + +if [ -e ~/.bash_profile ] +then + source ~/.bash_profile +fi + +if [ "x$JAVA_HOME" != "x" ] +then + echo "JAVA_HOME has been set " +else + export JAVA_HOME=/opt/taobao/java +fi +echo "JAVA_HOME =" $JAVA_HOME + +if [ "x$JSTORM_HOME" != "x" ] +then + echo "JSTORM_HOME has been set " +else + export JSTORM_HOME=/home/admin/jstorm +fi +echo "JSTORM_HOME =" $JSTORM_HOME + +if [ "x$JSTORM_CONF_DIR_PATH" != "x" ] +then + echo "JSTORM_CONF_DIR_PATH has been set " +else + export JSTORM_CONF_DIR_PATH=$JSTORM_HOME/conf +fi +echo "JSTORM_CONF_DIR_PATH =" $JSTORM_CONF_DIR_PATH + + + +export PATH=$JAVA_HOME/bin:$JSTORM_HOME/bin:$PATH + + +which java + +if [ $? -eq 0 ] +then + echo "Find java" +else + echo "No java, please install java firstly !!!" + exit 1 +fi + +function startJStorm() +{ + PROCESS=$1 + echo "start $PROCESS" + cd $JSTORM_HOME/bin; nohup $JSTORM_HOME/bin/jstorm $PROCESS >/dev/null 2>&1 & + sleep 4 + rm -rf nohup + ps -ef|grep $2 +} + + + +HOSTNAME=`hostname -i` +NIMBUS_HOST=`grep "nimbus.host:" $JSTORM_CONF_DIR_PATH/storm.yaml | grep -w $HOSTNAME` +SUPERVISOR_HOST_START=`grep "supervisor.host.start:" $JSTORM_CONF_DIR_PATH/storm.yaml | grep -w "false"` + +if [ "X${NIMBUS_HOST}" != "X" ] +then + startJStorm "nimbus" "NimbusServer" +fi + +if [ "X${SUPERVISOR_HOST_START}" == "X" ] +then + startJStorm "supervisor" "Supervisor" +fi + +echo "Successfully start jstorm daemon...." diff --git a/jstorm-server/bin/stop.sh b/bin/stop.sh old mode 100644 new mode 100755 similarity index 100% rename from jstorm-server/bin/stop.sh rename to bin/stop.sh diff --git a/jstorm-server/conf/cgconfig.conf b/conf/cgconfig.conf old mode 100644 new mode 100755 similarity index 100% rename from jstorm-server/conf/cgconfig.conf rename to conf/cgconfig.conf diff --git a/jstorm-server/conf/aloha_log4j.properties b/conf/client_log4j.properties old mode 100644 new mode 100755 similarity index 100% rename from jstorm-server/conf/aloha_log4j.properties rename to conf/client_log4j.properties diff --git a/conf/client_logback.xml b/conf/client_logback.xml new file mode 100755 index 000000000..02ef72e36 --- /dev/null +++ b/conf/client_logback.xml @@ -0,0 +1,18 @@ + + + + + + %-4r [%t] %-5p %c - %m%n + + + + + + + + + + + + \ No newline at end of file diff --git a/jstorm-server/conf/jstorm.log4j.properties b/conf/jstorm.log4j.properties old mode 100644 new mode 100755 similarity index 100% rename from jstorm-server/conf/jstorm.log4j.properties rename to conf/jstorm.log4j.properties diff --git a/jstorm-server/conf/jstorm.logback.xml b/conf/jstorm.logback.xml old mode 100644 new mode 100755 similarity index 85% rename from jstorm-server/conf/jstorm.logback.xml rename to conf/jstorm.logback.xml index c0dbb4471..bd097ea07 --- a/jstorm-server/conf/jstorm.logback.xml +++ b/conf/jstorm.logback.xml @@ -25,7 +25,7 @@ class="ch.qos.logback.core.rolling.RollingFileAppender"> ${jstorm.home}/logs/${logfile.name}.metrics - ${logfile.name}.metrics.%i + ${jstorm.home}/logs/${logfile.name}.metrics.%i 1 5 @@ -45,12 +45,12 @@ + additivity="false"> - @@ -68,6 +68,12 @@ + + + + + diff --git a/jstorm-server/conf/storm.yaml b/conf/storm.yaml old mode 100644 new mode 100755 similarity index 100% rename from jstorm-server/conf/storm.yaml rename to conf/storm.yaml diff --git a/dev-tools/add_apache_license.sh b/dev-tools/add_apache_license.sh new file mode 100755 index 000000000..26a005450 --- /dev/null +++ b/dev-tools/add_apache_license.sh @@ -0,0 +1 @@ +find . -name \*.java -exec sh -c "if ! grep -q 'LICENSE-2.0' '{}';then mv '{}' tmp && cp LICENSEHEADER.txt '{}' && cat tmp >> '{}' && rm tmp;fi" \; \ No newline at end of file diff --git a/dev-tools/java_license_header.txt b/dev-tools/java_license_header.txt new file mode 100755 index 000000000..7e66353b7 --- /dev/null +++ b/dev-tools/java_license_header.txt @@ -0,0 +1,17 @@ +/** + * 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. + */ diff --git a/dev-tools/py_license_header.txt b/dev-tools/py_license_header.txt new file mode 100755 index 000000000..0896fcd5a --- /dev/null +++ b/dev-tools/py_license_header.txt @@ -0,0 +1,18 @@ +#!/usr/bin/env python +# +# 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. + diff --git a/dev-tools/storm-eclipse-java-formatter.xml b/dev-tools/storm-eclipse-java-formatter.xml new file mode 100755 index 000000000..25e6f92f8 --- /dev/null +++ b/dev-tools/storm-eclipse-java-formatter.xml @@ -0,0 +1,291 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/docs/log.test.xlsx b/docs/log.test.xlsx new file mode 100755 index 000000000..b178ba7fc Binary files /dev/null and b/docs/log.test.xlsx differ diff --git a/example/sequence-split-merge/.classpath b/example/sequence-split-merge/.classpath new file mode 100755 index 000000000..f8ce0d30f --- /dev/null +++ b/example/sequence-split-merge/.classpath @@ -0,0 +1,31 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/example/sequence-split-merge/.gitignore b/example/sequence-split-merge/.gitignore new file mode 100755 index 000000000..1dd333108 --- /dev/null +++ b/example/sequence-split-merge/.gitignore @@ -0,0 +1,2 @@ +/target/ +/target/ diff --git a/example/sequence-split-merge/.project b/example/sequence-split-merge/.project new file mode 100755 index 000000000..4269e6725 --- /dev/null +++ b/example/sequence-split-merge/.project @@ -0,0 +1,23 @@ + + + sequence-split-merge + + + + + + org.eclipse.jdt.core.javabuilder + + + + + org.eclipse.m2e.core.maven2Builder + + + + + + org.eclipse.jdt.core.javanature + org.eclipse.m2e.core.maven2Nature + + diff --git a/example/sequence-split-merge/.settings/org.eclipse.core.resources.prefs b/example/sequence-split-merge/.settings/org.eclipse.core.resources.prefs new file mode 100755 index 000000000..8bc0e1c8b --- /dev/null +++ b/example/sequence-split-merge/.settings/org.eclipse.core.resources.prefs @@ -0,0 +1,5 @@ +eclipse.preferences.version=1 +encoding//src/main/java=UTF-8 +encoding//src/test/java=UTF-8 +encoding//src/test/resources=UTF-8 +encoding/=UTF-8 diff --git a/example/sequence-split-merge/.settings/org.eclipse.jdt.core.prefs b/example/sequence-split-merge/.settings/org.eclipse.jdt.core.prefs new file mode 100755 index 000000000..14f521d2a --- /dev/null +++ b/example/sequence-split-merge/.settings/org.eclipse.jdt.core.prefs @@ -0,0 +1,5 @@ +eclipse.preferences.version=1 +org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.6 +org.eclipse.jdt.core.compiler.compliance=1.6 +org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning +org.eclipse.jdt.core.compiler.source=1.6 diff --git a/example/sequence-split-merge/.settings/org.eclipse.m2e.core.prefs b/example/sequence-split-merge/.settings/org.eclipse.m2e.core.prefs new file mode 100755 index 000000000..14b697b7b --- /dev/null +++ b/example/sequence-split-merge/.settings/org.eclipse.m2e.core.prefs @@ -0,0 +1,4 @@ +activeProfiles= +eclipse.preferences.version=1 +resolveWorkspaceProjects=true +version=1 diff --git a/example/sequence-split-merge/conf/conf.prop b/example/sequence-split-merge/conf/conf.prop old mode 100644 new mode 100755 diff --git a/example/sequence-split-merge/conf/conf.yaml b/example/sequence-split-merge/conf/conf.yaml old mode 100644 new mode 100755 diff --git a/example/sequence-split-merge/conf/topology.yaml b/example/sequence-split-merge/conf/topology.yaml old mode 100644 new mode 100755 diff --git a/example/sequence-split-merge/drpc.sh b/example/sequence-split-merge/drpc.sh new file mode 100755 index 000000000..1492a51b7 --- /dev/null +++ b/example/sequence-split-merge/drpc.sh @@ -0,0 +1,3 @@ +#!/bin/bash + +jstorm jar target/sequence-split-merge-1.1.0-jar-with-dependencies.jar com.alipay.dw.jstorm.example.drpc.ReachTopology reach diff --git a/example/sequence-split-merge/pom.xml b/example/sequence-split-merge/pom.xml old mode 100644 new mode 100755 index 71f515421..afba08c36 --- a/example/sequence-split-merge/pom.xml +++ b/example/sequence-split-merge/pom.xml @@ -3,7 +3,7 @@ 4.0.0 storm sequence-split-merge - 1.0.8 + 1.1.0 jar sequence-split-merge @@ -11,7 +11,7 @@ UTF-8 - 0.9.6.3 + 2.0.4-SNAPSHOT storm-0.9.2-incubating @@ -34,33 +34,22 @@ com.alibaba.jstorm - jstorm-client-extension + jstorm-core ${jstorm.version} provided - - - com.alibaba.jstorm - jstorm-client - ${jstorm.version} - provided - - - org.slf4j - slf4j-log4j12 - - - - + + - com.alibaba.jstorm - jstorm-server - ${jstorm.version} - provided - + junit + junit + 4.10 + test + + - 4.0.0 - com.alibaba.jstorm - jstorm-client-extension - 0.9.6.3 - jar - ${project.artifactId}-${project.version} - - - - - maven-compiler-plugin - 2.3.2 - - 1.6 - 1.6 - - - - org.apache.maven.plugins - maven-source-plugin - - - attach-sources - - jar - - - - - - - - 1.4.11 - UTF-8 - - - - com.alibaba.jstorm - jstorm-client - ${project.version} - provided - - - com.alibaba - fastjson - 1.1.41 - - - com.sun.net.httpserver - http - 20070405 - - - org.powermock - powermock-module-junit4 - ${powermock.version} - test - - - com.codahale.metrics - metrics-core - 3.0.1 - - - com.codahale.metrics - metrics-jvm - 3.0.1 - - - \ No newline at end of file diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/BatchId.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/BatchId.java deleted file mode 100644 index e147db01d..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/BatchId.java +++ /dev/null @@ -1,66 +0,0 @@ -package com.alibaba.jstorm.batch; - -import java.io.Serializable; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; - -public class BatchId implements Serializable { - private static final long serialVersionUID = 5720810158625748049L; - protected final long id; - - protected BatchId(long id) { - this.id = id; - } - - public long getId() { - return id; - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + (int) (id ^ (id >>> 32)); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - BatchId other = (BatchId) obj; - if (id != other.id) - return false; - return true; - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this, - ToStringStyle.SHORT_PREFIX_STYLE); - } - - private static AtomicLong staticId = new AtomicLong(0); - - public static void updateId(long id) { - staticId.set(id); - } - - public static BatchId mkInstance() { - long id = staticId.incrementAndGet(); - - return new BatchId(id); - } - - public static BatchId incBatchId(BatchId old) { - long other = old.getId(); - return new BatchId(other + 1); - } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/BatchTopologyBuilder.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/BatchTopologyBuilder.java deleted file mode 100644 index dc599f187..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/BatchTopologyBuilder.java +++ /dev/null @@ -1,69 +0,0 @@ -package com.alibaba.jstorm.batch; - -import org.apache.log4j.Logger; - -import backtype.storm.topology.BoltDeclarer; -import backtype.storm.topology.IBasicBolt; -import backtype.storm.topology.SpoutDeclarer; -import backtype.storm.topology.TopologyBuilder; - -import com.alibaba.jstorm.batch.impl.BatchSpoutTrigger; -import com.alibaba.jstorm.batch.impl.CoordinatedBolt; -import com.alibaba.jstorm.batch.util.BatchDef; - -public class BatchTopologyBuilder { - private static final Logger LOG = Logger - .getLogger(BatchTopologyBuilder.class); - - private TopologyBuilder topologyBuilder; - - private SpoutDeclarer spoutDeclarer; - - public BatchTopologyBuilder(String topologyName) { - topologyBuilder = new TopologyBuilder(); - - spoutDeclarer = topologyBuilder.setSpout(BatchDef.SPOUT_TRIGGER, - new BatchSpoutTrigger(), 1); - } - - public BoltDeclarer setSpout(String id, IBatchSpout spout, int paralel) { - - BoltDeclarer boltDeclarer = this - .setBolt(id, (IBatchSpout) spout, paralel); - boltDeclarer.allGrouping(BatchDef.SPOUT_TRIGGER, - BatchDef.COMPUTING_STREAM_ID); - - return boltDeclarer; - } - - public BoltDeclarer setBolt(String id, IBasicBolt bolt, int paralel) { - CoordinatedBolt coordinatedBolt = new CoordinatedBolt(bolt); - - BoltDeclarer boltDeclarer = topologyBuilder.setBolt(id, - coordinatedBolt, paralel); - - if (bolt instanceof IPrepareCommit) { - boltDeclarer.allGrouping(BatchDef.SPOUT_TRIGGER, - BatchDef.PREPARE_STREAM_ID); - } - - if (bolt instanceof ICommitter) { - boltDeclarer.allGrouping(BatchDef.SPOUT_TRIGGER, - BatchDef.COMMIT_STREAM_ID); - boltDeclarer.allGrouping(BatchDef.SPOUT_TRIGGER, - BatchDef.REVERT_STREAM_ID); - } - - if (bolt instanceof IPostCommit) { - boltDeclarer.allGrouping(BatchDef.SPOUT_TRIGGER, - BatchDef.POST_STREAM_ID); - } - - return boltDeclarer; - } - - public TopologyBuilder getTopologyBuilder() { - return topologyBuilder; - } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/IBatchSpout.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/IBatchSpout.java deleted file mode 100644 index d3d117880..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/IBatchSpout.java +++ /dev/null @@ -1,37 +0,0 @@ -package com.alibaba.jstorm.batch; - -import java.io.Serializable; - -import backtype.storm.topology.IBasicBolt; - -public interface IBatchSpout extends IBasicBolt, ICommitter, Serializable { - - /** - * input's filed 0 is BatchId - * - * execute only receive trigger message - * - * do emitBatch operation in execute whose streamID is "batch/compute-stream" - */ - //void execute(Tuple input, IBasicOutputCollector collector); - /** - * begin to ack batchId's data - * - * return value will be stored in ZK, so sometimes don't need special action - * - * @param id - */ - //byte[] commit(BatchId id) throws FailedException; - - /** - * begin to revert batchId's data - * - * If current task fails to commit batchId, it won't call revert(batchId) - * If current task fails to revert batchId, JStorm won't call revert again. - * - * if not transaction, it can don't care revert - * - * @param id - */ - //void revert(BatchId id, byte[] commitResult); -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/ICommitter.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/ICommitter.java deleted file mode 100644 index 9492398f9..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/ICommitter.java +++ /dev/null @@ -1,36 +0,0 @@ -package com.alibaba.jstorm.batch; - -import java.io.Serializable; - -import backtype.storm.topology.FailedException; - -/** - * The less committer, the state is more stable. - * Don't need to do - * - * @author zhongyan.feng - * @version - */ -public interface ICommitter extends Serializable{ - /** - * begin to commit batchId's data, then return the commit result - * The commitResult will store into outside storage - * - * if failed to commit, please throw FailedException - * - * - * - * @param id - */ - byte[] commit(BatchId id) throws FailedException; - - /** - * begin to revert batchId's data - * - * If current task fails to commit batchId, it won't call revert(batchId) - * If current task fails to revert batchId, JStorm won't call revert again. - * - * @param id - */ - void revert(BatchId id, byte[] commitResult); -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/IPostCommit.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/IPostCommit.java deleted file mode 100644 index b408a3f20..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/IPostCommit.java +++ /dev/null @@ -1,14 +0,0 @@ -package com.alibaba.jstorm.batch; - -import backtype.storm.topology.BasicOutputCollector; - - -public interface IPostCommit { - /** - * Do after commit - * Don't care failure of postCommit - * - * @param id - */ - void postCommit(BatchId id, BasicOutputCollector collector); -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/IPrepareCommit.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/IPrepareCommit.java deleted file mode 100644 index dd3da44a4..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/IPrepareCommit.java +++ /dev/null @@ -1,20 +0,0 @@ -package com.alibaba.jstorm.batch; - -import backtype.storm.topology.BasicOutputCollector; -import backtype.storm.topology.FailedException; - -/** - * Called before commit, after finish batch - * - * @author zhongyan.feng - */ -public interface IPrepareCommit { - - /** - * Do prepare before commit - * - * @param id - * @param collector - */ - void prepareCommit(BatchId id, BasicOutputCollector collector) throws FailedException; -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/impl/BatchSpoutMsgId.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/impl/BatchSpoutMsgId.java deleted file mode 100644 index 63704fb4a..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/impl/BatchSpoutMsgId.java +++ /dev/null @@ -1,49 +0,0 @@ -package com.alibaba.jstorm.batch.impl; - -import java.io.Serializable; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; - -import com.alibaba.jstorm.batch.BatchId; -import com.alibaba.jstorm.batch.util.BatchStatus; - -public class BatchSpoutMsgId implements Serializable{ - - /** */ - private static final long serialVersionUID = 2899009971479957517L; - - private final BatchId batchId; - private BatchStatus batchStatus; - - protected BatchSpoutMsgId(BatchId batchId, BatchStatus batchStatus) { - this.batchId = batchId; - this.batchStatus = batchStatus; - } - - public static BatchSpoutMsgId mkInstance() { - BatchId batchId = BatchId.mkInstance(); - BatchStatus batchStatus = BatchStatus.COMPUTING; - - return new BatchSpoutMsgId(batchId, batchStatus); - } - - - public BatchStatus getBatchStatus() { - return batchStatus; - } - - public void setBatchStatus(BatchStatus batchStatus) { - this.batchStatus = batchStatus; - } - - public BatchId getBatchId() { - return batchId; - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this, - ToStringStyle.SHORT_PREFIX_STYLE); - } -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/impl/BatchSpoutTrigger.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/impl/BatchSpoutTrigger.java deleted file mode 100644 index d0c3e9497..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/impl/BatchSpoutTrigger.java +++ /dev/null @@ -1,312 +0,0 @@ -package com.alibaba.jstorm.batch.impl; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichSpout; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Values; - -import com.alibaba.jstorm.batch.BatchId; -import com.alibaba.jstorm.batch.util.BatchCommon; -import com.alibaba.jstorm.batch.util.BatchDef; -import com.alibaba.jstorm.batch.util.BatchStatus; -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.ClusterState; -import com.alibaba.jstorm.utils.IntervalCheck; -import com.alibaba.jstorm.utils.JStormUtils; - -/** - * Strong Sequence - * - * @author zhongyan.feng - * @version - */ -public class BatchSpoutTrigger implements IRichSpout { - /** */ - private static final long serialVersionUID = 7215109169247425954L; - - private static final Logger LOG = Logger.getLogger(BatchSpoutTrigger.class); - - private LinkedBlockingQueue batchQueue; - - private transient ClusterState zkClient; - - private transient SpoutOutputCollector collector; - - private static final String ZK_NODE_PATH = "/trigger"; - - private static BatchId currentBatchId = null; - - private Map conf; - - private String taskName; - - private IntervalCheck intervalCheck; - - /** - * @throws Exception - * - */ - public void initMsgId() throws Exception { - Long zkMsgId = null; - byte[] data = zkClient.get_data(ZK_NODE_PATH, false); - if (data != null) { - String value = new String(data); - try { - zkMsgId = Long.valueOf(value); - LOG.info("ZK msgId:" + zkMsgId); - } catch (Exception e) { - LOG.warn("Failed to get msgId ", e); - - } - - } - - if (zkMsgId != null) { - BatchId.updateId(zkMsgId); - } - - int max_spout_pending = JStormUtils.parseInt( - conf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING), 1); - - for (int i = 0; i < max_spout_pending; i++) { - BatchSpoutMsgId msgId = BatchSpoutMsgId.mkInstance(); - if (currentBatchId == null) { - currentBatchId = msgId.getBatchId(); - } - batchQueue.offer(msgId); - LOG.info("Push into queue," + msgId); - } - - } - - @Override - public void open(Map conf, TopologyContext context, - SpoutOutputCollector collector) { - batchQueue = new LinkedBlockingQueue(); - this.collector = collector; - this.conf = conf; - taskName = context.getThisComponentId() + "_" + context.getThisTaskId(); - - intervalCheck = new IntervalCheck(); - - try { - zkClient = BatchCommon.getZkClient(conf); - - initMsgId(); - - } catch (Exception e) { - LOG.error("", e); - throw new RuntimeException("Failed to init"); - } - LOG.info("Successfully open " + taskName); - } - - @Override - public void close() { - } - - @Override - public void activate() { - LOG.info("Activate " + taskName); - } - - @Override - public void deactivate() { - LOG.info("Deactivate " + taskName); - } - - protected String getStreamId(BatchStatus batchStatus) { - if (batchStatus == BatchStatus.COMPUTING) { - return BatchDef.COMPUTING_STREAM_ID; - } else if (batchStatus == BatchStatus.PREPARE_COMMIT) { - return BatchDef.PREPARE_STREAM_ID; - } else if (batchStatus == BatchStatus.COMMIT) { - return BatchDef.COMMIT_STREAM_ID; - } else if (batchStatus == BatchStatus.POST_COMMIT) { - return BatchDef.POST_STREAM_ID; - } else if (batchStatus == BatchStatus.REVERT_COMMIT) { - return BatchDef.REVERT_STREAM_ID; - } else { - LOG.error("Occur unkonw type BatchStatus " + batchStatus); - throw new RuntimeException(); - } - } - - protected boolean isCommitStatus(BatchStatus batchStatus) { - if (batchStatus == BatchStatus.COMMIT) { - return true; - } else if (batchStatus == BatchStatus.REVERT_COMMIT) { - return true; - } else { - return false; - } - } - - protected boolean isCommitWait(BatchSpoutMsgId msgId) { - - if (isCommitStatus(msgId.getBatchStatus()) == false) { - return false; - } - - // left status is commit status - if (currentBatchId.getId() >= msgId.getBatchId().getId()) { - return false; - } - - return true; - } - - @Override - public void nextTuple() { - BatchSpoutMsgId msgId = null; - try { - msgId = batchQueue.poll(10, TimeUnit.MILLISECONDS); - } catch (InterruptedException e) { - LOG.error("", e); - } - if (msgId == null) { - return; - } - - if (isCommitWait(msgId)) { - - batchQueue.offer(msgId); - if (intervalCheck.check()) { - LOG.info("Current msgId " + msgId - + ", but current commit BatchId is " + currentBatchId); - }else { - LOG.debug("Current msgId " + msgId - + ", but current commit BatchId is " + currentBatchId); - } - - return; - } - - String streamId = getStreamId(msgId.getBatchStatus()); - List outTasks = collector.emit(streamId, - new Values(msgId.getBatchId()), msgId); - if (outTasks.isEmpty()) { - forward(msgId); - } - return; - - } - - protected void mkMsgId(BatchSpoutMsgId oldMsgId) { - synchronized (BatchSpoutMsgId.class) { - if (currentBatchId.getId() <= oldMsgId.getBatchId().getId()) { - // this is normal case - - byte[] data = String.valueOf(currentBatchId.getId()).getBytes(); - try { - zkClient.set_data(ZK_NODE_PATH, data); - } catch (Exception e) { - LOG.error("Failed to update to ZK " + oldMsgId, e); - } - - currentBatchId = BatchId.incBatchId(oldMsgId.getBatchId()); - - } else { - // bigger batchId has been failed, when old msgId finish - // it will go here - - } - - } - - BatchSpoutMsgId newMsgId = BatchSpoutMsgId.mkInstance(); - batchQueue.offer(newMsgId); - StringBuilder sb = new StringBuilder(); - sb.append("Create new BatchId,"); - sb.append("old:").append(oldMsgId); - sb.append("new:").append(newMsgId); - sb.append("currentBatchId:").append(currentBatchId); - LOG.info(sb.toString()); - } - - protected void forward(BatchSpoutMsgId msgId) { - BatchStatus status = msgId.getBatchStatus(); - - BatchStatus newStatus = status.forward(); - if (newStatus == null) { - // create new status - mkMsgId(msgId); - LOG.info("Finish old batch " + msgId); - - } else { - msgId.setBatchStatus(newStatus); - batchQueue.offer(msgId); - LOG.info("Forward batch " + msgId); - } - } - - @Override - public void ack(Object msgId) { - if (msgId instanceof BatchSpoutMsgId) { - forward((BatchSpoutMsgId) msgId); - return; - } else { - LOG.warn("Unknown type msgId " + msgId.getClass().getName() + ":" - + msgId); - return; - } - } - - protected void handleFail(BatchSpoutMsgId msgId) { - LOG.info("Failed batch " + msgId); - BatchStatus status = msgId.getBatchStatus(); - - BatchStatus newStatus = status.error(); - if (newStatus == BatchStatus.ERROR) { - // create new status - mkMsgId(msgId); - - } else { - - msgId.setBatchStatus(newStatus); - batchQueue.offer(msgId); - - } - } - - @Override - public void fail(Object msgId) { - if (msgId instanceof BatchSpoutMsgId) { - handleFail((BatchSpoutMsgId) msgId); - } else { - LOG.warn("Unknown type msgId " + msgId.getClass().getName() + ":" - + msgId); - return; - } - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declareStream(BatchDef.COMPUTING_STREAM_ID, new Fields( - "BatchId")); - declarer.declareStream(BatchDef.PREPARE_STREAM_ID, - new Fields("BatchId")); - declarer.declareStream(BatchDef.COMMIT_STREAM_ID, new Fields("BatchId")); - declarer.declareStream(BatchDef.REVERT_STREAM_ID, new Fields("BatchId")); - declarer.declareStream(BatchDef.POST_STREAM_ID, new Fields("BatchId")); - } - - @Override - public Map getComponentConfiguration() { - Map map = new HashMap(); - ConfigExtension.setSpoutSingleThread(map, true); - return map; - } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/impl/CoordinatedBolt.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/impl/CoordinatedBolt.java deleted file mode 100644 index 0f4720b2f..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/impl/CoordinatedBolt.java +++ /dev/null @@ -1,281 +0,0 @@ -package com.alibaba.jstorm.batch.impl; - -import java.util.Collections; -import java.util.Comparator; -import java.util.List; -import java.util.Map; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.BasicOutputCollector; -import backtype.storm.topology.FailedException; -import backtype.storm.topology.IBasicBolt; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.ReportedFailedException; -import backtype.storm.tuple.Tuple; -import backtype.storm.utils.TimeCacheMap; - -import com.alibaba.jstorm.batch.BatchId; -import com.alibaba.jstorm.batch.ICommitter; -import com.alibaba.jstorm.batch.IPostCommit; -import com.alibaba.jstorm.batch.IPrepareCommit; -import com.alibaba.jstorm.batch.util.BatchCommon; -import com.alibaba.jstorm.batch.util.BatchDef; -import com.alibaba.jstorm.batch.util.BatchStatus; -import com.alibaba.jstorm.cluster.ClusterState; - -public class CoordinatedBolt implements IRichBolt { - private static final long serialVersionUID = 5720810158625748046L; - - public static Logger LOG = LoggerFactory.getLogger(CoordinatedBolt.class); - - private IBasicBolt delegate; - private BasicOutputCollector basicCollector; - private OutputCollector collector; - - private String taskId; - private String taskName; - - private boolean isCommiter = false; - private String zkCommitPath; - private TimeCacheMap commited; - - public CoordinatedBolt(IBasicBolt delegate) { - - this.delegate = delegate; - - } - - // use static variable to reduce zk connection - private static ClusterState zkClient = null; - - public void mkCommitDir(Map conf) { - - try { - zkClient = BatchCommon.getZkClient(conf); - - zkCommitPath = BatchDef.ZK_COMMIT_DIR + BatchDef.ZK_SEPERATOR - + taskId; - if (zkClient.node_existed(zkCommitPath, false)) { - zkClient.delete_node(zkCommitPath); - } - zkClient.mkdirs(zkCommitPath); - - LOG.info(taskName + " successfully create commit path" + zkCommitPath); - } catch (Exception e) { - LOG.error("Failed to create zk node", e); - throw new RuntimeException(); - } - } - - public void prepare(Map conf, TopologyContext context, - OutputCollector collector) { - - taskId = String.valueOf(context.getThisTaskId()); - taskName = context.getThisComponentId() + "_" + context.getThisTaskId(); - - this.basicCollector = new BasicOutputCollector(collector); - this.collector = collector; - - if (delegate instanceof ICommitter) { - isCommiter = true; - commited = new TimeCacheMap( - context.maxTopologyMessageTimeout()); - mkCommitDir(conf); - } - - delegate.prepare(conf, context); - - } - - public void removeUseless(String path, int reserveSize) throws Exception { - List childs = zkClient.get_children(path, false); - Collections.sort(childs, new Comparator() { - - @Override - public int compare(String o1, String o2) { - try { - Long v1 = Long.valueOf(o1); - Long v2 = Long.valueOf(o2); - return v1.compareTo(v2); - }catch(Exception e) { - return o1.compareTo(o2); - } - - } - - }); - - for (int index = 0; index < childs.size() - reserveSize; index++) { - zkClient.delete_node(path + BatchDef.ZK_SEPERATOR - + childs.get(index)); - } - } - - public String getCommitPath(BatchId id) { - return zkCommitPath + BatchDef.ZK_SEPERATOR + id.getId(); - } - - public void updateToZk(Object id, byte[] commitResult) { - try { - - removeUseless(zkCommitPath, BatchDef.ZK_COMMIT_RESERVER_NUM); - - String path = getCommitPath((BatchId)id); - byte[] data = commitResult; - if (data == null) { - data = new byte[0]; - } - zkClient.set_data(path, data); - LOG.info("Update " + path + " to zk"); - } catch (Exception e) { - LOG.warn("Failed to update to zk,", e); - - } - - } - - public byte[] getCommittedData(Object id) { - try { - String path = getCommitPath((BatchId)id); - byte[] data = zkClient.get_data(path, false); - - return data; - } catch (Exception e) { - LOG.error("Failed to visit ZK,", e); - return null; - } - } - - public void handleRegular(Tuple tuple) { - basicCollector.setContext(tuple); - try { - delegate.execute(tuple, basicCollector); - collector.ack(tuple); - } catch (FailedException e) { - if (e instanceof ReportedFailedException) { - collector.reportError(e); - } - collector.fail(tuple); - } - - } - - public void handlePrepareCommit(Tuple tuple) { - basicCollector.setContext(tuple); - try { - BatchId id = (BatchId) tuple.getValue(0); - ((IPrepareCommit) delegate).prepareCommit(id, basicCollector); - collector.ack(tuple); - } catch (FailedException e) { - if (e instanceof ReportedFailedException) { - collector.reportError(e); - } - collector.fail(tuple); - } - - } - - public void handleCommit(Tuple tuple) { - Object id = tuple.getValue(0); - try { - byte[] commitResult = ((ICommitter) delegate).commit((BatchId) id); - - collector.ack(tuple); - - updateToZk(id, commitResult); - commited.put(id, commitResult); - } catch (Exception e) { - LOG.error("Failed to commit ", e); - collector.fail(tuple); - } - } - - public void handleRevert(Tuple tuple) { - try { - Object id = tuple.getValue(0); - byte[] commitResult = null; - - if (commited.containsKey(id)) { - commitResult = (byte[]) commited.get(id); - } else { - commitResult = getCommittedData(id); - } - - if (commitResult != null) { - ((ICommitter) delegate).revert((BatchId) id, commitResult); - } - } catch (Exception e) { - LOG.error("Failed to revert,", e); - } - - collector.ack(tuple); - } - - public void handlePostCommit(Tuple tuple) { - - basicCollector.setContext(tuple); - try { - BatchId id = (BatchId) tuple.getValue(0); - ((IPostCommit) delegate).postCommit(id, basicCollector); - - } catch (Exception e) { - LOG.info("Failed to do postCommit,", e); - } - collector.ack(tuple); - } - - public void execute(Tuple tuple) { - - BatchStatus batchStatus = getBatchStatus(tuple); - - if (batchStatus == BatchStatus.COMPUTING) { - handleRegular(tuple); - } else if (batchStatus == BatchStatus.PREPARE_COMMIT) { - handlePrepareCommit(tuple); - } else if (batchStatus == BatchStatus.COMMIT) { - handleCommit(tuple); - } else if (batchStatus == BatchStatus.REVERT_COMMIT) { - handleRevert(tuple); - } else if (batchStatus == BatchStatus.POST_COMMIT) { - handlePostCommit(tuple); - } else { - throw new RuntimeException( - "Receive commit tuple, but not committer"); - } - } - - public void cleanup() { - delegate.cleanup(); - } - - public void declareOutputFields(OutputFieldsDeclarer declarer) { - delegate.declareOutputFields(declarer); - } - - @Override - public Map getComponentConfiguration() { - return delegate.getComponentConfiguration(); - } - - private BatchStatus getBatchStatus(Tuple tuple) { - String streamId = tuple.getSourceStreamId(); - - if (streamId.equals(BatchDef.PREPARE_STREAM_ID)) { - return BatchStatus.PREPARE_COMMIT; - } else if (streamId.equals(BatchDef.COMMIT_STREAM_ID)) { - return BatchStatus.COMMIT; - } else if (streamId.equals(BatchDef.REVERT_STREAM_ID)) { - return BatchStatus.REVERT_COMMIT; - } else if (streamId.equals(BatchDef.POST_STREAM_ID)) { - return BatchStatus.POST_COMMIT; - } else { - return BatchStatus.COMPUTING; - } - } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/util/BatchCommon.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/util/BatchCommon.java deleted file mode 100644 index f99edfa1f..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/util/BatchCommon.java +++ /dev/null @@ -1,65 +0,0 @@ -package com.alibaba.jstorm.batch.util; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; - -import com.alibaba.jstorm.cluster.ClusterState; -import com.alibaba.jstorm.cluster.DistributedClusterState; -import com.alibaba.jstorm.utils.JStormUtils; - -public class BatchCommon { - private static final Logger LOG = Logger.getLogger(BatchCommon.class); - - private static ClusterState zkClient = null; - - public static ClusterState getZkClient(Map conf) throws Exception { - synchronized (BatchCommon.class) { - if (zkClient != null) { - return zkClient; - } - - List zkServers = null; - if (conf.get(Config.TRANSACTIONAL_ZOOKEEPER_SERVERS) != null) { - zkServers = (List) conf - .get(Config.TRANSACTIONAL_ZOOKEEPER_SERVERS); - } else if (conf.get(Config.STORM_ZOOKEEPER_SERVERS) != null) { - zkServers = (List) conf - .get(Config.STORM_ZOOKEEPER_SERVERS); - } else { - throw new RuntimeException("No setting zk"); - } - - int port = 2181; - if (conf.get(Config.TRANSACTIONAL_ZOOKEEPER_PORT) != null) { - port = JStormUtils.parseInt( - conf.get(Config.TRANSACTIONAL_ZOOKEEPER_PORT), 2181); - } else if (conf.get(Config.STORM_ZOOKEEPER_PORT) != null) { - port = JStormUtils.parseInt( - conf.get(Config.STORM_ZOOKEEPER_PORT), 2181); - } - - String root = BatchDef.BATCH_ZK_ROOT; - if (conf.get(Config.TRANSACTIONAL_ZOOKEEPER_ROOT) != null) { - root = (String) conf.get(Config.TRANSACTIONAL_ZOOKEEPER_ROOT); - } - - root = root + BatchDef.ZK_SEPERATOR - + conf.get(Config.TOPOLOGY_NAME); - - Map tmpConf = new HashMap(); - tmpConf.putAll(conf); - tmpConf.put(Config.STORM_ZOOKEEPER_SERVERS, zkServers); - tmpConf.put(Config.STORM_ZOOKEEPER_ROOT, root); - zkClient = new DistributedClusterState(tmpConf); - - LOG.info("Successfully connect ZK"); - return zkClient; - } - - } -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/util/BatchDef.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/util/BatchDef.java deleted file mode 100644 index 7d87cfa8a..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/util/BatchDef.java +++ /dev/null @@ -1,26 +0,0 @@ -package com.alibaba.jstorm.batch.util; - - -public class BatchDef { - public static final String COMPUTING_STREAM_ID = "batch/compute-stream"; - - public static final String PREPARE_STREAM_ID = "batch/parepare-stream"; - - public static final String COMMIT_STREAM_ID = "batch/commit-stream"; - - public static final String REVERT_STREAM_ID = "batch/revert-stream"; - - public static final String POST_STREAM_ID = "batch/post-stream"; - - public static final String SPOUT_TRIGGER = "spout_trigger"; - - public static final String BATCH_ZK_ROOT = "batch"; - - public static final String ZK_COMMIT_DIR = "/commit"; - - public static final int ZK_COMMIT_RESERVER_NUM = 3; - - public static final String ZK_SEPERATOR = "/"; - - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/util/BatchStatus.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/util/BatchStatus.java deleted file mode 100644 index e02daeb2f..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/batch/util/BatchStatus.java +++ /dev/null @@ -1,33 +0,0 @@ -package com.alibaba.jstorm.batch.util; - -public enum BatchStatus { - COMPUTING, - PREPARE_COMMIT, - COMMIT, - REVERT_COMMIT, - POST_COMMIT, - ERROR; - - - - public BatchStatus forward() { - if (this == COMPUTING) { - return PREPARE_COMMIT; - }else if (this == PREPARE_COMMIT) { - return COMMIT; - }else if (this == COMMIT) { - return POST_COMMIT; - }else { - return null; - } - } - - public BatchStatus error() { - if (this == COMMIT) { - return REVERT_COMMIT; - }else { - return ERROR; - } - } - -}; diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/AsyncLoopDefaultKill.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/AsyncLoopDefaultKill.java deleted file mode 100644 index f392f2258..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/AsyncLoopDefaultKill.java +++ /dev/null @@ -1,25 +0,0 @@ -package com.alibaba.jstorm.callback; - -import com.alibaba.jstorm.utils.JStormUtils; - -/** - * Killer callback - * - * @author yannian - * - */ - -public class AsyncLoopDefaultKill extends RunnableCallback { - - @Override - public Object execute(T... args) { - Exception e = (Exception) args[0]; - JStormUtils.halt_process(1, "Async loop died!"); - return e; - } - - @Override - public void run() { - JStormUtils.halt_process(1, "Async loop died!"); - } -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/AsyncLoopRunnable.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/AsyncLoopRunnable.java deleted file mode 100644 index 1b1e588e1..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/AsyncLoopRunnable.java +++ /dev/null @@ -1,82 +0,0 @@ -package com.alibaba.jstorm.callback; - -import org.apache.log4j.Logger; - -import com.alibaba.jstorm.utils.JStormUtils; - -/** - * AsyncLoopThread 's runnable - * - * The class wrapper RunnableCallback fn, if occur exception, run killfn - * - * @author yannian - * - */ -public class AsyncLoopRunnable implements Runnable { - private static Logger LOG = Logger.getLogger(AsyncLoopRunnable.class); - - private RunnableCallback fn; - private RunnableCallback killfn; - - public AsyncLoopRunnable(RunnableCallback fn, RunnableCallback killfn) { - this.fn = fn; - this.killfn = killfn; - } - - private boolean needQuit(Object rtn) { - if (rtn != null) { - long sleepTime = Long.parseLong(String.valueOf(rtn)); - if (sleepTime < 0) { - return true; - }else if (sleepTime > 0) { - JStormUtils.sleepMs(sleepTime * 1000); - } - } - return false; - } - - @Override - public void run() { - - try { - while (true) { - Exception e = null; - - try { - if (fn == null) { - LOG.warn("fn==null"); - throw new RuntimeException("AsyncLoopRunnable no core function "); - } - - fn.run(); - - e = fn.error(); - - } catch (Exception ex) { - e = ex; - } - if (e != null) { - fn.shutdown(); - throw e; - } - Object rtn = fn.getResult(); - if (this.needQuit(rtn)) { - return; - } - - } - } catch (InterruptedException e) { - LOG.info("Async loop interrupted!"); - } catch (Throwable e) { - Object rtn = fn.getResult(); - if (this.needQuit(rtn)) { - return; - }else { - LOG.error("Async loop died!", e); - killfn.execute(e); - } - } - - } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/AsyncLoopThread.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/AsyncLoopThread.java deleted file mode 100644 index 534386d02..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/AsyncLoopThread.java +++ /dev/null @@ -1,122 +0,0 @@ -package com.alibaba.jstorm.callback; - -import java.lang.Thread.UncaughtExceptionHandler; - -import org.apache.log4j.Logger; - -import backtype.storm.utils.Time; - -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.SmartThread; - -/** - * Wrapper Timer thread Every several seconds execute afn, if something is run, - * run kill_fn - * - * - * @author yannian - * - */ -public class AsyncLoopThread implements SmartThread { - private static final Logger LOG = Logger.getLogger(AsyncLoopThread.class); - - private Thread thread; - - private RunnableCallback afn; - - public AsyncLoopThread(RunnableCallback afn) { - this.init(afn, false, Thread.NORM_PRIORITY, true); - } - - public AsyncLoopThread(RunnableCallback afn, boolean daemon, int priority, - boolean start) { - this.init(afn, daemon, priority, start); - } - - public AsyncLoopThread(RunnableCallback afn, boolean daemon, - RunnableCallback kill_fn, int priority, boolean start) { - this.init(afn, daemon, kill_fn, priority, start); - } - - public void init(RunnableCallback afn, boolean daemon, int priority, - boolean start) { - RunnableCallback kill_fn = new AsyncLoopDefaultKill(); - this.init(afn, daemon, kill_fn, priority, start); - } - - /** - * - * @param afn - * @param daemon - * @param kill_fn - * (Exception e) - * @param priority - * @param args_fn - * @param start - */ - private void init(RunnableCallback afn, boolean daemon, - RunnableCallback kill_fn, int priority, boolean start) { - if (kill_fn == null) { - kill_fn = new AsyncLoopDefaultKill(); - } - - Runnable runable = new AsyncLoopRunnable(afn, kill_fn); - thread = new Thread(runable); - String threadName = afn.getThreadName(); - if (threadName == null) { - threadName = afn.getClass().getSimpleName(); - } - thread.setName(threadName); - thread.setDaemon(daemon); - thread.setPriority(priority); - thread.setUncaughtExceptionHandler(new UncaughtExceptionHandler() { - @Override - public void uncaughtException(Thread t, Throwable e) { - LOG.error("UncaughtException", e); - JStormUtils.halt_process(1, "UncaughtException"); - } - }); - - this.afn = afn; - - if (start) { - thread.start(); - } - - } - - @Override - public void start() { - thread.start(); - } - - @Override - public void join() throws InterruptedException { - thread.join(); - } - - // for test - public void join(int times) throws InterruptedException { - thread.join(times); - } - - @Override - public void interrupt() { - thread.interrupt(); - } - - @Override - public Boolean isSleeping() { - return Time.isThreadWaiting(thread); - } - - public Thread getThread() { - return thread; - } - - @Override - public void cleanup() { - // TODO Auto-generated method stub - afn.cleanup(); - } -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/BaseCallback.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/BaseCallback.java deleted file mode 100644 index 31012faef..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/BaseCallback.java +++ /dev/null @@ -1,13 +0,0 @@ -package com.alibaba.jstorm.callback; - -import com.alibaba.jstorm.callback.Callback; - -public class BaseCallback implements Callback { - - @Override - public Object execute(T... args) { - // TODO Auto-generated method stub - return null; - } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/Callback.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/Callback.java deleted file mode 100644 index d832a7102..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/Callback.java +++ /dev/null @@ -1,13 +0,0 @@ -package com.alibaba.jstorm.callback; - -/** - * Callback interface - * - * @author lixin 2012-3-12 - * - */ -public interface Callback { - - public Object execute(T... args); - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/ClusterStateCallback.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/ClusterStateCallback.java deleted file mode 100644 index 2726b1d81..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/ClusterStateCallback.java +++ /dev/null @@ -1,7 +0,0 @@ -package com.alibaba.jstorm.callback; - -import com.alibaba.jstorm.callback.BaseCallback; - -public class ClusterStateCallback extends BaseCallback { - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/DefaultWatcherCallBack.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/DefaultWatcherCallBack.java deleted file mode 100644 index bd4ce257f..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/DefaultWatcherCallBack.java +++ /dev/null @@ -1,26 +0,0 @@ -package com.alibaba.jstorm.callback; - -import org.apache.log4j.Logger; -import org.apache.zookeeper.Watcher.Event.EventType; -import org.apache.zookeeper.Watcher.Event.KeeperState; - -import com.alibaba.jstorm.zk.ZkEventTypes; -import com.alibaba.jstorm.zk.ZkKeeperStates; - -/** - * Default ZK watch callback - * - * @author yannian - * - */ -public class DefaultWatcherCallBack implements WatcherCallBack { - - private static Logger LOG = Logger.getLogger(DefaultWatcherCallBack.class); - - @Override - public void execute(KeeperState state, EventType type, String path) { - LOG.info("Zookeeper state update:" + ZkKeeperStates.getStateName(state) - + "," + ZkEventTypes.getStateName(type) + "," + path); - } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/RunnableCallback.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/RunnableCallback.java deleted file mode 100644 index ccee6e240..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/RunnableCallback.java +++ /dev/null @@ -1,47 +0,0 @@ -package com.alibaba.jstorm.callback; - - -/** - * Base Runnable/Callback function - * - * @author yannian - * - */ -public class RunnableCallback implements Runnable, Callback { - - @Override - public Object execute(T... args) { - return null; - } - - @Override - public void run() { - - } - - public Exception error() { - return null; - } - - public Object getResult() { - return null; - } - - /** - * Called by exception - */ - public void shutdown() { - } - - /** - * Normal quit - */ - public void cleanup() { - - } - - public String getThreadName() { - return null; - } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/WatcherCallBack.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/WatcherCallBack.java deleted file mode 100644 index 382cc4b3d..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/callback/WatcherCallBack.java +++ /dev/null @@ -1,8 +0,0 @@ -package com.alibaba.jstorm.callback; - -import org.apache.zookeeper.Watcher.Event.EventType; -import org.apache.zookeeper.Watcher.Event.KeeperState; - -public interface WatcherCallBack { - public void execute(KeeperState state, EventType type, String path); -} 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 deleted file mode 100644 index f71a4ced3..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/ConfigExtension.java +++ /dev/null @@ -1,642 +0,0 @@ -package com.alibaba.jstorm.client; - -import java.security.InvalidParameterException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -import org.apache.commons.lang.StringUtils; - -import backtype.storm.Config; -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.utils.JStormUtils; - -public class ConfigExtension { - /** - * if this configure has been set, the spout or bolt will log all receive - * tuples - * - * topology.debug just for logging all sent tuples - */ - protected static final String TOPOLOGY_DEBUG_RECV_TUPLE = "topology.debug.recv.tuple"; - - public static void setTopologyDebugRecvTuple(Map conf, boolean debug) { - conf.put(TOPOLOGY_DEBUG_RECV_TUPLE, Boolean.valueOf(debug)); - } - - public static Boolean isTopologyDebugRecvTuple(Map conf) { - return JStormUtils.parseBoolean(conf.get(TOPOLOGY_DEBUG_RECV_TUPLE), - false); - } - - /** - * port number of deamon httpserver server - */ - private static final Integer DEFAULT_DEAMON_HTTPSERVER_PORT = 7621; - - protected static final String SUPERVISOR_DEAMON_HTTPSERVER_PORT = "supervisor.deamon.logview.port"; - - public static Integer getSupervisorDeamonHttpserverPort(Map conf) { - return JStormUtils.parseInt( - conf.get(SUPERVISOR_DEAMON_HTTPSERVER_PORT), - DEFAULT_DEAMON_HTTPSERVER_PORT + 1); - } - - protected static final String NIMBUS_DEAMON_HTTPSERVER_PORT = "nimbus.deamon.logview.port"; - - public static Integer getNimbusDeamonHttpserverPort(Map conf) { - return JStormUtils.parseInt(conf.get(NIMBUS_DEAMON_HTTPSERVER_PORT), - DEFAULT_DEAMON_HTTPSERVER_PORT); - } - - /** - * Worker gc parameter - * - * - */ - protected static final String WORKER_GC_CHILDOPTS = "worker.gc.childopts"; - - public static void setWorkerGc(Map conf, String gc) { - conf.put(WORKER_GC_CHILDOPTS, gc); - } - - public static String getWorkerGc(Map conf) { - return (String) conf.get(WORKER_GC_CHILDOPTS); - } - - protected static final String WOREKER_REDIRECT_OUTPUT = "worker.redirect.output"; - - public static boolean getWorkerRedirectOutput(Map conf) { - Object result = conf.get(WOREKER_REDIRECT_OUTPUT); - if (result == null) - return true; - return (Boolean) result; - } - - protected static final String WOREKER_REDIRECT_OUTPUT_FILE = "worker.redirect.output.file"; - - public static void setWorkerRedirectOutputFile(Map conf, String outputPath) { - conf.put(WOREKER_REDIRECT_OUTPUT_FILE, outputPath); - } - - public static String getWorkerRedirectOutputFile(Map conf) { - return (String)conf.get(WOREKER_REDIRECT_OUTPUT_FILE); - } - - /** - * Usually, spout finish prepare before bolt, so spout need wait several - * seconds so that bolt finish preparation - * - * By default, the setting is 30 seconds - */ - protected static final String SPOUT_DELAY_RUN = "spout.delay.run"; - - public static void setSpoutDelayRunSeconds(Map conf, int delay) { - conf.put(SPOUT_DELAY_RUN, Integer.valueOf(delay)); - } - - public static int getSpoutDelayRunSeconds(Map conf) { - return JStormUtils.parseInt(conf.get(SPOUT_DELAY_RUN), 30); - } - - /** - * Default ZMQ Pending queue size - */ - public static final int DEFAULT_ZMQ_MAX_QUEUE_MSG = 1000; - - /** - * One task will alloc how many memory slot, the default setting is 1 - */ - protected static final String MEM_SLOTS_PER_TASK = "memory.slots.per.task"; - - @Deprecated - public static void setMemSlotPerTask(Map conf, int slotNum) { - if (slotNum < 1) { - throw new InvalidParameterException(); - } - conf.put(MEM_SLOTS_PER_TASK, Integer.valueOf(slotNum)); - } - - /** - * One task will use cpu slot number, the default setting is 1 - */ - protected static final String CPU_SLOTS_PER_TASK = "cpu.slots.per.task"; - - @Deprecated - public static void setCpuSlotsPerTask(Map conf, int slotNum) { - if (slotNum < 1) { - throw new InvalidParameterException(); - } - conf.put(CPU_SLOTS_PER_TASK, Integer.valueOf(slotNum)); - } - - /** - * if the setting has been set, the component's task must run different node - * This is conflict with USE_SINGLE_NODE - */ - protected static final String TASK_ON_DIFFERENT_NODE = "task.on.differ.node"; - - public static void setTaskOnDifferentNode(Map conf, boolean isIsolate) { - conf.put(TASK_ON_DIFFERENT_NODE, Boolean.valueOf(isIsolate)); - } - - public static boolean isTaskOnDifferentNode(Map conf) { - return JStormUtils - .parseBoolean(conf.get(TASK_ON_DIFFERENT_NODE), false); - } - - protected static final String SUPERVISOR_ENABLE_CGROUP = "supervisor.enable.cgroup"; - - public static boolean isEnableCgroup(Map conf) { - return JStormUtils.parseBoolean(conf.get(SUPERVISOR_ENABLE_CGROUP), - false); - } - - /** - * If component or topology configuration set "use.old.assignment", will try - * use old assignment firstly - */ - protected static final String USE_OLD_ASSIGNMENT = "use.old.assignment"; - - 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 - */ - protected static final String SUPERVISOR_HOSTNAME = "supervisor.hostname"; - public static final Object SUPERVISOR_HOSTNAME_SCHEMA = String.class; - - public static String getSupervisorHost(Map conf) { - return (String) conf.get(SUPERVISOR_HOSTNAME); - } - - protected static final String SUPERVISOR_USE_IP = "supervisor.use.ip"; - - public static boolean isSupervisorUseIp(Map conf) { - return JStormUtils.parseBoolean(conf.get(SUPERVISOR_USE_IP), false); - } - - protected static final String NIMBUS_USE_IP = "nimbus.use.ip"; - - public static boolean isNimbusUseIp(Map conf) { - return JStormUtils.parseBoolean(conf.get(NIMBUS_USE_IP), false); - } - - protected static final String TOPOLOGY_ENABLE_CLASSLOADER = "topology.enable.classloader"; - - public static boolean isEnableTopologyClassLoader(Map conf) { - return JStormUtils.parseBoolean(conf.get(TOPOLOGY_ENABLE_CLASSLOADER), - false); - } - - 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"; - - /** - * Get to know whether nimbus is run under Apsara/Yarn container - * - * @param conf - * @return - */ - public static boolean isEnableContainerNimbus() { - String path = System.getenv(CONTAINER_NIMBUS_HEARTBEAT); - - if (StringUtils.isBlank(path)) { - return false; - } else { - return true; - } - } - - /** - * Get Apsara/Yarn nimbus container's hearbeat dir - * - * @param conf - * @return - */ - public static String getContainerNimbusHearbeat() { - return System.getenv(CONTAINER_NIMBUS_HEARTBEAT); - } - - protected static final String CONTAINER_SUPERVISOR_HEARTBEAT = "container.supervisor.heartbeat"; - - /** - * Get to know whether supervisor is run under Apsara/Yarn supervisor - * container - * - * @param conf - * @return - */ - public static boolean isEnableContainerSupervisor() { - String path = System.getenv(CONTAINER_SUPERVISOR_HEARTBEAT); - - if (StringUtils.isBlank(path)) { - return false; - } else { - return true; - } - } - - /** - * Get Apsara/Yarn supervisor container's hearbeat dir - * - * @param conf - * @return - */ - public static String getContainerSupervisorHearbeat() { - return (String) System.getenv(CONTAINER_SUPERVISOR_HEARTBEAT); - } - - protected static final String CONTAINER_HEARTBEAT_TIMEOUT_SECONDS = "container.heartbeat.timeout.seconds"; - - public static int getContainerHeartbeatTimeoutSeconds(Map conf) { - return JStormUtils.parseInt( - conf.get(CONTAINER_HEARTBEAT_TIMEOUT_SECONDS), 240); - } - - protected static final String CONTAINER_HEARTBEAT_FREQUENCE = "container.heartbeat.frequence"; - - public static int getContainerHeartbeatFrequence(Map conf) { - return JStormUtils - .parseInt(conf.get(CONTAINER_HEARTBEAT_FREQUENCE), 10); - } - - protected static final String JAVA_SANDBOX_ENABLE = "java.sandbox.enable"; - - public static boolean isJavaSandBoxEnable(Map conf) { - return JStormUtils.parseBoolean(conf.get(JAVA_SANDBOX_ENABLE), false); - } - - protected static String SPOUT_SINGLE_THREAD = "spout.single.thread"; - - public static boolean isSpoutSingleThread(Map conf) { - return JStormUtils.parseBoolean(conf.get(SPOUT_SINGLE_THREAD), false); - } - - public static void setSpoutSingleThread(Map conf, boolean enable) { - conf.put(SPOUT_SINGLE_THREAD, enable); - } - - protected static String WORKER_STOP_WITHOUT_SUPERVISOR = "worker.stop.without.supervisor"; - - public static boolean isWorkerStopWithoutSupervisor(Map conf) { - return JStormUtils.parseBoolean( - conf.get(WORKER_STOP_WITHOUT_SUPERVISOR), false); - } - - protected static String CGROUP_ROOT_DIR = "supervisor.cgroup.rootdir"; - - public static String getCgroupRootDir(Map conf) { - return (String) conf.get(CGROUP_ROOT_DIR); - } - - protected static String NETTY_TRANSFER_ASYNC_AND_BATCH = "storm.messaging.netty.transfer.async.batch"; - - public static boolean isNettyTransferAsyncBatch(Map conf) { - return JStormUtils.parseBoolean( - conf.get(NETTY_TRANSFER_ASYNC_AND_BATCH), true); - } - - protected static final String USE_USERDEFINE_ASSIGNMENT = "use.userdefine.assignment"; - - public static void setUserDefineAssignment(Map conf, - List userDefines) { - List ret = new ArrayList(); - for (WorkerAssignment worker : userDefines) { - ret.add(Utils.to_json(worker)); - } - 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"; - - public static void setMemSizePerWorker(Map conf, long memSize) { - conf.put(MEMSIZE_PER_WORKER, memSize); - } - - public static void setMemSizePerWorkerByKB(Map conf, long memSize) { - long size = memSize * 1024l; - setMemSizePerWorker(conf, size); - } - - public static void setMemSizePerWorkerByMB(Map conf, long memSize) { - long size = memSize * 1024l; - setMemSizePerWorkerByKB(conf, size); - } - - 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"; - - public static boolean isEnablePerformanceMetrics(Map conf) { - return JStormUtils.parseBoolean(conf.get(TOPOLOGY_PERFORMANCE_METRICS), - true); - } - - public static void setPerformanceMetrics(Map conf, boolean isEnable) { - conf.put(TOPOLOGY_PERFORMANCE_METRICS, isEnable); - } - - protected static String NETTY_BUFFER_THRESHOLD_SIZE = "storm.messaging.netty.buffer.threshold"; - - public static long getNettyBufferThresholdSize(Map conf) { - return JStormUtils.parseLong(conf.get(NETTY_BUFFER_THRESHOLD_SIZE), - 8 *JStormUtils.SIZE_1_M); - } - - public static void setNettyBufferThresholdSize(Map conf, long size) { - conf.put(NETTY_BUFFER_THRESHOLD_SIZE, size); - } - - protected static String NETTY_MAX_SEND_PENDING = "storm.messaging.netty.max.pending"; - - public static void setNettyMaxSendPending(Map conf, long pending) { - conf.put(NETTY_MAX_SEND_PENDING, pending); - } - - public static long getNettyMaxSendPending(Map conf) { - return JStormUtils.parseLong(conf.get(NETTY_MAX_SEND_PENDING), 16); - } - - protected static String DISRUPTOR_USE_SLEEP = "disruptor.use.sleep"; - - public static boolean isDisruptorUseSleep(Map conf) { - return JStormUtils.parseBoolean(conf.get(DISRUPTOR_USE_SLEEP), true); - } - - public static void setDisruptorUseSleep(Map conf, boolean useSleep) { - conf.put(DISRUPTOR_USE_SLEEP, useSleep); - } - - public static boolean isTopologyContainAcker(Map conf) { - int num = JStormUtils.parseInt(conf.get(Config.TOPOLOGY_ACKER_EXECUTORS), 1); - if (num > 0) { - return true; - }else { - return false; - } - } - - protected static String NETTY_SYNC_MODE = "storm.messaging.netty.sync.mode"; - - public static boolean isNettySyncMode(Map conf) { - return JStormUtils.parseBoolean(conf.get(NETTY_SYNC_MODE), false); - } - - public static void setNettySyncMode(Map conf, boolean sync) { - conf.put(NETTY_SYNC_MODE, sync); - } - - protected static String NETTY_ASYNC_BLOCK = "storm.messaging.netty.async.block"; - public static boolean isNettyASyncBlock(Map conf) { - return JStormUtils.parseBoolean(conf.get(NETTY_ASYNC_BLOCK), true); - } - - public static void setNettyASyncBlock(Map conf, boolean block) { - conf.put(NETTY_ASYNC_BLOCK, block); - } - - protected static String ALIMONITOR_METRICS_POST = "topology.alimonitor.metrics.post"; - - public static boolean isAlimonitorMetricsPost(Map conf) { - return JStormUtils.parseBoolean(conf.get(ALIMONITOR_METRICS_POST), true); - } - - public static void setAlimonitorMetricsPost(Map conf, boolean post) { - conf.put(ALIMONITOR_METRICS_POST, post); - } - - protected static String TASK_CLEANUP_TIMEOUT_SEC = "task.cleanup.timeout.sec"; - - public static int getTaskCleanupTimeoutSec(Map conf) { - return JStormUtils.parseInt(conf.get(TASK_CLEANUP_TIMEOUT_SEC), 10); - } - - public static void setTaskCleanupTimeoutSec(Map conf, int timeout) { - conf.put(TASK_CLEANUP_TIMEOUT_SEC, timeout); - } - - protected static String UI_CLUSTERS = "ui.clusters"; - protected static String UI_CLUSTER_NAME = "name"; - protected static String UI_CLUSTER_ZK_ROOT = "zkRoot"; - protected static String UI_CLUSTER_ZK_SERVERS = "zkServers"; - protected static String UI_CLUSTER_ZK_PORT = "zkPort"; - - public static List getUiClusters(Map conf) { - return (List) conf.get(UI_CLUSTERS); - } - - public static void setUiClusters(Map conf, List uiClusters) { - conf.put(UI_CLUSTERS, uiClusters); - } - - public static Map getUiClusterInfo(List uiClusters, String name) { - Map ret = null; - for (Map cluster : uiClusters) { - String clusterName = getUiClusterName(cluster); - if (clusterName.equals(name)) { - ret = cluster; - break; - } - } - - return ret; - } - - public static String getUiClusterName(Map uiCluster) { - return (String) uiCluster.get(UI_CLUSTER_NAME); - } - - public static String getUiClusterZkRoot(Map uiCluster) { - return (String) uiCluster.get(UI_CLUSTER_ZK_ROOT); - } - - public static List getUiClusterZkServers(Map uiCluster) { - return (List) uiCluster.get(UI_CLUSTER_ZK_SERVERS); - } - - public static Integer getUiClusterZkPort(Map uiCluster) { - return JStormUtils.parseInt(uiCluster.get(UI_CLUSTER_ZK_PORT)); - } - - protected static String SPOUT_PEND_FULL_SLEEP = "spout.pending.full.sleep"; - - public static boolean isSpoutPendFullSleep(Map conf) { - return JStormUtils.parseBoolean(conf.get(SPOUT_PEND_FULL_SLEEP), false); - } - - public static void setSpoutPendFullSleep(Map conf, boolean sleep) { - conf.put(SPOUT_PEND_FULL_SLEEP, sleep); - - } - - protected static String LOGVIEW_ENCODING = "supervisor.deamon.logview.encoding"; - protected static String UTF8 = "utf-8"; - - public static String getLogViewEncoding(Map conf) { - String ret = (String) conf.get(LOGVIEW_ENCODING); - if (ret == null) ret = UTF8; - return ret; - } - - public static void setLogViewEncoding(Map conf, String enc) { - conf.put(LOGVIEW_ENCODING, enc); - } - - public static String TASK_STATUS_ACTIVE = "Active"; - public static String TASK_STATUS_STARTING = "Starting"; - - protected static String ALIMONITOR_TOPO_METIRC_NAME = "topology.alimonitor.topo.metrics.name"; - protected static String ALIMONITOR_TASK_METIRC_NAME = "topology.alimonitor.task.metrics.name"; - protected static String ALIMONITOR_WORKER_METIRC_NAME = "topology.alimonitor.worker.metrics.name"; - protected static String ALIMONITOR_USER_METIRC_NAME = "topology.alimonitor.user.metrics.name"; - - public static String getAlmonTopoMetricName(Map conf) { - return (String) conf.get(ALIMONITOR_TOPO_METIRC_NAME); - } - - public static String getAlmonTaskMetricName(Map conf) { - return (String) conf.get(ALIMONITOR_TASK_METIRC_NAME); - } - - public static String getAlmonWorkerMetricName(Map conf) { - return (String) conf.get(ALIMONITOR_WORKER_METIRC_NAME); - } - - public static String getAlmonUserMetricName(Map conf) { - return (String) conf.get(ALIMONITOR_USER_METIRC_NAME); - } - - protected static String SPOUT_PARALLELISM = "topology.spout.parallelism"; - protected static String BOLT_PARALLELISM = "topology.bolt.parallelism"; - - public static Integer getSpoutParallelism(Map conf, String componentName) { - Integer ret = null; - Map map = (Map)(conf.get(SPOUT_PARALLELISM)); - if(map != null) ret = JStormUtils.parseInt(map.get(componentName)); - return ret; - } - - public static Integer getBoltParallelism(Map conf, String componentName) { - Integer ret = null; - Map map = (Map)(conf.get(BOLT_PARALLELISM)); - if(map != null) ret = JStormUtils.parseInt(map.get(componentName)); - return ret; - } - - protected static String TOPOLOGY_BUFFER_SIZE_LIMITED = "topology.buffer.size.limited"; - - public static void setTopologyBufferSizeLimited(Map conf, boolean limited) { - conf.put(TOPOLOGY_BUFFER_SIZE_LIMITED, limited); - } - - public static boolean getTopologyBufferSizeLimited(Map conf) { - boolean isSynchronized = isNettySyncMode(conf); - if (isSynchronized == true) { - return true; - } - - 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 deleted file mode 100644 index 9eac3269e..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/WorkerAssignment.java +++ /dev/null @@ -1,264 +0,0 @@ -package com.alibaba.jstorm.client; - -import java.io.Serializable; -import java.util.HashMap; -import java.util.Map; -import java.util.Map.Entry; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; -import org.apache.log4j.Logger; -import org.json.simple.JSONAware; - -import com.alibaba.jstorm.utils.JStormUtils; - -import backtype.storm.scheduler.WorkerSlot; -import backtype.storm.utils.Utils; - - -public class WorkerAssignment extends WorkerSlot implements Serializable, - JSONAware { - private static final Logger LOG = Logger.getLogger(WorkerAssignment.class); - - - private static final long serialVersionUID = -3483047434535537861L; - - private Map componentToNum = new HashMap(); - - private long mem; - - private int cpu; - - private String hostName; - - private String jvm; - - private static final String COMPONENTTONUM_TAG = "componentToNum"; - private static final String MEM_TAG = "mem"; - private static final String CPU_TAG = "cpu"; - private static final String HOSTNAME_TAG = "hostName"; - private static final String JVM_TAG = "jvm"; - private static final String NODEID_TAG = "nodeId"; - private static final String PORT_TAG = "port"; - - public WorkerAssignment(String nodeId, Number port) { - super(nodeId, port); - // TODO Auto-generated constructor stub - } - - public WorkerAssignment() { - - } - - public void addComponent(String compenentName, Integer num) { - componentToNum.put(compenentName, num); - } - - public Map getComponentToNum() { - return componentToNum; - } - - public String getHostName() { - return hostName; - } - - public void setHostName(String hostName) { - this.hostName = hostName; - } - - public void setJvm(String jvm) { - this.jvm = jvm; - } - - public String getJvm() { - return jvm; - } - - public long getMem() { - return mem; - } - - public void setMem(long mem) { - this.mem = mem; - } - - public int getCpu() { - return cpu; - } - - public void setCpu(int cpu) { - this.cpu = cpu; - } - - @Override - public String toJSONString() { -// StringBuilder sb = new StringBuilder(); - -// sb.append("["); -// sb.append("\"" + this.getNodeId() + "\""); -// sb.append(","); -// sb.append("\"" + this.hostName + "\""); -// sb.append(","); -// sb.append("\"" + String.valueOf(this.getPort()) + "\""); -// sb.append(","); -// sb.append("\"" + this.jvm + "\""); -// sb.append(","); -// sb.append("\"" + String.valueOf(this.mem) + "\""); -// sb.append(","); -// sb.append("\"" + String.valueOf(this.cpu) + "\""); -// sb.append(","); -// sb.append("{"); -// for (Entry entry : componentToNum.entrySet()) { -// sb.append("\"" + entry.getKey() + "\":"); -// sb.append("\"" + String.valueOf(entry.getValue()) + "\""); -// sb.append(","); -// } -// sb.append("}"); -// sb.append("]"); - - - - Map map = new HashMap(); - - map.put(COMPONENTTONUM_TAG, Utils.to_json(componentToNum)); - map.put(MEM_TAG, String.valueOf(mem)); - map.put(CPU_TAG, String.valueOf(cpu)); - map.put(HOSTNAME_TAG, hostName); - map.put(JVM_TAG, jvm); - map.put(NODEID_TAG, getNodeId()); - map.put(PORT_TAG, String.valueOf(getPort())); - - - return Utils.to_json(map); - } - - public static WorkerAssignment parseFromObj(Object obj) { - if (obj == null) { - return null; - } - - if (obj instanceof Map == false) { - return null; - } - - try { - Map map = (Map)obj; - - String supervisorId = map.get(NODEID_TAG); - String hostname = map.get(HOSTNAME_TAG); - Integer port = JStormUtils.parseInt(map.get(PORT_TAG)); - String jvm = map.get(JVM_TAG); - Long mem = JStormUtils.parseLong(map.get(MEM_TAG)); - Integer cpu = JStormUtils.parseInt(map.get(CPU_TAG)); - Map componentToNum = (Map)Utils.from_json(map.get(COMPONENTTONUM_TAG)); - - WorkerAssignment ret = new WorkerAssignment(supervisorId, port); - - - ret.hostName = hostname; - ret.setNodeId(supervisorId); - ret.setJvm(jvm); - if (port != null) { - ret.setPort(port); - } - if (mem != null) { - ret.setMem(mem); - } - if (cpu != null) { - ret.setCpu(cpu); - } - - for (Entry entry : componentToNum.entrySet()) { - ret.addComponent(entry.getKey(), - JStormUtils.parseInt(entry.getValue())); - } - return ret; - } catch (Exception e) { - LOG.error("Failed to convert to WorkerAssignment, raw:" + obj, e); - return null; - } - - } - - public static String getStringFromJson(String text) { - return text.equals("null") ? null : text; - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this, - ToStringStyle.SHORT_PREFIX_STYLE); - } - - @Override - public int hashCode() { - final int prime = 31; - int result = super.hashCode(); - result = prime * result - + ((componentToNum == null) ? 0 : componentToNum.hashCode()); - result = prime * result + cpu; - result = prime * result - + ((hostName == null) ? 0 : hostName.hashCode()); - result = prime * result + ((jvm == null) ? 0 : jvm.hashCode()); - result = prime * result + (int) (mem ^ (mem >>> 32)); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (!super.equals(obj)) - return false; - if (getClass() != obj.getClass()) - return false; - WorkerAssignment other = (WorkerAssignment) obj; - if (componentToNum == null) { - if (other.componentToNum != null) - return false; - } else if (!componentToNum.equals(other.componentToNum)) - return false; - if (cpu != other.cpu) - return false; - if (hostName == null) { - if (other.hostName != null) - return false; - } else if (!hostName.equals(other.hostName)) - return false; - if (jvm == null) { - if (other.jvm != null) - return false; - } else if (!jvm.equals(other.jvm)) - return false; - if (mem != other.mem) - return false; - return true; - } - - public static void main(String[] args) { - WorkerAssignment input = new WorkerAssignment(); - - input.setJvm("sb"); - - input.setCpu(1); - - input.setMem(2); - - input.addComponent("2b", 2); - - String outString = Utils.to_json(input); - - System.out.println(input); - - //String outString = "[componentToNum={},mem=1610612736,cpu=1,hostName=mobilejstorm-60-1,jvm=,nodeId=,port=0]"; - - Object object = Utils.from_json(outString); - System.out.println(object); - - System.out.println(parseFromObj(object)); - - System.out.print(input.equals(parseFromObj(object))); - } - - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/metric/MetricCallback.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/metric/MetricCallback.java deleted file mode 100644 index 964913ef8..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/metric/MetricCallback.java +++ /dev/null @@ -1,7 +0,0 @@ -package com.alibaba.jstorm.client.metric; - -import com.codahale.metrics.Metric; - -public interface MetricCallback { - void callback(T metric); -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/metric/MetricClient.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/metric/MetricClient.java deleted file mode 100644 index becc36510..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/metric/MetricClient.java +++ /dev/null @@ -1,66 +0,0 @@ -package com.alibaba.jstorm.client.metric; - -import backtype.storm.task.TopologyContext; - -import com.alibaba.jstorm.metric.Metrics; -import com.codahale.metrics.Counter; -import com.codahale.metrics.Gauge; -import com.codahale.metrics.Histogram; -import com.codahale.metrics.Meter; -import com.codahale.metrics.Timer; -import com.alibaba.jstorm.metric.JStormTimer; -import com.alibaba.jstorm.metric.JStormHistogram; - -public class MetricClient { - - private final int taskid; - - public MetricClient(TopologyContext context) { - taskid = context.getThisTaskId(); - } - - private String getMetricName(Integer taskid, String name) { - return "task-" + String.valueOf(taskid) + ":" + name; - } - - public Gauge registerGauge(String name, Gauge gauge, MetricCallback> callback) { - String userMetricName = getMetricName(taskid, name); - Gauge ret = Metrics.registerGauge(userMetricName, gauge); - Metrics.registerUserDefine(userMetricName, gauge, callback); - return ret; - } - - public Counter registerCounter(String name, MetricCallback callback) { - String userMetricName = getMetricName(taskid, name); - Counter ret = Metrics.registerCounter(userMetricName); - Metrics.registerUserDefine(userMetricName, ret, callback); - return ret; - } - - public Meter registerMeter(String name, MetricCallback callback) { - String userMetricName = getMetricName(taskid, name); - Meter ret = Metrics.registerMeter(userMetricName); - Metrics.registerUserDefine(userMetricName, ret, callback); - return ret; - } - - public JStormTimer registerTimer(String name, MetricCallback callback) { - String userMetricName = getMetricName(taskid, name); - JStormTimer ret = Metrics.registerTimer(userMetricName); - Metrics.registerUserDefine(userMetricName, ret, callback); - return ret; - } - - public JStormHistogram registerHistogram(String name, MetricCallback callback) { - String userMetricName = getMetricName(taskid, name); - JStormHistogram ret = Metrics.registerHistograms(userMetricName); - Metrics.registerUserDefine(userMetricName, ret, callback); - return ret; - } - - public boolean unregister(String name, Integer taskid) { - String userMetricName = getMetricName(taskid, name); - return Metrics.unregisterUserDefine(userMetricName); - } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/spout/IAckValueSpout.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/spout/IAckValueSpout.java deleted file mode 100644 index f14009898..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/spout/IAckValueSpout.java +++ /dev/null @@ -1,15 +0,0 @@ -package com.alibaba.jstorm.client.spout; - -import java.util.List; - -/** - * This interface will list emit values when tuple success - * - * if spout implement this interface, - * spout won't call ISpout.ack() when tuple success - * - * @author longda - */ -public interface IAckValueSpout { - void ack(Object msgId, List values); -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/spout/IFailValueSpout.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/spout/IFailValueSpout.java deleted file mode 100644 index 913709d6b..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/client/spout/IFailValueSpout.java +++ /dev/null @@ -1,15 +0,0 @@ -package com.alibaba.jstorm.client.spout; - -import java.util.List; - -/** - * This interface will list emit values when tuple fails - * - * if spout implement this interface, - * spout won't call ISpout.fail() when tuple fail - * - * @author longda - */ -public interface IFailValueSpout { - void fail(Object msgId, List values); -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/cluster/ClusterState.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/cluster/ClusterState.java deleted file mode 100644 index de64e2fda..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/cluster/ClusterState.java +++ /dev/null @@ -1,38 +0,0 @@ -package com.alibaba.jstorm.cluster; - -import java.util.List; -import java.util.UUID; - -import com.alibaba.jstorm.callback.ClusterStateCallback; - - -/** - * All ZK interface - * - * @author yannian - * - */ -public interface ClusterState { - public void set_ephemeral_node(String path, byte[] data) throws Exception; - - public void delete_node(String path) throws Exception; - - public void set_data(String path, byte[] data) throws Exception; - - public byte[] get_data(String path, boolean watch) throws Exception; - - public List get_children(String path, boolean watch) - throws Exception; - - public void mkdirs(String path) throws Exception; - - public void tryToBeLeader(String path, byte[] host) throws Exception; - - public void close(); - - public UUID register(ClusterStateCallback callback); - - public ClusterStateCallback unregister(UUID id); - - public boolean node_existed(String path, boolean watch) throws Exception; -} 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 deleted file mode 100644 index 95224f0dd..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/cluster/DistributedClusterState.java +++ /dev/null @@ -1,175 +0,0 @@ -package com.alibaba.jstorm.cluster; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.curator.framework.CuratorFramework; -import org.apache.log4j.Logger; -import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.Watcher.Event.EventType; -import org.apache.zookeeper.Watcher.Event.KeeperState; - -import backtype.storm.Config; - -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; - -/** - * All ZK interface implementation - * - * @author yannian.mu - * - */ -public class DistributedClusterState implements ClusterState { - - private static Logger LOG = Logger.getLogger(DistributedClusterState.class); - - private Zookeeper zkobj = new Zookeeper(); - private CuratorFramework zk; - private WatcherCallBack watcher; - - /** - * why run all callbacks, when receive one event - */ - private ConcurrentHashMap callbacks = new ConcurrentHashMap(); - - private Map conf; - private AtomicBoolean active; - - public DistributedClusterState(Map _conf) throws Exception { - conf = _conf; - - // just mkdir STORM_ZOOKEEPER_ROOT dir - CuratorFramework _zk = mkZk(); - String path = String.valueOf(conf.get(Config.STORM_ZOOKEEPER_ROOT)); - zkobj.mkdirs(_zk, path); - _zk.close(); - - active = new AtomicBoolean(true); - - watcher = new WatcherCallBack() { - @Override - public void execute(KeeperState state, EventType type, String path) { - if (active.get()) { - if (!(state.equals(KeeperState.SyncConnected))) { - LOG.warn("Received event " + state + ":" + type + ":" - + path + " with disconnected Zookeeper."); - } else { - LOG.info("Received event " + state + ":" + type + ":" - + path); - } - - if (!type.equals(EventType.None)) { - for (Entry e : callbacks - .entrySet()) { - ClusterStateCallback fn = e.getValue(); - fn.execute(type, path); - } - } - } - } - }; - zk = null; - zk = mkZk(watcher); - - } - - @SuppressWarnings("unchecked") - private CuratorFramework mkZk() throws IOException { - return zkobj.mkClient(conf, - (List) conf.get(Config.STORM_ZOOKEEPER_SERVERS), - conf.get(Config.STORM_ZOOKEEPER_PORT), ""); - } - - @SuppressWarnings("unchecked") - private CuratorFramework mkZk(WatcherCallBack watcher) - throws NumberFormatException, IOException { - return zkobj.mkClient(conf, - (List) conf.get(Config.STORM_ZOOKEEPER_SERVERS), - conf.get(Config.STORM_ZOOKEEPER_PORT), - String.valueOf(conf.get(Config.STORM_ZOOKEEPER_ROOT)), watcher); - } - - @Override - public void close() { - this.active.set(false); - zk.close(); - } - - @Override - public void delete_node(String path) throws Exception { - zkobj.deletereRcursive(zk, path); - } - - @Override - public List get_children(String path, boolean watch) - throws Exception { - return zkobj.getChildren(zk, path, watch); - } - - @Override - public byte[] get_data(String path, boolean watch) throws Exception { - return zkobj.getData(zk, path, watch); - } - - @Override - public void mkdirs(String path) throws Exception { - zkobj.mkdirs(zk, path); - - } - - @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 { - zkobj.mkdirs(zk, PathUtils.parent_path(path)); - zkobj.createNode(zk, path, data, CreateMode.PERSISTENT); - } - - } - - @Override - public void set_ephemeral_node(String path, byte[] data) throws Exception { - zkobj.mkdirs(zk, PathUtils.parent_path(path)); - if (zkobj.exists(zk, path, false)) { - zkobj.setData(zk, path, data); - } else { - zkobj.createNode(zk, path, data, CreateMode.EPHEMERAL); - } - } - - @Override - public UUID register(ClusterStateCallback callback) { - UUID id = UUID.randomUUID(); - this.callbacks.put(id, callback); - return id; - } - - @Override - public ClusterStateCallback unregister(UUID id) { - return this.callbacks.remove(id); - } - - @Override - public boolean node_existed(String path, boolean watch) throws Exception { - // TODO Auto-generated method stub - return zkobj.existsNode(zk, path, watch); - } - - @Override - public void tryToBeLeader(String path, byte[] host) throws Exception { - // TODO Auto-generated method stub - zkobj.createNode(zk, path, host, CreateMode.EPHEMERAL); - } -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/common/stats/StatBuckets.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/common/stats/StatBuckets.java deleted file mode 100644 index 26701fd26..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/common/stats/StatBuckets.java +++ /dev/null @@ -1,124 +0,0 @@ -package com.alibaba.jstorm.common.stats; - -import java.util.ArrayList; -import java.util.List; - -public class StatBuckets { - - public static final Integer NUM_STAT_BUCKETS = 20; - - public static final Integer MINUTE_WINDOW = 600; - public static final Integer HOUR_WINDOW = 10800; - public static final Integer DAY_WINDOW = 86400; - - public static final String MINUTE_WINDOW_STR = "0d0h10m0s"; - public static final String HOUR_WINDOW_STR = "0d3h0m0s"; - public static final String DAY_WINDOW_STR = "1d0h0m0s"; - public static final String ALL_WINDOW_STR = "All-time"; - - public static Integer[] STAT_BUCKETS = { MINUTE_WINDOW / NUM_STAT_BUCKETS, - HOUR_WINDOW / NUM_STAT_BUCKETS, DAY_WINDOW / NUM_STAT_BUCKETS }; - - private static final String[][] PRETTYSECDIVIDERS = { - new String[] { "s", "60" }, new String[] { "m", "60" }, - new String[] { "h", "24" }, new String[] { "d", null } }; - - /** - * Service b - * - * @param key - * @return - */ - public static String parseTimeKey(Integer key) { - if (key == 0) { - return ALL_WINDOW_STR; - } else { - return String.valueOf(key); - } - } - - /** - * - * Default is the latest result - * - * @param showKey - * @return - */ - public static String getTimeKey(String showKey) { - String window = null; - if (showKey == null) { - window = String.valueOf(MINUTE_WINDOW); - } else if (showKey.equals(MINUTE_WINDOW_STR)) { - window = String.valueOf(MINUTE_WINDOW); - } else if (showKey.equals(HOUR_WINDOW_STR)) { - window = String.valueOf(HOUR_WINDOW); - } else if (showKey.equals(DAY_WINDOW_STR)) { - window = String.valueOf(DAY_WINDOW); - } else if (showKey.equals(ALL_WINDOW_STR)) { - window = ALL_WINDOW_STR; - } else { - window = String.valueOf(MINUTE_WINDOW); - } - - return window; - } - - /** - * Default is the latest result - * - * @param showStr - * @return - */ - public static String getShowTimeStr(String showStr) { - if (showStr == null) { - return MINUTE_WINDOW_STR; - } else if (showStr.equals(MINUTE_WINDOW_STR) - || showStr.equals(HOUR_WINDOW_STR) - || showStr.equals(DAY_WINDOW_STR) - || showStr.equals(ALL_WINDOW_STR)) { - return showStr; - - } else { - return MINUTE_WINDOW_STR; - } - - } - - /** - * seconds to string like 1d20h30m40s - * - * @param secs - * @return - */ - public static String prettyUptimeStr(int secs) { - int diversize = PRETTYSECDIVIDERS.length; - - List tmp = new ArrayList(); - int div = secs; - for (int i = 0; i < diversize; i++) { - if (PRETTYSECDIVIDERS[i][1] != null) { - Integer d = Integer.parseInt(PRETTYSECDIVIDERS[i][1]); - tmp.add(div % d + PRETTYSECDIVIDERS[i][0]); - div = div / d; - } else { - tmp.add(div + PRETTYSECDIVIDERS[i][0]); - } - } - - String rtn = ""; - int tmpSzie = tmp.size(); - for (int j = tmpSzie - 1; j > -1; j--) { - rtn += tmp.get(j); - } - return rtn; - } - - /** - * @param args - */ - public static void main(String[] args) { - // TODO Auto-generated method stub - - } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/common/stats/StaticsType.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/common/stats/StaticsType.java deleted file mode 100644 index e9b76b326..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/common/stats/StaticsType.java +++ /dev/null @@ -1,5 +0,0 @@ -package com.alibaba.jstorm.common.stats; - -public enum StaticsType { - emitted, send_tps, recv_tps, acked, failed, transferred, process_latencies; -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/JStormHistogram.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/JStormHistogram.java deleted file mode 100644 index 863deaa5c..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/JStormHistogram.java +++ /dev/null @@ -1,39 +0,0 @@ -package com.alibaba.jstorm.metric; - -import com.codahale.metrics.Histogram; - -public class JStormHistogram { - private static boolean isEnable = true; - - public static boolean isEnable() { - return isEnable; - } - - public static void setEnable(boolean isEnable) { - JStormHistogram.isEnable = isEnable; - } - - private Histogram instance; - private String name; - - public JStormHistogram(String name, Histogram instance) { - this.name = name; - this.instance = instance; - } - - public void update(int value) { - if (isEnable == true) { - instance.update(value); - } - } - - public void update(long value) { - if (isEnable == true) { - instance.update(value); - } - } - - public Histogram getInstance() { - return instance; - } -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/JStormTimer.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/JStormTimer.java deleted file mode 100644 index dac94db07..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/JStormTimer.java +++ /dev/null @@ -1,61 +0,0 @@ -package com.alibaba.jstorm.metric; - - -import java.util.concurrent.atomic.AtomicReference; - -import org.apache.log4j.Logger; - -import com.codahale.metrics.Timer; - -public class JStormTimer { - private static final Logger LOG = Logger.getLogger(JStormTimer.class); - private static boolean isEnable = true; - - public static boolean isEnable() { - return isEnable; - } - - public static void setEnable(boolean isEnable) { - JStormTimer.isEnable = isEnable; - } - - - private Timer instance; - private String name; - public JStormTimer(String name, Timer instance) { - this.name = name; - this.instance = instance; - this.timerContext = new AtomicReference(); - } - - /** - * This logic isn't perfect, it will miss metrics when it is called - * in the same time. But this method performance is better than - * create a new instance wrapper Timer.Context - */ - private AtomicReference timerContext = null; - public void start() { - if (JStormTimer.isEnable == false) { - return ; - } - - if (timerContext.compareAndSet(null, instance.time()) == false) { - LOG.warn("Already start timer " + name); - return ; - } - - } - - public void stop() { - Timer.Context context = timerContext.getAndSet(null); - if (context != null) { - context.stop(); - } - } - - public Timer getInstance() { - return instance; - } - - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/MetricDef.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/MetricDef.java deleted file mode 100644 index 882057edb..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/MetricDef.java +++ /dev/null @@ -1,39 +0,0 @@ -package com.alibaba.jstorm.metric; - -public class MetricDef { - // metric name for task - public static final String DESERIALIZE_QUEUE = "Deserialize_Queue"; - public static final String DESERIALIZE_TIME = "Deserialize_Time"; - public static final String SERIALIZE_QUEUE = "Serialize_Queue"; - public static final String SERIALIZE_TIME = "Serialize_Time"; - public static final String EXECUTE_QUEUE = "Executor_Queue"; - public static final String EXECUTE_TIME = "Execute_Time"; - public static final String ACKER_TIME = "Acker_Time"; - public static final String EMPTY_CPU_RATIO = "Empty_Cpu_Ratio"; - public static final String PENDING_MAP = "Pending_Num"; - public static final String EMIT_TIME = "Emit_Time"; - - // metric name for worker - public static final String NETWORK_MSG_TRANS_TIME = "Network_Transmit_Time"; - public static final String NETTY_SERV_DECODE_TIME = "Netty_Server_Decode_Time"; - public static final String DISPATCH_TIME = "Virtual_Port_Dispatch_Time"; - public static final String DISPATCH_QUEUE = "Virtual_Port_Dispatch_Queue"; - public static final String BATCH_TUPLE_TIME = "Batch_Tuple_Time"; - public static final String BATCH_TUPLE_QUEUE = "Batch_Tuple_Queue"; - public static final String DRAINER_TIME = "Drainer_Time"; - public static final String DRAINER_QUEUE = "Drainer_Queue"; - public static final String NETTY_CLI_SEND_TIME = "Netty_Client_Send_Time"; - public static final String NETTY_CLI_BATCH_SIZE = "Netty_Client_Send_Batch_Size"; - public static final String NETTY_CLI_SEND_PENDING = "Netty_Client_Send_Pendings"; - public static final String NETTY_CLI_SYNC_BATCH_QUEUE = "Netty_Client_Sync_BatchQueue"; - public static final String NETTY_CLI_SYNC_DISR_QUEUE = "Netty_Client_Sync_DisrQueue"; - - public static final String ZMQ_SEND_TIME = "ZMQ_Send_Time"; - public static final String ZMQ_SEND_MSG_SIZE = "ZMQ_Send_MSG_Size"; - - public static final String CPU_USED_RATIO = "Used_Cpu"; - public static final String MEMORY_USED = "Used_Memory"; - - public static final String REMOTE_CLI_ADDR = "Remote_Client_Address"; - public static final String REMOTE_SERV_ADDR = "Remote_Server_Address"; -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/MetricInfo.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/MetricInfo.java deleted file mode 100644 index 09a2a107a..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/MetricInfo.java +++ /dev/null @@ -1,27 +0,0 @@ -package com.alibaba.jstorm.metric; - -import com.codahale.metrics.Metric; - -public class MetricInfo { - private Metric metric; - private String prefix; - private String name; - - public MetricInfo(String prefix, String name, Metric metric) { - this.prefix = prefix; - this.name = name; - this.metric = metric; - } - - public String getPrefix() { - return prefix; - } - - public String getName() { - return name; - } - - public Metric getMetric() { - return metric; - } -} \ No newline at end of file diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/MetricJstack.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/MetricJstack.java deleted file mode 100644 index c60525a5f..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/MetricJstack.java +++ /dev/null @@ -1,123 +0,0 @@ -package com.alibaba.jstorm.metric; - -import java.lang.management.ManagementFactory; -import java.lang.management.ThreadInfo; -import java.lang.management.ThreadMXBean; - -import com.codahale.metrics.Gauge; - -public class MetricJstack implements Gauge { - - private String getTaskName(long id, String name) { - if (name == null) { - return Long.toString(id); - } - return id + " (" + name + ")"; - } - - public String dumpThread() throws Exception { - StringBuilder writer = new StringBuilder(); - - ThreadMXBean threadMXBean = ManagementFactory.getThreadMXBean(); - - boolean contention = threadMXBean.isThreadContentionMonitoringEnabled(); - - long[] threadIds = threadMXBean.getAllThreadIds(); - writer.append(threadIds.length + " active threads:"); - for (long tid : threadIds) { - writer.append(tid).append(" "); - } - writer.append("\n"); - - long[] deadLockTids = threadMXBean.findDeadlockedThreads(); - if (deadLockTids != null) { - writer.append(threadIds.length + " deadlocked threads:"); - for (long tid : deadLockTids) { - writer.append(tid).append(" "); - } - writer.append("\n"); - } - - long[] deadLockMonitorTids = threadMXBean - .findMonitorDeadlockedThreads(); - if (deadLockMonitorTids != null) { - writer.append(threadIds.length + " deadlocked monitor threads:"); - for (long tid : deadLockMonitorTids) { - writer.append(tid).append(" "); - } - writer.append("\n"); - } - - for (long tid : threadIds) { - ThreadInfo info = threadMXBean - .getThreadInfo(tid, Integer.MAX_VALUE); - if (info == null) { - writer.append(" Inactive").append("\n"); - continue; - } - writer.append( - "Thread " - + getTaskName(info.getThreadId(), - info.getThreadName()) + ":").append("\n"); - Thread.State state = info.getThreadState(); - writer.append(" State: " + state).append("\n"); - writer.append(" Blocked count: " + info.getBlockedCount()).append( - "\n"); - writer.append(" Waited count: " + info.getWaitedCount()).append( - "\n"); - writer.append(" Cpu time:") - .append(threadMXBean.getThreadCpuTime(tid) / 1000000) - .append("ms").append("\n"); - writer.append(" User time:") - .append(threadMXBean.getThreadUserTime(tid) / 1000000) - .append("ms").append("\n"); - if (contention) { - writer.append(" Blocked time: " + info.getBlockedTime()) - .append("\n"); - writer.append(" Waited time: " + info.getWaitedTime()).append( - "\n"); - } - if (state == Thread.State.WAITING) { - writer.append(" Waiting on " + info.getLockName()) - .append("\n"); - } else if (state == Thread.State.BLOCKED) { - writer.append(" Blocked on " + info.getLockName()) - .append("\n"); - writer.append( - " Blocked by " - + getTaskName(info.getLockOwnerId(), - info.getLockOwnerName())).append("\n"); - } - - } - for (long tid : threadIds) { - ThreadInfo info = threadMXBean - .getThreadInfo(tid, Integer.MAX_VALUE); - if (info == null) { - writer.append(" Inactive").append("\n"); - continue; - } - - writer.append( - "Thread " - + getTaskName(info.getThreadId(), - info.getThreadName()) + ": Stack").append( - "\n"); - for (StackTraceElement frame : info.getStackTrace()) { - writer.append(" " + frame.toString()).append("\n"); - } - } - - return writer.toString(); - } - - @Override - public String getValue() { - try { - return dumpThread(); - } catch (Exception e) { - return "Failed to get jstack thread info"; - } - } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/Metrics.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/Metrics.java deleted file mode 100644 index 3e50c0ad4..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/Metrics.java +++ /dev/null @@ -1,330 +0,0 @@ -package com.alibaba.jstorm.metric; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -import org.apache.log4j.Logger; - -import backtype.storm.utils.DisruptorQueue; - -import com.alibaba.jstorm.client.metric.MetricCallback; -//import com.alibaba.jstorm.daemon.worker.Worker; -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.MetricRegistry; -import com.codahale.metrics.MetricSet; -import com.codahale.metrics.Snapshot; -import com.codahale.metrics.Timer; -import com.codahale.metrics.jvm.GarbageCollectorMetricSet; -import com.codahale.metrics.jvm.MemoryUsageGaugeSet; -import com.codahale.metrics.jvm.ThreadStatesGaugeSet; - -public class Metrics { - - public enum MetricType { - TASK, WORKER - } - - private static final Logger LOG = Logger.getLogger(Metrics.class); - //private static final Logger DEFAULT_LOG = Logger.getLogger(Worker.class); - - private static final MetricRegistry metrics = new MetricRegistry(); - - private static final MetricRegistry jstack = new MetricRegistry(); - - private static Map> taskMetricMap = new ConcurrentHashMap>(); - private static List workerMetricList = new ArrayList(); - private static UserDefMetric userDefMetric = new UserDefMetric(); - - static { - try { - registerAll("jvm-thread-state", new ThreadStatesGaugeSet()); - registerAll("jvm-mem", new MemoryUsageGaugeSet()); - registerAll("jvm-gc", new GarbageCollectorMetricSet()); - - jstack.register("jstack", new MetricJstack()); - } catch (Exception e) { - LOG.warn("Failed to regist jvm metrics"); - } - } - - public static MetricRegistry getMetrics() { - return metrics; - } - - public static MetricRegistry getJstack() { - return jstack; - } - - public static UserDefMetric getUserDefMetric() { - return userDefMetric; - } - - public static boolean unregister(String name) { - LOG.info("Unregister metric " + name); - return metrics.remove(name); - } - - public static boolean unregister(String prefix, String name, String id, Metrics.MetricType type) { - String MetricName; - if (prefix == null) - MetricName = name; - else - MetricName = prefix + "-" + name; - boolean ret = unregister(MetricName); - - if (ret == true) { - List metricList = null; - if (type == MetricType.WORKER) { - metricList = workerMetricList; - } else { - metricList = taskMetricMap.get(id); - } - - boolean found = false; - if (metricList != null) { - for (MetricInfo metric : metricList) { - if(metric.getName().equals(name)) { - if (prefix != null) { - if (metric.getPrefix().equals(prefix)) { - metricList.remove(metric); - found = true; - break; - } - } else { - if (metric.getPrefix() == null) { - metricList.remove(metric); - found = true; - break; - } - } - } - } - } - if (found != true) - LOG.warn("Name " + name + " is not found when unregister from metricList"); - } - return ret; - } - - public static boolean unregisterUserDefine(String name) { - boolean ret = unregister(name); - - if (ret == true) { - userDefMetric.remove(name); - userDefMetric.unregisterCallback(name); - } - - return ret; - } - - public static T register(String name, T metric) - throws IllegalArgumentException { - LOG.info("Register Metric " + name); - return metrics.register(name, metric); - } - - public static T register(String prefix, String name, T metric, - String idStr, MetricType metricType) throws IllegalArgumentException { - String metricName; - if (prefix == null) - metricName = name; - else - metricName = prefix + "-" + name; - T ret = register(metricName, metric); - updateMetric(prefix, name, metricType, ret, idStr); - return ret; - } - - public static void registerUserDefine(String name, Object metric, MetricCallback callback) { - if(metric instanceof Gauge) { - userDefMetric.addToGauge(name, (Gauge)metric); - } else if (metric instanceof Timer) { - userDefMetric.addToTimer(name, (Timer)metric); - } else if (metric instanceof Counter) { - userDefMetric.addToCounter(name, (Counter)metric); - } else if (metric instanceof Meter) { - userDefMetric.addToMeter(name, (Meter)metric); - } else if (metric instanceof Histogram) { - userDefMetric.addToHistogram(name, (Histogram)metric); - } else if (metric instanceof JStormTimer) { - userDefMetric.addToTimer(name, ((JStormTimer)metric).getInstance()); - } else if (metric instanceof JStormHistogram) { - userDefMetric.addToHistogram(name, ((JStormHistogram)metric).getInstance()); - } else { - LOG.warn("registerUserDefine, unknow Metric type, name=" + name); - } - - if (callback != null) { - userDefMetric.registerCallback(callback, name); - } - } - - - // copy from MetricRegistry - public static void registerAll(String prefix, MetricSet metrics) - throws IllegalArgumentException { - for (Map.Entry entry : metrics.getMetrics().entrySet()) { - if (entry.getValue() instanceof MetricSet) { - registerAll(MetricRegistry.name(prefix, entry.getKey()), - (MetricSet) entry.getValue()); - } else { - register(MetricRegistry.name(prefix, entry.getKey()), - entry.getValue()); - } - } - } - - private static void updateMetric(String prefix, String name, MetricType metricType, - Metric metric, String idStr) { - Map> metricMap; - List metricList; - if (metricType == MetricType.TASK) { - metricMap = taskMetricMap; - metricList = metricMap.get(idStr); - if (null == metricList) { - metricList = new ArrayList(); - metricMap.put(idStr, metricList); - } - } else if (metricType == MetricType.WORKER) { - metricList = workerMetricList; - } else { - LOG.error("updateMetricMap: unknown metric type"); - return; - } - - MetricInfo metricInfo = new MetricInfo(prefix, name, metric); - metricList.add(metricInfo); - - } - - public static Map> getTaskMetricMap() { - return taskMetricMap; - } - - public static List getWorkerMetricList() { - return workerMetricList; - } - - public static class QueueGauge implements Gauge { - DisruptorQueue queue; - String name; - - public QueueGauge(String name, DisruptorQueue queue) { - this.queue = queue; - this.name = name; - } - - @Override - public Float getValue() { - Float ret = queue.pctFull(); - if (ret > 0.8) { - //DEFAULT_LOG.info("Queue " + name + "is full " + ret); - } - - return ret; - } - - } - - public static Gauge registerQueue(String name, DisruptorQueue queue) { - LOG.info("Register Metric " + name); - return metrics.register(name, new QueueGauge(name, queue)); - } - - public static Gauge registerQueue(String prefix, String name, DisruptorQueue queue, - String idStr, MetricType metricType) { - String metricName; - if (prefix == null) - metricName = name; - else - metricName = prefix + "-" + name; - Gauge ret = registerQueue(metricName, queue); - updateMetric(prefix, name, metricType, ret, idStr); - return ret; - } - - public static Gauge registerGauge(String name, Gauge gauge) { - LOG.info("Register Metric " + name); - return metrics.register(name, gauge); - } - - public static Counter registerCounter(String name) { - LOG.info("Register Metric " + name); - return metrics.counter(name); - } - - public static Counter registerCounter(String prefix, String name, - String idStr, MetricType metricType) { - String metricName; - if (prefix == null) - metricName = name; - else - metricName = prefix + "-" + name; - Counter ret = registerCounter(metricName); - updateMetric(prefix, name, metricType, ret, idStr); - return ret; - } - - public static Meter registerMeter(String name) { - LOG.info("Register Metric " + name); - return metrics.meter(name); - } - - public static Meter registerMeter(String prefix, String name, - String idStr, MetricType metricType) { - String metricName; - if (prefix == null) - metricName = name; - else - metricName = prefix + "-" + name; - Meter ret = registerMeter(metricName); - updateMetric(prefix, name, metricType, ret, idStr); - return ret; - } - - public static JStormHistogram registerHistograms(String name) { - LOG.info("Register Metric " + name); - Histogram instance = metrics.histogram(name); - - return new JStormHistogram(name, instance); - } - - public static JStormHistogram registerHistograms(String prefix, String name, - String idStr, MetricType metricType) { - String metricName; - if (prefix == null) - metricName = name; - else - metricName = prefix + "-" + name; - JStormHistogram ret = registerHistograms(metricName); - updateMetric(prefix, name, metricType, ret.getInstance(), idStr); - return ret; - } - - public static JStormTimer registerTimer(String name) { - LOG.info("Register Metric " + name); - - Timer instance = metrics.timer(name); - return new JStormTimer(name, instance); - } - - public static JStormTimer registerTimer(String prefix, String name, - String idStr, MetricType metricType) { - String metricName; - if (prefix == null) - metricName = name; - else - metricName = prefix + "-" + name; - JStormTimer ret = registerTimer(metricName); - updateMetric(prefix, name, metricType, ret.getInstance(), idStr); - return ret; - } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/UserDefMetric.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/UserDefMetric.java deleted file mode 100644 index 51f787e40..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/UserDefMetric.java +++ /dev/null @@ -1,106 +0,0 @@ -package com.alibaba.jstorm.metric; - -import java.util.Map; -import java.util.HashMap; -import java.util.Map.Entry; -import java.io.Serializable; - -import com.codahale.metrics.Metric; -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.MetricInfo; - - -/** - * /storm-zk-root/Monitor/{topologyid}/UserDefMetrics/{workerid} data - */ -public class UserDefMetric { - - private static final long serialVersionUID = 4547327064057659279L; - - private Map> gaugeMap = new HashMap>(); - private Map counterMap = new HashMap(); - private Map histogramMap = new HashMap(); - private Map timerMap = new HashMap(); - private Map meterMap = new HashMap(); - private Map callbacks = new HashMap(); - - public UserDefMetric() { - } - - public Map> getGauge() { - return this.gaugeMap; - } - public void registerCallback(MetricCallback callback, String name) { - if (callbacks.containsKey(name) != true) { - callbacks.put(name, callback); - } - } - public void unregisterCallback(String name) { - callbacks.remove(name); - } - public Map getCallbacks() { - return callbacks; - } - public void addToGauge(String name, Gauge gauge) { - gaugeMap.put(name, gauge); - } - - public Map getCounter() { - return this.counterMap; - } - - public void addToCounter(String name, Counter counter) { - counterMap.put(name, counter); - } - - public Map getHistogram() { - return this.histogramMap; - } - - public void addToHistogram(String name, Histogram histogram) { - histogramMap.put(name, histogram); - } - - - public Map getTimer() { - return this.timerMap; - } - - public void addToTimer(String name, Timer timer) { - timerMap.put(name, timer); - } - - public Map getMeter() { - return this.meterMap; - } - - public void addToMeter(String name, Meter meter) { - meterMap.put(name, meter); - } - - public void remove(String name) { - if (gaugeMap.containsKey(name)) { - gaugeMap.remove(name); - } else if (counterMap.containsKey(name)) { - counterMap.remove(name); - } else if (histogramMap.containsKey(name)) { - histogramMap.remove(name); - } else if (timerMap.containsKey(name)) { - timerMap.remove(name); - } else if (meterMap.containsKey(name)) { - meterMap.remove(name); - } - - if (callbacks.containsKey(name)) { - callbacks.remove(name); - } - } - -} \ No newline at end of file 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 deleted file mode 100644 index 7ca0860cf..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/UserDefMetricData.java +++ /dev/null @@ -1,138 +0,0 @@ -package com.alibaba.jstorm.metric; - -import java.util.Map; -import java.util.HashMap; -import java.util.Map.Entry; -import java.io.Serializable; - -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.Timer; -import com.codahale.metrics.Counter; -import com.codahale.metrics.Histogram; -import com.codahale.metrics.Meter; -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; - - private Map gaugeDataMap = new HashMap(); - private Map counterDataMap = new HashMap(); - private Map timerDataMap = new HashMap(); - private Map meterDataMap = new HashMap(); - private Map histogramDataMap = new HashMap(); - - public UserDefMetricData() { - } - - public Map getGaugeDataMap() { - return gaugeDataMap; - } - - public Map getCounterDataMap() { - return counterDataMap; - } - - public Map getTimerDataMap() { - return timerDataMap; - } - - public Map getMeterDataMap() { - return meterDataMap; - } - - public Map getHistogramDataMap() { - return histogramDataMap; - } - - public void updateFromGauge(Map> gaugeMap) { - for(Entry> entry : gaugeMap.entrySet()) { - 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); - } - } - } - - public void updateFromCounter(Map counterMap) { - for(Entry entry : counterMap.entrySet()) { - CounterData counterData = new CounterData(); - counterData.setValue(entry.getValue().getCount()); - counterDataMap.put(entry.getKey(), counterData); - } - } - - public void updateFromMeterData(Map meterMap) { - for(Entry entry : meterMap.entrySet()) { - Meter meter = entry.getValue(); - MeterData meterData = new MeterData(); - meterData.setCount(meter.getCount()); - meterData.setMeanRate(meter.getMeanRate()); - meterData.setOneMinuteRate(meter.getOneMinuteRate()); - meterData.setFiveMinuteRate(meter.getFiveMinuteRate()); - meterData.setFifteenMinuteRate(meter.getFifteenMinuteRate()); - meterDataMap.put(entry.getKey(), meterData); - } - } - - public void updateFromHistogramData(Map histogramMap) { - for(Entry entry : histogramMap.entrySet()) { - Histogram histogram = entry.getValue(); - HistogramData histogramData = new HistogramData(); - histogramData.setCount(histogram.getCount()); - histogramData.setMax(histogram.getSnapshot().getMax()); - histogramData.setMin(histogram.getSnapshot().getMin()); - histogramData.setMean(histogram.getSnapshot().getMean()); - histogramData.setMedian(histogram.getSnapshot().getMedian()); - histogramData.setStdDev(histogram.getSnapshot().getStdDev()); - histogramData.setPercent75th(histogram.getSnapshot().get75thPercentile()); - histogramData.setPercent95th(histogram.getSnapshot().get95thPercentile()); - histogramData.setPercent98th(histogram.getSnapshot().get98thPercentile()); - histogramData.setPercent99th(histogram.getSnapshot().get99thPercentile()); - histogramData.setPercent999th(histogram.getSnapshot().get999thPercentile()); - histogramDataMap.put(entry.getKey(), histogramData); - } - } - - public void updateFromTimerData(Map timerMap) { - for(Entry entry : timerMap.entrySet()) { - Timer timer = entry.getValue(); - TimerData timerData = new TimerData(); - timerData.setCount(timer.getCount()); - timerData.setMax(timer.getSnapshot().getMax()); - timerData.setMin(timer.getSnapshot().getMin()); - timerData.setMean(timer.getSnapshot().getMean()); - timerData.setMedian(timer.getSnapshot().getMedian()); - timerData.setStdDev(timer.getSnapshot().getStdDev()); - timerData.setPercent75th(timer.getSnapshot().get75thPercentile()); - timerData.setPercent95th(timer.getSnapshot().get95thPercentile()); - timerData.setPercent98th(timer.getSnapshot().get98thPercentile()); - timerData.setPercent99th(timer.getSnapshot().get99thPercentile()); - timerData.setPercent999th(timer.getSnapshot().get999thPercentile()); - timerData.setMeanRate(timer.getMeanRate()); - timerData.setOneMinuteRate(timer.getOneMinuteRate()); - timerData.setFiveMinuteRate(timer.getFiveMinuteRate()); - timerData.setFifteenMinuteRate(timer.getFifteenMinuteRate()); - 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/metric/metrdata/CounterData.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/metrdata/CounterData.java deleted file mode 100644 index 727cb9da3..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/metrdata/CounterData.java +++ /dev/null @@ -1,23 +0,0 @@ -package com.alibaba.jstorm.metric.metrdata; - -import java.io.Serializable; - - -public class CounterData implements Serializable { - - private static final long serialVersionUID = 954627168057659219L; - - private long value; - - public CounterData () { - value = 0l; - } - - public long getValue() { - return value; - } - - public void setValue(long value) { - this.value = value; - } -} \ No newline at end of file diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/metrdata/GaugeData.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/metrdata/GaugeData.java deleted file mode 100644 index 9f64bf3db..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/metrdata/GaugeData.java +++ /dev/null @@ -1,23 +0,0 @@ -package com.alibaba.jstorm.metric.metrdata; - -import java.io.Serializable; - - -public class GaugeData implements Serializable { - - private static final long serialVersionUID = 954627168057659279L; - - private double value; - - public GaugeData () { - value = 0.0; - } - - public double getValue() { - return value; - } - - public void setValue(double value) { - this.value = value; - } -} \ No newline at end of file diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/metrdata/HistogramData.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/metrdata/HistogramData.java deleted file mode 100644 index ec3985148..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/metrdata/HistogramData.java +++ /dev/null @@ -1,112 +0,0 @@ -package com.alibaba.jstorm.metric.metrdata; - -import java.io.Serializable; - - -public class HistogramData implements Serializable { - - private static final long serialVersionUID = 954627168057639289L; - - private long count; - private long min; - private long max; - private double mean; - private double stdDev; - private double median; - private double percent75th; - private double percent95th; - private double percent98th; - private double percent99th; - private double percent999th; - - public HistogramData() { - } - - public long getCount() { - return count; - } - - public void setCount(long count) { - this.count = count; - } - - public long getMin() { - return min; - } - - public void setMin(long min) { - this.min = min; - } - - public long getMax() { - return max; - } - - public void setMax(long max) { - this.max = max; - } - - public double getMean() { - return mean; - } - - public void setMean(double mean) { - this.mean = mean; - } - - public double getStdDev() { - return stdDev; - } - - public void setStdDev(double stdDev) { - this.stdDev = stdDev; - } - - public double getMedian() { - return median; - } - - public void setMedian(double median) { - this.median = median; - } - - public double getPercent75th() { - return percent75th; - } - - public void setPercent75th(double percent75th) { - this.percent75th = percent75th; - } - - public double getPercent95th() { - return percent95th; - } - - public void setPercent95th(double percent95th) { - this.percent95th = percent95th; - } - - public double getPercent98th() { - return percent98th; - } - - public void setPercent98th(double percent98th) { - this.percent98th = percent98th; - } - - public double getPercent99th() { - return percent99th; - } - - public void setPercent99th(double percent99th) { - this.percent99th = percent99th; - } - - public double getPercent999th() { - return percent999th; - } - - public void setPercent999th(double percent999th) { - this.percent999th = percent999th; - } -} \ No newline at end of file diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/metrdata/MeterData.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/metrdata/MeterData.java deleted file mode 100644 index 865a3c418..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/metrdata/MeterData.java +++ /dev/null @@ -1,58 +0,0 @@ -package com.alibaba.jstorm.metric.metrdata; - -import java.io.Serializable; - - -public class MeterData implements Serializable { - - private static final long serialVersionUID = 954627168057659269L; - - private long count; - private double meanRate; - private double oneMinuteRate; - private double fiveMinuteRate; - private double fifteenMinuteRate; - - public MeterData() { - } - - public void setCount(long count) { - this.count = count; - } - - public long getCount() { - return this.count; - } - - public void setMeanRate(double meanRate) { - this.meanRate = meanRate; - } - - public double getMeanRate() { - return this.meanRate; - } - - public void setOneMinuteRate(double oneMinuteRate) { - this.oneMinuteRate = oneMinuteRate; - } - - public double getOneMinuteRate() { - return this.oneMinuteRate; - } - - public void setFiveMinuteRate(double fiveMinuteRate) { - this.fiveMinuteRate = fiveMinuteRate; - } - - public double getFiveMinuteRate() { - return this.fiveMinuteRate; - } - - public void setFifteenMinuteRate(double fifteenMinuteRate) { - this.fifteenMinuteRate = fifteenMinuteRate; - } - - public double getFifteenMinuteRate() { - return this.fifteenMinuteRate; - } -} \ No newline at end of file diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/metrdata/TimerData.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/metrdata/TimerData.java deleted file mode 100644 index 5aaab01b4..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/metric/metrdata/TimerData.java +++ /dev/null @@ -1,149 +0,0 @@ -package com.alibaba.jstorm.metric.metrdata; - -import java.io.Serializable; - - -public class TimerData implements Serializable { - - private static final long serialVersionUID = 954627168057659239L; - - private long count; - private double meanRate; - private double oneMinuteRate; - private double fiveMinuteRate; - private double fifteenMinuteRate; - private long min; - private long max; - private double mean; - private double stdDev; - private double median; - private double percent75th; - private double percent95th; - private double percent98th; - private double percent99th; - private double percent999th; - - public TimerData() { - - } - - public long getCount() { - return count; - } - - public void setCount(long count) { - this.count = count; - } - - public long getMin() { - return min; - } - - public void setMin(long min) { - this.min = min; - } - - public long getMax() { - return max; - } - - public void setMax(long max) { - this.max = max; - } - - public double getMean() { - return mean; - } - - public void setMean(double mean) { - this.mean = mean; - } - - public double getStdDev() { - return stdDev; - } - - public void setStdDev(double stdDev) { - this.stdDev = stdDev; - } - - public double getMedian() { - return median; - } - - public void setMedian(double median) { - this.median = median; - } - - public double getPercent75th() { - return percent75th; - } - - public void setPercent75th(double percent75th) { - this.percent75th = percent75th; - } - - public double getPercent95th() { - return percent95th; - } - - public void setPercent95th(double percent95th) { - this.percent95th = percent95th; - } - - public double getPercent98th() { - return percent98th; - } - - public void setPercent98th(double percent98th) { - this.percent98th = percent98th; - } - - public double getPercent99th() { - return percent99th; - } - - public void setPercent99th(double percent99th) { - this.percent99th = percent99th; - } - - public double getPercent999th() { - return percent999th; - } - - public void setPercent999th(double percent999th) { - this.percent999th = percent999th; - } - - public void setMeanRate(double meanRate) { - this.meanRate = meanRate; - } - - public double getMeanRate() { - return this.meanRate; - } - - public void setOneMinuteRate(double oneMinuteRate) { - this.oneMinuteRate = oneMinuteRate; - } - - public double getOneMinuteRate() { - return this.oneMinuteRate; - } - - public void setFiveMinuteRate(double fiveMinuteRate) { - this.fiveMinuteRate = fiveMinuteRate; - } - - public double getFiveMinuteRate() { - return this.fiveMinuteRate; - } - - public void setFifteenMinuteRate(double fifteenMinuteRate) { - this.fifteenMinuteRate = fifteenMinuteRate; - } - - public double getFifteenMinuteRate() { - return this.fifteenMinuteRate; - } -} \ No newline at end of file diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/DisruptorQueue.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/DisruptorQueue.java deleted file mode 100644 index b12709583..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/DisruptorQueue.java +++ /dev/null @@ -1,151 +0,0 @@ -//package com.alibaba.jstorm.utils; -// -//import java.util.ArrayList; -//import java.util.List; -//import java.util.concurrent.Executor; -//import java.util.concurrent.atomic.AtomicBoolean; -// -//import org.apache.commons.lang.mutable.MutableObject; -// -//import com.lmax.disruptor.EventFactory; -//import com.lmax.disruptor.ExceptionHandler; -//import com.lmax.disruptor.FatalExceptionHandler; -//import com.lmax.disruptor.RingBuffer; -//import com.lmax.disruptor.Sequence; -//import com.lmax.disruptor.SequenceBarrier; -//import com.lmax.disruptor.Sequencer; -//import com.lmax.disruptor.WaitStrategy; -//import com.lmax.disruptor.WorkHandler; -//import com.lmax.disruptor.WorkProcessor; -//import com.lmax.disruptor.util.Util; -// -//public class DisruptorQueue { -// private final RingBuffer ringBuffer; -// private final SequenceBarrier sequenceBarrier; -// private final ExceptionHandler exceptionHandler; -// private final List workProcessors; -// private final Sequence workSequence; -// private final AtomicBoolean started = new AtomicBoolean(false); -// -// public DisruptorQueue(boolean isMultiProducer, int bufferSize, -// WaitStrategy waitStrategy) { -// if (isMultiProducer) { -// ringBuffer = RingBuffer.createMultiProducer( -// new ObjectEventFactory(), bufferSize, waitStrategy); -// } else { -// ringBuffer = RingBuffer.createSingleProducer( -// new ObjectEventFactory(), bufferSize, waitStrategy); -// } -// -// sequenceBarrier = ringBuffer.newBarrier(); -// exceptionHandler = new FatalExceptionHandler(); -// workProcessors = new ArrayList(); -// workSequence = new Sequence(Sequencer.INITIAL_CURSOR_VALUE); -// } -// -// public void register(WorkHandler handler) { -// WorkProcessor workProcessor = new WorkProcessor(ringBuffer, -// sequenceBarrier, new HandleWraper(handler), exceptionHandler, -// workSequence); -// -// ringBuffer.addGatingSequences(workProcessor.getSequence()); -// -// workProcessors.add(workProcessor); -// } -// -// void cleanup() { -// -// } -// -// /** -// * Start the worker pool processing events in sequence. -// * -// * @param executor -// * providing threads for running the workers. -// * @return the {@link RingBuffer} used for the work queue. -// * @throws IllegalStateException -// * if the pool has already been started and not halted yet -// */ -// public void start() { -// if (!started.compareAndSet(false, true)) { -// throw new IllegalStateException( -// "WorkerPool has already been started and cannot be restarted until halted."); -// } -// -// final long cursor = ringBuffer.getCursor(); -// workSequence.set(cursor); -// -// for (WorkProcessor processor : workProcessors) { -// processor.getSequence().set(cursor); -// new Thread(processor).start(); -// } -// -// return; -// } -// -// public Sequence[] getWorkerSequences() { -// final Sequence[] sequences = new Sequence[workProcessors.size()]; -// for (int i = 0, size = workProcessors.size(); i < size; i++) { -// sequences[i] = workProcessors.get(i).getSequence(); -// } -// -// return sequences; -// } -// -// /** -// * Wait for the {@link RingBuffer} to drain of published events then halt -// * the workers. -// */ -// public void drainAndHalt() { -// Sequence[] workerSequences = getWorkerSequences(); -// while (ringBuffer.getCursor() > Util -// .getMinimumSequence(workerSequences)) { -// Thread.yield(); -// } -// -// for (WorkProcessor processor : workProcessors) { -// processor.halt(); -// } -// -// started.set(false); -// } -// -// /** -// * Halt all workers immediately at the end of their current cycle. -// */ -// public void halt() { -// for (WorkProcessor processor : workProcessors) { -// processor.halt(); -// } -// -// started.set(false); -// } -// -// public void offer(T o) { -// long sequence = ringBuffer.next(); -// ringBuffer.get(sequence).setValue(o); -// ringBuffer.publish(sequence); -// } -// -// public static class ObjectEventFactory implements -// EventFactory { -// -// public MutableObject newInstance() { -// return new MutableObject(); -// } -// } -// -// public static class HandleWraper implements WorkHandler { -// private WorkHandler handler; -// -// public HandleWraper(WorkHandler handler) { -// this.handler = handler; -// } -// -// public void onEvent(MutableObject event) throws Exception { -// // TODO Auto-generated method stub -// handler.onEvent((T) event.getValue()); -// } -// -// } -// } diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/EventSampler.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/EventSampler.java deleted file mode 100644 index c0bff679c..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/EventSampler.java +++ /dev/null @@ -1,100 +0,0 @@ -package com.alibaba.jstorm.utils; - -import java.util.Random; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; - -/** - * - * statistics tuples: sampling event - * - * - * @author yannian/Longda - * - */ -public class EventSampler { - private volatile int freq; - private AtomicInteger i = new AtomicInteger(0); - private volatile int target; - private Random r = new Random(); - - public EventSampler(int freq) { - this.freq = freq; - this.target = r.nextInt(freq); - - if (freq / 4 > 1) { - intervalCheck.setInterval(freq / 4); - } - } - - /** - * select 1/freq - * - * @return - */ - public boolean countCheck() { - i.incrementAndGet(); - if (i.get() > freq) { - target = r.nextInt(freq); - i.set(0); - } - if (i.get() == target) { - return true; - } - return false; - } - - private AtomicInteger counter = new AtomicInteger(0); - private AtomicLong sum = new AtomicLong(0); - private IntervalCheck intervalCheck = new IntervalCheck(); - - public Integer tpsCheck() { - int send = counter.incrementAndGet(); - - Double pastSeconds = intervalCheck.checkAndGet(); - if (pastSeconds != null) { - counter.set(0); - - return Integer.valueOf((int) (send / pastSeconds)); - - } - - return null; - } - - public Integer timesCheck() { - int send = counter.incrementAndGet(); - - Double pastSeconds = intervalCheck.checkAndGet(); - if (pastSeconds != null) { - counter.set(0); - - return send; - - } - - return null; - } - - public Pair avgCheck(long one) { - int send = counter.incrementAndGet(); - long total = sum.addAndGet(one); - - Double pastSeconds = intervalCheck.checkAndGet(); - if (pastSeconds != null) { - counter.set(0); - sum.set(0); - - Double avg = Double.valueOf(0); - if (send != 0) { - avg = ((double)total)/send; - } - - - return new Pair(send, avg); - - } - - return null; - } -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/ExpiredCallback.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/ExpiredCallback.java deleted file mode 100644 index 4ee450ba0..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/ExpiredCallback.java +++ /dev/null @@ -1,5 +0,0 @@ -package com.alibaba.jstorm.utils; - -public interface ExpiredCallback { - public void expire(K key, V val); -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/FileAttribute.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/FileAttribute.java deleted file mode 100644 index 46582b20e..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/FileAttribute.java +++ /dev/null @@ -1,118 +0,0 @@ -package com.alibaba.jstorm.utils; - -import java.io.Serializable; -import java.util.Date; -import java.util.HashMap; -import java.util.Map; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; -import org.json.simple.JSONAware; - -//import com.alibaba.fastjson.JSONAware; - -public class FileAttribute implements Serializable, JSONAware { - - /** */ - private static final long serialVersionUID = -5131640995402822835L; - - private String fileName; - private String isDir; - private String modifyTime; - private String size; - - private static final String FILE_NAME_FIELD = "fileName"; - private static final String IS_DIR_FIELD = "isDir"; - private static final String MODIFY_TIME_FIELD = "modifyTime"; - private static final String SIZE_FIELD = "size"; - - public String getFileName() { - return fileName; - } - - public void setFileName(String fileName) { - this.fileName = fileName; - } - - public String getIsDir() { - return isDir; - } - - public void setIsDir(String isDir) { - this.isDir = isDir; - } - - public String getModifyTime() { - return modifyTime; - } - - public void setModifyTime(String modifyTime) { - this.modifyTime = modifyTime; - } - - public String getSize() { - return size; - } - - public void setSize(String size) { - this.size = size; - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this, - ToStringStyle.SHORT_PREFIX_STYLE); - } - - @Override - public String toJSONString() { - Map map = new HashMap(); - - map.put(FILE_NAME_FIELD, fileName); - map.put(IS_DIR_FIELD, isDir); - map.put(MODIFY_TIME_FIELD, modifyTime); - map.put(SIZE_FIELD, size); - return JStormUtils.to_json(map); - } - - public static FileAttribute fromJSONObject(Map jobj) { - if (jobj == null) { - return null; - } - - FileAttribute attribute = new FileAttribute(); - - attribute.setFileName((String) jobj.get(FILE_NAME_FIELD)); - attribute.setIsDir((String) jobj.get(IS_DIR_FIELD)); - attribute.setModifyTime((String) jobj.get(MODIFY_TIME_FIELD)); - attribute.setSize((String) jobj.get(SIZE_FIELD)); - - return attribute; - } - - public static void main(String[] args) { - Map map = new HashMap(); - - FileAttribute attribute = new FileAttribute(); - attribute.setFileName("test"); - attribute.setIsDir("true"); - attribute.setModifyTime(new Date().toString()); - attribute.setSize("4096"); - - map.put("test", attribute); - - System.out.println("Before:" + map); - - String jsonString = JStormUtils.to_json(map); - - Map map2 = (Map) JStormUtils - .from_json(jsonString); - - Map jObject = map2.get("test"); - - FileAttribute attribute2 = FileAttribute.fromJSONObject(jObject); - - System.out.println("attribute2:" + attribute2); - } - -} 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 deleted file mode 100644 index ee7376dcb..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/HttpserverUtils.java +++ /dev/null @@ -1,30 +0,0 @@ -package com.alibaba.jstorm.utils; - -public class HttpserverUtils { - - public static final String HTTPSERVER_CONTEXT_PATH_LOGVIEW = "/logview"; - - public static final String HTTPSERVER_LOGVIEW_PARAM_CMD = "cmd"; - - public static final String HTTPSERVER_LOGVIEW_PARAM_CMD_LIST = "listDir"; - - public static final String HTTPSERVER_LOGVIEW_PARAM_CMD_SHOW = "showLog"; - - 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"; - - public static final String HTTPSERVER_LOGVIEW_PARAM_DIR = "dir"; - - public static final String HTTPSERVER_LOGVIEW_PARAM_WORKER_PORT = "workerPort"; - - public static final long HTTPSERVER_LOGVIEW_PAGESIZE = 16384; - - public static final String HTTPSERVER_LOGVIEW_PARAM_SIZE_FORMAT = "%016d\n"; - - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/IntervalCheck.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/IntervalCheck.java deleted file mode 100644 index 6d0acc2bd..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/IntervalCheck.java +++ /dev/null @@ -1,58 +0,0 @@ -package com.alibaba.jstorm.utils; - -import java.io.Serializable; - -public class IntervalCheck implements Serializable { - - /** - * - */ - private static final long serialVersionUID = 8952971673547362883L; - - long lastCheck = System.currentTimeMillis(); - - // default interval is 1 second - long interval = 1; - - /* - * if last check time is before interval seconds, return true, otherwise - * return false - */ - public boolean check() { - return checkAndGet() != null; - } - - /** - * - * @return - */ - public Double checkAndGet() { - long now = System.currentTimeMillis(); - - synchronized (this) { - if (now >= interval * 1000 + lastCheck) { - double pastSecond = ((double) (now - lastCheck)) / 1000; - lastCheck = now; - return pastSecond; - } - } - - return null; - } - - public long getInterval() { - return interval; - } - - public void setInterval(long interval) { - this.interval = interval; - } - - public void adjust(long addTimeMillis) { - lastCheck += addTimeMillis; - } - - public void start() { - lastCheck = System.currentTimeMillis(); - } -} 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 deleted file mode 100644 index 17b28cff3..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/JStormUtils.java +++ /dev/null @@ -1,1076 +0,0 @@ -package com.alibaba.jstorm.utils; - -import java.io.BufferedOutputStream; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.PrintStream; -import java.io.PrintWriter; -import java.io.StringWriter; -import java.lang.management.ManagementFactory; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Enumeration; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -import java.util.TreeMap; -import java.util.zip.ZipEntry; -import java.util.zip.ZipFile; - -import javax.management.ObjectName; - -import org.apache.commons.exec.CommandLine; -import org.apache.commons.exec.DefaultExecutor; -import org.apache.commons.exec.ExecuteException; -import org.apache.commons.exec.ExecuteResultHandler; -import org.apache.commons.exec.PumpStreamHandler; -import org.apache.commons.lang.StringUtils; -import org.apache.log4j.Appender; -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 - * - * @author yannian/Longda/Xin.Zhou/Xin.Li - * - */ -public class JStormUtils { - private static final Logger LOG = Logger.getLogger(JStormUtils.class); - - public static long SIZE_1_K = 1024; - public static long SIZE_1_M = SIZE_1_K * 1024; - public static long SIZE_1_G = SIZE_1_M * 1024; - public static long SIZE_1_T = SIZE_1_G * 1024; - public static long SIZE_1_P = SIZE_1_T * 1024; - - public static final int MIN_1 = 60; - 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 { - StringWriter sw = new StringWriter(); - PrintWriter pw = new PrintWriter(sw); - e.printStackTrace(pw); - return baseInfo + "\r\n" + sw.toString() + "\r\n"; - } catch (Exception e2) { - return baseInfo; - } - } - - public static String getErrorInfo(Throwable error) { - try { - StringWriter sw = new StringWriter(); - PrintWriter pw = new PrintWriter(sw); - error.printStackTrace(pw); - return sw.toString(); - } catch (Exception e1) { - return ""; - } - } - - /** - * filter the map - * - * @param filter - * @param all - * @return - */ - public static Map select_keys_pred(Set filter, Map all) { - Map filterMap = new HashMap(); - - for (Entry entry : all.entrySet()) { - if (!filter.contains(entry.getKey())) { - filterMap.put(entry.getKey(), entry.getValue()); - } - } - - return filterMap; - } - - public static byte[] barr(byte v) { - byte[] byteArray = new byte[1]; - byteArray[0] = v; - - return byteArray; - } - - public static byte[] barr(Short v) { - byte[] byteArray = new byte[Short.SIZE / 8]; - for (int i = 0; i < byteArray.length; i++) { - int off = (byteArray.length - 1 - i) * 8; - byteArray[i] = (byte) ((v >> off) & 0xFF); - } - return byteArray; - } - - public static byte[] barr(Integer v) { - byte[] byteArray = new byte[Integer.SIZE / 8]; - for (int i = 0; i < byteArray.length; i++) { - int off = (byteArray.length - 1 - i) * 8; - byteArray[i] = (byte) ((v >> off) & 0xFF); - } - return byteArray; - } - - // for test - public static int byteToInt2(byte[] b) { - - int iOutcome = 0; - byte bLoop; - - for (int i = 0; i < 4; i++) { - bLoop = b[i]; - int off = (b.length - 1 - i) * 8; - iOutcome += (bLoop & 0xFF) << off; - - } - - 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) { - } - if (localMode && val == 0) { - //throw new RuntimeException(msg); - }else { - haltProcess(val); - } - } - - /** - * "{:a 1 :b 1 :c 2} -> {1 [:a :b] 2 :c}" - * - * @param map - * @return - */ - public static HashMap> reverse_map(Map map) { - HashMap> rtn = new HashMap>(); - if (map == null) { - return rtn; - } - for (Entry entry : map.entrySet()) { - K key = entry.getKey(); - V val = entry.getValue(); - List list = rtn.get(val); - if (list == null) { - list = new ArrayList(); - rtn.put(entry.getValue(), list); - } - list.add(key); - - } - - return rtn; - } - - /** - * Gets the pid of this JVM, because Java doesn't provide a real way to do - * this. - * - * @return - */ - public static String process_pid() { - String name = ManagementFactory.getRuntimeMXBean().getName(); - String[] split = name.split("@"); - if (split.length != 2) { - throw new RuntimeException("Got unexpected process name: " + name); - } - - return split[0]; - } - - public static void exec_command(String command) throws ExecuteException, - IOException { - String[] cmdlist = command.split(" "); - CommandLine cmd = new CommandLine(cmdlist[0]); - for (int i = 1; i < cmdlist.length; i++) { - cmd.addArgument(cmdlist[i]); - } - - DefaultExecutor exec = new DefaultExecutor(); - exec.execute(cmd); - } - - /** - * Extra dir from the jar to destdir - * - * @param jarpath - * @param dir - * @param destdir - */ - public static void extract_dir_from_jar(String jarpath, String dir, - String destdir) { - String cmd = "unzip -qq " + jarpath + " " + dir + "/** -d " + destdir; - try { - exec_command(cmd); - } catch (Exception e) { - LOG.warn("No " + dir + " from " + jarpath + " by cmd:" + cmd + "!\n" - + e.getMessage()); - } - - } - - public static void ensure_process_killed(Integer pid) { - // in this function, just kill the process 5 times - // make sure the process be killed definitely - for (int i = 0; i < 5; i++) { - try { - exec_command("kill -9 " + pid); - LOG.info("kill -9 process " + pid); - sleepMs(100); - } catch (ExecuteException e) { - LOG.info("Error when trying to kill " + pid - + ". Process has been killed"); - } catch (Exception e) { - LOG.info("Error when trying to kill " + pid + ".Exception ", e); - } - } - } - - public static void process_killed(Integer pid) { - try { - exec_command("kill " + pid); - LOG.info("kill process " + pid); - } catch (ExecuteException e) { - LOG.info("Error when trying to kill " + pid - + ". Process has been killed. "); - } catch (Exception e) { - LOG.info("Error when trying to kill " + pid + ".Exception ", e); - } - } - - public static void kill(Integer pid) { - process_killed(pid); - - sleepMs(5 * 1000); - - ensure_process_killed(pid); - } - - public static void kill_signal(Integer pid, String signal) { - String cmd = "kill " + signal + " " + pid; - try { - exec_command(cmd); - LOG.info(cmd); - } catch (ExecuteException e) { - LOG.info("Error when run " + cmd + ". Process has been killed. "); - } catch (Exception e) { - LOG.info("Error when run " + cmd + ". Exception ", e); - } - } - - /** - * 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 - * - * This function don't care whether the command is successfully or not - * - * @param command - * @param environment - * @param workDir - * @param resultHandler - * @return - * @throws IOException - */ - public static ByteArrayOutputStream launchProcess(String command, final Map environment, - final String workDir, ExecuteResultHandler resultHandler) - throws IOException { - - String[] cmdlist = command.split(" "); - - CommandLine cmd = new CommandLine(cmdlist[0]); - for (String cmdItem : cmdlist) { - if (StringUtils.isBlank(cmdItem) == false) { - cmd.addArgument(cmdItem); - } - } - - DefaultExecutor executor = new DefaultExecutor(); - - executor.setExitValue(0); - if (StringUtils.isBlank(workDir) == false) { - executor.setWorkingDirectory(new File(workDir)); - } - - ByteArrayOutputStream out = new ByteArrayOutputStream(); - - PumpStreamHandler streamHandler = new PumpStreamHandler(out, out); - if (streamHandler != null) { - executor.setStreamHandler(streamHandler); - } - - try { - if (resultHandler == null) { - executor.execute(cmd, environment); - } else { - executor.execute(cmd, environment, resultHandler); - } - }catch(ExecuteException e) { - - // @@@@ - // failed to run command - } - - return out; - - } - - protected static java.lang.Process launchProcess(final String[] cmdlist, - final Map environment) throws IOException { - ArrayList buff = new ArrayList(); - for (String tok : cmdlist) { - if (!tok.isEmpty()) { - buff.add(tok); - } - } - - ProcessBuilder builder = new ProcessBuilder(buff); - builder.redirectErrorStream(true); - Map process_evn = builder.environment(); - for (Entry entry : environment.entrySet()) { - process_evn.put(entry.getKey(), entry.getValue()); - } - - return builder.start(); - } - - /** - * @@@ it should use DefaultExecutor to start a process, - * but some little problem have been found, such as exitCode/output string - * so still use the old method to start process - * - * @param command - * @param environment - * @param backend - * @return - * @throws IOException - */ - public static java.lang.Process launch_process(final String command, - final Map environment, boolean backend) throws IOException { - - if (backend == true) { - new Thread(new Runnable() { - - @Override - public void run() { - String[] cmdlist = (new String("nohup " + command + " &")).split(" "); - try { - launchProcess(cmdlist, environment); - } catch (IOException e) { - LOG.error("Failed to run " + command + ":" + e.getCause(), e); - } - } - }).start(); - return null; - }else { - String[] cmdlist = command.split(" "); - return launchProcess(cmdlist, environment); - } - } - - public static String current_classpath() { - return System.getProperty("java.class.path"); - } - - // public static String add_to_classpath(String classpath, String[] paths) { - // for (String path : paths) { - // classpath += ":" + path; - // } - // return classpath; - // } - - public static String to_json(Map m) { - return Utils.to_json(m); - } - - public static Object from_json(String json) { - return Utils.from_json(json); - } - - public static HashMap multi_set(List list) { - HashMap rtn = new HashMap(); - for (V v : list) { - int cnt = 1; - if (rtn.containsKey(v)) { - cnt += rtn.get(v); - } - rtn.put(v, cnt); - } - return rtn; - } - - /** - * - * if the list exist repeat string, return the repeated string - * - * this function will be used to check wheter bolt or spout exist same id - * - * @param sets - * @return - */ - public static List getRepeat(List list) { - - List rtn = new ArrayList(); - Set idSet = new HashSet(); - - for (String id : list) { - if (idSet.contains(id)) { - rtn.add(id); - } else { - idSet.add(id); - } - } - - return rtn; - } - - /** - * balance all T - * - * @param - * @param splitup - * @return - */ - public static List interleave_all(List> splitup) { - ArrayList rtn = new ArrayList(); - int maxLength = 0; - for (List e : splitup) { - int len = e.size(); - if (maxLength < len) { - maxLength = len; - } - } - - for (int i = 0; i < maxLength; i++) { - for (List e : splitup) { - if (e.size() > i) { - rtn.add(e.get(i)); - } - } - } - - return rtn; - } - - public static Long bit_xor_vals(Object... vals) { - Long rtn = 0l; - for (Object n : vals) { - rtn = bit_xor(rtn, n); - } - - return rtn; - } - - public static Long bit_xor_vals(java.util.List vals) { - Long rtn = 0l; - for (T n : vals) { - rtn = bit_xor(rtn, n); - } - - return rtn; - } - - public static Long bit_xor_vals_sets(java.util.Set vals) { - Long rtn = 0l; - for (T n : vals) { - rtn = bit_xor(rtn, n); - } - return rtn; - } - - public static Long bit_xor(Object a, Object b) { - Long rtn = 0l; - - if (a instanceof Long && b instanceof Long) { - rtn = ((Long) a) ^ ((Long) b); - return rtn; - } else if (b instanceof Set) { - Long bs = bit_xor_vals_sets((Set) b); - return bit_xor(a, bs); - } else if (a instanceof Set) { - Long as = bit_xor_vals_sets((Set) a); - return bit_xor(as, b); - } else { - Long ai = Long.parseLong(String.valueOf(a)); - Long bi = Long.parseLong(String.valueOf(b)); - rtn = ai ^ bi; - return rtn; - } - - } - - public static List mk_list(V... args) { - ArrayList rtn = new ArrayList(); - for (V o : args) { - rtn.add(o); - } - return rtn; - } - - public static List mk_list(java.util.Set args) { - ArrayList rtn = new ArrayList(); - if (args != null) { - for (V o : args) { - rtn.add(o); - } - } - return rtn; - } - - public static V[] mk_arr(V... args) { - return args; - } - - public static Long parseLong(Object o) { - if (o == null) { - return null; - } - - if (o instanceof String) { - return Long.valueOf(String.valueOf(o)); - } else if (o instanceof Integer) { - Integer value = (Integer) o; - return Long.valueOf((Integer) value); - } else if (o instanceof Long) { - return (Long) o; - } else { - throw new RuntimeException("Invalid value " - + 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) { - - if (o == null) { - return defaultValue; - } - - if (o instanceof String) { - return Long.valueOf(String.valueOf(o)); - } else if (o instanceof Integer) { - Integer value = (Integer) o; - return Long.valueOf((Integer) value); - } else if (o instanceof Long) { - return (Long) o; - } else { - return defaultValue; - } - } - - public static Integer parseInt(Object o) { - if (o == null) { - return null; - } - - if (o instanceof String) { - return Integer.parseInt(String.valueOf(o)); - } else if (o instanceof Long) { - long value = (Long) o; - return Integer.valueOf((int) value); - } else if (o instanceof Integer) { - return (Integer) o; - } else { - throw new RuntimeException("Invalid value " - + o.getClass().getName() + " " + o); - } - } - - public static Integer parseInt(Object o, int defaultValue) { - if (o == null) { - return defaultValue; - } - - if (o instanceof String) { - return Integer.parseInt(String.valueOf(o)); - } else if (o instanceof Long) { - long value = (Long) o; - return Integer.valueOf((int) value); - } else if (o instanceof Integer) { - return (Integer) o; - } else { - return defaultValue; - } - } - - public static boolean parseBoolean(Object o, boolean defaultValue) { - if (o == null) { - return defaultValue; - } - - if (o instanceof String) { - return Boolean.valueOf((String) o); - } else if (o instanceof Boolean) { - return (Boolean) o; - } else { - return defaultValue; - } - } - - public static Set listToSet(List list) { - if (list == null) { - return null; - } - - Set set = new HashSet(); - set.addAll(list); - return set; - } - - /** - * Check whether the zipfile contain the resources - * - * @param zipfile - * @param resources - * @return - */ - public static boolean zipContainsDir(String zipfile, String resources) { - - Enumeration entries = null; - try { - entries = (new ZipFile(zipfile)).entries(); - while (entries != null && entries.hasMoreElements()) { - ZipEntry ze = entries.nextElement(); - String name = ze.getName(); - if (name.startsWith(resources + "/")) { - return true; - } - } - } catch (IOException e) { - // TODO Auto-generated catch block - // e.printStackTrace(); - LOG.error(e + "zipContainsDir error"); - } - - return false; - } - - public static Object add(Object oldValue, Object newValue) { - if (oldValue == null) { - return newValue; - } - - if (oldValue instanceof Long) { - if (newValue == null) { - return (Long) oldValue; - } else { - return (Long) oldValue + (Long) newValue; - } - } else if (oldValue instanceof Double) { - if (newValue == null) { - return (Double) oldValue; - } else { - return (Double) oldValue + (Double) newValue; - } - } else { - return null; - } - } - - public static Object mergeList(List list) { - Object ret = null; - - for (Object value : list) { - ret = add(ret, value); - } - - return ret; - } - - public static List mergeList(List result, Object add) { - if (add instanceof Collection) { - for (Object o : (Collection) add) { - result.add(o); - } - } else if (add instanceof Set) { - for (Object o : (Collection) add) { - result.add(o); - } - } else { - result.add(add); - } - - return result; - } - - public static List distinctList(List input) { - List retList = new ArrayList(); - - for (Object object : input) { - if (retList.contains(object)) { - continue; - } else { - retList.add(object); - } - - } - - return retList; - } - - public static Map mergeMapList(List> list) { - Map ret = new HashMap(); - - for (Map listEntry : list) { - if (listEntry == null) { - continue; - } - for (Entry mapEntry : listEntry.entrySet()) { - K key = mapEntry.getKey(); - V value = mapEntry.getValue(); - - V retValue = (V) add(ret.get(key), value); - - ret.put(key, retValue); - } - } - - return ret; - } - - public static String formatSimpleDouble(Double value) { - try { - java.text.DecimalFormat form = new java.text.DecimalFormat( - "##0.000"); - String s = form.format(value); - return s; - } catch (Exception e) { - return "0.000"; - } - - } - - public static double formatDoubleDecPoint2(Double value) { - try { - java.text.DecimalFormat form = new java.text.DecimalFormat( - "##.00"); - String s = form.format(value); - return Double.valueOf(s); - } catch (Exception e) { - return 0.0; - } - } - - public static double formatDoubleDecPoint4(Double value) { - try { - java.text.DecimalFormat form = new java.text.DecimalFormat( - "###.0000"); - String s = form.format(value); - return Double.valueOf(s); - } catch (Exception e) { - return 0.0; - } - } - - public static Double convertToDouble(Object value) { - Double ret; - - if (value == null) { - ret = null; - } else { - if (value instanceof Integer) { - ret = ((Integer) value).doubleValue(); - } else if (value instanceof Long) { - ret = ((Long) value).doubleValue(); - } else if (value instanceof Float) { - ret = ((Float) value).doubleValue(); - } else if (value instanceof Double) { - ret = (Double) value; - } else { - ret = null; - } - } - - return ret; - } - - public static String formatValue(Object value) { - if (value == null) { - return "0"; - } - - if (value instanceof Long) { - return String.valueOf((Long) value); - } else if (value instanceof Double) { - return formatSimpleDouble((Double) value); - } else { - return String.valueOf(value); - } - } - - public static void sleepMs(long ms) { - try { - Thread.sleep(ms); - } catch (InterruptedException e) { - - } - } - - public static void sleepNs(int ns) { - try { - Thread.sleep(0, ns); - } catch (InterruptedException e) { - - } - } - - public static String HEXES = "0123456789ABCDEF"; - - public static String toPrintableString(byte[] buf) { - if (buf == null) { - return null; - } - - StringBuilder sb = new StringBuilder(); - int index = 0; - for (byte b : buf) { - if (index % 10 == 0) { - sb.append("\n"); - } - index++; - - sb.append(HEXES.charAt((b & 0xF0) >> 4)); - sb.append(HEXES.charAt((b & 0x0F))); - sb.append(" "); - - } - - return sb.toString(); - } - - /** - * @@@ Todo - * - * @return - */ - public static Long getPhysicMemorySize() { - Object object; - try { - object = ManagementFactory.getPlatformMBeanServer().getAttribute( - new ObjectName("java.lang", "type", "OperatingSystem"), - "TotalPhysicalMemorySize"); - } catch (Exception e) { - LOG.warn("Failed to get system physical memory size,", e); - return null; - } - - Long ret = (Long) object; - - return ret; - } - - public static String genLogName(String topology, Integer port) { - return topology + "-worker-" + port + ".log"; - } - - public static String getLogFileName() { - Enumeration enumAppender = Logger.getRootLogger() - .getAllAppenders(); - FileAppender fileAppender = null; - while (enumAppender.hasMoreElements()) { - Appender appender = enumAppender.nextElement(); - if (appender instanceof FileAppender) { - fileAppender = (FileAppender) appender; - break; - } - } - if (fileAppender != null) { - return fileAppender.getFile(); - - } - - return null; - } - - public static String getLogDir() { - String file = JStormUtils.getLogFileName(); - if (file != null) { - if (file.lastIndexOf(File.separator) < 0) - return ""; - return file.substring(0, file.lastIndexOf(File.separator)); - } - - String stormHome = System.getProperty("jstorm.home"); - if (stormHome == null) { - return "." + File.separator + "logs"; - } else { - return stormHome + File.separator + "logs"; - } - } - - public static void redirectOutput(String file) throws Exception { - - System.out.println("Redirect output to " + file); - - FileOutputStream workerOut = new FileOutputStream(new File(file)); - - PrintStream ps = new PrintStream(new BufferedOutputStream(workerOut), - true); - System.setOut(ps); - System.setErr(ps); - - LOG.info("Successfully redirect System.out to " + file); - - } - - public static RunnableCallback getDefaultKillfn() { - - return new AsyncLoopDefaultKill(); - } - - public static TreeMap integer_divided(int sum, - int num_pieces) { - return Utils.integerDivided(sum, num_pieces); - } - - public static HashMap filter_val(RunnableCallback fn, - Map amap) { - HashMap rtn = new HashMap(); - - for (Entry entry : amap.entrySet()) { - V value = entry.getValue(); - Object result = fn.execute(value); - - if (result == (Boolean) true) { - rtn.put(entry.getKey(), value); - } - } - 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 deleted file mode 100644 index 1a6c649b8..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/NetWorkUtils.java +++ /dev/null @@ -1,120 +0,0 @@ -package com.alibaba.jstorm.utils; - -import java.io.IOException; -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; - -/** - * Network utilis - * - * @author yannian - * - */ -public class NetWorkUtils { - private static Logger LOG = Logger.getLogger(NetWorkUtils.class); - - public static String hostname() { - String hostname = null; - try { - hostname = InetAddress.getLocalHost().getCanonicalHostName(); - } catch (UnknownHostException e) { - LOG.error("local_hostname", e); - } - return hostname; - } - - public static String ip() { - String hostname = null; - try { - hostname = InetAddress.getLocalHost().getHostAddress(); - } catch (UnknownHostException e) { - LOG.error("local_hostname", e); - } - return hostname; - } - - /** - * Check whether the port is available to binding - * - * @param port - * @return -1 means not available, others means available - * @throws IOException - */ - public static int tryPort(int port) throws IOException { - ServerSocket socket = new ServerSocket(port); - int rtn = socket.getLocalPort(); - socket.close(); - return rtn; - } - - /** - * get one available port - * - * @return -1 means failed, others means one availablePort - */ - public static int getAvailablePort() { - return availablePort(0); - } - - /** - * Check whether the port is available to binding - * - * @param prefered - * @return -1 means not available, others means available - */ - public static int availablePort(int prefered) { - int rtn = -1; - try { - rtn = tryPort(prefered); - } catch (IOException e) { - - } - return rtn; - } - - public static String host2Ip(String host) { - InetAddress address = null; - try { - address = InetAddress.getByName(host); - } catch (UnknownHostException e) { - LOG.warn("NetWorkUtil can't transfer hostname(" + host + ") to ip, return hostname", e); - return host; - } - return address.getHostAddress(); - } - - public static String ip2Host(String ip) { - InetAddress address = null; - try { - address = InetAddress.getByName(ip); - } catch (UnknownHostException e) { - LOG.warn("NetWorkUtil can't transfer ip(" + ip + ") to hostname, return ip", e); - return 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/OlderFileFilter.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/OlderFileFilter.java deleted file mode 100644 index a3063d6af..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/OlderFileFilter.java +++ /dev/null @@ -1,29 +0,0 @@ -package com.alibaba.jstorm.utils; - -import java.io.File; -import java.io.FileFilter; - -/** - * filter the older file, skip the files' modify time which is less sec than now - * - * @author lixin - * - */ -public class OlderFileFilter implements FileFilter { - - private int seconds; - - public OlderFileFilter(int seconds) { - this.seconds = seconds; - } - - @Override - public boolean accept(File pathname) { - - long current_time = System.currentTimeMillis(); - - return (pathname.isFile() && (pathname.lastModified() + seconds * 1000 <= current_time)) - || pathname.isDirectory(); - } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/Pair.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/Pair.java deleted file mode 100644 index 43aa5837d..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/Pair.java +++ /dev/null @@ -1,28 +0,0 @@ -package com.alibaba.jstorm.utils; - -public class Pair { - private F first; - private S second; - - public Pair(F first, S second) { - this.first = first; - this.second = second; - } - - public F getFirst() { - return first; - } - - public void setFirst(F first) { - this.first = first; - } - - public S getSecond() { - return second; - } - - public void setSecond(S second) { - this.second = second; - } - -} 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 deleted file mode 100644 index 26cddf589..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/PathUtils.java +++ /dev/null @@ -1,130 +0,0 @@ -package com.alibaba.jstorm.utils; - -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -import org.apache.commons.io.FileUtils; -import org.apache.log4j.Logger; - -/** - * - * @author yannian - * - */ -public class PathUtils { - static Logger LOG = Logger.getLogger(PathUtils.class); - - public static final String SEPERATOR = "/"; - - /** - * split path as list - * - * @param path - * @return - */ - public static List tokenize_path(String path) { - String[] toks = path.split(SEPERATOR); - java.util.ArrayList rtn = new ArrayList(); - for (String str : toks) { - if (!str.isEmpty()) { - rtn.add(str); - } - } - return rtn; - } - - public static String toks_to_path(List toks) { - StringBuffer buff = new StringBuffer(); - buff.append(SEPERATOR); - int size = toks.size(); - for (int i = 0; i < size; i++) { - buff.append(toks.get(i)); - if (i < (size - 1)) { - buff.append(SEPERATOR); - } - - } - return buff.toString(); - } - - public static String normalize_path(String path) { - String rtn = toks_to_path(tokenize_path(path)); - return rtn; - } - - public static String parent_path(String path) { - List toks = tokenize_path(path); - int size = toks.size(); - if (size > 0) { - toks.remove(size - 1); - } - return toks_to_path(toks); - } - - public static String full_path(String parent, String name) { - return normalize_path(parent + SEPERATOR + name); - } - - public static boolean exists_file(String path) { - return (new File(path)).exists(); - } - - public static void rmr(String path) throws IOException { - LOG.debug("Rmr path " + path); - if (exists_file(path)) { - FileUtils.forceDelete(new File(path)); - } - - } - - public static void local_mkdirs(String path) throws IOException { - LOG.debug("Making dirs at" + path); - FileUtils.forceMkdir(new File(path)); - } - - public static void rmpath(String path) { - LOG.debug("Removing path " + path); - boolean isdelete = (new File(path)).delete(); - if (!isdelete) { - throw new RuntimeException("Failed to delete " + path); - } - } - - public static void touch(String path) throws IOException { - LOG.debug("Touching file at" + path); - boolean success = (new File(path)).createNewFile(); - if (!success) { - throw new RuntimeException("Failed to touch " + path); - } - } - - public static List read_dir_contents(String dir) { - ArrayList rtn = new ArrayList(); - if (exists_file(dir)) { - File[] list = (new File(dir)).listFiles(); - for (File f : list) { - rtn.add(f.getName()); - } - } - 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/utils/RandomRange.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/RandomRange.java deleted file mode 100644 index b06ed5f1a..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/RandomRange.java +++ /dev/null @@ -1,57 +0,0 @@ -package com.alibaba.jstorm.utils; - -import java.util.ArrayList; - -/** - * Shuffle the Range, This class is used in shuffle grouping, it is better than - * random, which can't make sure balance. - * - * @author yannian - * - */ -public class RandomRange { - private ArrayList rr; - private Integer amt; - - public RandomRange(int amt) { - this.amt = amt; - this.rr = rotating_random_range(amt); - } - - public Integer nextInt() { - return this.acquire_random_range_id(); - } - - private ArrayList rotating_random_range(int amt) { - - ArrayList range = new ArrayList(); - for (int i = 0; i < amt; i++) { - range.add(i); - } - - ArrayList rtn = new ArrayList(); - for (int i = 0; i < amt; i++) { - int index = (int) (Math.random() * range.size()); - rtn.add(range.remove(index)); - } - - return rtn; - } - - private synchronized int acquire_random_range_id() { - int ret = this.rr.remove(0); - if (this.rr.size() == 0) { - this.rr.addAll(rotating_random_range(this.amt)); - } - return ret; - } - - public static void main(String[] args) { - RandomRange test = new RandomRange(10); - - for (int i = 0; i < 10; i++) { - System.out.println(test.acquire_random_range_id()); - } - } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/RotatingMap.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/RotatingMap.java deleted file mode 100644 index 78095a6d5..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/RotatingMap.java +++ /dev/null @@ -1,133 +0,0 @@ -package com.alibaba.jstorm.utils; - -import java.util.Deque; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.Map; -import java.util.Map.Entry; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.LinkedBlockingDeque; - -/** - * RotatingMap must be used under thread-safe environment - * - * Expires keys that have not been updated in the configured number of seconds. - * The algorithm used will take between expirationSecs and expirationSecs * (1 + - * 1 / (numBuckets-1)) to actually expire the message. - * - * get, put, remove, containsKey, and size take O(numBuckets) time to run. - * - */ -public class RotatingMap implements TimeOutMap{ - // this default ensures things expire at most 50% past the expiration time - private static final int DEFAULT_NUM_BUCKETS = 3; - - - private Deque> _buckets; - - private ExpiredCallback _callback; - - private final Object lock = new Object(); - - public RotatingMap(int numBuckets, ExpiredCallback callback, boolean isSingleThread) { - if (numBuckets < 2) { - throw new IllegalArgumentException("numBuckets must be >= 2"); - } - if (isSingleThread == true) { - _buckets = new LinkedList>(); - }else { - _buckets = new LinkedBlockingDeque>(); - } - - for (int i = 0; i < numBuckets; i++) { - _buckets.add(new ConcurrentHashMap()); - } - - _callback = callback; - } - - public RotatingMap(ExpiredCallback callback) { - this(DEFAULT_NUM_BUCKETS, callback, false); - } - - public RotatingMap(int numBuckets) { - this(numBuckets, null, false); - } - - public Map rotate() { - Map dead = _buckets.removeLast(); - _buckets.addFirst(new ConcurrentHashMap()); - if (_callback != null) { - for (Entry entry : dead.entrySet()) { - _callback.expire(entry.getKey(), entry.getValue()); - } - } - return dead; - } - - @Override - public boolean containsKey(K key) { - for (Map bucket : _buckets) { - if (bucket.containsKey(key)) { - return true; - } - } - return false; - } - - @Override - public V get(K key) { - for (Map bucket : _buckets) { - if (bucket.containsKey(key)) { - return bucket.get(key); - } - } - return null; - } - - @Override - public void putHead(K key, V value) { - _buckets.peekFirst().put(key, value); - } - - @Override - public void put(K key, V value) { - Iterator> it = _buckets.iterator(); - Map bucket = it.next(); - bucket.put(key, value); - while (it.hasNext()) { - bucket = it.next(); - bucket.remove(key); - } - } - - - /** - * Remove item from Rotate - * - * On the side of performance, scanning from header is faster On the side of - * logic, it should scan from the end to first. - * - * @param key - * @return - */ - @Override - public Object remove(K key) { - for (Map bucket : _buckets) { - Object value = bucket.remove(key); - if (value != null) { - return value; - } - } - return null; - } - - @Override - public int size() { - int size = 0; - for (Map bucket : _buckets) { - size += bucket.size(); - } - return size; - } -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/RunCounter.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/RunCounter.java deleted file mode 100644 index 69a1bb84f..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/RunCounter.java +++ /dev/null @@ -1,85 +0,0 @@ -package com.alibaba.jstorm.utils; - -import java.io.Serializable; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.log4j.Logger; - -public class RunCounter implements Serializable{ - - private static final long serialVersionUID = 2177944366059817622L; - private static final Logger LOG = Logger.getLogger(RunCounter.class); - private AtomicLong total = new AtomicLong(0); - private AtomicLong times = new AtomicLong(0); - private AtomicLong values = new AtomicLong(0); - - private IntervalCheck intervalCheck; - - private final String id; - - - public RunCounter() { - this("", RunCounter.class); - } - - public RunCounter(String id) { - this(id, RunCounter.class); - } - - public RunCounter(Class tclass) { - this(tclass.getName(), tclass); - - } - - public RunCounter(String id, Class tclass) { - this.id = id; - - intervalCheck = new IntervalCheck(); - intervalCheck.setInterval(60); - } - - public Double count(long value) { - long totalValue = total.incrementAndGet(); - long timesValue = times.incrementAndGet(); - long v = values.addAndGet(value); - - Double pass = intervalCheck.checkAndGet(); - if (pass != null) { - times.set(0); - values.set(0); - - Double tps = timesValue / pass; - - StringBuilder sb = new StringBuilder(); - sb.append(id); - sb.append(", tps:" + tps); - sb.append(", avg:" + ((double) v) / timesValue); - sb.append(", total:" + totalValue); - LOG.info(sb.toString()); - - return tps; - } - - return null; - } - - public void cleanup() { - - LOG.info(id + ", total:" + total); - } - - - - public IntervalCheck getIntervalCheck() { - return intervalCheck; - } - - /** - * @param args - */ - public static void main(String[] args) { - // TODO Auto-generated method stub - - } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/SmartThread.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/SmartThread.java deleted file mode 100644 index 22aa2f550..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/SmartThread.java +++ /dev/null @@ -1,18 +0,0 @@ -package com.alibaba.jstorm.utils; - -/** - * - * @author yannian - * - */ -public interface SmartThread { - public void start(); - - public void join() throws InterruptedException;; - - public void interrupt(); - - public Boolean isSleeping(); - - public void cleanup(); -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/TimeCacheMap.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/TimeCacheMap.java deleted file mode 100644 index 210e6c8c3..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/TimeCacheMap.java +++ /dev/null @@ -1,147 +0,0 @@ -package com.alibaba.jstorm.utils; - -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.Map; -import java.util.Map.Entry; - -/** - * Expires keys that have not been updated in the configured number of seconds. - * The algorithm used will take between expirationSecs and expirationSecs * (1 + - * 1 / (numBuckets-1)) to actually expire the message. - * - * get, put, remove, containsKey, and size take O(numBuckets) time to run. - * - * - */ -public class TimeCacheMap implements TimeOutMap { - // this default ensures things expire at most 50% past the expiration time - private static final int DEFAULT_NUM_BUCKETS = 3; - - private LinkedList> _buckets; - - private final Object _lock = new Object(); - private Thread _cleaner; - private ExpiredCallback _callback; - - public TimeCacheMap(int expirationSecs, int numBuckets, - ExpiredCallback callback) { - if (numBuckets < 2) { - throw new IllegalArgumentException("numBuckets must be >= 2"); - } - _buckets = new LinkedList>(); - for (int i = 0; i < numBuckets; i++) { - _buckets.add(new HashMap()); - } - - _callback = callback; - final long expirationMillis = expirationSecs * 1000L; - final long sleepTime = expirationMillis / (numBuckets - 1); - _cleaner = new Thread(new Runnable() { - public void run() { - - while (true) { - Map dead = null; - JStormUtils.sleepMs(sleepTime); - synchronized (_lock) { - dead = _buckets.removeLast(); - _buckets.addFirst(new HashMap()); - } - if (_callback != null) { - for (Entry entry : dead.entrySet()) { - _callback.expire(entry.getKey(), entry.getValue()); - } - } - } - } - }); - _cleaner.setDaemon(true); - _cleaner.start(); - } - - public TimeCacheMap(int expirationSecs, ExpiredCallback callback) { - this(expirationSecs, DEFAULT_NUM_BUCKETS, callback); - } - - public TimeCacheMap(int expirationSecs) { - this(expirationSecs, DEFAULT_NUM_BUCKETS); - } - - public TimeCacheMap(int expirationSecs, int numBuckets) { - this(expirationSecs, numBuckets, null); - } - - @Override - public boolean containsKey(K key) { - synchronized (_lock) { - for (HashMap bucket : _buckets) { - if (bucket.containsKey(key)) { - return true; - } - } - return false; - } - } - - @Override - public V get(K key) { - synchronized (_lock) { - for (HashMap bucket : _buckets) { - if (bucket.containsKey(key)) { - return bucket.get(key); - } - } - return null; - } - } - - @Override - public void putHead(K key, V value) { - synchronized (_lock) { - _buckets.getFirst().put(key, value); - } - } - - @Override - public void put(K key, V value) { - synchronized (_lock) { - Iterator> it = _buckets.iterator(); - HashMap bucket = it.next(); - bucket.put(key, value); - while (it.hasNext()) { - bucket = it.next(); - bucket.remove(key); - } - } - } - - @Override - public Object remove(K key) { - synchronized (_lock) { - for (HashMap bucket : _buckets) { - if (bucket.containsKey(key)) { - return bucket.remove(key); - } - } - return null; - } - } - - @Override - public int size() { - synchronized (_lock) { - int size = 0; - for (HashMap bucket : _buckets) { - size += bucket.size(); - } - return size; - } - } - - public void cleanup() { - _cleaner.interrupt(); - } - - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/TimeCacheQueue.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/TimeCacheQueue.java deleted file mode 100644 index acc82214f..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/TimeCacheQueue.java +++ /dev/null @@ -1,168 +0,0 @@ -package com.alibaba.jstorm.utils; - -import java.util.Iterator; -import java.util.LinkedList; -import java.util.concurrent.LinkedBlockingDeque; - -import org.apache.log4j.Logger; - -/** - * Expires keys that have not been updated in the configured number of seconds. - * The algorithm used will take between expirationSecs and expirationSecs * (1 + - * 1 / (numBuckets-1)) to actually expire the message. - * - * get, put, remove, containsKey, and size take O(numBuckets) time to run. - * - * The advantage of this design is that the expiration thread only locks the - * object for O(1) time, meaning the object is essentially always available for - * poll/offer - */ -public class TimeCacheQueue { - // this default ensures things expire at most 50% past the expiration time - public static final int DEFAULT_NUM_BUCKETS = 3; - - public static interface ExpiredCallback { - public void expire(K entry); - } - - public static class DefaultExpiredCallback implements ExpiredCallback { - protected static final Logger LOG = Logger - .getLogger(TimeCacheQueue.DefaultExpiredCallback.class); - - protected String queueName; - - public DefaultExpiredCallback(String queueName) { - this.queueName = queueName; - } - - public void expire(K entry) { - LOG.info("TimeCacheQueue " + queueName + " entry:" + entry - + ", timeout"); - } - } - - protected LinkedList> _buckets; - - protected final Object _lock = new Object(); - protected Thread _cleaner; - protected ExpiredCallback _callback; - - public TimeCacheQueue(int expirationSecs, int numBuckets, - ExpiredCallback callback) { - if (numBuckets < 2) { - throw new IllegalArgumentException("numBuckets must be >= 2"); - } - _buckets = new LinkedList>(); - for (int i = 0; i < numBuckets; i++) { - _buckets.add(new LinkedBlockingDeque()); - } - - _callback = callback; - final long expirationMillis = expirationSecs * 1000L; - final long sleepTime = expirationMillis / (numBuckets - 1); - _cleaner = new Thread(new Runnable() { - public void run() { - try { - while (true) { - LinkedBlockingDeque dead = null; - - Thread.sleep(sleepTime); - - synchronized (_lock) { - dead = _buckets.removeLast(); - _buckets.addFirst(new LinkedBlockingDeque()); - } - if (_callback != null) { - for (K entry : dead) { - _callback.expire(entry); - } - } - } - } catch (InterruptedException ex) { - - } - } - }); - _cleaner.setDaemon(true); - _cleaner.start(); - } - - public TimeCacheQueue(int expirationSecs, ExpiredCallback callback) { - this(expirationSecs, DEFAULT_NUM_BUCKETS, callback); - } - - public TimeCacheQueue(int expirationSecs) { - this(expirationSecs, DEFAULT_NUM_BUCKETS, null); - } - - public TimeCacheQueue(int expirationSecs, int numBuckets) { - this(expirationSecs, numBuckets, null); - } - - public boolean containsKey(K entry) { - synchronized (_lock) { - for (LinkedBlockingDeque bucket : _buckets) { - if (bucket.contains(entry)) { - return true; - } - } - return false; - } - } - - public K poll() { - synchronized (_lock) { - Iterator> itor = _buckets - .descendingIterator(); - while (itor.hasNext()) { - LinkedBlockingDeque bucket = itor.next(); - K entry = bucket.poll(); - if (entry != null) { - return entry; - } - } - - return null; - } - } - - public void offer(K entry) { - synchronized (_lock) { - LinkedBlockingDeque bucket = _buckets.getFirst(); - - bucket.offer(entry); - } - } - - public void remove(K entry) { - synchronized (_lock) { - for (LinkedBlockingDeque bucket : _buckets) { - if (bucket.contains(entry)) { - bucket.remove(entry); - return; - } - } - return; - } - } - - public int size() { - synchronized (_lock) { - int size = 0; - for (LinkedBlockingDeque bucket : _buckets) { - size += bucket.size(); - } - return size; - } - } - - @Override - protected void finalize() throws Throwable { - try { - _cleaner.interrupt(); - } finally { - super.finalize(); - } - } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/TimeFormat.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/TimeFormat.java deleted file mode 100644 index b82572b71..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/TimeFormat.java +++ /dev/null @@ -1,199 +0,0 @@ -package com.alibaba.jstorm.utils; - -import java.text.SimpleDateFormat; -import java.util.Calendar; -import java.util.Date; - -import org.apache.log4j.Logger; - -/** - * - * @author longda - * - */ -public class TimeFormat { - public static Logger log = Logger.getLogger(TimeFormat.class); - - public static final long ONE_SECOND_MILLISECONDS = 1000; - - public static final long ONE_MINUTE_SECONDS = 60; - - public static final long ONE_HOUR_MINUTES = 60; - - public static final long ONE_DAY_HOURS = 24; - - public static final long ONE_MINUTE_MILLISECONDS = ONE_MINUTE_SECONDS - * ONE_SECOND_MILLISECONDS; - - public static final long ONE_HOUR_MILLISECONDS = ONE_HOUR_MINUTES - * ONE_MINUTE_MILLISECONDS; - - public static final long ONE_DAY_MILLISECONDS = ONE_DAY_HOURS - * ONE_HOUR_MILLISECONDS; - - public static Date convertDate(String dateStr, String format) { - Date date = null; - try { - if (format != null) { - SimpleDateFormat simpleDateFormat = new SimpleDateFormat(format); - date = simpleDateFormat.parse(dateStr); - } else { - date = new Date(dateStr); - } - - } catch (Exception ex) { - log.error("Failed to convert " + dateStr + " to Date, format:" - + format); - return null; - } - return date; - } - - public static String convertStr(Date date, String format) { - String ret = null; - try { - - SimpleDateFormat sdf = new SimpleDateFormat(format); - - ret = sdf.format(date); - - } catch (Exception e) { - log.error("Failed to convert " + date + " to String, format:" - + format); - return null; - } - return ret; - } - - public static Date getYear(String dateStr) { - return convertDate(dateStr, "yyyy"); - } - - public static String getYear(Date date) { - return convertStr(date, "yyyy"); - } - - public static Date getMonth(String dateStr) { - return convertDate(dateStr, "yyyyMM"); - } - - public static String getMonth(Date date) { - return convertStr(date, "yyyyMM"); - } - - public static Date getDay(String dateStr) { - return convertDate(dateStr, "yyyyMMdd"); - } - - public static String getDay(Date date) { - return convertStr(date, "yyyyMMdd"); - } - - public static Date getHour(String dateStr) { - return convertDate(dateStr, "yyyyMMddHH"); - } - - public static String getHour(Date date) { - return convertStr(date, "yyyyMMddHH"); - } - - public static Date getMinute(String dateStr) { - return convertDate(dateStr, "yyyyMMddHHmm"); - } - - public static String getMinute(Date date) { - return convertStr(date, "yyyyMMddHHmm"); - } - - public static Date getSecond(String dateStr) { - return convertDate(dateStr, "yyyyMMddHHmmss"); - } - - public static String getSecond(Date date) { - return convertStr(date, "yyyyMMddHHmmss"); - } - - public static String getHourMin(String dateStr) { - Date date = convertDate(dateStr, null); - if (date == null) { - return null; - } - - return getHourMin(date); - } - - public static String getHourMin(Date date) { - String output = null; - try { - SimpleDateFormat sdf = new SimpleDateFormat("HH:mm"); - output = sdf.format(date); - } catch (Exception e) { - return null; - } - return output; - } - - public static Date getToday() { - Date now = new Date(); - - String todayStr = getDay(now); - - return getDay(todayStr); - } - - public static Date getYesterday() { - Date now = new Date(); - - Calendar yesterdayCal = Calendar.getInstance(); - yesterdayCal.setTime(now); - yesterdayCal.add(Calendar.DATE, -1); - - String yesterdayStr = getDay(yesterdayCal.getTime()); - - return getDay(yesterdayStr); - } - - /** - * get the days number pass from 1970-00-00 - * - * @return - */ - public static long getDayNum(Date date) { - long passMs = date.getTime() + (8 * 1000 * 60 * 60); - - return (passMs / 1000 / 60 / 60 / 24); - } - - /** - * @param args - */ - public static void main(String[] args) { - // TODO Auto-generated method stub - - Date date = new Date(); - - String dateStr = getDay(date); - - Date newDate = getDay(dateStr); - - System.out.println("new date:" + newDate); - - Date current = new Date(); - Calendar tomorrow = Calendar.getInstance(); - - tomorrow.setTime(current); - tomorrow.add(Calendar.DATE, 1); - tomorrow.set(Calendar.AM_PM, Calendar.AM); - tomorrow.set(Calendar.HOUR, 2); - tomorrow.set(Calendar.MINUTE, 0); - Date startTime = tomorrow.getTime(); - - long hourdiff = (startTime.getTime() - current.getTime()) - / ONE_HOUR_MILLISECONDS; - - System.out.println("Current:" + current + ", tomorrow" + startTime - + ", diff hour" + hourdiff); - - } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/TimeOutMap.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/TimeOutMap.java deleted file mode 100644 index 4bf214391..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/TimeOutMap.java +++ /dev/null @@ -1,18 +0,0 @@ -package com.alibaba.jstorm.utils; - - -public interface TimeOutMap { - - public boolean containsKey(K key); - - public V get(K key); - - public void putHead(K key, V value); - - public void put(K key, V value); - - public Object remove(K key); - - public int size() ; - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/TimeUtils.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/TimeUtils.java deleted file mode 100644 index 0536ceeb4..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/utils/TimeUtils.java +++ /dev/null @@ -1,34 +0,0 @@ -package com.alibaba.jstorm.utils; - -import backtype.storm.utils.Time; - -/** - * Time utils - * - * @author yannian - * - */ -public class TimeUtils { - - /** - * Take care of int overflow - * - * @return - */ - public static int current_time_secs() { - return (int)(Time.currentTimeMillis() / 1000); - } - - /** - * Take care of int overflow - * - * @return - */ - public static int time_delta(int time_secs) { - return current_time_secs() - time_secs; - } - - public static long time_delta_ms(long time_ms) { - return System.currentTimeMillis() - time_ms; - } -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/Factory.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/Factory.java deleted file mode 100644 index 2d203f3ef..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/Factory.java +++ /dev/null @@ -1,20 +0,0 @@ -package com.alibaba.jstorm.zk; - -import java.io.IOException; -import java.net.InetSocketAddress; - -import org.apache.zookeeper.server.NIOServerCnxnFactory; -import org.apache.zookeeper.server.ZooKeeperServer; - -public class Factory extends NIOServerCnxnFactory { - - public Factory(InetSocketAddress addr, int maxcc) throws IOException { - super(); - this.configure(addr, maxcc); - } - - public ZooKeeperServer getZooKeeperServer() { - return zkServer; - } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/ZkConstant.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/ZkConstant.java deleted file mode 100644 index c07add573..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/ZkConstant.java +++ /dev/null @@ -1,19 +0,0 @@ -package com.alibaba.jstorm.zk; - -public class ZkConstant { - - public static final String ZK_SEPERATOR = "/"; - - public static final String ASSIGNMENTS_BAK = "assignments_bak"; - - public static final String ASSIGNMENTS_BAK_SUBTREE; - - public static final String NIMBUS_SLAVE_ROOT = "nimbus_slave"; - - public static final String NIMBUS_SLAVE_SUBTREE; - - static { - ASSIGNMENTS_BAK_SUBTREE = ZK_SEPERATOR + ASSIGNMENTS_BAK; - NIMBUS_SLAVE_SUBTREE = ZK_SEPERATOR + NIMBUS_SLAVE_ROOT; - } -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/ZkCreateModes.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/ZkCreateModes.java deleted file mode 100644 index 886146f5e..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/ZkCreateModes.java +++ /dev/null @@ -1,21 +0,0 @@ -package com.alibaba.jstorm.zk; - -import java.util.HashMap; - -import org.apache.zookeeper.CreateMode; - -public class ZkCreateModes { - - private static HashMap map; - - static { - map = new HashMap(); - map.put(CreateMode.EPHEMERAL, ":ephemeral"); - map.put(CreateMode.PERSISTENT, ":persistent"); - } - - public static String getStateName(CreateMode mode) { - return map.get(mode); - } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/ZkEventTypes.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/ZkEventTypes.java deleted file mode 100644 index cd2bb4ee3..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/ZkEventTypes.java +++ /dev/null @@ -1,27 +0,0 @@ -package com.alibaba.jstorm.zk; - -import java.util.HashMap; - -import org.apache.zookeeper.Watcher; - -public class ZkEventTypes { - - private static HashMap map; - - static { - map = new HashMap(); - - map.put(Watcher.Event.EventType.None, ":none"); - map.put(Watcher.Event.EventType.NodeCreated, ":node-created"); - map.put(Watcher.Event.EventType.NodeDeleted, ":node-deleted"); - map.put(Watcher.Event.EventType.NodeDataChanged, ":node-data-changed"); - map.put(Watcher.Event.EventType.NodeChildrenChanged, - ":node-children-changed"); - - } - - public static String getStateName(Watcher.Event.EventType type) { - return map.get(type); - } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/ZkKeeperStates.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/ZkKeeperStates.java deleted file mode 100644 index 9a9a979dc..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/ZkKeeperStates.java +++ /dev/null @@ -1,24 +0,0 @@ -package com.alibaba.jstorm.zk; - -import java.util.HashMap; - -import org.apache.zookeeper.Watcher; - -public class ZkKeeperStates { - - private static HashMap map; - - static { - map = new HashMap(); - - map.put(Watcher.Event.KeeperState.AuthFailed, ":auth-failed"); - map.put(Watcher.Event.KeeperState.SyncConnected, ":connected"); - map.put(Watcher.Event.KeeperState.Disconnected, ":disconnected"); - map.put(Watcher.Event.KeeperState.Expired, ":expired"); - } - - public static String getStateName(Watcher.Event.KeeperState state) { - return map.get(state); - } - -} 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 deleted file mode 100644 index 2d504d92d..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/ZkTool.java +++ /dev/null @@ -1,201 +0,0 @@ -package com.alibaba.jstorm.zk; - -import java.util.List; -import java.util.Map; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.cluster.ClusterState; -import com.alibaba.jstorm.cluster.DistributedClusterState; -import com.google.common.collect.Maps; - -public class ZkTool { - private static Logger LOG = Logger.getLogger(ZkTool.class); - - public static final String READ_CMD = "read"; - - public static final String RM_CMD = "rm"; - - public static void usage() { - LOG.info("Read ZK node's data, please do as following:"); - LOG.info(ZkTool.class.getName() + " read zkpath"); - - LOG.info("\nDelete topology backup assignment, please do as following:"); - LOG.info(ZkTool.class.getName() + " rm topologyname"); - } - - public static String getData(DistributedClusterState zkClusterState, - String path) throws Exception { - byte[] data = zkClusterState.get_data(path, false); - if (data == null || data.length == 0) { - return null; - } - - Object obj = Utils.deserialize(data, null); - - return obj.toString(); - } - - public static void readData(String path) { - - DistributedClusterState zkClusterState = null; - - try { - conf.put(Config.STORM_ZOOKEEPER_ROOT, "/"); - - zkClusterState = new DistributedClusterState(conf); - - String data = getData(zkClusterState, path); - if (data == null) { - LOG.info("No data of " + path); - } - - StringBuilder sb = new StringBuilder(); - - sb.append("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!\n\n"); - sb.append("Zk node " + path + "\n"); - sb.append("Readable data:" + data + "\n"); - sb.append("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!\n\n"); - - LOG.info(sb.toString()); - - } catch (Exception e) { - if (zkClusterState == null) { - LOG.error("Failed to connect ZK ", e); - } else { - LOG.error("Failed to read data " + path + "\n", e); - } - } finally { - if (zkClusterState != null) { - zkClusterState.close(); - } - } - } - - public static void rmBakTopology(String topologyName) { - - DistributedClusterState zkClusterState = null; - - try { - - zkClusterState = new DistributedClusterState(conf); - - String path = ZkConstant.ASSIGNMENTS_BAK_SUBTREE; - List bakTopologys = zkClusterState - .get_children(path, false); - - for (String tid : bakTopologys) { - if (tid.equals(topologyName)) { - LOG.info("Find backup " + topologyName); - - String topologyPath = assignment_bak_path(topologyName); - zkClusterState.delete_node(topologyPath); - - LOG.info("Successfully delete topology " + topologyName - + " backup Assignment"); - - return; - } - } - - LOG.info("No backup topology " + topologyName + " Assignment"); - - } catch (Exception e) { - if (zkClusterState == null) { - LOG.error("Failed to connect ZK ", e); - } else { - LOG.error("Failed to delete old topology " + topologyName - + "\n", e); - } - } finally { - if (zkClusterState != null) { - zkClusterState.close(); - } - } - - } - - private static Map conf; - - /** - * @param args - * @throws Exception - */ - public static void main(String[] args) throws Exception { - // TODO Auto-generated method stub - - if (args.length < 2) { - LOG.info("Invalid parameter"); - usage(); - return; - } - - conf = Utils.readStormConfig(); - - if (args[0].equalsIgnoreCase(READ_CMD)) { - - readData(args[1]); - - } else if (args[0].equalsIgnoreCase(RM_CMD)) { - rmBakTopology(args[1]); - } - - } - - /*******************************************************************/ - - public static String assignment_bak_path(String id) { - return ZkConstant.ASSIGNMENTS_BAK_SUBTREE + ZkConstant.ZK_SEPERATOR - + id; - } - - @SuppressWarnings("rawtypes") - public static ClusterState mk_distributed_cluster_state(Map _conf) - throws Exception { - return new DistributedClusterState(_conf); - } - - public static Map get_followers(ClusterState cluster_state) - throws Exception { - Map ret = Maps.newHashMap(); - List followers = cluster_state.get_children( - ZkConstant.NIMBUS_SLAVE_SUBTREE, false); - if (followers == null || followers.size() == 0) { - return ret; - } - for (String follower : followers) { - if (follower != null) { - String uptime = new String(cluster_state.get_data( - ZkConstant.NIMBUS_SLAVE_SUBTREE + ZkConstant.ZK_SEPERATOR - + follower, false)); - ret.put(follower, uptime); - } - } - return ret; - } - - // public static List get_follower_hosts(ClusterState cluster_state) - // throws Exception { - // List followers = cluster_state.get_children( - // ZkConstant.NIMBUS_SLAVE_SUBTREE, false); - // if (followers == null || followers.size() == 0) { - // return Lists.newArrayList(); - // } - // return followers; - // } - // - // public static List get_follower_hbs(ClusterState cluster_state) - // throws Exception { - // List ret = Lists.newArrayList(); - // List followers = get_follower_hosts(cluster_state); - // for (String follower : followers) { - // ret.add(new String(cluster_state.get_data(ZkConstant.NIMBUS_SLAVE_SUBTREE - // + ZkConstant.ZK_SEPERATOR + follower, false))); - // } - // return ret; - // } - -} diff --git a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/Zookeeper.java b/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/Zookeeper.java deleted file mode 100644 index 5bf0b16a5..000000000 --- a/jstorm-client-extension/src/main/java/com/alibaba/jstorm/zk/Zookeeper.java +++ /dev/null @@ -1,217 +0,0 @@ -package com.alibaba.jstorm.zk; - -import java.io.File; -import java.io.IOException; -import java.net.InetSocketAddress; -import java.util.List; -import java.util.Map; - -import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.api.CuratorEvent; -import org.apache.curator.framework.api.CuratorEventType; -import org.apache.curator.framework.api.CuratorListener; -import org.apache.curator.framework.api.UnhandledErrorListener; -import org.apache.log4j.Logger; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.ZooDefs; -import org.apache.zookeeper.data.Stat; -import org.apache.zookeeper.server.ZooKeeperServer; - -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.callback.DefaultWatcherCallBack; -import com.alibaba.jstorm.callback.WatcherCallBack; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.PathUtils; - -/** - * ZK simple wrapper - * - * @author yannian - * - */ -public class Zookeeper { - - private static Logger LOG = Logger.getLogger(Zookeeper.class); - - public CuratorFramework mkClient(Map conf, List servers, - Object port, String root) { - return mkClient(conf, servers, port, root, new DefaultWatcherCallBack()); - } - - /** - * connect ZK, register Watch/unhandle Watch - * - * @return - */ - public CuratorFramework mkClient(Map conf, List servers, - Object port, String root, final WatcherCallBack watcher) { - - CuratorFramework fk = Utils.newCurator(conf, servers, port, root); - - fk.getCuratorListenable().addListener(new CuratorListener() { - @Override - public void eventReceived(CuratorFramework _fk, CuratorEvent e) - throws Exception { - if (e.getType().equals(CuratorEventType.WATCHED)) { - WatchedEvent event = e.getWatchedEvent(); - - watcher.execute(event.getState(), event.getType(), - event.getPath()); - } - - } - }); - - fk.getUnhandledErrorListenable().addListener( - new UnhandledErrorListener() { - @Override - public void unhandledError(String msg, Throwable error) { - String errmsg = "Unrecoverable Zookeeper error, halting process: " - + msg; - LOG.error(errmsg, error); - JStormUtils.halt_process(1, - "Unrecoverable Zookeeper error"); - - } - }); - fk.start(); - return fk; - } - - public String createNode(CuratorFramework zk, String path, byte[] data, - org.apache.zookeeper.CreateMode mode) throws Exception { - - String npath = PathUtils.normalize_path(path); - - return zk.create().withMode(mode).withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE) - .forPath(npath, data); - } - - public String createNode(CuratorFramework zk, String path, byte[] data) - throws Exception { - return createNode(zk, path, data, - org.apache.zookeeper.CreateMode.PERSISTENT); - } - - public boolean existsNode(CuratorFramework zk, String path, boolean watch) - throws Exception { - Stat stat = null; - if (watch) { - stat = zk.checkExists().watched() - .forPath(PathUtils.normalize_path(path)); - } else { - stat = zk.checkExists().forPath(PathUtils.normalize_path(path)); - } - return stat != null; - } - - public void deleteNode(CuratorFramework zk, String path) throws Exception { - zk.delete().forPath(PathUtils.normalize_path(path)); - } - - public void mkdirs(CuratorFramework zk, String path) throws Exception { - - String npath = PathUtils.normalize_path(path); - - // the node is "/" - if (npath.equals("/")) { - return; - } - - // the node exist - if (existsNode(zk, npath, false)) { - return; - } - - mkdirs(zk, PathUtils.parent_path(npath)); - try { - createNode(zk, npath, JStormUtils.barr((byte) 7), - org.apache.zookeeper.CreateMode.PERSISTENT); - } catch (KeeperException e) { - ;// this can happen when multiple clients doing mkdir at same - // time - LOG.warn("zookeeper mkdirs for path" + path, e); - - } - - } - - public byte[] getData(CuratorFramework zk, String path, boolean watch) - throws Exception { - String npath = PathUtils.normalize_path(path); - try { - if (existsNode(zk, npath, watch)) { - if (watch) { - return zk.getData().watched().forPath(npath); - } else { - return zk.getData().forPath(npath); - } - } - } catch (KeeperException e) { - LOG.error("zookeeper getdata for path" + path, e); - } - - return null; - } - - public List getChildren(CuratorFramework zk, String path, - boolean watch) throws Exception { - - String npath = PathUtils.normalize_path(path); - - if (watch) { - return zk.getChildren().watched().forPath(npath); - } else { - return zk.getChildren().forPath(npath); - } - } - - public Stat setData(CuratorFramework zk, String path, byte[] data) - throws Exception { - String npath = PathUtils.normalize_path(path); - return zk.setData().forPath(npath, data); - } - - public boolean exists(CuratorFramework zk, String path, boolean watch) - throws Exception { - return existsNode(zk, path, watch); - } - - public void deletereRcursive(CuratorFramework zk, String path) - throws Exception { - - String npath = PathUtils.normalize_path(path); - - if (existsNode(zk, npath, false)) { - - List childs = getChildren(zk, npath, false); - - for (String child : childs) { - - String childFullPath = PathUtils.full_path(npath, child); - - deletereRcursive(zk, childFullPath); - } - - deleteNode(zk, npath); - } - } - - public static Factory mkInprocessZookeeper(String localdir, int port) - throws IOException, InterruptedException { - LOG.info("Starting inprocess zookeeper at port " + port + " and dir " - + localdir); - File localfile = new File(localdir); - ZooKeeperServer zk = new ZooKeeperServer(localfile, localfile, 2000); - Factory factory = new Factory(new InetSocketAddress(port), 0); - factory.startup(zk); - return factory; - } - - public void shutdownInprocessZookeeper(Factory handle) { - handle.shutdown(); - } - -} diff --git a/jstorm-client/pom.xml b/jstorm-client/pom.xml deleted file mode 100644 index 6a0f4658c..000000000 --- a/jstorm-client/pom.xml +++ /dev/null @@ -1,194 +0,0 @@ - - - - - com.alibaba.jstorm - jstorm-all - 0.9.6.3 - .. - - - 4.0.0 - com.alibaba.jstorm - jstorm-client - 0.9.6.3 - jar - ${project.artifactId}-${project.version} - - - - - maven-compiler-plugin - 2.3.2 - - 1.6 - 1.6 - - - - org.apache.maven.plugins - maven-source-plugin - - - attach-sources - - jar - - - - - - - - UTF-8 - 1.4.11 - - - - - org.clojure - clojure - 1.5.1 - - - commons-io - commons-io - 2.4 - - - org.apache.commons - commons-exec - 1.1 - - - org.apache.httpcomponents - httpclient - 4.3.2 - - - storm - libthrift7 - 0.7.0 - - - org.slf4j - slf4j-api - - - - - clj-time - clj-time - 0.4.1 - - - org.apache.curator - curator-framework - 2.5.0 - - - log4j - log4j - - - org.slf4j - slf4j-api - - - - - backtype - jzmq - 2.1.0 - - - com.esotericsoftware.kryo - kryo - 2.17 - - - com.alibaba - fastjson - 1.1.41 - - - - com.googlecode.json-simple - json-simple - 1.1 - - - storm - carbonite - 1.5.0 - - - - org.yaml - snakeyaml - 1.11 - - - com.lmax - disruptor - 3.2.1 - - - io.netty - netty - 3.9.0.Final - - - storm - jgrapht - 0.8.3 - - - junit - junit - 4.10 - test - - - org.powermock - powermock-module-junit4 - ${powermock.version} - test - - - log4j - log4j - 1.2.16 - - - org.slf4j - slf4j-log4j12 - 1.7.5 - - - commons-cli - commons-cli - 1.2 - - - org.codehaus.plexus - plexus-compiler-javac - 1.8.1 - - - com.google.code.gson - gson - 2.3.1 - - - - - - diff --git a/jstorm-client/src/main/java/backtype/storm/Config.java b/jstorm-client/src/main/java/backtype/storm/Config.java deleted file mode 100644 index baebbe84a..000000000 --- a/jstorm-client/src/main/java/backtype/storm/Config.java +++ /dev/null @@ -1,1382 +0,0 @@ -package backtype.storm; - -import backtype.storm.ConfigValidation; -import backtype.storm.serialization.IKryoDecorator; -import backtype.storm.serialization.IKryoFactory; - -import com.esotericsoftware.kryo.Serializer; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * Topology configs are specified as a plain old map. This class provides a - * convenient way to create a topology config map by providing setter methods - * for all the configs that can be set. It also makes it easier to do things - * like add serializations. - * - *

- * This class also provides constants for all the configurations possible on a - * Storm cluster and Storm topology. Each constant is paired with a schema that - * defines the validity criterion of the corresponding field. Default values for - * these configs can be found in defaults.yaml. - *

- * - *

- * Note that you may put other configurations in any of the configs. Storm will - * ignore anything it doesn't recognize, but your topologies are free to make - * use of them by reading them in the prepare method of Bolts or the open method - * of Spouts. - *

- */ -public class Config extends HashMap { - /** - * The transporter for communication among Storm tasks - */ - public static final String STORM_MESSAGING_TRANSPORT = "storm.messaging.transport"; - public static final Object STORM_MESSAGING_TRANSPORT_SCHEMA = String.class; - - /** - * Netty based messaging: The buffer size for send/recv buffer - */ - public static final String STORM_MESSAGING_NETTY_BUFFER_SIZE = "storm.messaging.netty.buffer_size"; - public static final Object STORM_MESSAGING_NETTY_BUFFER_SIZE_SCHEMA = Number.class; - - /** - * Netty based messaging: The max # of retries that a peer will perform when - * a remote is not accessible - */ - public static final String STORM_MESSAGING_NETTY_MAX_RETRIES = "storm.messaging.netty.max_retries"; - public static final Object STORM_MESSAGING_NETTY_MAX_RETRIES_SCHEMA = Number.class; - - /** - * Netty based messaging: The min # of milliseconds that a peer will wait. - */ - public static final String STORM_MESSAGING_NETTY_MIN_SLEEP_MS = "storm.messaging.netty.min_wait_ms"; - public static final Object STORM_MESSAGING_NETTY_MIN_SLEEP_MS_SCHEMA = Number.class; - - /** - * Netty based messaging: The max # of milliseconds that a peer will wait. - */ - public static final String STORM_MESSAGING_NETTY_MAX_SLEEP_MS = "storm.messaging.netty.max_wait_ms"; - public static final Object STORM_MESSAGING_NETTY_MAX_SLEEP_MS_SCHEMA = Number.class; - - /** - * Netty based messaging: The # of worker threads for the server. - */ - public static final String STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS = "storm.messaging.netty.server_worker_threads"; - public static final Object STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS_SCHEMA = Number.class; - - /** - * Netty based messaging: The # of worker threads for the client. - */ - public static final String STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS = "storm.messaging.netty.client_worker_threads"; - public static final Object STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS_SCHEMA = Number.class; - - /** - * If the Netty messaging layer is busy, the Netty client will try to batch message as more as possible up to the size of STORM_NETTY_MESSAGE_BATCH_SIZE bytes - */ - public static final String STORM_NETTY_MESSAGE_BATCH_SIZE = "storm.messaging.netty.transfer.batch.size"; - public static final Object STORM_NETTY_MESSAGE_BATCH_SIZE_SCHEMA = Number.class; - - /** - * We check with this interval that whether the Netty channel is writable and try to write pending messages - */ - public static final String STORM_NETTY_FLUSH_CHECK_INTERVAL_MS = "storm.messaging.netty.flush.check.interval.ms"; - public static final Object STORM_NETTY_FLUSH_CHECK_INTERVAL_MS_SCHEMA = Number.class; - /** - * Netty based messaging: Is authentication required for Netty messaging from client worker process to server worker process. - */ - public static final String STORM_MESSAGING_NETTY_AUTHENTICATION = "storm.messaging.netty.authentication"; - public static final Object STORM_MESSAGING_NETTY_AUTHENTICATION_SCHEMA = Boolean.class; - - /** - * The delegate for serializing metadata, should be used for serialized objects stored in zookeeper and on disk. - * This is NOT used for compressing serialized tuples sent between topologies. - */ - public static final String STORM_META_SERIALIZATION_DELEGATE = "storm.meta.serialization.delegate"; - public static final Object STORM_META_SERIALIZATION_DELEGATE_SCHEMA = String.class; - - /** - * A list of hosts of ZooKeeper servers used to manage the cluster. - */ - public static final String STORM_ZOOKEEPER_SERVERS = "storm.zookeeper.servers"; - public static final Object STORM_ZOOKEEPER_SERVERS_SCHEMA = ConfigValidation.StringsValidator; - - /** - * The port Storm will use to connect to each of the ZooKeeper servers. - */ - public static final String STORM_ZOOKEEPER_PORT = "storm.zookeeper.port"; - public static final Object STORM_ZOOKEEPER_PORT_SCHEMA = Number.class; - - /** - * A directory on the local filesystem used by Storm for any local - * filesystem usage it needs. The directory must exist and the Storm daemons - * must have permission to read/write from this location. - */ - public static final String STORM_LOCAL_DIR = "storm.local.dir"; - public static final Object STORM_LOCAL_DIR_SCHEMA = String.class; - - /** - * A global task scheduler used to assign topologies's tasks to supervisors' - * wokers. - * - * If this is not set, a default system scheduler will be used. - */ - public static final String STORM_SCHEDULER = "storm.scheduler"; - public static final Object STORM_SCHEDULER_SCHEMA = String.class; - - /** - * The mode this Storm cluster is running in. Either "distributed" or - * "local". - */ - public static final String STORM_CLUSTER_MODE = "storm.cluster.mode"; - public static final Object STORM_CLUSTER_MODE_SCHEMA = String.class; - - /** - * The hostname the supervisors/workers should report to nimbus. If unset, - * Storm will get the hostname to report by calling - * InetAddress.getLocalHost().getCanonicalHostName(). - * - * You should set this config when you dont have a DNS which - * supervisors/workers can utilize to find each other based on hostname got - * from calls to - * InetAddress.getLocalHost().getCanonicalHostName(). - */ - public static final String STORM_LOCAL_HOSTNAME = "storm.local.hostname"; - public static final Object STORM_LOCAL_HOSTNAME_SCHEMA = String.class; - - /** - * The plugin that will convert a principal to a local user. - */ - public static final String STORM_PRINCIPAL_TO_LOCAL_PLUGIN = "storm.principal.tolocal"; - public static final Object STORM_PRINCIPAL_TO_LOCAL_PLUGIN_SCHEMA = String.class; - - /** - * The plugin that will provide user groups service - */ - public static final String STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN = "storm.group.mapping.service"; - public static final Object STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN_SCHEMA = String.class; - - /** - * Max no.of seconds group mapping service will cache user groups - */ - public static final String STORM_GROUP_MAPPING_SERVICE_CACHE_DURATION_SECS = "storm.group.mapping.service.cache.duration.secs"; - public static final Object STORM_GROUP_MAPPING_SERVICE_CACHE_DURATION_SECS_SCHEMA = Number.class; - - /** - * The transport plug-in for Thrift client/server communication - */ - public static final String STORM_THRIFT_TRANSPORT_PLUGIN = "storm.thrift.transport"; - public static final Object STORM_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class; - - /** - * The serializer class for ListDelegate (tuple payload). The default - * serializer will be ListDelegateSerializer - */ - public static final String TOPOLOGY_TUPLE_SERIALIZER = "topology.tuple.serializer"; - public static final Object TOPOLOGY_TUPLE_SERIALIZER_SCHEMA = String.class; - - /** - * Try to serialize all tuples, even for local transfers. This should only be used - * for testing, as a sanity check that all of your tuples are setup properly. - */ - public static final String TOPOLOGY_TESTING_ALWAYS_TRY_SERIALIZE = "topology.testing.always.try.serialize"; - public static final Object TOPOLOGY_TESTING_ALWAYS_TRY_SERIALIZE_SCHEMA = Boolean.class; - - /** - * Whether or not to use ZeroMQ for messaging in local mode. If this is set - * to false, then Storm will use a pure-Java messaging system. The purpose - * of this flag is to make it easy to run Storm in local mode by eliminating - * the need for native dependencies, which can be difficult to install. - * - * Defaults to false. - */ - public static final String STORM_LOCAL_MODE_ZMQ = "storm.local.mode.zmq"; - public static final Object STORM_LOCAL_MODE_ZMQ_SCHEMA = Boolean.class; - - /** - * The root location at which Storm stores data in ZooKeeper. - */ - public static final String STORM_ZOOKEEPER_ROOT = "storm.zookeeper.root"; - public static final Object STORM_ZOOKEEPER_ROOT_SCHEMA = String.class; - - /** - * The session timeout for clients to ZooKeeper. - */ - public static final String STORM_ZOOKEEPER_SESSION_TIMEOUT = "storm.zookeeper.session.timeout"; - public static final Object STORM_ZOOKEEPER_SESSION_TIMEOUT_SCHEMA = Number.class; - - /** - * The connection timeout for clients to ZooKeeper. - */ - public static final String STORM_ZOOKEEPER_CONNECTION_TIMEOUT = "storm.zookeeper.connection.timeout"; - public static final Object STORM_ZOOKEEPER_CONNECTION_TIMEOUT_SCHEMA = Number.class; - - /** - * The number of times to retry a Zookeeper operation. - */ - public static final String STORM_ZOOKEEPER_RETRY_TIMES = "storm.zookeeper.retry.times"; - public static final Object STORM_ZOOKEEPER_RETRY_TIMES_SCHEMA = Number.class; - - /** - * The interval between retries of a Zookeeper operation. - */ - public static final String STORM_ZOOKEEPER_RETRY_INTERVAL = "storm.zookeeper.retry.interval"; - public static final Object STORM_ZOOKEEPER_RETRY_INTERVAL_SCHEMA = Number.class; - - /** - * The ceiling of the interval between retries of a Zookeeper operation. - */ - public static final String STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING = "storm.zookeeper.retry.intervalceiling.millis"; - public static final Object STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING_SCHEMA = Number.class; - - /** - * The Zookeeper authentication scheme to use, e.g. "digest". Defaults to no - * authentication. - */ - public static final String STORM_ZOOKEEPER_AUTH_SCHEME = "storm.zookeeper.auth.scheme"; - public static final Object STORM_ZOOKEEPER_AUTH_SCHEME_SCHEMA = String.class; - - /** - * A string representing the payload for Zookeeper authentication. It gets - * serialized using UTF-8 encoding during authentication. - */ - public static final String STORM_ZOOKEEPER_AUTH_PAYLOAD = "storm.zookeeper.auth.payload"; - public static final Object STORM_ZOOKEEPER_AUTH_PAYLOAD_SCHEMA = String.class; - - /** - * The topology Zookeeper authentication scheme to use, e.g. "digest". Defaults to no authentication. - */ - public static final String STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME="storm.zookeeper.topology.auth.scheme"; - public static final Object STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME_SCHEMA = String.class; - - /** - * A string representing the payload for topology Zookeeper authentication. It gets serialized using UTF-8 encoding during authentication. - */ - public static final String STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD="storm.zookeeper.topology.auth.payload"; - public static final Object STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD_SCHEMA = String.class; - - /** - * The id assigned to a running topology. The id is the storm name with a - * unique nonce appended. - */ - public static final String TOPOLOGY_ID = "topology.id"; - public static final Object TOPOLOGY_ID_SCHEMA = String.class; - - /** - * The number of times to retry a Nimbus operation. - */ - public static final String STORM_NIMBUS_RETRY_TIMES="storm.nimbus.retry.times"; - public static final Object STORM_NIMBUS_RETRY_TIMES_SCHEMA = Number.class; - - /** - * The starting interval between exponential backoff retries of a Nimbus operation. - */ - public static final String STORM_NIMBUS_RETRY_INTERVAL="storm.nimbus.retry.interval.millis"; - public static final Object STORM_NIMBUS_RETRY_INTERVAL_SCHEMA = Number.class; - - /** - * The ceiling of the interval between retries of a client connect to Nimbus operation. - */ - public static final String STORM_NIMBUS_RETRY_INTERVAL_CEILING="storm.nimbus.retry.intervalceiling.millis"; - public static final Object STORM_NIMBUS_RETRY_INTERVAL_CEILING_SCHEMA = Number.class; - /** - * The host that the master server is running on. - */ - public static final String NIMBUS_HOST = "nimbus.host"; - public static final Object NIMBUS_HOST_SCHEMA = String.class; - - /** - * The Nimbus transport plug-in for Thrift client/server communication - */ - public static final String NIMBUS_THRIFT_TRANSPORT_PLUGIN = "nimbus.thrift.transport"; - public static final Object NIMBUS_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class; - - /** - * Which port the Thrift interface of Nimbus should run on. Clients should - * connect to this port to upload jars and submit topologies. - */ - public static final String NIMBUS_THRIFT_PORT = "nimbus.thrift.port"; - public static final Object NIMBUS_THRIFT_PORT_SCHEMA = Number.class; - - /** - * The number of threads that should be used by the nimbus thrift server. - */ - public static final String NIMBUS_THRIFT_THREADS = "nimbus.thrift.threads"; - public static final Object NIMBUS_THRIFT_THREADS_SCHEMA = Number.class; - - /** - * A list of users that are cluster admins and can run any command. To use this set - * nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer - */ - public static final String NIMBUS_ADMINS = "nimbus.admins"; - public static final Object NIMBUS_ADMINS_SCHEMA = ConfigValidation.StringsValidator; - - /** - * A list of users that run the supervisors and should be authorized to interact with - * nimbus as a supervisor would. To use this set - * nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer - */ - public static final String NIMBUS_SUPERVISOR_USERS = "nimbus.supervisor.users"; - public static final Object NIMBUS_SUPERVISOR_USERS_SCHEMA = ConfigValidation.StringsValidator; - /** - * The maximum buffer size thrift should use when reading messages. - */ - public static final String NIMBUS_THRIFT_MAX_BUFFER_SIZE = "nimbus.thrift.max_buffer_size"; - public static final Object NIMBUS_THRIFT_MAX_BUFFER_SIZE_SCHEMA = Number.class; - - /** - * This parameter is used by the storm-deploy project to configure the jvm - * options for the nimbus daemon. - */ - public static final String NIMBUS_CHILDOPTS = "nimbus.childopts"; - public static final Object NIMBUS_CHILDOPTS_SCHEMA = String.class; - - /** - * How long without heartbeating a task can go before nimbus will consider - * the task dead and reassign it to another location. - */ - public static final String NIMBUS_TASK_TIMEOUT_SECS = "nimbus.task.timeout.secs"; - public static final Object NIMBUS_TASK_TIMEOUT_SECS_SCHEMA = Number.class; - - /** - * How often nimbus should wake up to check heartbeats and do reassignments. - * Note that if a machine ever goes down Nimbus will immediately wake up and - * take action. This parameter is for checking for failures when there's no - * explicit event like that occuring. - */ - public static final String NIMBUS_MONITOR_FREQ_SECS = "nimbus.monitor.freq.secs"; - public static final Object NIMBUS_MONITOR_FREQ_SECS_SCHEMA = Number.class; - - /** - * How often nimbus should wake the cleanup thread to clean the inbox. - * - * @see NIMBUS_INBOX_JAR_EXPIRATION_SECS - */ - public static final String NIMBUS_CLEANUP_INBOX_FREQ_SECS = "nimbus.cleanup.inbox.freq.secs"; - public static final Object NIMBUS_CLEANUP_INBOX_FREQ_SECS_SCHEMA = Number.class; - - /** - * The length of time a jar file lives in the inbox before being deleted by - * the cleanup thread. - * - * Probably keep this value greater than or equal to - * NIMBUS_CLEANUP_INBOX_JAR_EXPIRATION_SECS. Note that the time it takes to - * delete an inbox jar file is going to be somewhat more than - * NIMBUS_CLEANUP_INBOX_JAR_EXPIRATION_SECS (depending on how often - * NIMBUS_CLEANUP_FREQ_SECS is set to). - * - * @see NIMBUS_CLEANUP_FREQ_SECS - */ - public static final String NIMBUS_INBOX_JAR_EXPIRATION_SECS = "nimbus.inbox.jar.expiration.secs"; - public static final Object NIMBUS_INBOX_JAR_EXPIRATION_SECS_SCHEMA = Number.class; - - /** - * How long before a supervisor can go without heartbeating before nimbus - * considers it dead and stops assigning new work to it. - */ - public static final String NIMBUS_SUPERVISOR_TIMEOUT_SECS = "nimbus.supervisor.timeout.secs"; - public static final Object NIMBUS_SUPERVISOR_TIMEOUT_SECS_SCHEMA = Number.class; - - /** - * A special timeout used when a task is initially launched. During launch, - * this is the timeout used until the first heartbeat, overriding - * nimbus.task.timeout.secs. - * - *

- * A separate timeout exists for launch because there can be quite a bit of - * overhead to launching new JVM's and configuring them. - *

- */ - public static final String NIMBUS_TASK_LAUNCH_SECS = "nimbus.task.launch.secs"; - public static final Object NIMBUS_TASK_LAUNCH_SECS_SCHEMA = Number.class; - - /** - * Whether or not nimbus should reassign tasks if it detects that a task - * goes down. Defaults to true, and it's not recommended to change this - * value. - */ - public static final String NIMBUS_REASSIGN = "nimbus.reassign"; - public static final Object NIMBUS_REASSIGN_SCHEMA = Boolean.class; - - /** - * During upload/download with the master, how long an upload or download - * connection is idle before nimbus considers it dead and drops the - * connection. - */ - public static final String NIMBUS_FILE_COPY_EXPIRATION_SECS = "nimbus.file.copy.expiration.secs"; - public static final Object NIMBUS_FILE_COPY_EXPIRATION_SECS_SCHEMA = Number.class; - - /** - * A custom class that implements ITopologyValidator that is run whenever a - * topology is submitted. Can be used to provide business-specific logic for - * whether topologies are allowed to run or not. - */ - public static final String NIMBUS_TOPOLOGY_VALIDATOR = "nimbus.topology.validator"; - public static final Object NIMBUS_TOPOLOGY_VALIDATOR_SCHEMA = String.class; - - /** - * Class name for authorization plugin for Nimbus - */ - public static final String NIMBUS_AUTHORIZER = "nimbus.authorizer"; - public static final Object NIMBUS_AUTHORIZER_SCHEMA = String.class; - - /** - * How often nimbus should wake up to renew credentials if needed. - */ - public static final String NIMBUS_CREDENTIAL_RENEW_FREQ_SECS = "nimbus.credential.renewers.freq.secs"; - public static final Object NIMBUS_CREDENTIAL_RENEW_FREQ_SECS_SCHEMA = Number.class; - - /** - * A list of credential renewers that nimbus should load. - */ - public static final String NIMBUS_CREDENTIAL_RENEWERS = "nimbus.credential.renewers.classes"; - public static final Object NIMBUS_CREDENTIAL_RENEWERS_SCHEMA = ConfigValidation.StringsValidator; - - /** - * A list of plugins that nimbus should load during submit topology to populate - * credentials on user's behalf. - */ - public static final String NIMBUS_AUTO_CRED_PLUGINS = "nimbus.autocredential.plugins.classes"; - public static final Object NIMBUS_AUTO_CRED_PLUGINS_SCHEMA = ConfigValidation.StringsValidator; - - /** - * Storm UI binds to this host/interface. - */ - public static final String UI_HOST = "ui.host"; - public static final Object UI_HOST_SCHEMA = String.class; - /** - * Storm UI binds to this port. - */ - public static final String UI_PORT = "ui.port"; - public static final Object UI_PORT_SCHEMA = Number.class; - - /** - * HTTP UI port for log viewer - */ - public static final String LOGVIEWER_PORT = "logviewer.port"; - public static final Object LOGVIEWER_PORT_SCHEMA = Number.class; - - /** - * Childopts for log viewer java process. - */ - public static final String LOGVIEWER_CHILDOPTS = "logviewer.childopts"; - public static final Object LOGVIEWER_CHILDOPTS_SCHEMA = String.class; - - /** - * Appender name used by log viewer to determine log directory. - */ - public static final String LOGVIEWER_APPENDER_NAME = "logviewer.appender.name"; - public static final Object LOGVIEWER_APPENDER_NAME_SCHEMA = String.class; - - /** - * Childopts for Storm UI Java process. - */ - public static final String UI_CHILDOPTS = "ui.childopts"; - public static final Object UI_CHILDOPTS_SCHEMA = String.class; - - /** - * List of DRPC servers so that the DRPCSpout knows who to talk to. - */ - public static final String DRPC_SERVERS = "drpc.servers"; - public static final Object DRPC_SERVERS_SCHEMA = ConfigValidation.StringsValidator; - - /** - * This port is used by Storm DRPC for receiving HTTP DPRC requests from clients. - */ - public static final String DRPC_HTTP_PORT = "drpc.http.port"; - public static final Object DRPC_HTTP_PORT_SCHEMA = Number.class; - - /** - * This port is used by Storm DRPC for receiving HTTPS (SSL) DPRC requests from clients. - */ - public static final String DRPC_HTTPS_PORT = "drpc.https.port"; - public static final Object DRPC_HTTPS_PORT_SCHEMA = Number.class; - - /** - * Path to the keystore used by Storm DRPC for setting up HTTPS (SSL). - */ - public static final String DRPC_HTTPS_KEYSTORE_PATH = "drpc.https.keystore.path"; - public static final Object DRPC_HTTPS_KEYSTORE_PATH_SCHEMA = String.class; - - /** - * Password to the keystore used by Storm DRPC for setting up HTTPS (SSL). - */ - public static final String DRPC_HTTPS_KEYSTORE_PASSWORD = "drpc.https.keystore.password"; - public static final Object DRPC_HTTPS_KEYSTORE_PASSWORD_SCHEMA = String.class; - - /** - * Type of keystore used by Storm DRPC for setting up HTTPS (SSL). - * see http://docs.oracle.com/javase/7/docs/api/java/security/KeyStore.html for more details. - */ - public static final String DRPC_HTTPS_KEYSTORE_TYPE = "drpc.https.keystore.type"; - public static final Object DRPC_HTTPS_KEYSTORE_TYPE_SCHEMA = String.class; - - /** - * The DRPC transport plug-in for Thrift client/server communication - */ - public static final String DRPC_THRIFT_TRANSPORT_PLUGIN = "drpc.thrift.transport"; - public static final Object DRPC_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class; - /** - * This port is used by Storm DRPC for receiving DPRC requests from clients. - */ - public static final String DRPC_PORT = "drpc.port"; - public static final Object DRPC_PORT_SCHEMA = Number.class; - - /** - * Class name for authorization plugin for DRPC client - */ - public static final String DRPC_AUTHORIZER = "drpc.authorizer"; - public static final Object DRPC_AUTHORIZER_SCHEMA = String.class; - - /** - * The Access Control List for the DRPC Authorizer. - * @see DRPCSimpleAclAuthorizer - */ - public static final String DRPC_AUTHORIZER_ACL = "drpc.authorizer.acl"; - public static final Object DRPC_AUTHORIZER_ACL_SCHEMA = Map.class; - - /** - * File name of the DRPC Authorizer ACL. - * @see DRPCSimpleAclAuthorizer - */ - public static final String DRPC_AUTHORIZER_ACL_FILENAME = "drpc.authorizer.acl.filename"; - public static final Object DRPC_AUTHORIZER_ACL_FILENAME_SCHEMA = String.class; - - /** - * Whether the DRPCSimpleAclAuthorizer should deny requests for operations - * involving functions that have no explicit ACL entry. When set to false - * (the default) DRPC functions that have no entry in the ACL will be - * permitted, which is appropriate for a development environment. When set - * to true, explicit ACL entries are required for every DRPC function, and - * any request for functions will be denied. - * @see DRPCSimpleAclAuthorizer - */ - public static final String DRPC_AUTHORIZER_ACL_STRICT = "drpc.authorizer.acl.strict"; - public static final Object DRPC_AUTHORIZER_ACL_STRICT_SCHEMA = Boolean.class; - - /** - * DRPC thrift server worker threads - */ - public static final String DRPC_WORKER_THREADS = "drpc.worker.threads"; - public static final Object DRPC_WORKER_THREADS_SCHEMA = Number.class; - - /** - * The maximum buffer size thrift should use when reading messages for DRPC. - */ - public static final String DRPC_MAX_BUFFER_SIZE = "drpc.max_buffer_size"; - public static final Object DRPC_MAX_BUFFER_SIZE_SCHEMA = Number.class; - - /** - * DRPC thrift server queue size - */ - public static final String DRPC_QUEUE_SIZE = "drpc.queue.size"; - public static final Object DRPC_QUEUE_SIZE_SCHEMA = Number.class; - - /** - * The DRPC invocations transport plug-in for Thrift client/server communication - */ - public static final String DRPC_INVOCATIONS_THRIFT_TRANSPORT_PLUGIN = "drpc.invocations.thrift.transport"; - public static final Object DRPC_INVOCATIONS_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class; - - /** - * This port on Storm DRPC is used by DRPC topologies to receive function - * invocations and send results back. - */ - public static final String DRPC_INVOCATIONS_PORT = "drpc.invocations.port"; - public static final Object DRPC_INVOCATIONS_PORT_SCHEMA = Number.class; - - /** - * DRPC invocations thrift server worker threads - */ - public static final String DRPC_INVOCATIONS_THREADS = "drpc.invocations.threads"; - public static final Object DRPC_INVOCATIONS_THREADS_SCHEMA = Number.class; - - /** - * The timeout on DRPC requests within the DRPC server. Defaults to 10 - * minutes. Note that requests can also timeout based on the socket timeout - * on the DRPC client, and separately based on the topology message timeout - * for the topology implementing the DRPC function. - */ - public static final String DRPC_REQUEST_TIMEOUT_SECS = "drpc.request.timeout.secs"; - public static final Object DRPC_REQUEST_TIMEOUT_SECS_SCHEMA = Number.class; - - /** - * Childopts for Storm DRPC Java process. - */ - public static final String DRPC_CHILDOPTS = "drpc.childopts"; - public static final Object DRPC_CHILDOPTS_SCHEMA = String.class; - - /** - * Class name of the HTTP credentials plugin for the UI. - */ - public static final String UI_HTTP_CREDS_PLUGIN = "ui.http.creds.plugin"; - public static final Object UI_HTTP_CREDS_PLUGIN_SCHEMA = String.class; - - /** - * Class name of the HTTP credentials plugin for DRPC. - */ - public static final String DRPC_HTTP_CREDS_PLUGIN = "drpc.http.creds.plugin"; - public static final Object DRPC_HTTP_CREDS_PLUGIN_SCHEMA = String.class; - /** - * the metadata configured on the supervisor - */ - public static final String SUPERVISOR_SCHEDULER_META = "supervisor.scheduler.meta"; - public static final Object SUPERVISOR_SCHEDULER_META_SCHEMA = Map.class; - - /** - * A list of ports that can run workers on this supervisor. Each worker uses - * one port, and the supervisor will only run one worker per port. Use this - * configuration to tune how many workers run on each machine. - */ - public static final String SUPERVISOR_SLOTS_PORTS = "supervisor.slots.ports"; - public static final Object SUPERVISOR_SLOTS_PORTS_SCHEMA = ConfigValidation.NumbersValidator; - - /** - * A number representing the maximum number of workers any single topology can acquire. - */ - public static final String NIMBUS_SLOTS_PER_TOPOLOGY = "nimbus.slots.perTopology"; - public static final Object NIMBUS_SLOTS_PER_TOPOLOGY_SCHEMA = Number.class; - - /** - * A class implementing javax.servlet.Filter for DRPC HTTP requests - */ - public static final String DRPC_HTTP_FILTER = "drpc.http.filter"; - public static final Object DRPC_HTTP_FILTER_SCHEMA = String.class; - - /** - * Initialization parameters for the javax.servlet.Filter of the DRPC HTTP - * service - */ - public static final String DRPC_HTTP_FILTER_PARAMS = "drpc.http.filter.params"; - public static final Object DRPC_HTTP_FILTER_PARAMS_SCHEMA = Map.class; - - /** - * A number representing the maximum number of executors any single topology can acquire. - */ - public static final String NIMBUS_EXECUTORS_PER_TOPOLOGY = "nimbus.executors.perTopology"; - public static final Object NIMBUS_EXECUTORS_PER_TOPOLOGY_SCHEMA = Number.class; - /** - * This parameter is used by the storm-deploy project to configure the - * jvm options for the supervisor daemon. - */ - public static final String SUPERVISOR_CHILDOPTS = "supervisor.childopts"; - public static final Object SUPERVISOR_CHILDOPTS_SCHEMA = String.class; - - /** - * How long a worker can go without heartbeating before the supervisor tries - * to restart the worker process. - */ - public static final String SUPERVISOR_WORKER_TIMEOUT_SECS = "supervisor.worker.timeout.secs"; - public static final Object SUPERVISOR_WORKER_TIMEOUT_SECS_SCHEMA = Number.class; - - /** - * How long a worker can go without heartbeating during the initial launch - * before the supervisor tries to restart the worker process. This value - * override supervisor.worker.timeout.secs during launch because there is - * additional overhead to starting and configuring the JVM on launch. - */ - public static final String SUPERVISOR_WORKER_START_TIMEOUT_SECS = "supervisor.worker.start.timeout.secs"; - public static final Object SUPERVISOR_WORKER_START_TIMEOUT_SECS_SCHEMA = Number.class; - - /** - * Whether or not the supervisor should launch workers assigned to it. - * Defaults to true -- and you should probably never change this value. This - * configuration is used in the Storm unit tests. - */ - public static final String SUPERVISOR_ENABLE = "supervisor.enable"; - public static final Object SUPERVISOR_ENABLE_SCHEMA = Boolean.class; - - /** - * how often the supervisor sends a heartbeat to the master. - */ - public static final String SUPERVISOR_HEARTBEAT_FREQUENCY_SECS = "supervisor.heartbeat.frequency.secs"; - public static final Object SUPERVISOR_HEARTBEAT_FREQUENCY_SECS_SCHEMA = Number.class; - - /** - * How often the supervisor checks the worker heartbeats to see if any of - * them need to be restarted. - */ - public static final String SUPERVISOR_MONITOR_FREQUENCY_SECS = "supervisor.monitor.frequency.secs"; - public static final Object SUPERVISOR_MONITOR_FREQUENCY_SECS_SCHEMA = Number.class; - - /** - * Should the supervior try to run the worker as the lauching user or not. Defaults to false. - */ - public static final String SUPERVISOR_RUN_WORKER_AS_USER = "supervisor.run.worker.as.user"; - public static final Object SUPERVISOR_RUN_WORKER_AS_USER_SCHEMA = Boolean.class; - - /** - * Full path to the worker-laucher executable that will be used to lauch workers when - * SUPERVISOR_RUN_WORKER_AS_USER is set to true. - */ - public static final String SUPERVISOR_WORKER_LAUNCHER = "supervisor.worker.launcher"; - public static final Object SUPERVISOR_WORKER_LAUNCHER_SCHEMA = String.class; - - /** - * The jvm opts provided to workers launched by this supervisor. All "%ID%" - * substrings are replaced with an identifier for this worker. - */ - public static final String WORKER_CHILDOPTS = "worker.childopts"; - public static final Object WORKER_CHILDOPTS_SCHEMA = String.class; - - /** - * control how many worker receiver threads we need per worker - */ - public static final String WORKER_RECEIVER_THREAD_COUNT = "topology.worker.receiver.thread.count"; - public static final Object WORKER_RECEIVER_THREAD_COUNT_SCHEMA = Number.class; - - /** - * How often this worker should heartbeat to the supervisor. - */ - public static final String WORKER_HEARTBEAT_FREQUENCY_SECS = "worker.heartbeat.frequency.secs"; - public static final Object WORKER_HEARTBEAT_FREQUENCY_SECS_SCHEMA = Number.class; - - /** - * How often a task should heartbeat its status to the master. - */ - public static final String TASK_HEARTBEAT_FREQUENCY_SECS = "task.heartbeat.frequency.secs"; - public static final Object TASK_HEARTBEAT_FREQUENCY_SECS_SCHEMA = Number.class; - - /** - * How often a task should sync its connections with other tasks (if a task - * is reassigned, the other tasks sending messages to it need to refresh - * their connections). In general though, when a reassignment happens other - * tasks will be notified almost immediately. This configuration is here - * just in case that notification doesn't come through. - */ - public static final String TASK_REFRESH_POLL_SECS = "task.refresh.poll.secs"; - public static final Object TASK_REFRESH_POLL_SECS_SCHEMA = Number.class; - - - /** - * A list of users that are allowed to interact with the topology. To use this set - * nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer - */ - public static final String TOPOLOGY_USERS = "topology.users"; - public static final Object TOPOLOGY_USERS_SCHEMA = ConfigValidation.StringsValidator; - - /** - * A list of groups that are allowed to interact with the topology. To use this set - * nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer - */ - public static final String TOPOLOGY_GROUPS = "topology.groups"; - public static final Object TOPOLOGY_GROUPS_SCHEMA = ConfigValidation.StringsValidator; - /** - * True if Storm should timeout messages or not. Defaults to true. This is - * meant to be used in unit tests to prevent tuples from being accidentally - * timed out during the test. - */ - public static final String TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS = "topology.enable.message.timeouts"; - public static final Object TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS_SCHEMA = Boolean.class; - - /** - * When set to true, Storm will log every message that's emitted. - */ - public static final String TOPOLOGY_DEBUG = "topology.debug"; - public static final Object TOPOLOGY_DEBUG_SCHEMA = Boolean.class; - - /** - * Whether or not the master should optimize topologies by running multiple - * tasks in a single thread where appropriate. - */ - public static final String TOPOLOGY_OPTIMIZE = "topology.optimize"; - public static final Object TOPOLOGY_OPTIMIZE_SCHEMA = Boolean.class; - - /** - * How many processes should be spawned around the cluster to execute this - * topology. Each process will execute some number of tasks as threads - * within them. This parameter should be used in conjunction with the - * parallelism hints on each component in the topology to tune the - * performance of a topology. - */ - public static final String TOPOLOGY_WORKERS = "topology.workers"; - public static final Object TOPOLOGY_WORKERS_SCHEMA = Number.class; - - /** - * How many instances to create for a spout/bolt. A task runs on a thread - * with zero or more other tasks for the same spout/bolt. The number of - * tasks for a spout/bolt is always the same throughout the lifetime of a - * topology, but the number of executors (threads) for a spout/bolt can - * change over time. This allows a topology to scale to more or less - * resources without redeploying the topology or violating the constraints - * of Storm (such as a fields grouping guaranteeing that the same value goes - * to the same task). - */ - public static final String TOPOLOGY_TASKS = "topology.tasks"; - public static final Object TOPOLOGY_TASKS_SCHEMA = Number.class; - - /** - * How many executors to spawn for ackers. - * - *

- * If this is set to 0, then Storm will immediately ack tuples as soon as - * they come off the spout, effectively disabling reliability. - *

- */ - public static final String TOPOLOGY_ACKER_EXECUTORS = "topology.acker.executors"; - public static final Object TOPOLOGY_ACKER_EXECUTORS_SCHEMA = Number.class; - - /** - * Add TOPOLOGY_ACKERS is to compatible old storm code - */ - public static final String TOPOLOGY_ACKERS = TOPOLOGY_ACKER_EXECUTORS; - public static final Object TOPOLOGY_ACKER_SCHEMA = Number.class; - - /** - * The maximum amount of time given to the topology to fully process a - * message emitted by a spout. If the message is not acked within this time - * frame, Storm will fail the message on the spout. Some spouts - * implementations will then replay the message at a later time. - */ - public static final String TOPOLOGY_MESSAGE_TIMEOUT_SECS = "topology.message.timeout.secs"; - public static final Object TOPOLOGY_MESSAGE_TIMEOUT_SECS_SCHEMA = Number.class; - - /** - * A list of serialization registrations for Kryo ( - * http://code.google.com/p/kryo/ ), the underlying serialization framework - * for Storm. A serialization can either be the name of a class (in which - * case Kryo will automatically create a serializer for the class that saves - * all the object's fields), or an implementation of - * com.esotericsoftware.kryo.Serializer. - * - * See Kryo's documentation for more information about writing custom - * serializers. - */ - public static final String TOPOLOGY_KRYO_REGISTER = "topology.kryo.register"; - public static final Object TOPOLOGY_KRYO_REGISTER_SCHEMA = ConfigValidation.StringsValidator; - - /** - * A list of classes that customize storm's kryo instance during start-up. - * Each listed class name must implement IKryoDecorator. During start-up the - * listed class is instantiated with 0 arguments, then its 'decorate' method - * is called with storm's kryo instance as the only argument. - */ - public static final String TOPOLOGY_KRYO_DECORATORS = "topology.kryo.decorators"; - public static final Object TOPOLOGY_KRYO_DECORATORS_SCHEMA = ConfigValidation.StringsValidator; - - /** - * Class that specifies how to create a Kryo instance for serialization. - * Storm will then apply topology.kryo.register and topology.kryo.decorators - * on top of this. The default implementation implements - * topology.fall.back.on.java.serialization and turns references off. - */ - public static final String TOPOLOGY_KRYO_FACTORY = "topology.kryo.factory"; - public static final Object TOPOLOGY_KRYO_FACTORY_SCHEMA = String.class; - - /** - * Whether or not Storm should skip the loading of kryo registrations for - * which it does not know the class or have the serializer implementation. - * Otherwise, the task will fail to load and will throw an error at runtime. - * The use case of this is if you want to declare your serializations on the - * storm.yaml files on the cluster rather than every single time you submit - * a topology. Different applications may use different serializations and - * so a single application may not have the code for the other serializers - * used by other apps. By setting this config to true, Storm will ignore - * that it doesn't have those other serializations rather than throw an - * error. - */ - public static final String TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS = "topology.skip.missing.kryo.registrations"; - public static final Object TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS_SCHEMA = Boolean.class; - - /* - * A list of classes implementing IMetricsConsumer (See storm.yaml.example - * for exact config format). Each listed class will be routed all the - * metrics data generated by the storm metrics API. Each listed class maps - * 1:1 to a system bolt named __metrics_ClassName#N, and it's parallelism is - * configurable. - */ - public static final String TOPOLOGY_METRICS_CONSUMER_REGISTER = "topology.metrics.consumer.register"; - public static final Object TOPOLOGY_METRICS_CONSUMER_REGISTER_SCHEMA = ConfigValidation.MapsValidator; - - /** - * The maximum parallelism allowed for a component in this topology. This - * configuration is typically used in testing to limit the number of threads - * spawned in local mode. - */ - public static final String TOPOLOGY_MAX_TASK_PARALLELISM = "topology.max.task.parallelism"; - public static final Object TOPOLOGY_MAX_TASK_PARALLELISM_SCHEMA = Number.class; - - /** - * The maximum number of tuples that can be pending on a spout task at any - * given time. This config applies to individual tasks, not to spouts or - * topologies as a whole. - * - * A pending tuple is one that has been emitted from a spout but has not - * been acked or failed yet. Note that this config parameter has no effect - * for unreliable spouts that don't tag their tuples with a message id. - */ - public static final String TOPOLOGY_MAX_SPOUT_PENDING = "topology.max.spout.pending"; - public static final Object TOPOLOGY_MAX_SPOUT_PENDING_SCHEMA = Number.class; - - /** - * A class that implements a strategy for what to do when a spout needs to - * wait. Waiting is triggered in one of two conditions: - * - * 1. nextTuple emits no tuples 2. The spout has hit maxSpoutPending and - * can't emit any more tuples - */ - public static final String TOPOLOGY_SPOUT_WAIT_STRATEGY = "topology.spout.wait.strategy"; - public static final Object TOPOLOGY_SPOUT_WAIT_STRATEGY_SCHEMA = String.class; - - /** - * The amount of milliseconds the SleepEmptyEmitStrategy should sleep for. - */ - public static final String TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS = "topology.sleep.spout.wait.strategy.time.ms"; - public static final Object TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS_SCHEMA = Number.class; - - /** - * The maximum amount of time a component gives a source of state to - * synchronize before it requests synchronization again. - */ - public static final String TOPOLOGY_STATE_SYNCHRONIZATION_TIMEOUT_SECS = "topology.state.synchronization.timeout.secs"; - public static final Object TOPOLOGY_STATE_SYNCHRONIZATION_TIMEOUT_SECS_SCHEMA = Number.class; - - /** - * The percentage of tuples to sample to produce stats for a task. - */ - public static final String TOPOLOGY_STATS_SAMPLE_RATE = "topology.stats.sample.rate"; - public static final Object TOPOLOGY_STATS_SAMPLE_RATE_SCHEMA = Number.class; - - /** - * The time period that builtin metrics data in bucketed into. - */ - public static final String TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS = "topology.builtin.metrics.bucket.size.secs"; - public static final Object TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS_SCHEMA = Number.class; - - /** - * Whether or not to use Java serialization in a topology. - */ - public static final String TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION = "topology.fall.back.on.java.serialization"; - public static final Object TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION_SCHEMA = Boolean.class; - - /** - * Topology-specific options for the worker child process. This is used in - * addition to WORKER_CHILDOPTS. - */ - public static final String TOPOLOGY_WORKER_CHILDOPTS = "topology.worker.childopts"; - public static final Object TOPOLOGY_WORKER_CHILDOPTS_SCHEMA = String.class; - - /** - * Topology-specific environment variables for the worker child process. - * This is added to the existing environment (that of the supervisor) - */ - public static final String TOPOLOGY_ENVIRONMENT="topology.environment"; - public static final Object TOPOLOGY_ENVIRONMENT_SCHEMA = Map.class; - - /* - * Topology-specific option to disable/enable bolt's outgoing overflow buffer. - * Enabling this option ensures that the bolt can always clear the incoming messages, - * preventing live-lock for the topology with cyclic flow. - * The overflow buffer can fill degrading the performance gradually, - * eventually running out of memory. - */ - public static final String TOPOLOGY_BOLTS_OUTGOING_OVERFLOW_BUFFER_ENABLE="topology.bolts.outgoing.overflow.buffer.enable"; - public static final Object TOPOLOGY_BOLTS_OUTGOING_OVERFLOW_BUFFER_ENABLE_SCHEMA = Boolean.class; - - /** - * This config is available for TransactionalSpouts, and contains the id ( a - * String) for the transactional topology. This id is used to store the - * state of the transactional topology in Zookeeper. - */ - public static final String TOPOLOGY_TRANSACTIONAL_ID = "topology.transactional.id"; - public static final Object TOPOLOGY_TRANSACTIONAL_ID_SCHEMA = String.class; - - /** - * A list of task hooks that are automatically added to every spout and bolt - * in the topology. An example of when you'd do this is to add a hook that - * integrates with your internal monitoring system. These hooks are - * instantiated using the zero-arg constructor. - */ - public static final String TOPOLOGY_AUTO_TASK_HOOKS = "topology.auto.task.hooks"; - public static final Object TOPOLOGY_AUTO_TASK_HOOKS_SCHEMA = ConfigValidation.StringsValidator; - - /** - * The size of the Disruptor receive queue for each executor. Must be a - * power of 2. - */ - public static final String TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE = "topology.executor.receive.buffer.size"; - public static final Object TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE_SCHEMA = ConfigValidation.PowerOf2Validator; - - /** - * The maximum number of messages to batch from the thread receiving off the - * network to the executor queues. Must be a power of 2. - */ - public static final String TOPOLOGY_RECEIVER_BUFFER_SIZE = "topology.receiver.buffer.size"; - public static final Object TOPOLOGY_RECEIVER_BUFFER_SIZE_SCHEMA = ConfigValidation.PowerOf2Validator; - - /** - * The size of the Disruptor send queue for each executor. Must be a power - * of 2. - */ - public static final String TOPOLOGY_EXECUTOR_SEND_BUFFER_SIZE = "topology.executor.send.buffer.size"; - public static final Object TOPOLOGY_EXECUTOR_SEND_BUFFER_SIZE_SCHEMA = ConfigValidation.PowerOf2Validator; - - /** - * The size of the Disruptor transfer queue for each worker. - */ - public static final String TOPOLOGY_TRANSFER_BUFFER_SIZE = "topology.transfer.buffer.size"; - public static final Object TOPOLOGY_TRANSFER_BUFFER_SIZE_SCHEMA = Number.class; - - /** - * How often a tick tuple from the "__system" component and "__tick" stream - * should be sent to tasks. Meant to be used as a component-specific - * configuration. - */ - public static final String TOPOLOGY_TICK_TUPLE_FREQ_SECS = "topology.tick.tuple.freq.secs"; - public static final Object TOPOLOGY_TICK_TUPLE_FREQ_SECS_SCHEMA = Number.class; - - /** - * Configure the wait strategy used for internal queuing. Can be used to - * tradeoff latency vs. throughput - */ - public static final String TOPOLOGY_DISRUPTOR_WAIT_STRATEGY = "topology.disruptor.wait.strategy"; - public static final Object TOPOLOGY_DISRUPTOR_WAIT_STRATEGY_SCHEMA = String.class; - - /** - * The size of the shared thread pool for worker tasks to make use of. The - * thread pool can be accessed via the TopologyContext. - */ - public static final String TOPOLOGY_WORKER_SHARED_THREAD_POOL_SIZE = "topology.worker.shared.thread.pool.size"; - public static final Object TOPOLOGY_WORKER_SHARED_THREAD_POOL_SIZE_SCHEMA = Number.class; - - /** - * The interval in seconds to use for determining whether to throttle error - * reported to Zookeeper. For example, an interval of 10 seconds with - * topology.max.error.report.per.interval set to 5 will only allow 5 errors - * to be reported to Zookeeper per task for every 10 second interval of - * time. - */ - public static final String TOPOLOGY_ERROR_THROTTLE_INTERVAL_SECS = "topology.error.throttle.interval.secs"; - public static final Object TOPOLOGY_ERROR_THROTTLE_INTERVAL_SECS_SCHEMA = Number.class; - - /** - * See doc for TOPOLOGY_ERROR_THROTTLE_INTERVAL_SECS - */ - public static final String TOPOLOGY_MAX_ERROR_REPORT_PER_INTERVAL = "topology.max.error.report.per.interval"; - public static final Object TOPOLOGY_MAX_ERROR_REPORT_PER_INTERVAL_SCHEMA = Number.class; - - /** - * How often a batch can be emitted in a Trident topology. - */ - public static final String TOPOLOGY_TRIDENT_BATCH_EMIT_INTERVAL_MILLIS = "topology.trident.batch.emit.interval.millis"; - public static final Object TOPOLOGY_TRIDENT_BATCH_EMIT_INTERVAL_MILLIS_SCHEMA = Number.class; - - /** - * Name of the topology. This config is automatically set by Storm when the - * topology is submitted. - */ - public static final String TOPOLOGY_NAME = "topology.name"; - public static final Object TOPOLOGY_NAME_SCHEMA = String.class; - - /** - * The principal who submitted a topology - */ - public final static String TOPOLOGY_SUBMITTER_PRINCIPAL = "topology.submitter.principal"; - public static final Object TOPOLOGY_SUBMITTER_PRINCIPAL_SCHEMA = String.class; - - /** - * The local user name of the user who submitted a topology. - */ - public static final String TOPOLOGY_SUBMITTER_USER = "topology.submitter.user"; - public static final Object TOPOLOGY_SUBMITTER_USER_SCHEMA = String.class; - - /** - * Array of components that scheduler should try to place on separate hosts. - */ - public static final String TOPOLOGY_SPREAD_COMPONENTS = "topology.spread.components"; - public static final Object TOPOLOGY_SPREAD_COMPONENTS_SCHEMA = ConfigValidation.StringsValidator; - - /** - * A list of IAutoCredentials that the topology should load and use. - */ - public static final String TOPOLOGY_AUTO_CREDENTIALS = "topology.auto-credentials"; - public static final Object TOPOLOGY_AUTO_CREDENTIALS_SCHEMA = ConfigValidation.StringsValidator; - - /** - * Max pending tuples in one ShellBolt - */ - public static final String TOPOLOGY_SHELLBOLT_MAX_PENDING="topology.shellbolt.max.pending"; - public static final Object TOPOLOGY_SHELLBOLT_MAX_PENDING_SCHEMA = Number.class; - - /** - * The root directory in ZooKeeper for metadata about TransactionalSpouts. - */ - public static final String TRANSACTIONAL_ZOOKEEPER_ROOT = "transactional.zookeeper.root"; - public static final Object TRANSACTIONAL_ZOOKEEPER_ROOT_SCHEMA = String.class; - - /** - * The list of zookeeper servers in which to keep the transactional state. - * If null (which is default), will use storm.zookeeper.servers - */ - public static final String TRANSACTIONAL_ZOOKEEPER_SERVERS = "transactional.zookeeper.servers"; - public static final Object TRANSACTIONAL_ZOOKEEPER_SERVERS_SCHEMA = ConfigValidation.StringsValidator; - - /** - * The port to use to connect to the transactional zookeeper servers. If - * null (which is default), will use storm.zookeeper.port - */ - public static final String TRANSACTIONAL_ZOOKEEPER_PORT = "transactional.zookeeper.port"; - public static final Object TRANSACTIONAL_ZOOKEEPER_PORT_SCHEMA = Number.class; - - /** - * The number of threads that should be used by the zeromq context in each - * worker process. - */ - public static final String ZMQ_THREADS = "zmq.threads"; - public static final Object ZMQ_THREADS_SCHEMA = Number.class; - - /** - * How long a connection should retry sending messages to a target host when - * the connection is closed. This is an advanced configuration and can - * almost certainly be ignored. - */ - public static final String ZMQ_LINGER_MILLIS = "zmq.linger.millis"; - public static final Object ZMQ_LINGER_MILLIS_SCHEMA = Number.class; - - /** - * The high water for the ZeroMQ push sockets used for networking. Use this - * config to prevent buffer explosion on the networking layer. - */ - public static final String ZMQ_HWM = "zmq.hwm"; - public static final Object ZMQ_HWM_SCHEMA = Number.class; - - /** - * This value is passed to spawned JVMs (e.g., Nimbus, Supervisor, and - * Workers) for the java.library.path value. java.library.path tells the JVM - * where to look for native libraries. It is necessary to set this config - * correctly since Storm uses the ZeroMQ and JZMQ native libs. - */ - public static final String JAVA_LIBRARY_PATH = "java.library.path"; - public static final Object JAVA_LIBRARY_PATH_SCHEMA = String.class; - - /** - * The path to use as the zookeeper dir when running a zookeeper server via - * "storm dev-zookeeper". This zookeeper instance is only intended for - * development; it is not a production grade zookeeper setup. - */ - public static final String DEV_ZOOKEEPER_PATH = "dev.zookeeper.path"; - public static final Object DEV_ZOOKEEPER_PATH_SCHEMA = String.class; - - /** - * A map from topology name to the number of machines that should be - * dedicated for that topology. Set storm.scheduler to - * backtype.storm.scheduler.IsolationScheduler to make use of the isolation - * scheduler. - */ - public static final String ISOLATION_SCHEDULER_MACHINES = "isolation.scheduler.machines"; - public static final Object ISOLATION_SCHEDULER_MACHINES_SCHEMA = List.class; - - /** - * A map from the user name to the number of machines that should that user is allowed to use. Set storm.scheduler - * to backtype.storm.scheduler.multitenant.MultitenantScheduler - */ - public static final String MULTITENANT_SCHEDULER_USER_POOLS = "multitenant.scheduler.user.pools"; - public static final Object MULTITENANT_SCHEDULER_USER_POOLS_SCHEMA = Map.class; - - /** - * The number of machines that should be used by this topology to isolate it from all others. Set storm.scheduler - * to backtype.storm.scheduler.multitenant.MultitenantScheduler - */ - public static final String TOPOLOGY_ISOLATED_MACHINES = "topology.isolate.machines"; - public static final Object TOPOLOGY_ISOLATED_MACHINES_SCHEMA = Number.class; - - public static final String WORKER_CLASSPATH = "worker.classpath"; - public static final Object WORKER_CLASSPATH_SCHEMA = String.class; - - /** - * The serializer for communication between shell components and non-JVM - * processes - */ - public static final String TOPOLOGY_MULTILANG_SERIALIZER = "topology.multilang.serializer"; - public static final Object TOPOLOGY_MULTILANG_SERIALIZER_SCHEMA = String.class; - - /** - * HDFS information, used to get the delegation token on behalf of the topology - * submitter user and renew the tokens. see {@link backtype.storm.security.auth.hadoop.AutoHDFS} - * kerberos principal name with realm should be provided. - */ - public static final Object TOPOLOGY_HDFS_PRINCIPAL = "topology.hdfs.user"; - public static final Object TOPOLOGY_HDFS_PRINCIPAL_SCHEMA = String.class; - - /** - * The HDFS URI to be used by AutoHDFS.java to grab the delegation token on topology - * submitter user's behalf by the nimbus. If this is not provided the default URI provided - * in the hdfs configuration files will be used. - */ - public static final Object TOPOLOGY_HDFS_URI = "topology.hdfs.uri"; - public static final Object TOPOLOGY_HDFS_URI_SCHEMA = String.class; - - public static void setDebug(Map conf, boolean isOn) { - conf.put(Config.TOPOLOGY_DEBUG, isOn); - } - - public void setDebug(boolean isOn) { - setDebug(this, isOn); - } - - @Deprecated - public void setOptimize(boolean isOn) { - put(Config.TOPOLOGY_OPTIMIZE, isOn); - } - - public static void setNumWorkers(Map conf, int workers) { - conf.put(Config.TOPOLOGY_WORKERS, workers); - } - - public void setNumWorkers(int workers) { - setNumWorkers(this, workers); - } - - public static void setNumAckers(Map conf, int numExecutors) { - conf.put(Config.TOPOLOGY_ACKER_EXECUTORS, numExecutors); - } - - public void setNumAckers(int numExecutors) { - setNumAckers(this, numExecutors); - } - - public static void setMessageTimeoutSecs(Map conf, int secs) { - conf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, secs); - } - - public void setMessageTimeoutSecs(int secs) { - setMessageTimeoutSecs(this, secs); - } - - public static void registerSerialization(Map conf, Class klass) { - getRegisteredSerializations(conf).add(klass.getName()); - } - - public void registerSerialization(Class klass) { - registerSerialization(this, klass); - } - - public static void registerSerialization(Map conf, Class klass, - Class serializerClass) { - Map register = new HashMap(); - register.put(klass.getName(), serializerClass.getName()); - getRegisteredSerializations(conf).add(register); - } - - public void registerSerialization(Class klass, - Class serializerClass) { - registerSerialization(this, klass, serializerClass); - } - - public static void registerMetricsConsumer(Map conf, Class klass, Object argument, long parallelismHint) { - HashMap m = new HashMap(); - m.put("class", klass.getCanonicalName()); - m.put("parallelism.hint", parallelismHint); - m.put("argument", argument); - - List l = (List)conf.get(TOPOLOGY_METRICS_CONSUMER_REGISTER); - if (l == null) { l = new ArrayList(); } - l.add(m); - conf.put(TOPOLOGY_METRICS_CONSUMER_REGISTER, l); - } - - public void registerMetricsConsumer(Class klass, Object argument, long parallelismHint) { - registerMetricsConsumer(this, klass, argument, parallelismHint); - } - - public static void registerMetricsConsumer(Map conf, Class klass, long parallelismHint) { - registerMetricsConsumer(conf, klass, null, parallelismHint); - } - - public void registerMetricsConsumer(Class klass, long parallelismHint) { - registerMetricsConsumer(this, klass, parallelismHint); - } - - public static void registerMetricsConsumer(Map conf, Class klass) { - registerMetricsConsumer(conf, klass, null, 1L); - } - - public void registerMetricsConsumer(Class klass) { - registerMetricsConsumer(this, klass); - } - - public static void registerDecorator(Map conf, - Class klass) { - getRegisteredDecorators(conf).add(klass.getName()); - } - - public void registerDecorator(Class klass) { - registerDecorator(this, klass); - } - - public static void setKryoFactory(Map conf, - Class klass) { - conf.put(Config.TOPOLOGY_KRYO_FACTORY, klass.getName()); - } - - public void setKryoFactory(Class klass) { - setKryoFactory(this, klass); - } - - public static void setSkipMissingKryoRegistrations(Map conf, boolean skip) { - conf.put(Config.TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS, skip); - } - - public void setSkipMissingKryoRegistrations(boolean skip) { - setSkipMissingKryoRegistrations(this, skip); - } - - public static void setMaxTaskParallelism(Map conf, int max) { - conf.put(Config.TOPOLOGY_MAX_TASK_PARALLELISM, max); - } - - public void setMaxTaskParallelism(int max) { - setMaxTaskParallelism(this, max); - } - - public static void setMaxSpoutPending(Map conf, int max) { - conf.put(Config.TOPOLOGY_MAX_SPOUT_PENDING, max); - } - - public void setMaxSpoutPending(int max) { - setMaxSpoutPending(this, max); - } - - public static void setStatsSampleRate(Map conf, double rate) { - conf.put(Config.TOPOLOGY_STATS_SAMPLE_RATE, rate); - } - - public void setStatsSampleRate(double rate) { - setStatsSampleRate(this, rate); - } - - public static void setFallBackOnJavaSerialization(Map conf, boolean fallback) { - conf.put(Config.TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION, fallback); - } - - public void setFallBackOnJavaSerialization(boolean fallback) { - setFallBackOnJavaSerialization(this, fallback); - } - - private static List getRegisteredSerializations(Map conf) { - List ret; - if (!conf.containsKey(Config.TOPOLOGY_KRYO_REGISTER)) { - ret = new ArrayList(); - } else { - ret = new ArrayList((List) conf.get(Config.TOPOLOGY_KRYO_REGISTER)); - } - conf.put(Config.TOPOLOGY_KRYO_REGISTER, ret); - return ret; - } - - private static List getRegisteredDecorators(Map conf) { - List ret; - if (!conf.containsKey(Config.TOPOLOGY_KRYO_DECORATORS)) { - ret = new ArrayList(); - } else { - ret = new ArrayList( - (List) conf.get(Config.TOPOLOGY_KRYO_DECORATORS)); - } - conf.put(Config.TOPOLOGY_KRYO_DECORATORS, ret); - return ret; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/ConfigValidation.java b/jstorm-client/src/main/java/backtype/storm/ConfigValidation.java deleted file mode 100644 index 0825eb966..000000000 --- a/jstorm-client/src/main/java/backtype/storm/ConfigValidation.java +++ /dev/null @@ -1,100 +0,0 @@ -package backtype.storm; - -import java.util.Map; - -/** - * Provides functionality for validating configuration fields. - */ -public class ConfigValidation { - - /** - * Declares methods for validating configuration values. - */ - public static interface FieldValidator { - /** - * Validates the given field. - * - * @param name - * the name of the field. - * @param field - * The field to be validated. - * @throws IllegalArgumentException - * if the field fails validation. - */ - public void validateField(String name, Object field) - throws IllegalArgumentException; - } - - /** - * Returns a new FieldValidator for a List of the given Class. - * - * @param cls - * the Class of elements composing the list - * @return a FieldValidator for a list of the given class - */ - static FieldValidator FieldListValidatorFactory(final Class cls) { - return new FieldValidator() { - @Override - public void validateField(String name, Object field) - throws IllegalArgumentException { - if (field == null) { - // A null value is acceptable. - return; - } - if (field instanceof Iterable) { - for (Object e : (Iterable) field) { - if (!cls.isInstance(e)) { - throw new IllegalArgumentException( - "Each element of the list " + name - + " must be a " + cls.getName() - + "."); - } - } - return; - } - throw new IllegalArgumentException("Field " + name - + " must be an Iterable of " + cls.getName()); - } - }; - } - - /** - * Validates a list of Numbers. - */ - public static Object NumbersValidator = FieldListValidatorFactory(Number.class); - - /** - * Validates is a list of Strings. - */ - public static Object StringsValidator = FieldListValidatorFactory(String.class); - - /** - * Validates is a list of Maps. - */ - public static Object MapsValidator = FieldListValidatorFactory(Map.class); - - /** - * Validates a power of 2. - */ - public static Object PowerOf2Validator = new FieldValidator() { - @Override - public void validateField(String name, Object o) - throws IllegalArgumentException { - if (o == null) { - // A null value is acceptable. - return; - } - final long i; - if (o instanceof Number - && (i = ((Number) o).longValue()) == ((Number) o) - .doubleValue()) { - // Test whether the integer is a power of 2. - if (i > 0 && (i & (i - 1)) == 0) { - return; - } - } - throw new IllegalArgumentException("Field " + name - + " must be a power of 2."); - } - }; -} diff --git a/jstorm-client/src/main/java/backtype/storm/Constants.java b/jstorm-client/src/main/java/backtype/storm/Constants.java deleted file mode 100644 index b657ee623..000000000 --- a/jstorm-client/src/main/java/backtype/storm/Constants.java +++ /dev/null @@ -1,20 +0,0 @@ -package backtype.storm; - -import backtype.storm.coordination.CoordinatedBolt; -import clojure.lang.RT; - -public class Constants { - public static final String COORDINATED_STREAM_ID = CoordinatedBolt.class - .getName() + "/coord-stream"; - - public static final long SYSTEM_TASK_ID = -1; - public static final Object SYSTEM_EXECUTOR_ID = RT.readString("[-1 -1]"); - public static final String SYSTEM_COMPONENT_ID = "__system"; - public static final String SYSTEM_TICK_STREAM_ID = "__tick"; - public static final String METRICS_COMPONENT_ID_PREFIX = "__metrics"; - public static final String METRICS_STREAM_ID = "__metrics"; - public static final String METRICS_TICK_STREAM_ID = "__metrics_tick"; - public static final String CREDENTIALS_CHANGED_STREAM_ID = "__credentials"; - - public static final String JSTORM_CONF_DIR = "JSTORM_CONF_DIR"; -} diff --git a/jstorm-client/src/main/java/backtype/storm/GenericOptionsParser.java b/jstorm-client/src/main/java/backtype/storm/GenericOptionsParser.java deleted file mode 100644 index 8ca7a1d4c..000000000 --- a/jstorm-client/src/main/java/backtype/storm/GenericOptionsParser.java +++ /dev/null @@ -1,296 +0,0 @@ -package backtype.storm; - -import java.io.File; -import java.io.FileInputStream; -import java.io.FileNotFoundException; -import java.io.InputStreamReader; -import java.io.IOException; -import java.io.PrintStream; -import java.nio.charset.Charset; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.LinkedHashMap; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.commons.cli.CommandLine; -import org.apache.commons.cli.CommandLineParser; -import org.apache.commons.cli.GnuParser; -import org.apache.commons.cli.Option; -import org.apache.commons.cli.OptionBuilder; -import org.apache.commons.cli.Options; -import org.apache.commons.cli.ParseException; -import org.yaml.snakeyaml.Yaml; - -/** - * GenericOptionsParser is a utility to parse command line - * arguments generic to Storm. - * - * GenericOptionsParser recognizes several standard command line - * arguments, enabling applications to easily specify additional jar files, - * configuration resources, data files etc. - * - *

Generic Options

- * - *

- * The supported generic options are: - *

- *

- *

- * - *
- * -conf <conf.xml>                           load configurations from
- *                                            <conf.xml>
- * -conf <conf.yaml>                          load configurations from
- *                                            <conf.yaml>
- * -D <key=value>                             set <key> in configuration to
- *                                            <value> (preserve value's type)
- * -libjars <comma separated list of jars>    specify comma separated jars to be
- *                                            used by the submitted topology
- * 
- * - *
- *

- * - * Note: The XML configuration file specified by -conf shall - * be readable by Hadoop's Configuration class. Also note that all configuration - * values of an XML file will be treated as strings, and not as specific - * types. - * - *

- * The general command line syntax is: - *

- *

- *

- *     storm jar app.jar [genericOptions] [commandOptions]
- * 
- *

- * - *

- * Generic command line arguments might modify - * Config objects, given to constructors. - *

- * - *

Configuration priority

- * - * The following list defines the priorities of different configuration sources, - * in ascending order. Thus, if a configuration appears in more than one of - * them, only the last one will take effect. - * - *
    - *
  • defaults.yaml in classpath. - *
  • storm.yaml in classpath. - *
  • Configurations from files specified with the -conf option, - * in the order of appearance. - *
  • Configurations defined with the -D option, in order of - * appearance. - *
- * - *

- * The functionality is implemented using Commons CLI. - *

- * - * @see Tool - * @see ToolRunner - */ - -public class GenericOptionsParser { - static final Logger LOG = LoggerFactory - .getLogger(GenericOptionsParser.class); - - static final Charset UTF8 = Charset.forName("UTF-8"); - - public static final String TOPOLOGY_LIB_PATH = "topology.lib.path"; - - public static final String TOPOLOGY_LIB_NAME = "topology.lib.name"; - - Config conf; - - CommandLine commandLine; - - // Order in this map is important for these purposes: - // - configuration priority - static final LinkedHashMap optionProcessors = new LinkedHashMap(); - - public GenericOptionsParser(Config conf, String[] args) - throws ParseException { - this(conf, new Options(), args); - } - - public GenericOptionsParser(Config conf, Options options, String[] args) - throws ParseException { - this.conf = conf; - parseGeneralOptions(options, conf, args); - } - - public String[] getRemainingArgs() { - return commandLine.getArgs(); - } - - public Config getConfiguration() { - return conf; - } - - static Options buildGeneralOptions(Options opts) { - Options r = new Options(); - - for (Object o : opts.getOptions()) - r.addOption((Option) o); - - Option libjars = OptionBuilder - .withArgName("paths") - .hasArg() - .withDescription( - "comma separated jars to be used by the submitted topology") - .create("libjars"); - r.addOption(libjars); - optionProcessors.put("libjars", new LibjarsProcessor()); - - Option conf = OptionBuilder.withArgName("configuration file").hasArg() - .withDescription("an application configuration file") - .create("conf"); - r.addOption(conf); - optionProcessors.put("conf", new ConfFileProcessor()); - - // Must come after `conf': this option is of higher priority - Option extraConfig = OptionBuilder.withArgName("D").hasArg() - .withDescription("extra configurations (preserving types)") - .create("D"); - r.addOption(extraConfig); - optionProcessors.put("D", new ExtraConfigProcessor()); - - return r; - } - - void parseGeneralOptions(Options opts, Config conf, String[] args) - throws ParseException { - opts = buildGeneralOptions(opts); - CommandLineParser parser = new GnuParser(); - commandLine = parser.parse(opts, args, true); - processGeneralOptions(conf, commandLine); - } - - void processGeneralOptions(Config conf, CommandLine commandLine) - throws ParseException { - for (Map.Entry e : optionProcessors.entrySet()) - if (commandLine.hasOption(e.getKey())) - e.getValue().process(conf, commandLine); - } - - static List validateFiles(String pathList) throws IOException { - List l = new ArrayList(); - - for (String s : pathList.split(",")) { - File file = new File(s); - if (!file.exists()) - throw new FileNotFoundException("File `" - + file.getAbsolutePath() + "' does not exist"); - - l.add(file); - } - - return l; - } - - public static void printGenericCommandUsage(PrintStream out) { - String[] strs = new String[] { - "Generic options supported are", - " -conf load configurations from", - " ", - " -conf load configurations from", - " ", - " -D = set in configuration", - " to (preserve value's type)", - " -libjars specify comma separated", - " jars to be used by", - " the submitted topology", }; - for (String s : strs) - out.println(s); - } - - static interface OptionProcessor { - public void process(Config conf, CommandLine commandLine) - throws ParseException; - } - - static class LibjarsProcessor implements OptionProcessor { - @Override - public void process(Config conf, CommandLine commandLine) - throws ParseException { - try { - List jarFiles = validateFiles(commandLine - .getOptionValue("libjars")); - Map jars = new HashMap(jarFiles.size()); - List names = new ArrayList(jarFiles.size()); - for (File f : jarFiles) { - jars.put(f.getName(), f.getAbsolutePath()); - names.add(f.getName()); - } - conf.put(TOPOLOGY_LIB_PATH, jars); - conf.put(TOPOLOGY_LIB_NAME, names); - - } catch (IOException e) { - throw new ParseException(e.getMessage()); - } - } - } - - static class ExtraConfigProcessor implements OptionProcessor { - static final Yaml yaml = new Yaml(); - - @Override - public void process(Config conf, CommandLine commandLine) - throws ParseException { - for (String s : commandLine.getOptionValues("D")) { - String[] keyval = s.split("=", 2); - if (keyval.length != 2) - throw new ParseException("Invalid option value `" + s + "'"); - - conf.putAll((Map) yaml.load(keyval[0] + ": " + keyval[1])); - } - } - } - - static class ConfFileProcessor implements OptionProcessor { - static final Yaml yaml = new Yaml(); - - static Map loadYamlConf(String f) throws IOException { - InputStreamReader reader = null; - try { - FileInputStream fis = new FileInputStream(f); - reader = new InputStreamReader(fis, UTF8); - return (Map) yaml.load(reader); - } finally { - if (reader != null) - reader.close(); - } - } - - static Map loadConf(String f) throws IOException { - if (f.endsWith(".yaml")) - return loadYamlConf(f); - throw new IOException("Unknown configuration file type: " + f - + " does not end with either .yaml"); - } - - @Override - public void process(Config conf, CommandLine commandLine) - throws ParseException { - try { - for (String f : commandLine.getOptionValues("conf")) { - Map m = loadConf(f); - if (m == null) - throw new ParseException("Empty configuration file " - + f); - conf.putAll(m); - } - } catch (IOException e) { - throw new ParseException(e.getMessage()); - } - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/ILocalCluster.java b/jstorm-client/src/main/java/backtype/storm/ILocalCluster.java deleted file mode 100644 index 03d9054d1..000000000 --- a/jstorm-client/src/main/java/backtype/storm/ILocalCluster.java +++ /dev/null @@ -1,46 +0,0 @@ -package backtype.storm; - -import backtype.storm.generated.AlreadyAliveException; -import backtype.storm.generated.ClusterSummary; -import backtype.storm.generated.InvalidTopologyException; -import backtype.storm.generated.KillOptions; -import backtype.storm.generated.SubmitOptions; -import backtype.storm.generated.NotAliveException; -import backtype.storm.generated.RebalanceOptions; -import backtype.storm.generated.StormTopology; -import backtype.storm.generated.TopologyInfo; - -import java.util.Map; - -public interface ILocalCluster { - void submitTopology(String topologyName, Map conf, StormTopology topology) - throws AlreadyAliveException, InvalidTopologyException; - - void submitTopologyWithOpts(String topologyName, Map conf, - StormTopology topology, SubmitOptions submitOpts) - throws AlreadyAliveException, InvalidTopologyException; - - void killTopology(String topologyName) throws NotAliveException; - - void killTopologyWithOpts(String name, KillOptions options) - throws NotAliveException; - - void activate(String topologyName) throws NotAliveException; - - void deactivate(String topologyName) throws NotAliveException; - - void rebalance(String name, RebalanceOptions options) - throws NotAliveException; - - void shutdown(); - - String getTopologyConf(String id); - - StormTopology getTopology(String id); - - ClusterSummary getClusterInfo(); - - TopologyInfo getTopologyInfo(String id); - - Map getState(); -} diff --git a/jstorm-client/src/main/java/backtype/storm/ILocalDRPC.java b/jstorm-client/src/main/java/backtype/storm/ILocalDRPC.java deleted file mode 100644 index 80fc8426d..000000000 --- a/jstorm-client/src/main/java/backtype/storm/ILocalDRPC.java +++ /dev/null @@ -1,10 +0,0 @@ -package backtype.storm; - -import backtype.storm.daemon.Shutdownable; -import backtype.storm.generated.DistributedRPC; -import backtype.storm.generated.DistributedRPCInvocations; - -public interface ILocalDRPC extends DistributedRPC.Iface, - DistributedRPCInvocations.Iface, Shutdownable { - public String getServiceId(); -} diff --git a/jstorm-client/src/main/java/backtype/storm/StormSubmitter.java b/jstorm-client/src/main/java/backtype/storm/StormSubmitter.java deleted file mode 100644 index 25a9b4c0e..000000000 --- a/jstorm-client/src/main/java/backtype/storm/StormSubmitter.java +++ /dev/null @@ -1,375 +0,0 @@ -package backtype.storm; - -import java.io.File; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.thrift7.TException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import backtype.storm.generated.AlreadyAliveException; -import backtype.storm.generated.ClusterSummary; -import backtype.storm.generated.InvalidTopologyException; -import backtype.storm.generated.Nimbus; -import backtype.storm.generated.StormTopology; -import backtype.storm.generated.SubmitOptions; -import backtype.storm.generated.TopologyAssignException; -import backtype.storm.generated.TopologySummary; -import backtype.storm.utils.BufferFileInputStream; -import backtype.storm.utils.NimbusClient; -import backtype.storm.utils.Utils; - -/** - * Use this class to submit topologies to run on the Storm cluster. You should - * run your program with the "storm jar" command from the command-line, and then - * use this class to submit your topologies. - */ -public class StormSubmitter { - public static Logger LOG = LoggerFactory.getLogger(StormSubmitter.class); - - private static Nimbus.Iface localNimbus = null; - - public static void setLocalNimbus(Nimbus.Iface localNimbusHandler) { - StormSubmitter.localNimbus = localNimbusHandler; - } - - /** - * Submits a topology to run on the cluster. A topology runs forever or - * until explicitly killed. - * - * - * @param name - * the name of the storm. - * @param stormConf - * the topology-specific configuration. See {@link Config}. - * @param topology - * the processing to execute. - * @throws AlreadyAliveException - * if a topology with this name is already running - * @throws InvalidTopologyException - * if an invalid topology was submitted - */ - public static void submitTopology(String name, Map stormConf, - StormTopology topology) throws AlreadyAliveException, - InvalidTopologyException { - submitTopology(name, stormConf, topology, null); - } - - public static void submitTopology(String name, Map stormConf, - StormTopology topology, SubmitOptions opts, List jarFiles) - throws AlreadyAliveException, InvalidTopologyException { - if (jarFiles == null) { - jarFiles = new ArrayList(); - } - Map jars = new HashMap(jarFiles.size()); - List names = new ArrayList(jarFiles.size()); - - for (File f : jarFiles) { - if (!f.exists()) { - LOG.info(f.getName() + " is not existed: " - + f.getAbsolutePath()); - continue; - } - jars.put(f.getName(), f.getAbsolutePath()); - names.add(f.getName()); - } - LOG.info("Files: " + names + " will be loaded"); - stormConf.put(GenericOptionsParser.TOPOLOGY_LIB_PATH, jars); - stormConf.put(GenericOptionsParser.TOPOLOGY_LIB_NAME, names); - submitTopology(name, stormConf, topology, opts); - } - - public static void submitTopology(String name, Map stormConf, - StormTopology topology, SubmitOptions opts, - ProgressListener listener) throws AlreadyAliveException, - InvalidTopologyException { - submitTopology(name, stormConf, topology, opts); - } - - /** - * Submits a topology to run on the cluster. A topology runs forever or - * until explicitly killed. - * - * - * @param name - * the name of the storm. - * @param stormConf - * the topology-specific configuration. See {@link Config}. - * @param topology - * the processing to execute. - * @param options - * to manipulate the starting of the topology - * @throws AlreadyAliveException - * if a topology with this name is already running - * @throws InvalidTopologyException - * if an invalid topology was submitted - */ - public static void submitTopology(String name, Map stormConf, - StormTopology topology, SubmitOptions opts) - throws AlreadyAliveException, InvalidTopologyException { - if (!Utils.isValidConf(stormConf)) { - throw new IllegalArgumentException( - "Storm conf is not valid. Must be json-serializable"); - } - stormConf = new HashMap(stormConf); - stormConf.putAll(Utils.readCommandLineOpts()); - Map conf = Utils.readStormConfig(); - conf.putAll(stormConf); - putUserInfo(conf, stormConf); - try { - String serConf = Utils.to_json(stormConf); - if (localNimbus != null) { - LOG.info("Submitting topology " + name + " in local mode"); - localNimbus.submitTopology(name, null, serConf, topology); - } else { - NimbusClient client = NimbusClient.getConfiguredClient(conf); - if (topologyNameExists(conf, name)) { - throw new RuntimeException("Topology with name `" + name - + "` already exists on cluster"); - } - - submitJar(conf); - try { - LOG.info("Submitting topology " + name - + " in distributed mode with conf " + serConf); - if (opts != null) { - client.getClient().submitTopologyWithOpts(name, path, - serConf, topology, opts); - } else { - // this is for backwards compatibility - client.getClient().submitTopology(name, path, serConf, - topology); - } - } finally { - client.close(); - } - } - LOG.info("Finished submitting topology: " + name); - } catch (InvalidTopologyException e) { - LOG.warn("Topology submission exception", e); - throw e; - } catch (AlreadyAliveException e) { - LOG.warn("Topology already alive exception", e); - throw e; - } catch (TopologyAssignException e) { - LOG.warn("Failed to assign " + e.get_msg(), e); - throw new RuntimeException(e); - } catch (TException e) { - LOG.warn("Failed to assign ", e); - throw new RuntimeException(e); - } - } - - /** - * Submits a topology to run on the cluster with a progress bar. A topology - * runs forever or until explicitly killed. - * - * - * @param name - * the name of the storm. - * @param stormConf - * the topology-specific configuration. See {@link Config}. - * @param topology - * the processing to execute. - * @throws AlreadyAliveException - * if a topology with this name is already running - * @throws InvalidTopologyException - * if an invalid topology was submitted - * @throws TopologyAssignException - */ - - public static void submitTopologyWithProgressBar(String name, - Map stormConf, StormTopology topology) - throws AlreadyAliveException, InvalidTopologyException { - submitTopologyWithProgressBar(name, stormConf, topology, null); - } - - /** - * Submits a topology to run on the cluster with a progress bar. A topology - * runs forever or until explicitly killed. - * - * - * @param name - * the name of the storm. - * @param stormConf - * the topology-specific configuration. See {@link Config}. - * @param topology - * the processing to execute. - * @param opts - * to manipulate the starting of the topology - * @throws AlreadyAliveException - * if a topology with this name is already running - * @throws InvalidTopologyException - * if an invalid topology was submitted - * @throws TopologyAssignException - */ - - public static void submitTopologyWithProgressBar(String name, - Map stormConf, StormTopology topology, SubmitOptions opts) - throws AlreadyAliveException, InvalidTopologyException { - - /** - * remove progress bar in jstorm - */ - submitTopology(name, stormConf, topology, opts); - } - - private static boolean topologyNameExists(Map conf, String name) { - NimbusClient client = NimbusClient.getConfiguredClient(conf); - try { - ClusterSummary summary = client.getClient().getClusterInfo(); - for (TopologySummary s : summary.get_topologies()) { - if (s.get_name().equals(name)) { - return true; - } - } - return false; - - } catch (Exception e) { - throw new RuntimeException(e); - } finally { - client.close(); - } - } - - private static String submittedJar = null; - private static String path = null; - - private static void submitJar(Map conf) { - if (submittedJar == null) { - NimbusClient client = NimbusClient.getConfiguredClient(conf); - try { - LOG.info("Jar not uploaded to master yet. Submitting jar..."); - String localJar = System.getProperty("storm.jar"); - path = client.getClient().beginFileUpload(); - String[] pathCache = path.split("/"); - String uploadLocation = path + "/stormjar-" - + pathCache[pathCache.length - 1] + ".jar"; - List lib = (List) conf - .get(GenericOptionsParser.TOPOLOGY_LIB_NAME); - Map libPath = (Map) conf - .get(GenericOptionsParser.TOPOLOGY_LIB_PATH); - if (lib != null && lib.size() != 0) { - for (String libName : lib) { - String jarPath = path + "/lib/" + libName; - client.getClient().beginLibUpload(jarPath); - submitJar(conf, libPath.get(libName), jarPath, client); - } - - } else { - if (localJar == null) { - // no lib, no client jar - throw new RuntimeException("No client app jar, please upload it"); - } - } - - if (localJar != null) { - submittedJar = submitJar(conf, localJar, - uploadLocation, client); - }else { - // no client jar, but with lib jar - client.getClient().finishFileUpload(uploadLocation); - } - } catch (Exception e) { - throw new RuntimeException(e); - } finally { - client.close(); - } - } else { - LOG.info("Jar already uploaded to master. Not submitting jar."); - } - } - - public static String submitJar(Map conf, String localJar, - String uploadLocation, NimbusClient client) { - if (localJar == null) { - throw new RuntimeException( - "Must submit topologies using the 'storm' client script so that StormSubmitter knows which jar to upload."); - } - - try { - - LOG.info("Uploading topology jar " + localJar - + " to assigned location: " + uploadLocation); - int bufferSize = 512 * 1024; - Object maxBufSizeObject = conf - .get(Config.NIMBUS_THRIFT_MAX_BUFFER_SIZE); - if (maxBufSizeObject != null) { - bufferSize = Utils.getInt(maxBufSizeObject) / 2; - } - - BufferFileInputStream is = new BufferFileInputStream(localJar, - bufferSize); - while (true) { - byte[] toSubmit = is.read(); - if (toSubmit.length == 0) - break; - client.getClient().uploadChunk(uploadLocation, - ByteBuffer.wrap(toSubmit)); - } - client.getClient().finishFileUpload(uploadLocation); - LOG.info("Successfully uploaded topology jar to assigned location: " - + uploadLocation); - return uploadLocation; - } catch (Exception e) { - throw new RuntimeException(e); - } finally { - - } - } - - private static void putUserInfo(Map conf, Map stormConf) { - stormConf.put("user.group", conf.get("user.group")); - stormConf.put("user.name", conf.get("user.name")); - stormConf.put("user.password", conf.get("user.password")); - } - - /** - * Interface use to track progress of file upload - */ - public interface ProgressListener { - /** - * called before file is uploaded - * - * @param srcFile - * - jar file to be uploaded - * @param targetFile - * - destination file - * @param totalBytes - * - total number of bytes of the file - */ - public void onStart(String srcFile, String targetFile, long totalBytes); - - /** - * called whenever a chunk of bytes is uploaded - * - * @param srcFile - * - jar file to be uploaded - * @param targetFile - * - destination file - * @param bytesUploaded - * - number of bytes transferred so far - * @param totalBytes - * - total number of bytes of the file - */ - public void onProgress(String srcFile, String targetFile, - long bytesUploaded, long totalBytes); - - /** - * called when the file is uploaded - * - * @param srcFile - * - jar file to be uploaded - * @param targetFile - * - destination file - * @param totalBytes - * - total number of bytes of the file - */ - public void onCompleted(String srcFile, String targetFile, - long totalBytes); - } -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/Tool.java b/jstorm-client/src/main/java/backtype/storm/Tool.java deleted file mode 100644 index 5fa7ccc26..000000000 --- a/jstorm-client/src/main/java/backtype/storm/Tool.java +++ /dev/null @@ -1,47 +0,0 @@ -package backtype.storm; - -/** - * A tool abstract class that supports handling of generic - * command-line options. - * - *

Here is how a typical Tool is implemented:

- *

- *     public class TopologyApp extends Tool {
- *         {@literal @}Override
- *         public int run(String[] args) throws Exception {
- *             // Config processed by ToolRunner
- *             Config conf = getConf();
- *
- *             // Other setups go here
- *             String name = "topology";
- *             StormTopology topology = buildTopology(args);
- *             StormSubmitter.submitTopology(name, conf, topology);
- *             return 0;
- *         }
- *
- *         StormTopology buildTopology(String[] args) { ... }
- *
- *         public static void main(String[] args) throws Exception {
- *             // Use ToolRunner to handle generic command-line options
- *             ToolRunner.run(new TopologyApp(), args);
- *         }
- *     }
- * 

- * - * @see GenericOptionsParser - * @see ToolRunner - */ - -public abstract class Tool { - Config config; - - public abstract int run(String[] args) throws Exception; - - public Config getConf() { - return config; - } - - public void setConf(Config config) { - this.config = config; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/ToolRunner.java b/jstorm-client/src/main/java/backtype/storm/ToolRunner.java deleted file mode 100644 index 30940daef..000000000 --- a/jstorm-client/src/main/java/backtype/storm/ToolRunner.java +++ /dev/null @@ -1,53 +0,0 @@ -package backtype.storm; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.commons.cli.ParseException; - -import backtype.storm.utils.Utils; - -/** - * A utility to help run {@link Tool}s - * - *

ToolRunner can be used to run classes extending the - * Tool abstract class. It works in conjunction with - * {@link GenericOptionsParser} to parse the - * generic storm command line arguments and modifies the - * Config of the Tool. The - * application-specific options are passed along without being - * modified. - * - * @see Tool - * @see GenericOptionsParser - */ - -public class ToolRunner { - static final Logger LOG = LoggerFactory.getLogger(ToolRunner.class); - - public static void run(Tool tool, String[] args) { - run(tool.getConf(), tool, args); - } - - public static void run(Config conf, Tool tool, String[] args) { - try { - if (conf == null) { - conf = new Config(); - conf.putAll(Utils.readStormConfig()); - } - - GenericOptionsParser parser = new GenericOptionsParser(conf, args); - tool.setConf(conf); - - System.exit(tool.run(parser.getRemainingArgs())); - } catch (ParseException e) { - LOG.error("Error parsing generic options: {}", e.getMessage()); - GenericOptionsParser.printGenericCommandUsage(System.err); - System.exit(2); - } catch (Exception e) { - LOG.error("Error running tool", e); - System.exit(1); - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/clojure/ClojureBolt.java b/jstorm-client/src/main/java/backtype/storm/clojure/ClojureBolt.java deleted file mode 100644 index bd3873f0b..000000000 --- a/jstorm-client/src/main/java/backtype/storm/clojure/ClojureBolt.java +++ /dev/null @@ -1,108 +0,0 @@ -package backtype.storm.clojure; - -import backtype.storm.coordination.CoordinatedBolt.FinishedCallback; -import backtype.storm.generated.StreamInfo; -import backtype.storm.task.IBolt; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.utils.Utils; -import clojure.lang.IFn; -import clojure.lang.PersistentArrayMap; -import clojure.lang.Keyword; -import clojure.lang.Symbol; -import clojure.lang.RT; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -public class ClojureBolt implements IRichBolt, FinishedCallback { - Map _fields; - List _fnSpec; - List _confSpec; - List _params; - - IBolt _bolt; - - public ClojureBolt(List fnSpec, List confSpec, List params, - Map fields) { - _fnSpec = fnSpec; - _confSpec = confSpec; - _params = params; - _fields = fields; - } - - @Override - public void prepare(final Map stormConf, final TopologyContext context, - final OutputCollector collector) { - IFn hof = Utils.loadClojureFn(_fnSpec.get(0), _fnSpec.get(1)); - try { - IFn preparer = (IFn) hof.applyTo(RT.seq(_params)); - final Map collectorMap = new PersistentArrayMap( - new Object[] { - Keyword.intern(Symbol.create("output-collector")), - collector, - Keyword.intern(Symbol.create("context")), context }); - List args = new ArrayList() { - { - add(stormConf); - add(context); - add(collectorMap); - } - }; - - _bolt = (IBolt) preparer.applyTo(RT.seq(args)); - // this is kind of unnecessary for clojure - try { - _bolt.prepare(stormConf, context, collector); - } catch (AbstractMethodError ame) { - - } - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - @Override - public void execute(Tuple input) { - _bolt.execute(input); - } - - @Override - public void cleanup() { - try { - _bolt.cleanup(); - } catch (AbstractMethodError ame) { - - } - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - for (String stream : _fields.keySet()) { - StreamInfo info = _fields.get(stream); - declarer.declareStream(stream, info.is_direct(), - new Fields(info.get_output_fields())); - } - } - - @Override - public void finishedId(Object id) { - if (_bolt instanceof FinishedCallback) { - ((FinishedCallback) _bolt).finishedId(id); - } - } - - @Override - public Map getComponentConfiguration() { - IFn hof = Utils.loadClojureFn(_confSpec.get(0), _confSpec.get(1)); - try { - return (Map) hof.applyTo(RT.seq(_params)); - } catch (Exception e) { - throw new RuntimeException(e); - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/clojure/ClojureSpout.java b/jstorm-client/src/main/java/backtype/storm/clojure/ClojureSpout.java deleted file mode 100644 index 360625291..000000000 --- a/jstorm-client/src/main/java/backtype/storm/clojure/ClojureSpout.java +++ /dev/null @@ -1,142 +0,0 @@ -package backtype.storm.clojure; - -import backtype.storm.generated.StreamInfo; -import backtype.storm.spout.ISpout; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichSpout; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import backtype.storm.utils.Utils; -import clojure.lang.IFn; -import clojure.lang.PersistentArrayMap; -import clojure.lang.Keyword; -import clojure.lang.Symbol; -import clojure.lang.RT; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -public class ClojureSpout implements IRichSpout { - Map _fields; - List _fnSpec; - List _confSpec; - List _params; - - ISpout _spout; - - public ClojureSpout(List fnSpec, List confSpec, List params, - Map fields) { - _fnSpec = fnSpec; - _confSpec = confSpec; - _params = params; - _fields = fields; - } - - @Override - public void open(final Map conf, final TopologyContext context, - final SpoutOutputCollector collector) { - IFn hof = Utils.loadClojureFn(_fnSpec.get(0), _fnSpec.get(1)); - try { - IFn preparer = (IFn) hof.applyTo(RT.seq(_params)); - final Map collectorMap = new PersistentArrayMap( - new Object[] { - Keyword.intern(Symbol.create("output-collector")), - collector, - Keyword.intern(Symbol.create("context")), context }); - List args = new ArrayList() { - { - add(conf); - add(context); - add(collectorMap); - } - }; - - _spout = (ISpout) preparer.applyTo(RT.seq(args)); - // this is kind of unnecessary for clojure - try { - _spout.open(conf, context, collector); - } catch (AbstractMethodError ame) { - - } - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - @Override - public void close() { - try { - _spout.close(); - } catch (AbstractMethodError ame) { - - } - } - - @Override - public void nextTuple() { - try { - _spout.nextTuple(); - } catch (AbstractMethodError ame) { - - } - - } - - @Override - public void ack(Object msgId) { - try { - _spout.ack(msgId); - } catch (AbstractMethodError ame) { - - } - - } - - @Override - public void fail(Object msgId) { - try { - _spout.fail(msgId); - } catch (AbstractMethodError ame) { - - } - - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - for (String stream : _fields.keySet()) { - StreamInfo info = _fields.get(stream); - declarer.declareStream(stream, info.is_direct(), - new Fields(info.get_output_fields())); - } - } - - @Override - public Map getComponentConfiguration() { - IFn hof = Utils.loadClojureFn(_confSpec.get(0), _confSpec.get(1)); - try { - return (Map) hof.applyTo(RT.seq(_params)); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - @Override - public void activate() { - try { - _spout.activate(); - } catch (AbstractMethodError ame) { - - } - } - - @Override - public void deactivate() { - try { - _spout.deactivate(); - } catch (AbstractMethodError ame) { - - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/clojure/RichShellBolt.java b/jstorm-client/src/main/java/backtype/storm/clojure/RichShellBolt.java deleted file mode 100644 index 92894483c..000000000 --- a/jstorm-client/src/main/java/backtype/storm/clojure/RichShellBolt.java +++ /dev/null @@ -1,36 +0,0 @@ -package backtype.storm.clojure; - -import backtype.storm.generated.StreamInfo; -import backtype.storm.task.ShellBolt; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import java.util.Map; - -public class RichShellBolt extends ShellBolt implements IRichBolt { - private Map _outputs; - - public RichShellBolt(String[] command, Map outputs) { - super(command); - _outputs = outputs; - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - for (String stream : _outputs.keySet()) { - StreamInfo def = _outputs.get(stream); - if (def.is_direct()) { - declarer.declareStream(stream, true, - new Fields(def.get_output_fields())); - } else { - declarer.declareStream(stream, - new Fields(def.get_output_fields())); - } - } - } - - @Override - public Map getComponentConfiguration() { - return null; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/clojure/RichShellSpout.java b/jstorm-client/src/main/java/backtype/storm/clojure/RichShellSpout.java deleted file mode 100644 index 98763a550..000000000 --- a/jstorm-client/src/main/java/backtype/storm/clojure/RichShellSpout.java +++ /dev/null @@ -1,36 +0,0 @@ -package backtype.storm.clojure; - -import backtype.storm.generated.StreamInfo; -import backtype.storm.spout.ShellSpout; -import backtype.storm.topology.IRichSpout; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import java.util.Map; - -public class RichShellSpout extends ShellSpout implements IRichSpout { - private Map _outputs; - - public RichShellSpout(String[] command, Map outputs) { - super(command); - _outputs = outputs; - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - for (String stream : _outputs.keySet()) { - StreamInfo def = _outputs.get(stream); - if (def.is_direct()) { - declarer.declareStream(stream, true, - new Fields(def.get_output_fields())); - } else { - declarer.declareStream(stream, - new Fields(def.get_output_fields())); - } - } - } - - @Override - public Map getComponentConfiguration() { - return null; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/command/activate.java b/jstorm-client/src/main/java/backtype/storm/command/activate.java deleted file mode 100644 index ed8c33a81..000000000 --- a/jstorm-client/src/main/java/backtype/storm/command/activate.java +++ /dev/null @@ -1,49 +0,0 @@ -package backtype.storm.command; - -import java.security.InvalidParameterException; -import java.util.Map; - -import backtype.storm.utils.NimbusClient; -import backtype.storm.utils.Utils; - -/** - * Activate topology - * - * @author longda - * - */ -public class activate { - - /** - * @param args - */ - public static void main(String[] args) { - // TODO Auto-generated method stub - if (args == null || args.length == 0) { - throw new InvalidParameterException("Should input topology name"); - } - - String topologyName = args[0]; - - NimbusClient client = null; - try { - - Map conf = Utils.readStormConfig(); - client = NimbusClient.getConfiguredClient(conf); - - client.getClient().activate(topologyName); - - System.out.println("Successfully submit command activate " - + topologyName); - } catch (Exception e) { - System.out.println(e.getMessage()); - e.printStackTrace(); - throw new RuntimeException(e); - } finally { - if (client != null) { - client.close(); - } - } - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/command/config_value.java b/jstorm-client/src/main/java/backtype/storm/command/config_value.java deleted file mode 100644 index 90a3ed040..000000000 --- a/jstorm-client/src/main/java/backtype/storm/command/config_value.java +++ /dev/null @@ -1,33 +0,0 @@ -package backtype.storm.command; - -import java.security.InvalidParameterException; -import java.util.Map; - -import backtype.storm.utils.NimbusClient; -import backtype.storm.utils.Utils; - -/** - * Get configuration - * - * @author longda - * - */ -public class config_value { - - /** - * @param args - */ - public static void main(String[] args) { - // TODO Auto-generated method stub - if (args == null || args.length == 0) { - throw new InvalidParameterException("Should input key name"); - } - - String key = args[0]; - - Map conf = Utils.readStormConfig(); - - System.out.print("VALUE: " + String.valueOf(conf.get(key))); - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/command/deactivate.java b/jstorm-client/src/main/java/backtype/storm/command/deactivate.java deleted file mode 100644 index 845f4565b..000000000 --- a/jstorm-client/src/main/java/backtype/storm/command/deactivate.java +++ /dev/null @@ -1,49 +0,0 @@ -package backtype.storm.command; - -import java.security.InvalidParameterException; -import java.util.Map; - -import backtype.storm.utils.NimbusClient; -import backtype.storm.utils.Utils; - -/** - * Deactivate topology - * - * @author longda - * - */ -public class deactivate { - - /** - * @param args - */ - public static void main(String[] args) { - // TODO Auto-generated method stub - if (args == null || args.length == 0) { - throw new InvalidParameterException("Should input topology name"); - } - - String topologyName = args[0]; - - NimbusClient client = null; - try { - - Map conf = Utils.readStormConfig(); - client = NimbusClient.getConfiguredClient(conf); - - client.getClient().deactivate(topologyName); - - System.out.println("Successfully submit command deactivate " - + topologyName); - } catch (Exception e) { - System.out.println(e.getMessage()); - e.printStackTrace(); - throw new RuntimeException(e); - } finally { - if (client != null) { - client.close(); - } - } - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/command/kill_topology.java b/jstorm-client/src/main/java/backtype/storm/command/kill_topology.java deleted file mode 100644 index 0c950cbc9..000000000 --- a/jstorm-client/src/main/java/backtype/storm/command/kill_topology.java +++ /dev/null @@ -1,61 +0,0 @@ -package backtype.storm.command; - -import java.security.InvalidParameterException; -import java.util.Map; - -import backtype.storm.generated.KillOptions; -import backtype.storm.utils.NimbusClient; -import backtype.storm.utils.Utils; - -/** - * Kill topology - * - * @author longda - * - */ -public class kill_topology { - - /** - * @param args - */ - public static void main(String[] args) { - // TODO Auto-generated method stub - if (args == null || args.length == 0) { - throw new InvalidParameterException("Should input topology name"); - } - - String topologyName = args[0]; - - NimbusClient client = null; - try { - - Map conf = Utils.readStormConfig(); - client = NimbusClient.getConfiguredClient(conf); - - if (args.length == 1) { - - client.getClient().killTopology(topologyName); - } else { - int delaySeconds = Integer.parseInt(args[1]); - - KillOptions options = new KillOptions(); - options.set_wait_secs(delaySeconds); - - client.getClient().killTopologyWithOpts(topologyName, options); - - } - - System.out.println("Successfully submit command kill " - + topologyName); - } catch (Exception e) { - System.out.println(e.getMessage()); - e.printStackTrace(); - throw new RuntimeException(e); - } finally { - if (client != null) { - client.close(); - } - } - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/command/list.java b/jstorm-client/src/main/java/backtype/storm/command/list.java deleted file mode 100644 index b0e40bc39..000000000 --- a/jstorm-client/src/main/java/backtype/storm/command/list.java +++ /dev/null @@ -1,59 +0,0 @@ -package backtype.storm.command; - -import java.util.Map; - -import org.apache.commons.lang.StringUtils; - -import backtype.storm.generated.ClusterSummary; -import backtype.storm.generated.TopologyInfo; -import backtype.storm.utils.NimbusClient; -import backtype.storm.utils.Utils; - -/** - * Activate topology - * - * @author longda - * - */ -public class list { - - - - - /** - * @param args - */ - public static void main(String[] args) { - - NimbusClient client = null; - try { - - Map conf = Utils.readStormConfig(); - client = NimbusClient.getConfiguredClient(conf); - - if (args.length > 0 && StringUtils.isBlank(args[0]) == false) { - String topologyName = args[0]; - TopologyInfo info = client.getClient().getTopologyInfoByName(topologyName); - - System.out.println("Successfully get topology info \n" - + Utils.toPrettyJsonString(info)); - }else { - ClusterSummary clusterSummary = client.getClient().getClusterInfo(); - - System.out.println("Successfully get cluster info \n" - + Utils.toPrettyJsonString(clusterSummary)); - } - - - } catch (Exception e) { - System.out.println(e.getMessage()); - e.printStackTrace(); - throw new RuntimeException(e); - } finally { - if (client != null) { - client.close(); - } - } - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/command/metrics_monitor.java b/jstorm-client/src/main/java/backtype/storm/command/metrics_monitor.java deleted file mode 100644 index bb339d462..000000000 --- a/jstorm-client/src/main/java/backtype/storm/command/metrics_monitor.java +++ /dev/null @@ -1,56 +0,0 @@ -package backtype.storm.command; - -import java.util.Map; -import java.security.InvalidParameterException; - -import backtype.storm.generated.MonitorOptions; -import backtype.storm.utils.NimbusClient; -import backtype.storm.utils.Utils; - -/** - * Monitor topology - * - * @author Basti - * - */ -public class metrics_monitor { - - /** - * @param args - */ - public static void main(String[] args) { - // TODO Auto-generated method stub - if (args == null || args.length <= 1) { - throw new InvalidParameterException("Should input topology name and enable flag"); - } - - String topologyName = args[0]; - - NimbusClient client = null; - try { - - Map conf = Utils.readStormConfig(); - client = NimbusClient.getConfiguredClient(conf); - - boolean isEnable = Boolean.valueOf(args[1]).booleanValue(); - - MonitorOptions options = new MonitorOptions(); - options.set_isEnable(isEnable); - - client.getClient().metricMonitor(topologyName, options); - - String str = (isEnable) ? "enable" : "disable"; - System.out.println("Successfully submit command to " + str - + " the monitor of " + topologyName); - } catch (Exception e) { - System.out.println(e.getMessage()); - e.printStackTrace(); - throw new RuntimeException(e); - } finally { - if (client != null) { - client.close(); - } - } - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/command/rebalance.java b/jstorm-client/src/main/java/backtype/storm/command/rebalance.java deleted file mode 100644 index 72c3f2614..000000000 --- a/jstorm-client/src/main/java/backtype/storm/command/rebalance.java +++ /dev/null @@ -1,60 +0,0 @@ -package backtype.storm.command; - -import java.security.InvalidParameterException; -import java.util.Map; - -import backtype.storm.generated.RebalanceOptions; -import backtype.storm.utils.NimbusClient; -import backtype.storm.utils.Utils; - -/** - * Active topology - * - * @author longda - * - */ -public class rebalance { - - /** - * @param args - */ - public static void main(String[] args) { - // TODO Auto-generated method stub - if (args == null || args.length == 0) { - throw new InvalidParameterException("Should input topology name"); - } - - String topologyName = args[0]; - - NimbusClient client = null; - try { - - Map conf = Utils.readStormConfig(); - client = NimbusClient.getConfiguredClient(conf); - - if (args.length == 1) { - - client.getClient().rebalance(topologyName, null); - } else { - int delaySeconds = Integer.parseInt(args[1]); - - RebalanceOptions options = new RebalanceOptions(); - options.set_wait_secs(delaySeconds); - - client.getClient().rebalance(topologyName, options); - } - - System.out.println("Successfully submit command rebalance " - + topologyName); - } catch (Exception e) { - System.out.println(e.getMessage()); - e.printStackTrace(); - throw new RuntimeException(e); - } finally { - if (client != null) { - client.close(); - } - } - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/command/restart.java b/jstorm-client/src/main/java/backtype/storm/command/restart.java deleted file mode 100644 index 1eb577deb..000000000 --- a/jstorm-client/src/main/java/backtype/storm/command/restart.java +++ /dev/null @@ -1,120 +0,0 @@ -package backtype.storm.command; - -import java.io.FileInputStream; -import java.io.FileNotFoundException; -import java.io.InputStream; -import java.security.InvalidParameterException; -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; - -import org.yaml.snakeyaml.Yaml; - -import backtype.storm.utils.NimbusClient; -import backtype.storm.utils.Utils; - -/** - * Active topology - * - * @author basti - * - */ -public class restart { - private static Map LoadProperty(String prop) { - Map ret = new HashMap(); - Properties properties = new Properties(); - - try { - InputStream stream = new FileInputStream(prop); - properties.load(stream); - if (properties.size() == 0) { - System.out.println("WARN: Config file is empty"); - return null; - } else { - ret.putAll(properties); - } - } catch (FileNotFoundException e) { - System.out.println("No such file " + prop); - throw new RuntimeException(e.getMessage()); - } catch (Exception e1) { - e1.printStackTrace(); - throw new RuntimeException(e1.getMessage()); - } - - return ret; - } - - private static Map LoadYaml(String confPath) { - Map ret = new HashMap(); - Yaml yaml = new Yaml(); - - try { - InputStream stream = new FileInputStream(confPath); - ret = (Map) yaml.load(stream); - if (ret == null || ret.isEmpty() == true) { - System.out.println("WARN: Config file is empty"); - return null; - } - } catch (FileNotFoundException e) { - System.out.println("No such file " + confPath); - throw new RuntimeException("No config file"); - } catch (Exception e1) { - e1.printStackTrace(); - throw new RuntimeException("Failed to read config file"); - } - - return ret; - } - - private static Map LoadConf(String arg) { - Map ret = null; - if (arg.endsWith("yaml")) { - ret = LoadYaml(arg); - } else { - ret = LoadProperty(arg); - } - return ret; - } - - /** - * @param args - */ - public static void main(String[] args) { - // TODO Auto-generated method stub - if (args == null || args.length == 0) { - throw new InvalidParameterException("Should input topology name"); - } - - String topologyName = args[0]; - - NimbusClient client = null; - try { - Map conf = Utils.readStormConfig(); - client = NimbusClient.getConfiguredClient(conf); - - System.out.println("It will take 15 ~ 100 seconds to restart, please wait patiently\n"); - - if (args.length == 1) { - client.getClient().restart(topologyName, null); - } else { - Map loadConf = LoadConf(args[1]); - String jsonConf = Utils.to_json(loadConf); - System.out.println("New configuration:\n" + jsonConf); - - client.getClient().restart(topologyName, jsonConf); - } - - System.out.println("Successfully submit command restart " - + topologyName); - } catch (Exception e) { - System.out.println(e.getMessage()); - e.printStackTrace(); - throw new RuntimeException(e); - } finally { - if (client != null) { - client.close(); - } - } - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/coordination/BatchBoltExecutor.java b/jstorm-client/src/main/java/backtype/storm/coordination/BatchBoltExecutor.java deleted file mode 100644 index 5502975a2..000000000 --- a/jstorm-client/src/main/java/backtype/storm/coordination/BatchBoltExecutor.java +++ /dev/null @@ -1,92 +0,0 @@ -package backtype.storm.coordination; - -import backtype.storm.coordination.CoordinatedBolt.FinishedCallback; -import backtype.storm.coordination.CoordinatedBolt.TimeoutCallback; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.FailedException; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Tuple; -import backtype.storm.utils.Utils; -import java.util.HashMap; -import java.util.Map; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class BatchBoltExecutor implements IRichBolt, FinishedCallback, - TimeoutCallback { - public static Logger LOG = LoggerFactory.getLogger(BatchBoltExecutor.class); - - byte[] _boltSer; - Map _openTransactions; - Map _conf; - TopologyContext _context; - BatchOutputCollectorImpl _collector; - - public BatchBoltExecutor(IBatchBolt bolt) { - _boltSer = Utils.serialize(bolt); - } - - @Override - public void prepare(Map conf, TopologyContext context, - OutputCollector collector) { - _conf = conf; - _context = context; - _collector = new BatchOutputCollectorImpl(collector); - _openTransactions = new HashMap(); - } - - @Override - public void execute(Tuple input) { - Object id = input.getValue(0); - IBatchBolt bolt = getBatchBolt(id); - try { - bolt.execute(input); - _collector.ack(input); - } catch (FailedException e) { - LOG.error("Failed to process tuple in batch", e); - _collector.fail(input); - } - } - - @Override - public void cleanup() { - } - - @Override - public void finishedId(Object id) { - IBatchBolt bolt = getBatchBolt(id); - _openTransactions.remove(id); - bolt.finishBatch(); - } - - @Override - public void timeoutId(Object attempt) { - _openTransactions.remove(attempt); - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - newTransactionalBolt().declareOutputFields(declarer); - } - - @Override - public Map getComponentConfiguration() { - return newTransactionalBolt().getComponentConfiguration(); - } - - private IBatchBolt getBatchBolt(Object id) { - IBatchBolt bolt = _openTransactions.get(id); - if (bolt == null) { - bolt = newTransactionalBolt(); - bolt.prepare(_conf, _context, _collector, id); - _openTransactions.put(id, bolt); - } - return bolt; - } - - private IBatchBolt newTransactionalBolt() { - return (IBatchBolt) Utils.deserialize(_boltSer); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/coordination/BatchOutputCollector.java b/jstorm-client/src/main/java/backtype/storm/coordination/BatchOutputCollector.java deleted file mode 100644 index e087f5fb0..000000000 --- a/jstorm-client/src/main/java/backtype/storm/coordination/BatchOutputCollector.java +++ /dev/null @@ -1,31 +0,0 @@ -package backtype.storm.coordination; - -import backtype.storm.utils.Utils; -import java.util.List; - -public abstract class BatchOutputCollector { - - /** - * Emits a tuple to the default output stream. - */ - public List emit(List tuple) { - return emit(Utils.DEFAULT_STREAM_ID, tuple); - } - - public abstract List emit(String streamId, List tuple); - - /** - * Emits a tuple to the specified task on the default output stream. This - * output stream must have been declared as a direct stream, and the - * specified task must use a direct grouping on this stream to receive the - * message. - */ - public void emitDirect(int taskId, List tuple) { - emitDirect(taskId, Utils.DEFAULT_STREAM_ID, tuple); - } - - public abstract void emitDirect(int taskId, String streamId, - List tuple); - - public abstract void reportError(Throwable error); -} diff --git a/jstorm-client/src/main/java/backtype/storm/coordination/BatchOutputCollectorImpl.java b/jstorm-client/src/main/java/backtype/storm/coordination/BatchOutputCollectorImpl.java deleted file mode 100644 index 73eff73c8..000000000 --- a/jstorm-client/src/main/java/backtype/storm/coordination/BatchOutputCollectorImpl.java +++ /dev/null @@ -1,36 +0,0 @@ -package backtype.storm.coordination; - -import backtype.storm.task.OutputCollector; -import backtype.storm.tuple.Tuple; -import java.util.List; - -public class BatchOutputCollectorImpl extends BatchOutputCollector { - OutputCollector _collector; - - public BatchOutputCollectorImpl(OutputCollector collector) { - _collector = collector; - } - - @Override - public List emit(String streamId, List tuple) { - return _collector.emit(streamId, tuple); - } - - @Override - public void emitDirect(int taskId, String streamId, List tuple) { - _collector.emitDirect(taskId, streamId, tuple); - } - - @Override - public void reportError(Throwable error) { - _collector.reportError(error); - } - - public void ack(Tuple tup) { - _collector.ack(tup); - } - - public void fail(Tuple tup) { - _collector.fail(tup); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/coordination/BatchSubtopologyBuilder.java b/jstorm-client/src/main/java/backtype/storm/coordination/BatchSubtopologyBuilder.java deleted file mode 100644 index eff05b4b6..000000000 --- a/jstorm-client/src/main/java/backtype/storm/coordination/BatchSubtopologyBuilder.java +++ /dev/null @@ -1,469 +0,0 @@ -package backtype.storm.coordination; - -import backtype.storm.Constants; -import backtype.storm.coordination.CoordinatedBolt.SourceArgs; -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.generated.Grouping; -import backtype.storm.grouping.CustomStreamGrouping; -import backtype.storm.topology.BaseConfigurationDeclarer; -import backtype.storm.topology.BasicBoltExecutor; -import backtype.storm.topology.BoltDeclarer; -import backtype.storm.topology.IBasicBolt; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.InputDeclarer; -import backtype.storm.topology.TopologyBuilder; -import backtype.storm.tuple.Fields; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - - -public class BatchSubtopologyBuilder { - Map _bolts = new HashMap(); - Component _masterBolt; - String _masterId; - - public BatchSubtopologyBuilder(String masterBoltId, IBasicBolt masterBolt, - Number boltParallelism) { - Integer p = boltParallelism == null ? null : boltParallelism.intValue(); - _masterBolt = new Component(new BasicBoltExecutor(masterBolt), p); - _masterId = masterBoltId; - } - - public BatchSubtopologyBuilder(String masterBoltId, IBasicBolt masterBolt) { - this(masterBoltId, masterBolt, null); - } - - public BoltDeclarer getMasterDeclarer() { - return new BoltDeclarerImpl(_masterBolt); - } - - public BoltDeclarer setBolt(String id, IBatchBolt bolt) { - return setBolt(id, bolt, null); - } - - public BoltDeclarer setBolt(String id, IBatchBolt bolt, Number parallelism) { - return setBolt(id, new BatchBoltExecutor(bolt), parallelism); - } - - public BoltDeclarer setBolt(String id, IBasicBolt bolt) { - return setBolt(id, bolt, null); - } - - public BoltDeclarer setBolt(String id, IBasicBolt bolt, Number parallelism) { - return setBolt(id, new BasicBoltExecutor(bolt), parallelism); - } - - private BoltDeclarer setBolt(String id, IRichBolt bolt, Number parallelism) { - Integer p = null; - if (parallelism != null) - p = parallelism.intValue(); - Component component = new Component(bolt, p); - _bolts.put(id, component); - return new BoltDeclarerImpl(component); - } - - public void extendTopology(TopologyBuilder builder) { - BoltDeclarer declarer = builder.setBolt(_masterId, new CoordinatedBolt( - _masterBolt.bolt), _masterBolt.parallelism); - for (InputDeclaration decl : _masterBolt.declarations) { - decl.declare(declarer); - } - for (Map conf : _masterBolt.componentConfs) { - declarer.addConfigurations(conf); - } - for (String id : _bolts.keySet()) { - Component component = _bolts.get(id); - Map coordinatedArgs = new HashMap(); - for (String c : componentBoltSubscriptions(component)) { - SourceArgs source; - if (c.equals(_masterId)) { - source = SourceArgs.single(); - } else { - source = SourceArgs.all(); - } - coordinatedArgs.put(c, source); - } - - BoltDeclarer input = builder.setBolt(id, new CoordinatedBolt( - component.bolt, coordinatedArgs, null), - component.parallelism); - for (Map conf : component.componentConfs) { - input.addConfigurations(conf); - } - for (String c : componentBoltSubscriptions(component)) { - input.directGrouping(c, Constants.COORDINATED_STREAM_ID); - } - for (InputDeclaration d : component.declarations) { - d.declare(input); - } - } - } - - private Set componentBoltSubscriptions(Component component) { - Set ret = new HashSet(); - for (InputDeclaration d : component.declarations) { - ret.add(d.getComponent()); - } - return ret; - } - - private static class Component { - public IRichBolt bolt; - public Integer parallelism; - public List declarations = new ArrayList(); - public List componentConfs = new ArrayList(); - - public Component(IRichBolt bolt, Integer parallelism) { - this.bolt = bolt; - this.parallelism = parallelism; - } - } - - private static interface InputDeclaration { - void declare(InputDeclarer declarer); - - String getComponent(); - } - - private class BoltDeclarerImpl extends - BaseConfigurationDeclarer implements BoltDeclarer { - Component _component; - - public BoltDeclarerImpl(Component component) { - _component = component; - } - - @Override - public BoltDeclarer fieldsGrouping(final String component, - final Fields fields) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.fieldsGrouping(component, fields); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer fieldsGrouping(final String component, - final String streamId, final Fields fields) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.fieldsGrouping(component, streamId, fields); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer globalGrouping(final String component) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.globalGrouping(component); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer globalGrouping(final String component, - final String streamId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.globalGrouping(component, streamId); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer shuffleGrouping(final String component) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.shuffleGrouping(component); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer shuffleGrouping(final String component, - final String streamId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.shuffleGrouping(component, streamId); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer localOrShuffleGrouping(final String component) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.localOrShuffleGrouping(component); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer localOrShuffleGrouping(final String component, - final String streamId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.localOrShuffleGrouping(component, streamId); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer localFirstGrouping(final String componentId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.localFirstGrouping(componentId); - } - - @Override - public String getComponent() { - return componentId; - } - }); - return this; - } - - @Override - public BoltDeclarer localFirstGrouping(final String component, - final String streamId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.localFirstGrouping(component, streamId); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer noneGrouping(final String component) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.noneGrouping(component); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer noneGrouping(final String component, - final String streamId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.noneGrouping(component, streamId); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer allGrouping(final String component) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.allGrouping(component); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer allGrouping(final String component, - final String streamId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.allGrouping(component, streamId); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer directGrouping(final String component) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.directGrouping(component); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer directGrouping(final String component, - final String streamId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.directGrouping(component, streamId); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer customGrouping(final String component, - final CustomStreamGrouping grouping) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.customGrouping(component, grouping); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer customGrouping(final String component, - final String streamId, final CustomStreamGrouping grouping) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.customGrouping(component, streamId, grouping); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer grouping(final GlobalStreamId stream, - final Grouping grouping) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.grouping(stream, grouping); - } - - @Override - public String getComponent() { - return stream.get_componentId(); - } - }); - return this; - } - - private void addDeclaration(InputDeclaration declaration) { - _component.declarations.add(declaration); - } - - @Override - public BoltDeclarer addConfigurations(Map conf) { - _component.componentConfs.add(conf); - return this; - } - - - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/coordination/CoordinatedBolt.java b/jstorm-client/src/main/java/backtype/storm/coordination/CoordinatedBolt.java deleted file mode 100644 index 1e789328a..000000000 --- a/jstorm-client/src/main/java/backtype/storm/coordination/CoordinatedBolt.java +++ /dev/null @@ -1,379 +0,0 @@ -package backtype.storm.coordination; - -import backtype.storm.topology.FailedException; -import java.util.Map.Entry; -import backtype.storm.tuple.Values; -import backtype.storm.generated.GlobalStreamId; -import java.util.Collection; -import backtype.storm.Constants; -import backtype.storm.generated.Grouping; -import backtype.storm.task.IOutputCollector; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.utils.TimeCacheMap; -import backtype.storm.utils.Utils; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import static backtype.storm.utils.Utils.get; - -/** - * Coordination requires the request ids to be globally unique for awhile. This - * is so it doesn't get confused in the case of retries. - */ -public class CoordinatedBolt implements IRichBolt { - public static Logger LOG = LoggerFactory.getLogger(CoordinatedBolt.class); - - public static interface FinishedCallback { - void finishedId(Object id); - } - - public static interface TimeoutCallback { - void timeoutId(Object id); - } - - public static class SourceArgs implements Serializable { - public boolean singleCount; - - protected SourceArgs(boolean singleCount) { - this.singleCount = singleCount; - } - - public static SourceArgs single() { - return new SourceArgs(true); - } - - public static SourceArgs all() { - return new SourceArgs(false); - } - - @Override - public String toString() { - return ""; - } - } - - public class CoordinatedOutputCollector implements IOutputCollector { - IOutputCollector _delegate; - - public CoordinatedOutputCollector(IOutputCollector delegate) { - _delegate = delegate; - } - - public List emit(String stream, Collection anchors, - List tuple) { - List tasks = _delegate.emit(stream, anchors, tuple); - updateTaskCounts(tuple.get(0), tasks); - return tasks; - } - - public void emitDirect(int task, String stream, - Collection anchors, List tuple) { - updateTaskCounts(tuple.get(0), Arrays.asList(task)); - _delegate.emitDirect(task, stream, anchors, tuple); - } - - public void ack(Tuple tuple) { - Object id = tuple.getValue(0); - synchronized (_tracked) { - TrackingInfo track = _tracked.get(id); - if (track != null) - track.receivedTuples++; - } - boolean failed = checkFinishId(tuple, TupleType.REGULAR); - if (failed) { - _delegate.fail(tuple); - } else { - _delegate.ack(tuple); - } - } - - public void fail(Tuple tuple) { - Object id = tuple.getValue(0); - synchronized (_tracked) { - TrackingInfo track = _tracked.get(id); - if (track != null) - track.failed = true; - } - checkFinishId(tuple, TupleType.REGULAR); - _delegate.fail(tuple); - } - - public void reportError(Throwable error) { - _delegate.reportError(error); - } - - private void updateTaskCounts(Object id, List tasks) { - synchronized (_tracked) { - TrackingInfo track = _tracked.get(id); - if (track != null) { - Map taskEmittedTuples = track.taskEmittedTuples; - for (Integer task : tasks) { - int newCount = get(taskEmittedTuples, task, 0) + 1; - taskEmittedTuples.put(task, newCount); - } - } - } - } - } - - private Map _sourceArgs; - private IdStreamSpec _idStreamSpec; - private IRichBolt _delegate; - private Integer _numSourceReports; - private List _countOutTasks = new ArrayList();; - private OutputCollector _collector; - private TimeCacheMap _tracked; - - public static class TrackingInfo { - int reportCount = 0; - int expectedTupleCount = 0; - int receivedTuples = 0; - boolean failed = false; - Map taskEmittedTuples = new HashMap(); - boolean receivedId = false; - boolean finished = false; - List ackTuples = new ArrayList(); - - @Override - public String toString() { - return "reportCount: " + reportCount + "\n" - + "expectedTupleCount: " + expectedTupleCount + "\n" - + "receivedTuples: " + receivedTuples + "\n" + "failed: " - + failed + "\n" + taskEmittedTuples.toString(); - } - } - - public static class IdStreamSpec implements Serializable { - GlobalStreamId _id; - - public GlobalStreamId getGlobalStreamId() { - return _id; - } - - public static IdStreamSpec makeDetectSpec(String component, - String stream) { - return new IdStreamSpec(component, stream); - } - - protected IdStreamSpec(String component, String stream) { - _id = new GlobalStreamId(component, stream); - } - } - - public CoordinatedBolt(IRichBolt delegate) { - this(delegate, null, null); - } - - public CoordinatedBolt(IRichBolt delegate, String sourceComponent, - SourceArgs sourceArgs, IdStreamSpec idStreamSpec) { - this(delegate, singleSourceArgs(sourceComponent, sourceArgs), - idStreamSpec); - } - - public CoordinatedBolt(IRichBolt delegate, - Map sourceArgs, IdStreamSpec idStreamSpec) { - _sourceArgs = sourceArgs; - if (_sourceArgs == null) - _sourceArgs = new HashMap(); - _delegate = delegate; - _idStreamSpec = idStreamSpec; - } - - public void prepare(Map config, TopologyContext context, - OutputCollector collector) { - TimeCacheMap.ExpiredCallback callback = null; - if (_delegate instanceof TimeoutCallback) { - callback = new TimeoutItems(); - } - _tracked = new TimeCacheMap( - context.maxTopologyMessageTimeout(), callback); - _collector = collector; - _delegate.prepare(config, context, new OutputCollector( - new CoordinatedOutputCollector(collector))); - for (String component : Utils.get(context.getThisTargets(), - Constants.COORDINATED_STREAM_ID, - new HashMap()).keySet()) { - for (Integer task : context.getComponentTasks(component)) { - _countOutTasks.add(task); - } - } - if (!_sourceArgs.isEmpty()) { - _numSourceReports = 0; - for (Entry entry : _sourceArgs.entrySet()) { - if (entry.getValue().singleCount) { - _numSourceReports += 1; - } else { - _numSourceReports += context.getComponentTasks( - entry.getKey()).size(); - } - } - } - } - - private boolean checkFinishId(Tuple tup, TupleType type) { - Object id = tup.getValue(0); - boolean failed = false; - - synchronized (_tracked) { - TrackingInfo track = _tracked.get(id); - try { - if (track != null) { - boolean delayed = false; - if (_idStreamSpec == null && type == TupleType.COORD - || _idStreamSpec != null && type == TupleType.ID) { - track.ackTuples.add(tup); - delayed = true; - } - if (track.failed) { - failed = true; - for (Tuple t : track.ackTuples) { - _collector.fail(t); - } - _tracked.remove(id); - } else if (track.receivedId - && (_sourceArgs.isEmpty() || track.reportCount == _numSourceReports - && track.expectedTupleCount == track.receivedTuples)) { - if (_delegate instanceof FinishedCallback) { - ((FinishedCallback) _delegate).finishedId(id); - } - if (!(_sourceArgs.isEmpty() || type != TupleType.REGULAR)) { - throw new IllegalStateException( - "Coordination condition met on a non-coordinating tuple. Should be impossible"); - } - Iterator outTasks = _countOutTasks.iterator(); - while (outTasks.hasNext()) { - int task = outTasks.next(); - int numTuples = get(track.taskEmittedTuples, task, - 0); - _collector.emitDirect(task, - Constants.COORDINATED_STREAM_ID, tup, - new Values(id, numTuples)); - } - for (Tuple t : track.ackTuples) { - _collector.ack(t); - } - track.finished = true; - _tracked.remove(id); - } - if (!delayed && type != TupleType.REGULAR) { - if (track.failed) { - _collector.fail(tup); - } else { - _collector.ack(tup); - } - } - } else { - if (type != TupleType.REGULAR) - _collector.fail(tup); - } - } catch (FailedException e) { - LOG.error("Failed to finish batch", e); - for (Tuple t : track.ackTuples) { - _collector.fail(t); - } - _tracked.remove(id); - failed = true; - } - } - return failed; - } - - public void execute(Tuple tuple) { - Object id = tuple.getValue(0); - TrackingInfo track; - TupleType type = getTupleType(tuple); - synchronized (_tracked) { - track = _tracked.get(id); - if (track == null) { - track = new TrackingInfo(); - if (_idStreamSpec == null) - track.receivedId = true; - _tracked.put(id, track); - } - } - - if (type == TupleType.ID) { - synchronized (_tracked) { - track.receivedId = true; - } - checkFinishId(tuple, type); - } else if (type == TupleType.COORD) { - int count = (Integer) tuple.getValue(1); - synchronized (_tracked) { - track.reportCount++; - track.expectedTupleCount += count; - } - checkFinishId(tuple, type); - } else { - synchronized (_tracked) { - _delegate.execute(tuple); - } - } - } - - public void cleanup() { - _delegate.cleanup(); - _tracked.cleanup(); - } - - public void declareOutputFields(OutputFieldsDeclarer declarer) { - _delegate.declareOutputFields(declarer); - declarer.declareStream(Constants.COORDINATED_STREAM_ID, true, - new Fields("id", "count")); - } - - @Override - public Map getComponentConfiguration() { - return _delegate.getComponentConfiguration(); - } - - private static Map singleSourceArgs( - String sourceComponent, SourceArgs sourceArgs) { - Map ret = new HashMap(); - ret.put(sourceComponent, sourceArgs); - return ret; - } - - private class TimeoutItems implements - TimeCacheMap.ExpiredCallback { - @Override - public void expire(Object id, TrackingInfo val) { - synchronized (_tracked) { - // the combination of the lock and the finished flag ensure that - // an id is never timed out if it has been finished - val.failed = true; - if (!val.finished) { - ((TimeoutCallback) _delegate).timeoutId(id); - } - } - } - } - - private TupleType getTupleType(Tuple tuple) { - if (_idStreamSpec != null - && tuple.getSourceGlobalStreamid().equals(_idStreamSpec._id)) { - return TupleType.ID; - } else if (!_sourceArgs.isEmpty() - && tuple.getSourceStreamId().equals( - Constants.COORDINATED_STREAM_ID)) { - return TupleType.COORD; - } else { - return TupleType.REGULAR; - } - } - - static enum TupleType { - REGULAR, ID, COORD - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/coordination/IBatchBolt.java b/jstorm-client/src/main/java/backtype/storm/coordination/IBatchBolt.java deleted file mode 100644 index f8a9386e5..000000000 --- a/jstorm-client/src/main/java/backtype/storm/coordination/IBatchBolt.java +++ /dev/null @@ -1,16 +0,0 @@ -package backtype.storm.coordination; - -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IComponent; -import backtype.storm.tuple.Tuple; -import java.io.Serializable; -import java.util.Map; - -public interface IBatchBolt extends Serializable, IComponent { - void prepare(Map conf, TopologyContext context, - BatchOutputCollector collector, T id); - - void execute(Tuple tuple); - - void finishBatch(); -} diff --git a/jstorm-client/src/main/java/backtype/storm/daemon/Shutdownable.java b/jstorm-client/src/main/java/backtype/storm/daemon/Shutdownable.java deleted file mode 100644 index 1e6151225..000000000 --- a/jstorm-client/src/main/java/backtype/storm/daemon/Shutdownable.java +++ /dev/null @@ -1,5 +0,0 @@ -package backtype.storm.daemon; - -public interface Shutdownable { - public void shutdown(); -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/drpc/DRPCInvocationsClient.java b/jstorm-client/src/main/java/backtype/storm/drpc/DRPCInvocationsClient.java deleted file mode 100644 index 756b8aa2a..000000000 --- a/jstorm-client/src/main/java/backtype/storm/drpc/DRPCInvocationsClient.java +++ /dev/null @@ -1,78 +0,0 @@ -package backtype.storm.drpc; - -import org.apache.thrift7.TException; -import org.apache.thrift7.protocol.TBinaryProtocol; -import org.apache.thrift7.transport.TFramedTransport; -import org.apache.thrift7.transport.TSocket; -import org.apache.thrift7.transport.TTransport; - -import backtype.storm.generated.DRPCRequest; -import backtype.storm.generated.DistributedRPCInvocations; - -public class DRPCInvocationsClient implements DistributedRPCInvocations.Iface { - private TTransport conn; - private DistributedRPCInvocations.Client client; - private String host; - private int port; - - public DRPCInvocationsClient(String host, int port) { - try { - this.host = host; - this.port = port; - connect(); - } catch (TException e) { - throw new RuntimeException(e); - } - } - - private void connect() throws TException { - conn = new TFramedTransport(new TSocket(host, port)); - client = new DistributedRPCInvocations.Client(new TBinaryProtocol(conn)); - conn.open(); - } - - public String getHost() { - return host; - } - - public int getPort() { - return port; - } - - public void result(String id, String result) throws TException { - try { - if (client == null) - connect(); - client.result(id, result); - } catch (TException e) { - client = null; - throw e; - } - } - - public DRPCRequest fetchRequest(String func) throws TException { - try { - if (client == null) - connect(); - return client.fetchRequest(func); - } catch (TException e) { - client = null; - throw e; - } - } - - public void failRequest(String id) throws TException { - try { - if (client == null) - connect(); - client.failRequest(id); - } catch (TException e) { - client = null; - throw e; - } - } - - public void close() { - conn.close(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/drpc/DRPCSpout.java b/jstorm-client/src/main/java/backtype/storm/drpc/DRPCSpout.java deleted file mode 100644 index a68b008ed..000000000 --- a/jstorm-client/src/main/java/backtype/storm/drpc/DRPCSpout.java +++ /dev/null @@ -1,162 +0,0 @@ -package backtype.storm.drpc; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.thrift7.TException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import backtype.storm.Config; -import backtype.storm.ILocalDRPC; -import backtype.storm.generated.DRPCRequest; -import backtype.storm.generated.DistributedRPCInvocations; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseRichSpout; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Values; -import backtype.storm.utils.ServiceRegistry; -import backtype.storm.utils.Utils; - -public class DRPCSpout extends BaseRichSpout { - public static Logger LOG = LoggerFactory.getLogger(DRPCSpout.class); - - SpoutOutputCollector _collector; - List _clients = new ArrayList(); - String _function; - String _local_drpc_id = null; - - private static class DRPCMessageId { - String id; - int index; - - public DRPCMessageId(String id, int index) { - this.id = id; - this.index = index; - } - } - - public DRPCSpout(String function) { - _function = function; - } - - public DRPCSpout(String function, ILocalDRPC drpc) { - _function = function; - _local_drpc_id = drpc.getServiceId(); - } - - @Override - public void open(Map conf, TopologyContext context, - SpoutOutputCollector collector) { - _collector = collector; - if (_local_drpc_id == null) { - int numTasks = context.getComponentTasks( - context.getThisComponentId()).size(); - int index = context.getThisTaskIndex(); - - int port = Utils.getInt(conf.get(Config.DRPC_INVOCATIONS_PORT)); - List servers = (List) conf.get(Config.DRPC_SERVERS); - if (servers == null || servers.isEmpty()) { - throw new RuntimeException( - "No DRPC servers configured for topology"); - } - if (numTasks < servers.size()) { - for (String s : servers) { - _clients.add(new DRPCInvocationsClient(s, port)); - } - } else { - int i = index % servers.size(); - _clients.add(new DRPCInvocationsClient(servers.get(i), port)); - } - } - - } - - @Override - public void close() { - for (DRPCInvocationsClient client : _clients) { - client.close(); - } - } - - @Override - public void nextTuple() { - boolean gotRequest = false; - if (_local_drpc_id == null) { - for (int i = 0; i < _clients.size(); i++) { - DRPCInvocationsClient client = _clients.get(i); - try { - DRPCRequest req = client.fetchRequest(_function); - if (req.get_request_id().length() > 0) { - Map returnInfo = new HashMap(); - returnInfo.put("id", req.get_request_id()); - returnInfo.put("host", client.getHost()); - returnInfo.put("port", client.getPort()); - gotRequest = true; - _collector.emit(new Values(req.get_func_args(), - Utils.to_json(returnInfo)), - new DRPCMessageId(req.get_request_id(), i)); - break; - } - } catch (TException e) { - LOG.error("Failed to fetch DRPC result from DRPC server", e); - } - } - } else { - DistributedRPCInvocations.Iface drpc = (DistributedRPCInvocations.Iface) ServiceRegistry - .getService(_local_drpc_id); - if (drpc != null) { // can happen during shutdown of drpc while - // topology is still up - try { - DRPCRequest req = drpc.fetchRequest(_function); - if (req.get_request_id().length() > 0) { - Map returnInfo = new HashMap(); - returnInfo.put("id", req.get_request_id()); - returnInfo.put("host", _local_drpc_id); - returnInfo.put("port", 0); - gotRequest = true; - _collector.emit(new Values(req.get_func_args(), - Utils.to_json(returnInfo)), - new DRPCMessageId(req.get_request_id(), 0)); - } - } catch (TException e) { - throw new RuntimeException(e); - } - } - } - if (!gotRequest) { - Utils.sleep(1); - } - } - - @Override - public void ack(Object msgId) { - } - - @Override - public void fail(Object msgId) { - DRPCMessageId did = (DRPCMessageId) msgId; - DistributedRPCInvocations.Iface client; - - if (_local_drpc_id == null) { - client = _clients.get(did.index); - } else { - client = (DistributedRPCInvocations.Iface) ServiceRegistry - .getService(_local_drpc_id); - } - try { - client.failRequest(did.id); - } catch (TException e) { - LOG.error("Failed to fail request", e); - } - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(new Fields("args", "return-info")); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/drpc/JoinResult.java b/jstorm-client/src/main/java/backtype/storm/drpc/JoinResult.java deleted file mode 100644 index b6733a30c..000000000 --- a/jstorm-client/src/main/java/backtype/storm/drpc/JoinResult.java +++ /dev/null @@ -1,59 +0,0 @@ -package backtype.storm.drpc; - -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.tuple.Values; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class JoinResult extends BaseRichBolt { - public static Logger LOG = LoggerFactory.getLogger(JoinResult.class); - - String returnComponent; - Map returns = new HashMap(); - Map results = new HashMap(); - OutputCollector _collector; - - public JoinResult(String returnComponent) { - this.returnComponent = returnComponent; - } - - public void prepare(Map map, TopologyContext context, - OutputCollector collector) { - _collector = collector; - } - - public void execute(Tuple tuple) { - Object requestId = tuple.getValue(0); - if (tuple.getSourceComponent().equals(returnComponent)) { - returns.put(requestId, tuple); - } else { - results.put(requestId, tuple); - } - - if (returns.containsKey(requestId) && results.containsKey(requestId)) { - Tuple result = results.remove(requestId); - Tuple returner = returns.remove(requestId); - LOG.debug(result.getValue(1).toString()); - List anchors = new ArrayList(); - anchors.add(result); - anchors.add(returner); - _collector.emit(anchors, new Values("" + result.getValue(1), - returner.getValue(1))); - _collector.ack(result); - _collector.ack(returner); - } - } - - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(new Fields("result", "return-info")); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/drpc/KeyedFairBolt.java b/jstorm-client/src/main/java/backtype/storm/drpc/KeyedFairBolt.java deleted file mode 100644 index 287168f9c..000000000 --- a/jstorm-client/src/main/java/backtype/storm/drpc/KeyedFairBolt.java +++ /dev/null @@ -1,75 +0,0 @@ -package backtype.storm.drpc; - -import backtype.storm.coordination.CoordinatedBolt.FinishedCallback; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.BasicBoltExecutor; -import backtype.storm.topology.IBasicBolt; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Tuple; -import backtype.storm.utils.KeyedRoundRobinQueue; -import java.util.HashMap; -import java.util.Map; - -public class KeyedFairBolt implements IRichBolt, FinishedCallback { - IRichBolt _delegate; - KeyedRoundRobinQueue _rrQueue; - Thread _executor; - FinishedCallback _callback; - - public KeyedFairBolt(IRichBolt delegate) { - _delegate = delegate; - } - - public KeyedFairBolt(IBasicBolt delegate) { - this(new BasicBoltExecutor(delegate)); - } - - public void prepare(Map stormConf, TopologyContext context, - OutputCollector collector) { - if (_delegate instanceof FinishedCallback) { - _callback = (FinishedCallback) _delegate; - } - _delegate.prepare(stormConf, context, collector); - _rrQueue = new KeyedRoundRobinQueue(); - _executor = new Thread(new Runnable() { - public void run() { - try { - while (true) { - _delegate.execute(_rrQueue.take()); - } - } catch (InterruptedException e) { - - } - } - }); - _executor.setDaemon(true); - _executor.start(); - } - - public void execute(Tuple input) { - Object key = input.getValue(0); - _rrQueue.add(key, input); - } - - public void cleanup() { - _executor.interrupt(); - _delegate.cleanup(); - } - - public void declareOutputFields(OutputFieldsDeclarer declarer) { - _delegate.declareOutputFields(declarer); - } - - public void finishedId(Object id) { - if (_callback != null) { - _callback.finishedId(id); - } - } - - @Override - public Map getComponentConfiguration() { - return new HashMap(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/drpc/LinearDRPCInputDeclarer.java b/jstorm-client/src/main/java/backtype/storm/drpc/LinearDRPCInputDeclarer.java deleted file mode 100644 index 5277cff6f..000000000 --- a/jstorm-client/src/main/java/backtype/storm/drpc/LinearDRPCInputDeclarer.java +++ /dev/null @@ -1,42 +0,0 @@ -package backtype.storm.drpc; - -import backtype.storm.grouping.CustomStreamGrouping; -import backtype.storm.topology.ComponentConfigurationDeclarer; -import backtype.storm.tuple.Fields; - -public interface LinearDRPCInputDeclarer extends - ComponentConfigurationDeclarer { - public LinearDRPCInputDeclarer fieldsGrouping(Fields fields); - - public LinearDRPCInputDeclarer fieldsGrouping(String streamId, Fields fields); - - public LinearDRPCInputDeclarer globalGrouping(); - - public LinearDRPCInputDeclarer globalGrouping(String streamId); - - public LinearDRPCInputDeclarer shuffleGrouping(); - - public LinearDRPCInputDeclarer shuffleGrouping(String streamId); - - public LinearDRPCInputDeclarer localOrShuffleGrouping(); - - public LinearDRPCInputDeclarer localOrShuffleGrouping(String streamId); - - public LinearDRPCInputDeclarer noneGrouping(); - - public LinearDRPCInputDeclarer noneGrouping(String streamId); - - public LinearDRPCInputDeclarer allGrouping(); - - public LinearDRPCInputDeclarer allGrouping(String streamId); - - public LinearDRPCInputDeclarer directGrouping(); - - public LinearDRPCInputDeclarer directGrouping(String streamId); - - public LinearDRPCInputDeclarer customGrouping(CustomStreamGrouping grouping); - - public LinearDRPCInputDeclarer customGrouping(String streamId, - CustomStreamGrouping grouping); - -} diff --git a/jstorm-client/src/main/java/backtype/storm/drpc/LinearDRPCTopologyBuilder.java b/jstorm-client/src/main/java/backtype/storm/drpc/LinearDRPCTopologyBuilder.java deleted file mode 100644 index 67bf325d6..000000000 --- a/jstorm-client/src/main/java/backtype/storm/drpc/LinearDRPCTopologyBuilder.java +++ /dev/null @@ -1,378 +0,0 @@ -package backtype.storm.drpc; - -import backtype.storm.Constants; -import backtype.storm.ILocalDRPC; -import backtype.storm.coordination.BatchBoltExecutor; -import backtype.storm.coordination.CoordinatedBolt; -import backtype.storm.coordination.CoordinatedBolt.FinishedCallback; -import backtype.storm.coordination.CoordinatedBolt.IdStreamSpec; -import backtype.storm.coordination.CoordinatedBolt.SourceArgs; -import backtype.storm.coordination.IBatchBolt; -import backtype.storm.generated.StormTopology; -import backtype.storm.generated.StreamInfo; -import backtype.storm.grouping.CustomStreamGrouping; -import backtype.storm.topology.BaseConfigurationDeclarer; -import backtype.storm.topology.BasicBoltExecutor; -import backtype.storm.topology.BoltDeclarer; -import backtype.storm.topology.IBasicBolt; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.InputDeclarer; -import backtype.storm.topology.OutputFieldsGetter; -import backtype.storm.topology.TopologyBuilder; -import backtype.storm.tuple.Fields; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -// Trident subsumes the functionality provided by this class, so it's deprecated -@Deprecated -public class LinearDRPCTopologyBuilder { - String _function; - List _components = new ArrayList(); - - public LinearDRPCTopologyBuilder(String function) { - _function = function; - } - - public LinearDRPCInputDeclarer addBolt(IBatchBolt bolt, Number parallelism) { - return addBolt(new BatchBoltExecutor(bolt), parallelism); - } - - public LinearDRPCInputDeclarer addBolt(IBatchBolt bolt) { - return addBolt(bolt, 1); - } - - @Deprecated - public LinearDRPCInputDeclarer addBolt(IRichBolt bolt, Number parallelism) { - if (parallelism == null) - parallelism = 1; - Component component = new Component(bolt, parallelism.intValue()); - _components.add(component); - return new InputDeclarerImpl(component); - } - - @Deprecated - public LinearDRPCInputDeclarer addBolt(IRichBolt bolt) { - return addBolt(bolt, null); - } - - public LinearDRPCInputDeclarer addBolt(IBasicBolt bolt, Number parallelism) { - return addBolt(new BasicBoltExecutor(bolt), parallelism); - } - - public LinearDRPCInputDeclarer addBolt(IBasicBolt bolt) { - return addBolt(bolt, null); - } - - public StormTopology createLocalTopology(ILocalDRPC drpc) { - return createTopology(new DRPCSpout(_function, drpc)); - } - - public StormTopology createRemoteTopology() { - return createTopology(new DRPCSpout(_function)); - } - - private StormTopology createTopology(DRPCSpout spout) { - final String SPOUT_ID = "spout"; - final String PREPARE_ID = "prepare-request"; - - TopologyBuilder builder = new TopologyBuilder(); - builder.setSpout(SPOUT_ID, spout); - builder.setBolt(PREPARE_ID, new PrepareRequest()) - .noneGrouping(SPOUT_ID); - int i = 0; - for (; i < _components.size(); i++) { - Component component = _components.get(i); - - Map source = new HashMap(); - if (i == 1) { - source.put(boltId(i - 1), SourceArgs.single()); - } else if (i >= 2) { - source.put(boltId(i - 1), SourceArgs.all()); - } - IdStreamSpec idSpec = null; - if (i == _components.size() - 1 - && component.bolt instanceof FinishedCallback) { - idSpec = IdStreamSpec.makeDetectSpec(PREPARE_ID, - PrepareRequest.ID_STREAM); - } - BoltDeclarer declarer = builder.setBolt(boltId(i), - new CoordinatedBolt(component.bolt, source, idSpec), - component.parallelism); - - for (Map conf : component.componentConfs) { - declarer.addConfigurations(conf); - } - - if (idSpec != null) { - declarer.fieldsGrouping(idSpec.getGlobalStreamId() - .get_componentId(), PrepareRequest.ID_STREAM, - new Fields("request")); - } - if (i == 0 && component.declarations.isEmpty()) { - declarer.noneGrouping(PREPARE_ID, PrepareRequest.ARGS_STREAM); - } else { - String prevId; - if (i == 0) { - prevId = PREPARE_ID; - } else { - prevId = boltId(i - 1); - } - for (InputDeclaration declaration : component.declarations) { - declaration.declare(prevId, declarer); - } - } - if (i > 0) { - declarer.directGrouping(boltId(i - 1), - Constants.COORDINATED_STREAM_ID); - } - } - - IRichBolt lastBolt = _components.get(_components.size() - 1).bolt; - OutputFieldsGetter getter = new OutputFieldsGetter(); - lastBolt.declareOutputFields(getter); - Map streams = getter.getFieldsDeclaration(); - if (streams.size() != 1) { - throw new RuntimeException( - "Must declare exactly one stream from last bolt in LinearDRPCTopology"); - } - String outputStream = streams.keySet().iterator().next(); - List fields = streams.get(outputStream).get_output_fields(); - if (fields.size() != 2) { - throw new RuntimeException( - "Output stream of last component in LinearDRPCTopology must contain exactly two fields. The first should be the request id, and the second should be the result."); - } - - builder.setBolt(boltId(i), new JoinResult(PREPARE_ID)) - .fieldsGrouping(boltId(i - 1), outputStream, - new Fields(fields.get(0))) - .fieldsGrouping(PREPARE_ID, PrepareRequest.RETURN_STREAM, - new Fields("request")); - i++; - builder.setBolt(boltId(i), new ReturnResults()).noneGrouping( - boltId(i - 1)); - return builder.createTopology(); - } - - private static String boltId(int index) { - return "bolt" + index; - } - - private static class Component { - public IRichBolt bolt; - public int parallelism; - public List componentConfs; - public List declarations = new ArrayList(); - - public Component(IRichBolt bolt, int parallelism) { - this.bolt = bolt; - this.parallelism = parallelism; - this.componentConfs = new ArrayList(); - } - } - - private static interface InputDeclaration { - public void declare(String prevComponent, InputDeclarer declarer); - } - - private class InputDeclarerImpl extends - BaseConfigurationDeclarer implements - LinearDRPCInputDeclarer { - Component _component; - - public InputDeclarerImpl(Component component) { - _component = component; - } - - @Override - public LinearDRPCInputDeclarer fieldsGrouping(final Fields fields) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(String prevComponent, InputDeclarer declarer) { - declarer.fieldsGrouping(prevComponent, fields); - } - }); - return this; - } - - @Override - public LinearDRPCInputDeclarer fieldsGrouping(final String streamId, - final Fields fields) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(String prevComponent, InputDeclarer declarer) { - declarer.fieldsGrouping(prevComponent, streamId, fields); - } - }); - return this; - } - - @Override - public LinearDRPCInputDeclarer globalGrouping() { - addDeclaration(new InputDeclaration() { - @Override - public void declare(String prevComponent, InputDeclarer declarer) { - declarer.globalGrouping(prevComponent); - } - }); - return this; - } - - @Override - public LinearDRPCInputDeclarer globalGrouping(final String streamId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(String prevComponent, InputDeclarer declarer) { - declarer.globalGrouping(prevComponent, streamId); - } - }); - return this; - } - - @Override - public LinearDRPCInputDeclarer shuffleGrouping() { - addDeclaration(new InputDeclaration() { - @Override - public void declare(String prevComponent, InputDeclarer declarer) { - declarer.shuffleGrouping(prevComponent); - } - }); - return this; - } - - @Override - public LinearDRPCInputDeclarer shuffleGrouping(final String streamId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(String prevComponent, InputDeclarer declarer) { - declarer.shuffleGrouping(prevComponent, streamId); - } - }); - return this; - } - - @Override - public LinearDRPCInputDeclarer localOrShuffleGrouping() { - addDeclaration(new InputDeclaration() { - @Override - public void declare(String prevComponent, InputDeclarer declarer) { - declarer.localOrShuffleGrouping(prevComponent); - } - }); - return this; - } - - @Override - public LinearDRPCInputDeclarer localOrShuffleGrouping( - final String streamId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(String prevComponent, InputDeclarer declarer) { - declarer.localOrShuffleGrouping(prevComponent, streamId); - } - }); - return this; - } - - @Override - public LinearDRPCInputDeclarer noneGrouping() { - addDeclaration(new InputDeclaration() { - @Override - public void declare(String prevComponent, InputDeclarer declarer) { - declarer.noneGrouping(prevComponent); - } - }); - return this; - } - - @Override - public LinearDRPCInputDeclarer noneGrouping(final String streamId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(String prevComponent, InputDeclarer declarer) { - declarer.noneGrouping(prevComponent, streamId); - } - }); - return this; - } - - @Override - public LinearDRPCInputDeclarer allGrouping() { - addDeclaration(new InputDeclaration() { - @Override - public void declare(String prevComponent, InputDeclarer declarer) { - declarer.allGrouping(prevComponent); - } - }); - return this; - } - - @Override - public LinearDRPCInputDeclarer allGrouping(final String streamId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(String prevComponent, InputDeclarer declarer) { - declarer.allGrouping(prevComponent, streamId); - } - }); - return this; - } - - @Override - public LinearDRPCInputDeclarer directGrouping() { - addDeclaration(new InputDeclaration() { - @Override - public void declare(String prevComponent, InputDeclarer declarer) { - declarer.directGrouping(prevComponent); - } - }); - return this; - } - - @Override - public LinearDRPCInputDeclarer directGrouping(final String streamId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(String prevComponent, InputDeclarer declarer) { - declarer.directGrouping(prevComponent, streamId); - } - }); - return this; - } - - @Override - public LinearDRPCInputDeclarer customGrouping( - final CustomStreamGrouping grouping) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(String prevComponent, InputDeclarer declarer) { - declarer.customGrouping(prevComponent, grouping); - } - }); - return this; - } - - @Override - public LinearDRPCInputDeclarer customGrouping(final String streamId, - final CustomStreamGrouping grouping) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(String prevComponent, InputDeclarer declarer) { - declarer.customGrouping(prevComponent, streamId, grouping); - } - }); - return this; - } - - private void addDeclaration(InputDeclaration declaration) { - _component.declarations.add(declaration); - } - - @Override - public LinearDRPCInputDeclarer addConfigurations(Map conf) { - _component.componentConfs.add(conf); - return this; - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/drpc/PrepareRequest.java b/jstorm-client/src/main/java/backtype/storm/drpc/PrepareRequest.java deleted file mode 100644 index 95b92fbfa..000000000 --- a/jstorm-client/src/main/java/backtype/storm/drpc/PrepareRequest.java +++ /dev/null @@ -1,41 +0,0 @@ -package backtype.storm.drpc; - -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.BasicOutputCollector; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseBasicBolt; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; -import java.util.Map; -import java.util.Random; -import backtype.storm.utils.Utils; - -public class PrepareRequest extends BaseBasicBolt { - public static final String ARGS_STREAM = Utils.DEFAULT_STREAM_ID; - public static final String RETURN_STREAM = "ret"; - public static final String ID_STREAM = "id"; - - Random rand; - - @Override - public void prepare(Map map, TopologyContext context) { - rand = new Random(); - } - - @Override - public void execute(Tuple tuple, BasicOutputCollector collector) { - String args = tuple.getString(0); - String returnInfo = tuple.getString(1); - long requestId = rand.nextLong(); - collector.emit(ARGS_STREAM, new Values(requestId, args)); - collector.emit(RETURN_STREAM, new Values(requestId, returnInfo)); - collector.emit(ID_STREAM, new Values(requestId)); - } - - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declareStream(ARGS_STREAM, new Fields("request", "args")); - declarer.declareStream(RETURN_STREAM, new Fields("request", "return")); - declarer.declareStream(ID_STREAM, new Fields("request")); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/drpc/ReturnResults.java b/jstorm-client/src/main/java/backtype/storm/drpc/ReturnResults.java deleted file mode 100644 index 1b9a15b00..000000000 --- a/jstorm-client/src/main/java/backtype/storm/drpc/ReturnResults.java +++ /dev/null @@ -1,83 +0,0 @@ -package backtype.storm.drpc; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.thrift7.TException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import backtype.storm.Config; -import backtype.storm.generated.DistributedRPCInvocations; -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.Tuple; -import backtype.storm.utils.ServiceRegistry; -import backtype.storm.utils.Utils; - -public class ReturnResults extends BaseRichBolt { - public static final Logger LOG = LoggerFactory - .getLogger(ReturnResults.class); - OutputCollector _collector; - boolean local; - - Map _clients = new HashMap(); - - @Override - public void prepare(Map stormConf, TopologyContext context, - OutputCollector collector) { - _collector = collector; - local = stormConf.get(Config.STORM_CLUSTER_MODE).equals("local"); - } - - @Override - public void execute(Tuple input) { - String result = (String) input.getValue(0); - String returnInfo = (String) input.getValue(1); - if (returnInfo != null) { - Map retMap = (Map) Utils.from_json(returnInfo); - final String host = (String) retMap.get("host"); - final int port = Utils.getInt(retMap.get("port")); - String id = (String) retMap.get("id"); - DistributedRPCInvocations.Iface client; - if (local) { - client = (DistributedRPCInvocations.Iface) ServiceRegistry - .getService(host); - } else { - List server = new ArrayList() { - { - add(host); - add(port); - } - }; - - if (!_clients.containsKey(server)) { - _clients.put(server, new DRPCInvocationsClient(host, port)); - } - client = _clients.get(server); - } - - try { - client.result(id, result); - _collector.ack(input); - } catch (TException e) { - LOG.error("Failed to return results to DRPC server", e); - _collector.fail(input); - } - } - } - - @Override - public void cleanup() { - for (DRPCInvocationsClient c : _clients.values()) { - c.close(); - } - } - - public void declareOutputFields(OutputFieldsDeclarer declarer) { - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/generated/AlreadyAliveException.java b/jstorm-client/src/main/java/backtype/storm/generated/AlreadyAliveException.java deleted file mode 100644 index 9e1d6079a..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/AlreadyAliveException.java +++ /dev/null @@ -1,328 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class AlreadyAliveException extends Exception implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("AlreadyAliveException"); - - private static final org.apache.thrift7.protocol.TField MSG_FIELD_DESC = new org.apache.thrift7.protocol.TField("msg", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String msg; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - MSG((short)1, "msg"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // MSG - return MSG; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.MSG, new org.apache.thrift7.meta_data.FieldMetaData("msg", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(AlreadyAliveException.class, metaDataMap); - } - - public AlreadyAliveException() { - } - - public AlreadyAliveException( - String msg) - { - this(); - this.msg = msg; - } - - /** - * Performs a deep copy on other. - */ - public AlreadyAliveException(AlreadyAliveException other) { - if (other.is_set_msg()) { - this.msg = other.msg; - } - } - - public AlreadyAliveException deepCopy() { - return new AlreadyAliveException(this); - } - - @Override - public void clear() { - this.msg = null; - } - - public String get_msg() { - return this.msg; - } - - public void set_msg(String msg) { - this.msg = msg; - } - - public void unset_msg() { - this.msg = null; - } - - /** Returns true if field msg is set (has been assigned a value) and false otherwise */ - public boolean is_set_msg() { - return this.msg != null; - } - - public void set_msg_isSet(boolean value) { - if (!value) { - this.msg = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case MSG: - if (value == null) { - unset_msg(); - } else { - set_msg((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case MSG: - return get_msg(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case MSG: - return is_set_msg(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof AlreadyAliveException) - return this.equals((AlreadyAliveException)that); - return false; - } - - public boolean equals(AlreadyAliveException that) { - if (that == null) - return false; - - boolean this_present_msg = true && this.is_set_msg(); - boolean that_present_msg = true && that.is_set_msg(); - if (this_present_msg || that_present_msg) { - if (!(this_present_msg && that_present_msg)) - return false; - if (!this.msg.equals(that.msg)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_msg = true && (is_set_msg()); - builder.append(present_msg); - if (present_msg) - builder.append(msg); - - return builder.toHashCode(); - } - - public int compareTo(AlreadyAliveException other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - AlreadyAliveException typedOther = (AlreadyAliveException)other; - - lastComparison = Boolean.valueOf(is_set_msg()).compareTo(typedOther.is_set_msg()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_msg()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.msg, typedOther.msg); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // MSG - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.msg = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.msg != null) { - oprot.writeFieldBegin(MSG_FIELD_DESC); - oprot.writeString(this.msg); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("AlreadyAliveException("); - boolean first = true; - - sb.append("msg:"); - if (this.msg == null) { - sb.append("null"); - } else { - sb.append(this.msg); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_msg()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/AuthorizationException.java b/jstorm-client/src/main/java/backtype/storm/generated/AuthorizationException.java deleted file mode 100644 index 9cd7e5239..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/AuthorizationException.java +++ /dev/null @@ -1,328 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class AuthorizationException extends Exception implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("AuthorizationException"); - - private static final org.apache.thrift7.protocol.TField MSG_FIELD_DESC = new org.apache.thrift7.protocol.TField("msg", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String msg; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - MSG((short)1, "msg"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // MSG - return MSG; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.MSG, new org.apache.thrift7.meta_data.FieldMetaData("msg", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(AuthorizationException.class, metaDataMap); - } - - public AuthorizationException() { - } - - public AuthorizationException( - String msg) - { - this(); - this.msg = msg; - } - - /** - * Performs a deep copy on other. - */ - public AuthorizationException(AuthorizationException other) { - if (other.is_set_msg()) { - this.msg = other.msg; - } - } - - public AuthorizationException deepCopy() { - return new AuthorizationException(this); - } - - @Override - public void clear() { - this.msg = null; - } - - public String get_msg() { - return this.msg; - } - - public void set_msg(String msg) { - this.msg = msg; - } - - public void unset_msg() { - this.msg = null; - } - - /** Returns true if field msg is set (has been assigned a value) and false otherwise */ - public boolean is_set_msg() { - return this.msg != null; - } - - public void set_msg_isSet(boolean value) { - if (!value) { - this.msg = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case MSG: - if (value == null) { - unset_msg(); - } else { - set_msg((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case MSG: - return get_msg(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case MSG: - return is_set_msg(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof AuthorizationException) - return this.equals((AuthorizationException)that); - return false; - } - - public boolean equals(AuthorizationException that) { - if (that == null) - return false; - - boolean this_present_msg = true && this.is_set_msg(); - boolean that_present_msg = true && that.is_set_msg(); - if (this_present_msg || that_present_msg) { - if (!(this_present_msg && that_present_msg)) - return false; - if (!this.msg.equals(that.msg)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_msg = true && (is_set_msg()); - builder.append(present_msg); - if (present_msg) - builder.append(msg); - - return builder.toHashCode(); - } - - public int compareTo(AuthorizationException other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - AuthorizationException typedOther = (AuthorizationException)other; - - lastComparison = Boolean.valueOf(is_set_msg()).compareTo(typedOther.is_set_msg()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_msg()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.msg, typedOther.msg); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // MSG - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.msg = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.msg != null) { - oprot.writeFieldBegin(MSG_FIELD_DESC); - oprot.writeString(this.msg); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("AuthorizationException("); - boolean first = true; - - sb.append("msg:"); - if (this.msg == null) { - sb.append("null"); - } else { - sb.append(this.msg); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_msg()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/Bolt.java b/jstorm-client/src/main/java/backtype/storm/generated/Bolt.java deleted file mode 100644 index 44bf51488..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/Bolt.java +++ /dev/null @@ -1,427 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class Bolt implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("Bolt"); - - private static final org.apache.thrift7.protocol.TField BOLT_OBJECT_FIELD_DESC = new org.apache.thrift7.protocol.TField("bolt_object", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift7.protocol.TField COMMON_FIELD_DESC = new org.apache.thrift7.protocol.TField("common", org.apache.thrift7.protocol.TType.STRUCT, (short)2); - - private ComponentObject bolt_object; // required - private ComponentCommon common; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - BOLT_OBJECT((short)1, "bolt_object"), - COMMON((short)2, "common"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // BOLT_OBJECT - return BOLT_OBJECT; - case 2: // COMMON - return COMMON; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.BOLT_OBJECT, new org.apache.thrift7.meta_data.FieldMetaData("bolt_object", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ComponentObject.class))); - tmpMap.put(_Fields.COMMON, new org.apache.thrift7.meta_data.FieldMetaData("common", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ComponentCommon.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(Bolt.class, metaDataMap); - } - - public Bolt() { - } - - public Bolt( - ComponentObject bolt_object, - ComponentCommon common) - { - this(); - this.bolt_object = bolt_object; - this.common = common; - } - - /** - * Performs a deep copy on other. - */ - public Bolt(Bolt other) { - if (other.is_set_bolt_object()) { - this.bolt_object = new ComponentObject(other.bolt_object); - } - if (other.is_set_common()) { - this.common = new ComponentCommon(other.common); - } - } - - public Bolt deepCopy() { - return new Bolt(this); - } - - @Override - public void clear() { - this.bolt_object = null; - this.common = null; - } - - public ComponentObject get_bolt_object() { - return this.bolt_object; - } - - public void set_bolt_object(ComponentObject bolt_object) { - this.bolt_object = bolt_object; - } - - public void unset_bolt_object() { - this.bolt_object = null; - } - - /** Returns true if field bolt_object is set (has been assigned a value) and false otherwise */ - public boolean is_set_bolt_object() { - return this.bolt_object != null; - } - - public void set_bolt_object_isSet(boolean value) { - if (!value) { - this.bolt_object = null; - } - } - - public ComponentCommon get_common() { - return this.common; - } - - public void set_common(ComponentCommon common) { - this.common = common; - } - - public void unset_common() { - this.common = null; - } - - /** Returns true if field common is set (has been assigned a value) and false otherwise */ - public boolean is_set_common() { - return this.common != null; - } - - public void set_common_isSet(boolean value) { - if (!value) { - this.common = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case BOLT_OBJECT: - if (value == null) { - unset_bolt_object(); - } else { - set_bolt_object((ComponentObject)value); - } - break; - - case COMMON: - if (value == null) { - unset_common(); - } else { - set_common((ComponentCommon)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case BOLT_OBJECT: - return get_bolt_object(); - - case COMMON: - return get_common(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case BOLT_OBJECT: - return is_set_bolt_object(); - case COMMON: - return is_set_common(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof Bolt) - return this.equals((Bolt)that); - return false; - } - - public boolean equals(Bolt that) { - if (that == null) - return false; - - boolean this_present_bolt_object = true && this.is_set_bolt_object(); - boolean that_present_bolt_object = true && that.is_set_bolt_object(); - if (this_present_bolt_object || that_present_bolt_object) { - if (!(this_present_bolt_object && that_present_bolt_object)) - return false; - if (!this.bolt_object.equals(that.bolt_object)) - return false; - } - - boolean this_present_common = true && this.is_set_common(); - boolean that_present_common = true && that.is_set_common(); - if (this_present_common || that_present_common) { - if (!(this_present_common && that_present_common)) - return false; - if (!this.common.equals(that.common)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_bolt_object = true && (is_set_bolt_object()); - builder.append(present_bolt_object); - if (present_bolt_object) - builder.append(bolt_object); - - boolean present_common = true && (is_set_common()); - builder.append(present_common); - if (present_common) - builder.append(common); - - return builder.toHashCode(); - } - - public int compareTo(Bolt other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - Bolt typedOther = (Bolt)other; - - lastComparison = Boolean.valueOf(is_set_bolt_object()).compareTo(typedOther.is_set_bolt_object()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_bolt_object()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.bolt_object, typedOther.bolt_object); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_common()).compareTo(typedOther.is_set_common()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_common()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.common, typedOther.common); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // BOLT_OBJECT - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.bolt_object = new ComponentObject(); - this.bolt_object.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // COMMON - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.common = new ComponentCommon(); - this.common.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.bolt_object != null) { - oprot.writeFieldBegin(BOLT_OBJECT_FIELD_DESC); - this.bolt_object.write(oprot); - oprot.writeFieldEnd(); - } - if (this.common != null) { - oprot.writeFieldBegin(COMMON_FIELD_DESC); - this.common.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("Bolt("); - boolean first = true; - - sb.append("bolt_object:"); - if (this.bolt_object == null) { - sb.append("null"); - } else { - sb.append(this.bolt_object); - } - first = false; - if (!first) sb.append(", "); - sb.append("common:"); - if (this.common == null) { - sb.append("null"); - } else { - sb.append(this.common); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_bolt_object()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'bolt_object' is unset! Struct:" + toString()); - } - - if (!is_set_common()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'common' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/BoltStats.java b/jstorm-client/src/main/java/backtype/storm/generated/BoltStats.java deleted file mode 100644 index 31dced070..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/BoltStats.java +++ /dev/null @@ -1,1111 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class BoltStats implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("BoltStats"); - - private static final org.apache.thrift7.protocol.TField ACKED_FIELD_DESC = new org.apache.thrift7.protocol.TField("acked", org.apache.thrift7.protocol.TType.MAP, (short)1); - private static final org.apache.thrift7.protocol.TField FAILED_FIELD_DESC = new org.apache.thrift7.protocol.TField("failed", org.apache.thrift7.protocol.TType.MAP, (short)2); - private static final org.apache.thrift7.protocol.TField PROCESS_MS_AVG_FIELD_DESC = new org.apache.thrift7.protocol.TField("process_ms_avg", org.apache.thrift7.protocol.TType.MAP, (short)3); - private static final org.apache.thrift7.protocol.TField EXECUTED_FIELD_DESC = new org.apache.thrift7.protocol.TField("executed", org.apache.thrift7.protocol.TType.MAP, (short)4); - private static final org.apache.thrift7.protocol.TField EXECUTE_MS_AVG_FIELD_DESC = new org.apache.thrift7.protocol.TField("execute_ms_avg", org.apache.thrift7.protocol.TType.MAP, (short)5); - - private Map> acked; // required - private Map> failed; // required - private Map> process_ms_avg; // required - private Map> executed; // required - private Map> execute_ms_avg; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - ACKED((short)1, "acked"), - FAILED((short)2, "failed"), - PROCESS_MS_AVG((short)3, "process_ms_avg"), - EXECUTED((short)4, "executed"), - EXECUTE_MS_AVG((short)5, "execute_ms_avg"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // ACKED - return ACKED; - case 2: // FAILED - return FAILED; - case 3: // PROCESS_MS_AVG - return PROCESS_MS_AVG; - case 4: // EXECUTED - return EXECUTED; - case 5: // EXECUTE_MS_AVG - return EXECUTE_MS_AVG; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.ACKED, new org.apache.thrift7.meta_data.FieldMetaData("acked", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, GlobalStreamId.class), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I64))))); - tmpMap.put(_Fields.FAILED, new org.apache.thrift7.meta_data.FieldMetaData("failed", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, GlobalStreamId.class), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I64))))); - tmpMap.put(_Fields.PROCESS_MS_AVG, new org.apache.thrift7.meta_data.FieldMetaData("process_ms_avg", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, GlobalStreamId.class), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE))))); - tmpMap.put(_Fields.EXECUTED, new org.apache.thrift7.meta_data.FieldMetaData("executed", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, GlobalStreamId.class), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I64))))); - tmpMap.put(_Fields.EXECUTE_MS_AVG, new org.apache.thrift7.meta_data.FieldMetaData("execute_ms_avg", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, GlobalStreamId.class), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE))))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(BoltStats.class, metaDataMap); - } - - public BoltStats() { - } - - public BoltStats( - Map> acked, - Map> failed, - Map> process_ms_avg, - Map> executed, - Map> execute_ms_avg) - { - this(); - this.acked = acked; - this.failed = failed; - this.process_ms_avg = process_ms_avg; - this.executed = executed; - this.execute_ms_avg = execute_ms_avg; - } - - /** - * Performs a deep copy on other. - */ - public BoltStats(BoltStats other) { - if (other.is_set_acked()) { - Map> __this__acked = new HashMap>(); - for (Map.Entry> other_element : other.acked.entrySet()) { - - String other_element_key = other_element.getKey(); - Map other_element_value = other_element.getValue(); - - String __this__acked_copy_key = other_element_key; - - Map __this__acked_copy_value = new HashMap(); - for (Map.Entry other_element_value_element : other_element_value.entrySet()) { - - GlobalStreamId other_element_value_element_key = other_element_value_element.getKey(); - Long other_element_value_element_value = other_element_value_element.getValue(); - - GlobalStreamId __this__acked_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key); - - Long __this__acked_copy_value_copy_value = other_element_value_element_value; - - __this__acked_copy_value.put(__this__acked_copy_value_copy_key, __this__acked_copy_value_copy_value); - } - - __this__acked.put(__this__acked_copy_key, __this__acked_copy_value); - } - this.acked = __this__acked; - } - if (other.is_set_failed()) { - Map> __this__failed = new HashMap>(); - for (Map.Entry> other_element : other.failed.entrySet()) { - - String other_element_key = other_element.getKey(); - Map other_element_value = other_element.getValue(); - - String __this__failed_copy_key = other_element_key; - - Map __this__failed_copy_value = new HashMap(); - for (Map.Entry other_element_value_element : other_element_value.entrySet()) { - - GlobalStreamId other_element_value_element_key = other_element_value_element.getKey(); - Long other_element_value_element_value = other_element_value_element.getValue(); - - GlobalStreamId __this__failed_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key); - - Long __this__failed_copy_value_copy_value = other_element_value_element_value; - - __this__failed_copy_value.put(__this__failed_copy_value_copy_key, __this__failed_copy_value_copy_value); - } - - __this__failed.put(__this__failed_copy_key, __this__failed_copy_value); - } - this.failed = __this__failed; - } - if (other.is_set_process_ms_avg()) { - Map> __this__process_ms_avg = new HashMap>(); - for (Map.Entry> other_element : other.process_ms_avg.entrySet()) { - - String other_element_key = other_element.getKey(); - Map other_element_value = other_element.getValue(); - - String __this__process_ms_avg_copy_key = other_element_key; - - Map __this__process_ms_avg_copy_value = new HashMap(); - for (Map.Entry other_element_value_element : other_element_value.entrySet()) { - - GlobalStreamId other_element_value_element_key = other_element_value_element.getKey(); - Double other_element_value_element_value = other_element_value_element.getValue(); - - GlobalStreamId __this__process_ms_avg_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key); - - Double __this__process_ms_avg_copy_value_copy_value = other_element_value_element_value; - - __this__process_ms_avg_copy_value.put(__this__process_ms_avg_copy_value_copy_key, __this__process_ms_avg_copy_value_copy_value); - } - - __this__process_ms_avg.put(__this__process_ms_avg_copy_key, __this__process_ms_avg_copy_value); - } - this.process_ms_avg = __this__process_ms_avg; - } - if (other.is_set_executed()) { - Map> __this__executed = new HashMap>(); - for (Map.Entry> other_element : other.executed.entrySet()) { - - String other_element_key = other_element.getKey(); - Map other_element_value = other_element.getValue(); - - String __this__executed_copy_key = other_element_key; - - Map __this__executed_copy_value = new HashMap(); - for (Map.Entry other_element_value_element : other_element_value.entrySet()) { - - GlobalStreamId other_element_value_element_key = other_element_value_element.getKey(); - Long other_element_value_element_value = other_element_value_element.getValue(); - - GlobalStreamId __this__executed_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key); - - Long __this__executed_copy_value_copy_value = other_element_value_element_value; - - __this__executed_copy_value.put(__this__executed_copy_value_copy_key, __this__executed_copy_value_copy_value); - } - - __this__executed.put(__this__executed_copy_key, __this__executed_copy_value); - } - this.executed = __this__executed; - } - if (other.is_set_execute_ms_avg()) { - Map> __this__execute_ms_avg = new HashMap>(); - for (Map.Entry> other_element : other.execute_ms_avg.entrySet()) { - - String other_element_key = other_element.getKey(); - Map other_element_value = other_element.getValue(); - - String __this__execute_ms_avg_copy_key = other_element_key; - - Map __this__execute_ms_avg_copy_value = new HashMap(); - for (Map.Entry other_element_value_element : other_element_value.entrySet()) { - - GlobalStreamId other_element_value_element_key = other_element_value_element.getKey(); - Double other_element_value_element_value = other_element_value_element.getValue(); - - GlobalStreamId __this__execute_ms_avg_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key); - - Double __this__execute_ms_avg_copy_value_copy_value = other_element_value_element_value; - - __this__execute_ms_avg_copy_value.put(__this__execute_ms_avg_copy_value_copy_key, __this__execute_ms_avg_copy_value_copy_value); - } - - __this__execute_ms_avg.put(__this__execute_ms_avg_copy_key, __this__execute_ms_avg_copy_value); - } - this.execute_ms_avg = __this__execute_ms_avg; - } - } - - public BoltStats deepCopy() { - return new BoltStats(this); - } - - @Override - public void clear() { - this.acked = null; - this.failed = null; - this.process_ms_avg = null; - this.executed = null; - this.execute_ms_avg = null; - } - - public int get_acked_size() { - return (this.acked == null) ? 0 : this.acked.size(); - } - - public void put_to_acked(String key, Map val) { - if (this.acked == null) { - this.acked = new HashMap>(); - } - this.acked.put(key, val); - } - - public Map> get_acked() { - return this.acked; - } - - public void set_acked(Map> acked) { - this.acked = acked; - } - - public void unset_acked() { - this.acked = null; - } - - /** Returns true if field acked is set (has been assigned a value) and false otherwise */ - public boolean is_set_acked() { - return this.acked != null; - } - - public void set_acked_isSet(boolean value) { - if (!value) { - this.acked = null; - } - } - - public int get_failed_size() { - return (this.failed == null) ? 0 : this.failed.size(); - } - - public void put_to_failed(String key, Map val) { - if (this.failed == null) { - this.failed = new HashMap>(); - } - this.failed.put(key, val); - } - - public Map> get_failed() { - return this.failed; - } - - public void set_failed(Map> failed) { - this.failed = failed; - } - - public void unset_failed() { - this.failed = null; - } - - /** Returns true if field failed is set (has been assigned a value) and false otherwise */ - public boolean is_set_failed() { - return this.failed != null; - } - - public void set_failed_isSet(boolean value) { - if (!value) { - this.failed = null; - } - } - - public int get_process_ms_avg_size() { - return (this.process_ms_avg == null) ? 0 : this.process_ms_avg.size(); - } - - public void put_to_process_ms_avg(String key, Map val) { - if (this.process_ms_avg == null) { - this.process_ms_avg = new HashMap>(); - } - this.process_ms_avg.put(key, val); - } - - public Map> get_process_ms_avg() { - return this.process_ms_avg; - } - - public void set_process_ms_avg(Map> process_ms_avg) { - this.process_ms_avg = process_ms_avg; - } - - public void unset_process_ms_avg() { - this.process_ms_avg = null; - } - - /** Returns true if field process_ms_avg is set (has been assigned a value) and false otherwise */ - public boolean is_set_process_ms_avg() { - return this.process_ms_avg != null; - } - - public void set_process_ms_avg_isSet(boolean value) { - if (!value) { - this.process_ms_avg = null; - } - } - - public int get_executed_size() { - return (this.executed == null) ? 0 : this.executed.size(); - } - - public void put_to_executed(String key, Map val) { - if (this.executed == null) { - this.executed = new HashMap>(); - } - this.executed.put(key, val); - } - - public Map> get_executed() { - return this.executed; - } - - public void set_executed(Map> executed) { - this.executed = executed; - } - - public void unset_executed() { - this.executed = null; - } - - /** Returns true if field executed is set (has been assigned a value) and false otherwise */ - public boolean is_set_executed() { - return this.executed != null; - } - - public void set_executed_isSet(boolean value) { - if (!value) { - this.executed = null; - } - } - - public int get_execute_ms_avg_size() { - return (this.execute_ms_avg == null) ? 0 : this.execute_ms_avg.size(); - } - - public void put_to_execute_ms_avg(String key, Map val) { - if (this.execute_ms_avg == null) { - this.execute_ms_avg = new HashMap>(); - } - this.execute_ms_avg.put(key, val); - } - - public Map> get_execute_ms_avg() { - return this.execute_ms_avg; - } - - public void set_execute_ms_avg(Map> execute_ms_avg) { - this.execute_ms_avg = execute_ms_avg; - } - - public void unset_execute_ms_avg() { - this.execute_ms_avg = null; - } - - /** Returns true if field execute_ms_avg is set (has been assigned a value) and false otherwise */ - public boolean is_set_execute_ms_avg() { - return this.execute_ms_avg != null; - } - - public void set_execute_ms_avg_isSet(boolean value) { - if (!value) { - this.execute_ms_avg = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case ACKED: - if (value == null) { - unset_acked(); - } else { - set_acked((Map>)value); - } - break; - - case FAILED: - if (value == null) { - unset_failed(); - } else { - set_failed((Map>)value); - } - break; - - case PROCESS_MS_AVG: - if (value == null) { - unset_process_ms_avg(); - } else { - set_process_ms_avg((Map>)value); - } - break; - - case EXECUTED: - if (value == null) { - unset_executed(); - } else { - set_executed((Map>)value); - } - break; - - case EXECUTE_MS_AVG: - if (value == null) { - unset_execute_ms_avg(); - } else { - set_execute_ms_avg((Map>)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case ACKED: - return get_acked(); - - case FAILED: - return get_failed(); - - case PROCESS_MS_AVG: - return get_process_ms_avg(); - - case EXECUTED: - return get_executed(); - - case EXECUTE_MS_AVG: - return get_execute_ms_avg(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case ACKED: - return is_set_acked(); - case FAILED: - return is_set_failed(); - case PROCESS_MS_AVG: - return is_set_process_ms_avg(); - case EXECUTED: - return is_set_executed(); - case EXECUTE_MS_AVG: - return is_set_execute_ms_avg(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof BoltStats) - return this.equals((BoltStats)that); - return false; - } - - public boolean equals(BoltStats that) { - if (that == null) - return false; - - boolean this_present_acked = true && this.is_set_acked(); - boolean that_present_acked = true && that.is_set_acked(); - if (this_present_acked || that_present_acked) { - if (!(this_present_acked && that_present_acked)) - return false; - if (!this.acked.equals(that.acked)) - return false; - } - - boolean this_present_failed = true && this.is_set_failed(); - boolean that_present_failed = true && that.is_set_failed(); - if (this_present_failed || that_present_failed) { - if (!(this_present_failed && that_present_failed)) - return false; - if (!this.failed.equals(that.failed)) - return false; - } - - boolean this_present_process_ms_avg = true && this.is_set_process_ms_avg(); - boolean that_present_process_ms_avg = true && that.is_set_process_ms_avg(); - if (this_present_process_ms_avg || that_present_process_ms_avg) { - if (!(this_present_process_ms_avg && that_present_process_ms_avg)) - return false; - if (!this.process_ms_avg.equals(that.process_ms_avg)) - return false; - } - - boolean this_present_executed = true && this.is_set_executed(); - boolean that_present_executed = true && that.is_set_executed(); - if (this_present_executed || that_present_executed) { - if (!(this_present_executed && that_present_executed)) - return false; - if (!this.executed.equals(that.executed)) - return false; - } - - boolean this_present_execute_ms_avg = true && this.is_set_execute_ms_avg(); - boolean that_present_execute_ms_avg = true && that.is_set_execute_ms_avg(); - if (this_present_execute_ms_avg || that_present_execute_ms_avg) { - if (!(this_present_execute_ms_avg && that_present_execute_ms_avg)) - return false; - if (!this.execute_ms_avg.equals(that.execute_ms_avg)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_acked = true && (is_set_acked()); - builder.append(present_acked); - if (present_acked) - builder.append(acked); - - boolean present_failed = true && (is_set_failed()); - builder.append(present_failed); - if (present_failed) - builder.append(failed); - - boolean present_process_ms_avg = true && (is_set_process_ms_avg()); - builder.append(present_process_ms_avg); - if (present_process_ms_avg) - builder.append(process_ms_avg); - - boolean present_executed = true && (is_set_executed()); - builder.append(present_executed); - if (present_executed) - builder.append(executed); - - boolean present_execute_ms_avg = true && (is_set_execute_ms_avg()); - builder.append(present_execute_ms_avg); - if (present_execute_ms_avg) - builder.append(execute_ms_avg); - - return builder.toHashCode(); - } - - public int compareTo(BoltStats other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - BoltStats typedOther = (BoltStats)other; - - lastComparison = Boolean.valueOf(is_set_acked()).compareTo(typedOther.is_set_acked()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_acked()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.acked, typedOther.acked); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_failed()).compareTo(typedOther.is_set_failed()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_failed()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.failed, typedOther.failed); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_process_ms_avg()).compareTo(typedOther.is_set_process_ms_avg()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_process_ms_avg()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.process_ms_avg, typedOther.process_ms_avg); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_executed()).compareTo(typedOther.is_set_executed()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_executed()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.executed, typedOther.executed); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_execute_ms_avg()).compareTo(typedOther.is_set_execute_ms_avg()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_execute_ms_avg()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.execute_ms_avg, typedOther.execute_ms_avg); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // ACKED - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map45 = iprot.readMapBegin(); - this.acked = new HashMap>(2*_map45.size); - for (int _i46 = 0; _i46 < _map45.size; ++_i46) - { - String _key47; // required - Map _val48; // required - _key47 = iprot.readString(); - { - org.apache.thrift7.protocol.TMap _map49 = iprot.readMapBegin(); - _val48 = new HashMap(2*_map49.size); - for (int _i50 = 0; _i50 < _map49.size; ++_i50) - { - GlobalStreamId _key51; // required - long _val52; // required - _key51 = new GlobalStreamId(); - _key51.read(iprot); - _val52 = iprot.readI64(); - _val48.put(_key51, _val52); - } - iprot.readMapEnd(); - } - this.acked.put(_key47, _val48); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // FAILED - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map53 = iprot.readMapBegin(); - this.failed = new HashMap>(2*_map53.size); - for (int _i54 = 0; _i54 < _map53.size; ++_i54) - { - String _key55; // required - Map _val56; // required - _key55 = iprot.readString(); - { - org.apache.thrift7.protocol.TMap _map57 = iprot.readMapBegin(); - _val56 = new HashMap(2*_map57.size); - for (int _i58 = 0; _i58 < _map57.size; ++_i58) - { - GlobalStreamId _key59; // required - long _val60; // required - _key59 = new GlobalStreamId(); - _key59.read(iprot); - _val60 = iprot.readI64(); - _val56.put(_key59, _val60); - } - iprot.readMapEnd(); - } - this.failed.put(_key55, _val56); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 3: // PROCESS_MS_AVG - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map61 = iprot.readMapBegin(); - this.process_ms_avg = new HashMap>(2*_map61.size); - for (int _i62 = 0; _i62 < _map61.size; ++_i62) - { - String _key63; // required - Map _val64; // required - _key63 = iprot.readString(); - { - org.apache.thrift7.protocol.TMap _map65 = iprot.readMapBegin(); - _val64 = new HashMap(2*_map65.size); - for (int _i66 = 0; _i66 < _map65.size; ++_i66) - { - GlobalStreamId _key67; // required - double _val68; // required - _key67 = new GlobalStreamId(); - _key67.read(iprot); - _val68 = iprot.readDouble(); - _val64.put(_key67, _val68); - } - iprot.readMapEnd(); - } - this.process_ms_avg.put(_key63, _val64); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 4: // EXECUTED - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map69 = iprot.readMapBegin(); - this.executed = new HashMap>(2*_map69.size); - for (int _i70 = 0; _i70 < _map69.size; ++_i70) - { - String _key71; // required - Map _val72; // required - _key71 = iprot.readString(); - { - org.apache.thrift7.protocol.TMap _map73 = iprot.readMapBegin(); - _val72 = new HashMap(2*_map73.size); - for (int _i74 = 0; _i74 < _map73.size; ++_i74) - { - GlobalStreamId _key75; // required - long _val76; // required - _key75 = new GlobalStreamId(); - _key75.read(iprot); - _val76 = iprot.readI64(); - _val72.put(_key75, _val76); - } - iprot.readMapEnd(); - } - this.executed.put(_key71, _val72); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 5: // EXECUTE_MS_AVG - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map77 = iprot.readMapBegin(); - this.execute_ms_avg = new HashMap>(2*_map77.size); - for (int _i78 = 0; _i78 < _map77.size; ++_i78) - { - String _key79; // required - Map _val80; // required - _key79 = iprot.readString(); - { - org.apache.thrift7.protocol.TMap _map81 = iprot.readMapBegin(); - _val80 = new HashMap(2*_map81.size); - for (int _i82 = 0; _i82 < _map81.size; ++_i82) - { - GlobalStreamId _key83; // required - double _val84; // required - _key83 = new GlobalStreamId(); - _key83.read(iprot); - _val84 = iprot.readDouble(); - _val80.put(_key83, _val84); - } - iprot.readMapEnd(); - } - this.execute_ms_avg.put(_key79, _val80); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.acked != null) { - oprot.writeFieldBegin(ACKED_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.acked.size())); - for (Map.Entry> _iter85 : this.acked.entrySet()) - { - oprot.writeString(_iter85.getKey()); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRUCT, org.apache.thrift7.protocol.TType.I64, _iter85.getValue().size())); - for (Map.Entry _iter86 : _iter85.getValue().entrySet()) - { - _iter86.getKey().write(oprot); - oprot.writeI64(_iter86.getValue()); - } - oprot.writeMapEnd(); - } - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.failed != null) { - oprot.writeFieldBegin(FAILED_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.failed.size())); - for (Map.Entry> _iter87 : this.failed.entrySet()) - { - oprot.writeString(_iter87.getKey()); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRUCT, org.apache.thrift7.protocol.TType.I64, _iter87.getValue().size())); - for (Map.Entry _iter88 : _iter87.getValue().entrySet()) - { - _iter88.getKey().write(oprot); - oprot.writeI64(_iter88.getValue()); - } - oprot.writeMapEnd(); - } - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.process_ms_avg != null) { - oprot.writeFieldBegin(PROCESS_MS_AVG_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.process_ms_avg.size())); - for (Map.Entry> _iter89 : this.process_ms_avg.entrySet()) - { - oprot.writeString(_iter89.getKey()); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRUCT, org.apache.thrift7.protocol.TType.DOUBLE, _iter89.getValue().size())); - for (Map.Entry _iter90 : _iter89.getValue().entrySet()) - { - _iter90.getKey().write(oprot); - oprot.writeDouble(_iter90.getValue()); - } - oprot.writeMapEnd(); - } - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.executed != null) { - oprot.writeFieldBegin(EXECUTED_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.executed.size())); - for (Map.Entry> _iter91 : this.executed.entrySet()) - { - oprot.writeString(_iter91.getKey()); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRUCT, org.apache.thrift7.protocol.TType.I64, _iter91.getValue().size())); - for (Map.Entry _iter92 : _iter91.getValue().entrySet()) - { - _iter92.getKey().write(oprot); - oprot.writeI64(_iter92.getValue()); - } - oprot.writeMapEnd(); - } - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.execute_ms_avg != null) { - oprot.writeFieldBegin(EXECUTE_MS_AVG_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.execute_ms_avg.size())); - for (Map.Entry> _iter93 : this.execute_ms_avg.entrySet()) - { - oprot.writeString(_iter93.getKey()); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRUCT, org.apache.thrift7.protocol.TType.DOUBLE, _iter93.getValue().size())); - for (Map.Entry _iter94 : _iter93.getValue().entrySet()) - { - _iter94.getKey().write(oprot); - oprot.writeDouble(_iter94.getValue()); - } - oprot.writeMapEnd(); - } - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("BoltStats("); - boolean first = true; - - sb.append("acked:"); - if (this.acked == null) { - sb.append("null"); - } else { - sb.append(this.acked); - } - first = false; - if (!first) sb.append(", "); - sb.append("failed:"); - if (this.failed == null) { - sb.append("null"); - } else { - sb.append(this.failed); - } - first = false; - if (!first) sb.append(", "); - sb.append("process_ms_avg:"); - if (this.process_ms_avg == null) { - sb.append("null"); - } else { - sb.append(this.process_ms_avg); - } - first = false; - if (!first) sb.append(", "); - sb.append("executed:"); - if (this.executed == null) { - sb.append("null"); - } else { - sb.append(this.executed); - } - first = false; - if (!first) sb.append(", "); - sb.append("execute_ms_avg:"); - if (this.execute_ms_avg == null) { - sb.append("null"); - } else { - sb.append(this.execute_ms_avg); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_acked()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'acked' is unset! Struct:" + toString()); - } - - if (!is_set_failed()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'failed' is unset! Struct:" + toString()); - } - - if (!is_set_process_ms_avg()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'process_ms_avg' is unset! Struct:" + toString()); - } - - if (!is_set_executed()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'executed' is unset! Struct:" + toString()); - } - - if (!is_set_execute_ms_avg()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'execute_ms_avg' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/ClusterSummary.java b/jstorm-client/src/main/java/backtype/storm/generated/ClusterSummary.java deleted file mode 100644 index 9e4214013..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/ClusterSummary.java +++ /dev/null @@ -1,693 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class ClusterSummary implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("ClusterSummary"); - - private static final org.apache.thrift7.protocol.TField SUPERVISORS_FIELD_DESC = new org.apache.thrift7.protocol.TField("supervisors", org.apache.thrift7.protocol.TType.LIST, (short)1); - private static final org.apache.thrift7.protocol.TField NIMBUS_UPTIME_SECS_FIELD_DESC = new org.apache.thrift7.protocol.TField("nimbus_uptime_secs", org.apache.thrift7.protocol.TType.I32, (short)2); - private static final org.apache.thrift7.protocol.TField TOPOLOGIES_FIELD_DESC = new org.apache.thrift7.protocol.TField("topologies", org.apache.thrift7.protocol.TType.LIST, (short)3); - private static final org.apache.thrift7.protocol.TField VERSION_FIELD_DESC = new org.apache.thrift7.protocol.TField("version", org.apache.thrift7.protocol.TType.STRING, (short)4); - - private List supervisors; // required - private int nimbus_uptime_secs; // required - private List topologies; // required - private String version; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - SUPERVISORS((short)1, "supervisors"), - NIMBUS_UPTIME_SECS((short)2, "nimbus_uptime_secs"), - TOPOLOGIES((short)3, "topologies"), - VERSION((short)4, "version"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SUPERVISORS - return SUPERVISORS; - case 2: // NIMBUS_UPTIME_SECS - return NIMBUS_UPTIME_SECS; - case 3: // TOPOLOGIES - return TOPOLOGIES; - case 4: // VERSION - return VERSION; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __NIMBUS_UPTIME_SECS_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUPERVISORS, new org.apache.thrift7.meta_data.FieldMetaData("supervisors", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.ListMetaData(org.apache.thrift7.protocol.TType.LIST, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, SupervisorSummary.class)))); - tmpMap.put(_Fields.NIMBUS_UPTIME_SECS, new org.apache.thrift7.meta_data.FieldMetaData("nimbus_uptime_secs", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - tmpMap.put(_Fields.TOPOLOGIES, new org.apache.thrift7.meta_data.FieldMetaData("topologies", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.ListMetaData(org.apache.thrift7.protocol.TType.LIST, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, TopologySummary.class)))); - tmpMap.put(_Fields.VERSION, new org.apache.thrift7.meta_data.FieldMetaData("version", org.apache.thrift7.TFieldRequirementType.OPTIONAL, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(ClusterSummary.class, metaDataMap); - } - - public ClusterSummary() { - } - - public ClusterSummary( - List supervisors, - int nimbus_uptime_secs, - List topologies) - { - this(); - this.supervisors = supervisors; - this.nimbus_uptime_secs = nimbus_uptime_secs; - set_nimbus_uptime_secs_isSet(true); - this.topologies = topologies; - } - - /** - * Performs a deep copy on other. - */ - public ClusterSummary(ClusterSummary other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); - if (other.is_set_supervisors()) { - List __this__supervisors = new ArrayList(); - for (SupervisorSummary other_element : other.supervisors) { - __this__supervisors.add(new SupervisorSummary(other_element)); - } - this.supervisors = __this__supervisors; - } - this.nimbus_uptime_secs = other.nimbus_uptime_secs; - if (other.is_set_topologies()) { - List __this__topologies = new ArrayList(); - for (TopologySummary other_element : other.topologies) { - __this__topologies.add(new TopologySummary(other_element)); - } - this.topologies = __this__topologies; - } - if (other.is_set_version()) { - this.version = other.version; - } - } - - public ClusterSummary deepCopy() { - return new ClusterSummary(this); - } - - @Override - public void clear() { - this.supervisors = null; - set_nimbus_uptime_secs_isSet(false); - this.nimbus_uptime_secs = 0; - this.topologies = null; - this.version = null; - } - - public int get_supervisors_size() { - return (this.supervisors == null) ? 0 : this.supervisors.size(); - } - - public java.util.Iterator get_supervisors_iterator() { - return (this.supervisors == null) ? null : this.supervisors.iterator(); - } - - public void add_to_supervisors(SupervisorSummary elem) { - if (this.supervisors == null) { - this.supervisors = new ArrayList(); - } - this.supervisors.add(elem); - } - - public List get_supervisors() { - return this.supervisors; - } - - public void set_supervisors(List supervisors) { - this.supervisors = supervisors; - } - - public void unset_supervisors() { - this.supervisors = null; - } - - /** Returns true if field supervisors is set (has been assigned a value) and false otherwise */ - public boolean is_set_supervisors() { - return this.supervisors != null; - } - - public void set_supervisors_isSet(boolean value) { - if (!value) { - this.supervisors = null; - } - } - - public int get_nimbus_uptime_secs() { - return this.nimbus_uptime_secs; - } - - public void set_nimbus_uptime_secs(int nimbus_uptime_secs) { - this.nimbus_uptime_secs = nimbus_uptime_secs; - set_nimbus_uptime_secs_isSet(true); - } - - public void unset_nimbus_uptime_secs() { - __isset_bit_vector.clear(__NIMBUS_UPTIME_SECS_ISSET_ID); - } - - /** Returns true if field nimbus_uptime_secs is set (has been assigned a value) and false otherwise */ - public boolean is_set_nimbus_uptime_secs() { - return __isset_bit_vector.get(__NIMBUS_UPTIME_SECS_ISSET_ID); - } - - public void set_nimbus_uptime_secs_isSet(boolean value) { - __isset_bit_vector.set(__NIMBUS_UPTIME_SECS_ISSET_ID, value); - } - - public int get_topologies_size() { - return (this.topologies == null) ? 0 : this.topologies.size(); - } - - public java.util.Iterator get_topologies_iterator() { - return (this.topologies == null) ? null : this.topologies.iterator(); - } - - public void add_to_topologies(TopologySummary elem) { - if (this.topologies == null) { - this.topologies = new ArrayList(); - } - this.topologies.add(elem); - } - - public List get_topologies() { - return this.topologies; - } - - public void set_topologies(List topologies) { - this.topologies = topologies; - } - - public void unset_topologies() { - this.topologies = null; - } - - /** Returns true if field topologies is set (has been assigned a value) and false otherwise */ - public boolean is_set_topologies() { - return this.topologies != null; - } - - public void set_topologies_isSet(boolean value) { - if (!value) { - this.topologies = null; - } - } - - public String get_version() { - return this.version; - } - - public void set_version(String version) { - this.version = version; - } - - public void unset_version() { - this.version = null; - } - - /** Returns true if field version is set (has been assigned a value) and false otherwise */ - public boolean is_set_version() { - return this.version != null; - } - - public void set_version_isSet(boolean value) { - if (!value) { - this.version = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUPERVISORS: - if (value == null) { - unset_supervisors(); - } else { - set_supervisors((List)value); - } - break; - - case NIMBUS_UPTIME_SECS: - if (value == null) { - unset_nimbus_uptime_secs(); - } else { - set_nimbus_uptime_secs((Integer)value); - } - break; - - case TOPOLOGIES: - if (value == null) { - unset_topologies(); - } else { - set_topologies((List)value); - } - break; - - case VERSION: - if (value == null) { - unset_version(); - } else { - set_version((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUPERVISORS: - return get_supervisors(); - - case NIMBUS_UPTIME_SECS: - return Integer.valueOf(get_nimbus_uptime_secs()); - - case TOPOLOGIES: - return get_topologies(); - - case VERSION: - return get_version(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUPERVISORS: - return is_set_supervisors(); - case NIMBUS_UPTIME_SECS: - return is_set_nimbus_uptime_secs(); - case TOPOLOGIES: - return is_set_topologies(); - case VERSION: - return is_set_version(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof ClusterSummary) - return this.equals((ClusterSummary)that); - return false; - } - - public boolean equals(ClusterSummary that) { - if (that == null) - return false; - - boolean this_present_supervisors = true && this.is_set_supervisors(); - boolean that_present_supervisors = true && that.is_set_supervisors(); - if (this_present_supervisors || that_present_supervisors) { - if (!(this_present_supervisors && that_present_supervisors)) - return false; - if (!this.supervisors.equals(that.supervisors)) - return false; - } - - boolean this_present_nimbus_uptime_secs = true; - boolean that_present_nimbus_uptime_secs = true; - if (this_present_nimbus_uptime_secs || that_present_nimbus_uptime_secs) { - if (!(this_present_nimbus_uptime_secs && that_present_nimbus_uptime_secs)) - return false; - if (this.nimbus_uptime_secs != that.nimbus_uptime_secs) - return false; - } - - boolean this_present_topologies = true && this.is_set_topologies(); - boolean that_present_topologies = true && that.is_set_topologies(); - if (this_present_topologies || that_present_topologies) { - if (!(this_present_topologies && that_present_topologies)) - return false; - if (!this.topologies.equals(that.topologies)) - return false; - } - - boolean this_present_version = true && this.is_set_version(); - boolean that_present_version = true && that.is_set_version(); - if (this_present_version || that_present_version) { - if (!(this_present_version && that_present_version)) - return false; - if (!this.version.equals(that.version)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_supervisors = true && (is_set_supervisors()); - builder.append(present_supervisors); - if (present_supervisors) - builder.append(supervisors); - - boolean present_nimbus_uptime_secs = true; - builder.append(present_nimbus_uptime_secs); - if (present_nimbus_uptime_secs) - builder.append(nimbus_uptime_secs); - - boolean present_topologies = true && (is_set_topologies()); - builder.append(present_topologies); - if (present_topologies) - builder.append(topologies); - - boolean present_version = true && (is_set_version()); - builder.append(present_version); - if (present_version) - builder.append(version); - - return builder.toHashCode(); - } - - public int compareTo(ClusterSummary other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - ClusterSummary typedOther = (ClusterSummary)other; - - lastComparison = Boolean.valueOf(is_set_supervisors()).compareTo(typedOther.is_set_supervisors()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_supervisors()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.supervisors, typedOther.supervisors); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_nimbus_uptime_secs()).compareTo(typedOther.is_set_nimbus_uptime_secs()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_nimbus_uptime_secs()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.nimbus_uptime_secs, typedOther.nimbus_uptime_secs); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_topologies()).compareTo(typedOther.is_set_topologies()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_topologies()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.topologies, typedOther.topologies); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_version()).compareTo(typedOther.is_set_version()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_version()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.version, typedOther.version); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // SUPERVISORS - if (field.type == org.apache.thrift7.protocol.TType.LIST) { - { - org.apache.thrift7.protocol.TList _list37 = iprot.readListBegin(); - this.supervisors = new ArrayList(_list37.size); - for (int _i38 = 0; _i38 < _list37.size; ++_i38) - { - SupervisorSummary _elem39; // required - _elem39 = new SupervisorSummary(); - _elem39.read(iprot); - this.supervisors.add(_elem39); - } - iprot.readListEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // NIMBUS_UPTIME_SECS - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.nimbus_uptime_secs = iprot.readI32(); - set_nimbus_uptime_secs_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 3: // TOPOLOGIES - if (field.type == org.apache.thrift7.protocol.TType.LIST) { - { - org.apache.thrift7.protocol.TList _list40 = iprot.readListBegin(); - this.topologies = new ArrayList(_list40.size); - for (int _i41 = 0; _i41 < _list40.size; ++_i41) - { - TopologySummary _elem42; // required - _elem42 = new TopologySummary(); - _elem42.read(iprot); - this.topologies.add(_elem42); - } - iprot.readListEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 4: // VERSION - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.version = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.supervisors != null) { - oprot.writeFieldBegin(SUPERVISORS_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift7.protocol.TList(org.apache.thrift7.protocol.TType.STRUCT, this.supervisors.size())); - for (SupervisorSummary _iter43 : this.supervisors) - { - _iter43.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldBegin(NIMBUS_UPTIME_SECS_FIELD_DESC); - oprot.writeI32(this.nimbus_uptime_secs); - oprot.writeFieldEnd(); - if (this.topologies != null) { - oprot.writeFieldBegin(TOPOLOGIES_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift7.protocol.TList(org.apache.thrift7.protocol.TType.STRUCT, this.topologies.size())); - for (TopologySummary _iter44 : this.topologies) - { - _iter44.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (this.version != null) { - if (is_set_version()) { - oprot.writeFieldBegin(VERSION_FIELD_DESC); - oprot.writeString(this.version); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("ClusterSummary("); - boolean first = true; - - sb.append("supervisors:"); - if (this.supervisors == null) { - sb.append("null"); - } else { - sb.append(this.supervisors); - } - first = false; - if (!first) sb.append(", "); - sb.append("nimbus_uptime_secs:"); - sb.append(this.nimbus_uptime_secs); - first = false; - if (!first) sb.append(", "); - sb.append("topologies:"); - if (this.topologies == null) { - sb.append("null"); - } else { - sb.append(this.topologies); - } - first = false; - if (is_set_version()) { - if (!first) sb.append(", "); - sb.append("version:"); - if (this.version == null) { - sb.append("null"); - } else { - sb.append(this.version); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_supervisors()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'supervisors' is unset! Struct:" + toString()); - } - - if (!is_set_nimbus_uptime_secs()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'nimbus_uptime_secs' is unset! Struct:" + toString()); - } - - if (!is_set_topologies()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'topologies' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bit_vector = new BitSet(1); - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/ComponentCommon.java b/jstorm-client/src/main/java/backtype/storm/generated/ComponentCommon.java deleted file mode 100644 index 273f8be42..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/ComponentCommon.java +++ /dev/null @@ -1,707 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class ComponentCommon implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("ComponentCommon"); - - private static final org.apache.thrift7.protocol.TField INPUTS_FIELD_DESC = new org.apache.thrift7.protocol.TField("inputs", org.apache.thrift7.protocol.TType.MAP, (short)1); - private static final org.apache.thrift7.protocol.TField STREAMS_FIELD_DESC = new org.apache.thrift7.protocol.TField("streams", org.apache.thrift7.protocol.TType.MAP, (short)2); - private static final org.apache.thrift7.protocol.TField PARALLELISM_HINT_FIELD_DESC = new org.apache.thrift7.protocol.TField("parallelism_hint", org.apache.thrift7.protocol.TType.I32, (short)3); - private static final org.apache.thrift7.protocol.TField JSON_CONF_FIELD_DESC = new org.apache.thrift7.protocol.TField("json_conf", org.apache.thrift7.protocol.TType.STRING, (short)4); - - private Map inputs; // required - private Map streams; // required - private int parallelism_hint; // required - private String json_conf; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - INPUTS((short)1, "inputs"), - STREAMS((short)2, "streams"), - PARALLELISM_HINT((short)3, "parallelism_hint"), - JSON_CONF((short)4, "json_conf"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // INPUTS - return INPUTS; - case 2: // STREAMS - return STREAMS; - case 3: // PARALLELISM_HINT - return PARALLELISM_HINT; - case 4: // JSON_CONF - return JSON_CONF; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __PARALLELISM_HINT_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.INPUTS, new org.apache.thrift7.meta_data.FieldMetaData("inputs", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, GlobalStreamId.class), - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, Grouping.class)))); - tmpMap.put(_Fields.STREAMS, new org.apache.thrift7.meta_data.FieldMetaData("streams", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, StreamInfo.class)))); - tmpMap.put(_Fields.PARALLELISM_HINT, new org.apache.thrift7.meta_data.FieldMetaData("parallelism_hint", org.apache.thrift7.TFieldRequirementType.OPTIONAL, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - tmpMap.put(_Fields.JSON_CONF, new org.apache.thrift7.meta_data.FieldMetaData("json_conf", org.apache.thrift7.TFieldRequirementType.OPTIONAL, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(ComponentCommon.class, metaDataMap); - } - - public ComponentCommon() { - } - - public ComponentCommon( - Map inputs, - Map streams) - { - this(); - this.inputs = inputs; - this.streams = streams; - } - - /** - * Performs a deep copy on other. - */ - public ComponentCommon(ComponentCommon other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); - if (other.is_set_inputs()) { - Map __this__inputs = new HashMap(); - for (Map.Entry other_element : other.inputs.entrySet()) { - - GlobalStreamId other_element_key = other_element.getKey(); - Grouping other_element_value = other_element.getValue(); - - GlobalStreamId __this__inputs_copy_key = new GlobalStreamId(other_element_key); - - Grouping __this__inputs_copy_value = new Grouping(other_element_value); - - __this__inputs.put(__this__inputs_copy_key, __this__inputs_copy_value); - } - this.inputs = __this__inputs; - } - if (other.is_set_streams()) { - Map __this__streams = new HashMap(); - for (Map.Entry other_element : other.streams.entrySet()) { - - String other_element_key = other_element.getKey(); - StreamInfo other_element_value = other_element.getValue(); - - String __this__streams_copy_key = other_element_key; - - StreamInfo __this__streams_copy_value = new StreamInfo(other_element_value); - - __this__streams.put(__this__streams_copy_key, __this__streams_copy_value); - } - this.streams = __this__streams; - } - this.parallelism_hint = other.parallelism_hint; - if (other.is_set_json_conf()) { - this.json_conf = other.json_conf; - } - } - - public ComponentCommon deepCopy() { - return new ComponentCommon(this); - } - - @Override - public void clear() { - this.inputs = null; - this.streams = null; - set_parallelism_hint_isSet(false); - this.parallelism_hint = 0; - this.json_conf = null; - } - - public int get_inputs_size() { - return (this.inputs == null) ? 0 : this.inputs.size(); - } - - public void put_to_inputs(GlobalStreamId key, Grouping val) { - if (this.inputs == null) { - this.inputs = new HashMap(); - } - this.inputs.put(key, val); - } - - public Map get_inputs() { - return this.inputs; - } - - public void set_inputs(Map inputs) { - this.inputs = inputs; - } - - public void unset_inputs() { - this.inputs = null; - } - - /** Returns true if field inputs is set (has been assigned a value) and false otherwise */ - public boolean is_set_inputs() { - return this.inputs != null; - } - - public void set_inputs_isSet(boolean value) { - if (!value) { - this.inputs = null; - } - } - - public int get_streams_size() { - return (this.streams == null) ? 0 : this.streams.size(); - } - - public void put_to_streams(String key, StreamInfo val) { - if (this.streams == null) { - this.streams = new HashMap(); - } - this.streams.put(key, val); - } - - public Map get_streams() { - return this.streams; - } - - public void set_streams(Map streams) { - this.streams = streams; - } - - public void unset_streams() { - this.streams = null; - } - - /** Returns true if field streams is set (has been assigned a value) and false otherwise */ - public boolean is_set_streams() { - return this.streams != null; - } - - public void set_streams_isSet(boolean value) { - if (!value) { - this.streams = null; - } - } - - public int get_parallelism_hint() { - return this.parallelism_hint; - } - - public void set_parallelism_hint(int parallelism_hint) { - this.parallelism_hint = parallelism_hint; - set_parallelism_hint_isSet(true); - } - - public void unset_parallelism_hint() { - __isset_bit_vector.clear(__PARALLELISM_HINT_ISSET_ID); - } - - /** Returns true if field parallelism_hint is set (has been assigned a value) and false otherwise */ - public boolean is_set_parallelism_hint() { - return __isset_bit_vector.get(__PARALLELISM_HINT_ISSET_ID); - } - - public void set_parallelism_hint_isSet(boolean value) { - __isset_bit_vector.set(__PARALLELISM_HINT_ISSET_ID, value); - } - - public String get_json_conf() { - return this.json_conf; - } - - public void set_json_conf(String json_conf) { - this.json_conf = json_conf; - } - - public void unset_json_conf() { - this.json_conf = null; - } - - /** Returns true if field json_conf is set (has been assigned a value) and false otherwise */ - public boolean is_set_json_conf() { - return this.json_conf != null; - } - - public void set_json_conf_isSet(boolean value) { - if (!value) { - this.json_conf = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case INPUTS: - if (value == null) { - unset_inputs(); - } else { - set_inputs((Map)value); - } - break; - - case STREAMS: - if (value == null) { - unset_streams(); - } else { - set_streams((Map)value); - } - break; - - case PARALLELISM_HINT: - if (value == null) { - unset_parallelism_hint(); - } else { - set_parallelism_hint((Integer)value); - } - break; - - case JSON_CONF: - if (value == null) { - unset_json_conf(); - } else { - set_json_conf((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case INPUTS: - return get_inputs(); - - case STREAMS: - return get_streams(); - - case PARALLELISM_HINT: - return Integer.valueOf(get_parallelism_hint()); - - case JSON_CONF: - return get_json_conf(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case INPUTS: - return is_set_inputs(); - case STREAMS: - return is_set_streams(); - case PARALLELISM_HINT: - return is_set_parallelism_hint(); - case JSON_CONF: - return is_set_json_conf(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof ComponentCommon) - return this.equals((ComponentCommon)that); - return false; - } - - public boolean equals(ComponentCommon that) { - if (that == null) - return false; - - boolean this_present_inputs = true && this.is_set_inputs(); - boolean that_present_inputs = true && that.is_set_inputs(); - if (this_present_inputs || that_present_inputs) { - if (!(this_present_inputs && that_present_inputs)) - return false; - if (!this.inputs.equals(that.inputs)) - return false; - } - - boolean this_present_streams = true && this.is_set_streams(); - boolean that_present_streams = true && that.is_set_streams(); - if (this_present_streams || that_present_streams) { - if (!(this_present_streams && that_present_streams)) - return false; - if (!this.streams.equals(that.streams)) - return false; - } - - boolean this_present_parallelism_hint = true && this.is_set_parallelism_hint(); - boolean that_present_parallelism_hint = true && that.is_set_parallelism_hint(); - if (this_present_parallelism_hint || that_present_parallelism_hint) { - if (!(this_present_parallelism_hint && that_present_parallelism_hint)) - return false; - if (this.parallelism_hint != that.parallelism_hint) - return false; - } - - boolean this_present_json_conf = true && this.is_set_json_conf(); - boolean that_present_json_conf = true && that.is_set_json_conf(); - if (this_present_json_conf || that_present_json_conf) { - if (!(this_present_json_conf && that_present_json_conf)) - return false; - if (!this.json_conf.equals(that.json_conf)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_inputs = true && (is_set_inputs()); - builder.append(present_inputs); - if (present_inputs) - builder.append(inputs); - - boolean present_streams = true && (is_set_streams()); - builder.append(present_streams); - if (present_streams) - builder.append(streams); - - boolean present_parallelism_hint = true && (is_set_parallelism_hint()); - builder.append(present_parallelism_hint); - if (present_parallelism_hint) - builder.append(parallelism_hint); - - boolean present_json_conf = true && (is_set_json_conf()); - builder.append(present_json_conf); - if (present_json_conf) - builder.append(json_conf); - - return builder.toHashCode(); - } - - public int compareTo(ComponentCommon other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - ComponentCommon typedOther = (ComponentCommon)other; - - lastComparison = Boolean.valueOf(is_set_inputs()).compareTo(typedOther.is_set_inputs()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_inputs()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.inputs, typedOther.inputs); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_streams()).compareTo(typedOther.is_set_streams()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_streams()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.streams, typedOther.streams); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_parallelism_hint()).compareTo(typedOther.is_set_parallelism_hint()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_parallelism_hint()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.parallelism_hint, typedOther.parallelism_hint); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_json_conf()).compareTo(typedOther.is_set_json_conf()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_json_conf()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.json_conf, typedOther.json_conf); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // INPUTS - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map12 = iprot.readMapBegin(); - this.inputs = new HashMap(2*_map12.size); - for (int _i13 = 0; _i13 < _map12.size; ++_i13) - { - GlobalStreamId _key14; // required - Grouping _val15; // required - _key14 = new GlobalStreamId(); - _key14.read(iprot); - _val15 = new Grouping(); - _val15.read(iprot); - this.inputs.put(_key14, _val15); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // STREAMS - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map16 = iprot.readMapBegin(); - this.streams = new HashMap(2*_map16.size); - for (int _i17 = 0; _i17 < _map16.size; ++_i17) - { - String _key18; // required - StreamInfo _val19; // required - _key18 = iprot.readString(); - _val19 = new StreamInfo(); - _val19.read(iprot); - this.streams.put(_key18, _val19); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 3: // PARALLELISM_HINT - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.parallelism_hint = iprot.readI32(); - set_parallelism_hint_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 4: // JSON_CONF - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.json_conf = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.inputs != null) { - oprot.writeFieldBegin(INPUTS_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRUCT, org.apache.thrift7.protocol.TType.STRUCT, this.inputs.size())); - for (Map.Entry _iter20 : this.inputs.entrySet()) - { - _iter20.getKey().write(oprot); - _iter20.getValue().write(oprot); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.streams != null) { - oprot.writeFieldBegin(STREAMS_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.STRUCT, this.streams.size())); - for (Map.Entry _iter21 : this.streams.entrySet()) - { - oprot.writeString(_iter21.getKey()); - _iter21.getValue().write(oprot); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (is_set_parallelism_hint()) { - oprot.writeFieldBegin(PARALLELISM_HINT_FIELD_DESC); - oprot.writeI32(this.parallelism_hint); - oprot.writeFieldEnd(); - } - if (this.json_conf != null) { - if (is_set_json_conf()) { - oprot.writeFieldBegin(JSON_CONF_FIELD_DESC); - oprot.writeString(this.json_conf); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("ComponentCommon("); - boolean first = true; - - sb.append("inputs:"); - if (this.inputs == null) { - sb.append("null"); - } else { - sb.append(this.inputs); - } - first = false; - if (!first) sb.append(", "); - sb.append("streams:"); - if (this.streams == null) { - sb.append("null"); - } else { - sb.append(this.streams); - } - first = false; - if (is_set_parallelism_hint()) { - if (!first) sb.append(", "); - sb.append("parallelism_hint:"); - sb.append(this.parallelism_hint); - first = false; - } - if (is_set_json_conf()) { - if (!first) sb.append(", "); - sb.append("json_conf:"); - if (this.json_conf == null) { - sb.append("null"); - } else { - sb.append(this.json_conf); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_inputs()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'inputs' is unset! Struct:" + toString()); - } - - if (!is_set_streams()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'streams' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bit_vector = new BitSet(1); - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/ComponentObject.java b/jstorm-client/src/main/java/backtype/storm/generated/ComponentObject.java deleted file mode 100644 index 0509519f1..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/ComponentObject.java +++ /dev/null @@ -1,385 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class ComponentObject extends org.apache.thrift7.TUnion { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("ComponentObject"); - private static final org.apache.thrift7.protocol.TField SERIALIZED_JAVA_FIELD_DESC = new org.apache.thrift7.protocol.TField("serialized_java", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField SHELL_FIELD_DESC = new org.apache.thrift7.protocol.TField("shell", org.apache.thrift7.protocol.TType.STRUCT, (short)2); - private static final org.apache.thrift7.protocol.TField JAVA_OBJECT_FIELD_DESC = new org.apache.thrift7.protocol.TField("java_object", org.apache.thrift7.protocol.TType.STRUCT, (short)3); - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - SERIALIZED_JAVA((short)1, "serialized_java"), - SHELL((short)2, "shell"), - JAVA_OBJECT((short)3, "java_object"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SERIALIZED_JAVA - return SERIALIZED_JAVA; - case 2: // SHELL - return SHELL; - case 3: // JAVA_OBJECT - return JAVA_OBJECT; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SERIALIZED_JAVA, new org.apache.thrift7.meta_data.FieldMetaData("serialized_java", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING , true))); - tmpMap.put(_Fields.SHELL, new org.apache.thrift7.meta_data.FieldMetaData("shell", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ShellComponent.class))); - tmpMap.put(_Fields.JAVA_OBJECT, new org.apache.thrift7.meta_data.FieldMetaData("java_object", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, JavaObject.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(ComponentObject.class, metaDataMap); - } - - public ComponentObject() { - super(); - } - - public ComponentObject(_Fields setField, Object value) { - super(setField, value); - } - - public ComponentObject(ComponentObject other) { - super(other); - } - public ComponentObject deepCopy() { - return new ComponentObject(this); - } - - public static ComponentObject serialized_java(ByteBuffer value) { - ComponentObject x = new ComponentObject(); - x.set_serialized_java(value); - return x; - } - - public static ComponentObject serialized_java(byte[] value) { - ComponentObject x = new ComponentObject(); - x.set_serialized_java(ByteBuffer.wrap(value)); - return x; - } - - public static ComponentObject shell(ShellComponent value) { - ComponentObject x = new ComponentObject(); - x.set_shell(value); - return x; - } - - public static ComponentObject java_object(JavaObject value) { - ComponentObject x = new ComponentObject(); - x.set_java_object(value); - return x; - } - - - @Override - protected void checkType(_Fields setField, Object value) throws ClassCastException { - switch (setField) { - case SERIALIZED_JAVA: - if (value instanceof ByteBuffer) { - break; - } - throw new ClassCastException("Was expecting value of type ByteBuffer for field 'serialized_java', but got " + value.getClass().getSimpleName()); - case SHELL: - if (value instanceof ShellComponent) { - break; - } - throw new ClassCastException("Was expecting value of type ShellComponent for field 'shell', but got " + value.getClass().getSimpleName()); - case JAVA_OBJECT: - if (value instanceof JavaObject) { - break; - } - throw new ClassCastException("Was expecting value of type JavaObject for field 'java_object', but got " + value.getClass().getSimpleName()); - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected Object readValue(org.apache.thrift7.protocol.TProtocol iprot, org.apache.thrift7.protocol.TField field) throws org.apache.thrift7.TException { - _Fields setField = _Fields.findByThriftId(field.id); - if (setField != null) { - switch (setField) { - case SERIALIZED_JAVA: - if (field.type == SERIALIZED_JAVA_FIELD_DESC.type) { - ByteBuffer serialized_java; - serialized_java = iprot.readBinary(); - return serialized_java; - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case SHELL: - if (field.type == SHELL_FIELD_DESC.type) { - ShellComponent shell; - shell = new ShellComponent(); - shell.read(iprot); - return shell; - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case JAVA_OBJECT: - if (field.type == JAVA_OBJECT_FIELD_DESC.type) { - JavaObject java_object; - java_object = new JavaObject(); - java_object.read(iprot); - return java_object; - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - } - - @Override - protected void writeValue(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - switch (setField_) { - case SERIALIZED_JAVA: - ByteBuffer serialized_java = (ByteBuffer)value_; - oprot.writeBinary(serialized_java); - return; - case SHELL: - ShellComponent shell = (ShellComponent)value_; - shell.write(oprot); - return; - case JAVA_OBJECT: - JavaObject java_object = (JavaObject)value_; - java_object.write(oprot); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected org.apache.thrift7.protocol.TField getFieldDesc(_Fields setField) { - switch (setField) { - case SERIALIZED_JAVA: - return SERIALIZED_JAVA_FIELD_DESC; - case SHELL: - return SHELL_FIELD_DESC; - case JAVA_OBJECT: - return JAVA_OBJECT_FIELD_DESC; - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected org.apache.thrift7.protocol.TStruct getStructDesc() { - return STRUCT_DESC; - } - - @Override - protected _Fields enumForId(short id) { - return _Fields.findByThriftIdOrThrow(id); - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - - public byte[] get_serialized_java() { - set_serialized_java(org.apache.thrift7.TBaseHelper.rightSize(buffer_for_serialized_java())); - ByteBuffer b = buffer_for_serialized_java(); - return b == null ? null : b.array(); - } - - public ByteBuffer buffer_for_serialized_java() { - if (getSetField() == _Fields.SERIALIZED_JAVA) { - return (ByteBuffer)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'serialized_java' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void set_serialized_java(byte[] value) { - set_serialized_java(ByteBuffer.wrap(value)); - } - - public void set_serialized_java(ByteBuffer value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.SERIALIZED_JAVA; - value_ = value; - } - - public ShellComponent get_shell() { - if (getSetField() == _Fields.SHELL) { - return (ShellComponent)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'shell' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void set_shell(ShellComponent value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.SHELL; - value_ = value; - } - - public JavaObject get_java_object() { - if (getSetField() == _Fields.JAVA_OBJECT) { - return (JavaObject)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'java_object' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void set_java_object(JavaObject value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.JAVA_OBJECT; - value_ = value; - } - - public boolean is_set_serialized_java() { - return setField_ == _Fields.SERIALIZED_JAVA; - } - - - public boolean is_set_shell() { - return setField_ == _Fields.SHELL; - } - - - public boolean is_set_java_object() { - return setField_ == _Fields.JAVA_OBJECT; - } - - - public boolean equals(Object other) { - if (other instanceof ComponentObject) { - return equals((ComponentObject)other); - } else { - return false; - } - } - - public boolean equals(ComponentObject other) { - return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); - } - - @Override - public int compareTo(ComponentObject other) { - int lastComparison = org.apache.thrift7.TBaseHelper.compareTo(getSetField(), other.getSetField()); - if (lastComparison == 0) { - return org.apache.thrift7.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); - } - return lastComparison; - } - - - @Override - public int hashCode() { - HashCodeBuilder hcb = new HashCodeBuilder(); - hcb.append(this.getClass().getName()); - org.apache.thrift7.TFieldIdEnum setField = getSetField(); - if (setField != null) { - hcb.append(setField.getThriftFieldId()); - Object value = getFieldValue(); - if (value instanceof org.apache.thrift7.TEnum) { - hcb.append(((org.apache.thrift7.TEnum)getFieldValue()).getValue()); - } else { - hcb.append(value); - } - } - return hcb.toHashCode(); - } - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - -} diff --git a/jstorm-client/src/main/java/backtype/storm/generated/DRPCExecutionException.java b/jstorm-client/src/main/java/backtype/storm/generated/DRPCExecutionException.java deleted file mode 100644 index 4dbac482d..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/DRPCExecutionException.java +++ /dev/null @@ -1,328 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class DRPCExecutionException extends Exception implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("DRPCExecutionException"); - - private static final org.apache.thrift7.protocol.TField MSG_FIELD_DESC = new org.apache.thrift7.protocol.TField("msg", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String msg; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - MSG((short)1, "msg"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // MSG - return MSG; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.MSG, new org.apache.thrift7.meta_data.FieldMetaData("msg", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(DRPCExecutionException.class, metaDataMap); - } - - public DRPCExecutionException() { - } - - public DRPCExecutionException( - String msg) - { - this(); - this.msg = msg; - } - - /** - * Performs a deep copy on other. - */ - public DRPCExecutionException(DRPCExecutionException other) { - if (other.is_set_msg()) { - this.msg = other.msg; - } - } - - public DRPCExecutionException deepCopy() { - return new DRPCExecutionException(this); - } - - @Override - public void clear() { - this.msg = null; - } - - public String get_msg() { - return this.msg; - } - - public void set_msg(String msg) { - this.msg = msg; - } - - public void unset_msg() { - this.msg = null; - } - - /** Returns true if field msg is set (has been assigned a value) and false otherwise */ - public boolean is_set_msg() { - return this.msg != null; - } - - public void set_msg_isSet(boolean value) { - if (!value) { - this.msg = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case MSG: - if (value == null) { - unset_msg(); - } else { - set_msg((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case MSG: - return get_msg(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case MSG: - return is_set_msg(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof DRPCExecutionException) - return this.equals((DRPCExecutionException)that); - return false; - } - - public boolean equals(DRPCExecutionException that) { - if (that == null) - return false; - - boolean this_present_msg = true && this.is_set_msg(); - boolean that_present_msg = true && that.is_set_msg(); - if (this_present_msg || that_present_msg) { - if (!(this_present_msg && that_present_msg)) - return false; - if (!this.msg.equals(that.msg)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_msg = true && (is_set_msg()); - builder.append(present_msg); - if (present_msg) - builder.append(msg); - - return builder.toHashCode(); - } - - public int compareTo(DRPCExecutionException other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - DRPCExecutionException typedOther = (DRPCExecutionException)other; - - lastComparison = Boolean.valueOf(is_set_msg()).compareTo(typedOther.is_set_msg()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_msg()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.msg, typedOther.msg); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // MSG - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.msg = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.msg != null) { - oprot.writeFieldBegin(MSG_FIELD_DESC); - oprot.writeString(this.msg); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("DRPCExecutionException("); - boolean first = true; - - sb.append("msg:"); - if (this.msg == null) { - sb.append("null"); - } else { - sb.append(this.msg); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_msg()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/DRPCRequest.java b/jstorm-client/src/main/java/backtype/storm/generated/DRPCRequest.java deleted file mode 100644 index ce8cfb5ff..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/DRPCRequest.java +++ /dev/null @@ -1,425 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class DRPCRequest implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("DRPCRequest"); - - private static final org.apache.thrift7.protocol.TField FUNC_ARGS_FIELD_DESC = new org.apache.thrift7.protocol.TField("func_args", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField REQUEST_ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("request_id", org.apache.thrift7.protocol.TType.STRING, (short)2); - - private String func_args; // required - private String request_id; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - FUNC_ARGS((short)1, "func_args"), - REQUEST_ID((short)2, "request_id"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // FUNC_ARGS - return FUNC_ARGS; - case 2: // REQUEST_ID - return REQUEST_ID; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.FUNC_ARGS, new org.apache.thrift7.meta_data.FieldMetaData("func_args", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.REQUEST_ID, new org.apache.thrift7.meta_data.FieldMetaData("request_id", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(DRPCRequest.class, metaDataMap); - } - - public DRPCRequest() { - } - - public DRPCRequest( - String func_args, - String request_id) - { - this(); - this.func_args = func_args; - this.request_id = request_id; - } - - /** - * Performs a deep copy on other. - */ - public DRPCRequest(DRPCRequest other) { - if (other.is_set_func_args()) { - this.func_args = other.func_args; - } - if (other.is_set_request_id()) { - this.request_id = other.request_id; - } - } - - public DRPCRequest deepCopy() { - return new DRPCRequest(this); - } - - @Override - public void clear() { - this.func_args = null; - this.request_id = null; - } - - public String get_func_args() { - return this.func_args; - } - - public void set_func_args(String func_args) { - this.func_args = func_args; - } - - public void unset_func_args() { - this.func_args = null; - } - - /** Returns true if field func_args is set (has been assigned a value) and false otherwise */ - public boolean is_set_func_args() { - return this.func_args != null; - } - - public void set_func_args_isSet(boolean value) { - if (!value) { - this.func_args = null; - } - } - - public String get_request_id() { - return this.request_id; - } - - public void set_request_id(String request_id) { - this.request_id = request_id; - } - - public void unset_request_id() { - this.request_id = null; - } - - /** Returns true if field request_id is set (has been assigned a value) and false otherwise */ - public boolean is_set_request_id() { - return this.request_id != null; - } - - public void set_request_id_isSet(boolean value) { - if (!value) { - this.request_id = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case FUNC_ARGS: - if (value == null) { - unset_func_args(); - } else { - set_func_args((String)value); - } - break; - - case REQUEST_ID: - if (value == null) { - unset_request_id(); - } else { - set_request_id((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case FUNC_ARGS: - return get_func_args(); - - case REQUEST_ID: - return get_request_id(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case FUNC_ARGS: - return is_set_func_args(); - case REQUEST_ID: - return is_set_request_id(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof DRPCRequest) - return this.equals((DRPCRequest)that); - return false; - } - - public boolean equals(DRPCRequest that) { - if (that == null) - return false; - - boolean this_present_func_args = true && this.is_set_func_args(); - boolean that_present_func_args = true && that.is_set_func_args(); - if (this_present_func_args || that_present_func_args) { - if (!(this_present_func_args && that_present_func_args)) - return false; - if (!this.func_args.equals(that.func_args)) - return false; - } - - boolean this_present_request_id = true && this.is_set_request_id(); - boolean that_present_request_id = true && that.is_set_request_id(); - if (this_present_request_id || that_present_request_id) { - if (!(this_present_request_id && that_present_request_id)) - return false; - if (!this.request_id.equals(that.request_id)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_func_args = true && (is_set_func_args()); - builder.append(present_func_args); - if (present_func_args) - builder.append(func_args); - - boolean present_request_id = true && (is_set_request_id()); - builder.append(present_request_id); - if (present_request_id) - builder.append(request_id); - - return builder.toHashCode(); - } - - public int compareTo(DRPCRequest other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - DRPCRequest typedOther = (DRPCRequest)other; - - lastComparison = Boolean.valueOf(is_set_func_args()).compareTo(typedOther.is_set_func_args()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_func_args()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.func_args, typedOther.func_args); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_request_id()).compareTo(typedOther.is_set_request_id()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_request_id()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.request_id, typedOther.request_id); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // FUNC_ARGS - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.func_args = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // REQUEST_ID - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.request_id = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.func_args != null) { - oprot.writeFieldBegin(FUNC_ARGS_FIELD_DESC); - oprot.writeString(this.func_args); - oprot.writeFieldEnd(); - } - if (this.request_id != null) { - oprot.writeFieldBegin(REQUEST_ID_FIELD_DESC); - oprot.writeString(this.request_id); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("DRPCRequest("); - boolean first = true; - - sb.append("func_args:"); - if (this.func_args == null) { - sb.append("null"); - } else { - sb.append(this.func_args); - } - first = false; - if (!first) sb.append(", "); - sb.append("request_id:"); - if (this.request_id == null) { - sb.append("null"); - } else { - sb.append(this.request_id); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_func_args()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'func_args' is unset! Struct:" + toString()); - } - - if (!is_set_request_id()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'request_id' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/DistributedRPC.java b/jstorm-client/src/main/java/backtype/storm/generated/DistributedRPC.java deleted file mode 100644 index c2e332111..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/DistributedRPC.java +++ /dev/null @@ -1,964 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class DistributedRPC { - - public interface Iface { - - public String execute(String functionName, String funcArgs) throws DRPCExecutionException, org.apache.thrift7.TException; - - } - - public interface AsyncIface { - - public void execute(String functionName, String funcArgs, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - } - - public static class Client extends org.apache.thrift7.TServiceClient implements Iface { - public static class Factory implements org.apache.thrift7.TServiceClientFactory { - public Factory() {} - public Client getClient(org.apache.thrift7.protocol.TProtocol prot) { - return new Client(prot); - } - public Client getClient(org.apache.thrift7.protocol.TProtocol iprot, org.apache.thrift7.protocol.TProtocol oprot) { - return new Client(iprot, oprot); - } - } - - public Client(org.apache.thrift7.protocol.TProtocol prot) - { - super(prot, prot); - } - - public Client(org.apache.thrift7.protocol.TProtocol iprot, org.apache.thrift7.protocol.TProtocol oprot) { - super(iprot, oprot); - } - - public String execute(String functionName, String funcArgs) throws DRPCExecutionException, org.apache.thrift7.TException - { - send_execute(functionName, funcArgs); - return recv_execute(); - } - - public void send_execute(String functionName, String funcArgs) throws org.apache.thrift7.TException - { - execute_args args = new execute_args(); - args.set_functionName(functionName); - args.set_funcArgs(funcArgs); - sendBase("execute", args); - } - - public String recv_execute() throws DRPCExecutionException, org.apache.thrift7.TException - { - execute_result result = new execute_result(); - receiveBase(result, "execute"); - if (result.is_set_success()) { - return result.success; - } - if (result.e != null) { - throw result.e; - } - throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "execute failed: unknown result"); - } - - } - public static class AsyncClient extends org.apache.thrift7.async.TAsyncClient implements AsyncIface { - public static class Factory implements org.apache.thrift7.async.TAsyncClientFactory { - private org.apache.thrift7.async.TAsyncClientManager clientManager; - private org.apache.thrift7.protocol.TProtocolFactory protocolFactory; - public Factory(org.apache.thrift7.async.TAsyncClientManager clientManager, org.apache.thrift7.protocol.TProtocolFactory protocolFactory) { - this.clientManager = clientManager; - this.protocolFactory = protocolFactory; - } - public AsyncClient getAsyncClient(org.apache.thrift7.transport.TNonblockingTransport transport) { - return new AsyncClient(protocolFactory, clientManager, transport); - } - } - - public AsyncClient(org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.async.TAsyncClientManager clientManager, org.apache.thrift7.transport.TNonblockingTransport transport) { - super(protocolFactory, clientManager, transport); - } - - public void execute(String functionName, String funcArgs, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - execute_call method_call = new execute_call(functionName, funcArgs, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class execute_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String functionName; - private String funcArgs; - public execute_call(String functionName, String funcArgs, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.functionName = functionName; - this.funcArgs = funcArgs; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("execute", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - execute_args args = new execute_args(); - args.set_functionName(functionName); - args.set_funcArgs(funcArgs); - args.write(prot); - prot.writeMessageEnd(); - } - - public String getResult() throws DRPCExecutionException, org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_execute(); - } - } - - } - - public static class Processor extends org.apache.thrift7.TBaseProcessor implements org.apache.thrift7.TProcessor { - private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName()); - public Processor(I iface) { - super(iface, getProcessMap(new HashMap>())); - } - - protected Processor(I iface, Map> processMap) { - super(iface, getProcessMap(processMap)); - } - - private static Map> getProcessMap(Map> processMap) { - processMap.put("execute", new execute()); - return processMap; - } - - private static class execute extends org.apache.thrift7.ProcessFunction { - public execute() { - super("execute"); - } - - protected execute_args getEmptyArgsInstance() { - return new execute_args(); - } - - protected execute_result getResult(I iface, execute_args args) throws org.apache.thrift7.TException { - execute_result result = new execute_result(); - try { - result.success = iface.execute(args.functionName, args.funcArgs); - } catch (DRPCExecutionException e) { - result.e = e; - } - return result; - } - } - - } - - public static class execute_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("execute_args"); - - private static final org.apache.thrift7.protocol.TField FUNCTION_NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("functionName", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField FUNC_ARGS_FIELD_DESC = new org.apache.thrift7.protocol.TField("funcArgs", org.apache.thrift7.protocol.TType.STRING, (short)2); - - private String functionName; // required - private String funcArgs; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - FUNCTION_NAME((short)1, "functionName"), - FUNC_ARGS((short)2, "funcArgs"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // FUNCTION_NAME - return FUNCTION_NAME; - case 2: // FUNC_ARGS - return FUNC_ARGS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.FUNCTION_NAME, new org.apache.thrift7.meta_data.FieldMetaData("functionName", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.FUNC_ARGS, new org.apache.thrift7.meta_data.FieldMetaData("funcArgs", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(execute_args.class, metaDataMap); - } - - public execute_args() { - } - - public execute_args( - String functionName, - String funcArgs) - { - this(); - this.functionName = functionName; - this.funcArgs = funcArgs; - } - - /** - * Performs a deep copy on other. - */ - public execute_args(execute_args other) { - if (other.is_set_functionName()) { - this.functionName = other.functionName; - } - if (other.is_set_funcArgs()) { - this.funcArgs = other.funcArgs; - } - } - - public execute_args deepCopy() { - return new execute_args(this); - } - - @Override - public void clear() { - this.functionName = null; - this.funcArgs = null; - } - - public String get_functionName() { - return this.functionName; - } - - public void set_functionName(String functionName) { - this.functionName = functionName; - } - - public void unset_functionName() { - this.functionName = null; - } - - /** Returns true if field functionName is set (has been assigned a value) and false otherwise */ - public boolean is_set_functionName() { - return this.functionName != null; - } - - public void set_functionName_isSet(boolean value) { - if (!value) { - this.functionName = null; - } - } - - public String get_funcArgs() { - return this.funcArgs; - } - - public void set_funcArgs(String funcArgs) { - this.funcArgs = funcArgs; - } - - public void unset_funcArgs() { - this.funcArgs = null; - } - - /** Returns true if field funcArgs is set (has been assigned a value) and false otherwise */ - public boolean is_set_funcArgs() { - return this.funcArgs != null; - } - - public void set_funcArgs_isSet(boolean value) { - if (!value) { - this.funcArgs = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case FUNCTION_NAME: - if (value == null) { - unset_functionName(); - } else { - set_functionName((String)value); - } - break; - - case FUNC_ARGS: - if (value == null) { - unset_funcArgs(); - } else { - set_funcArgs((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case FUNCTION_NAME: - return get_functionName(); - - case FUNC_ARGS: - return get_funcArgs(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case FUNCTION_NAME: - return is_set_functionName(); - case FUNC_ARGS: - return is_set_funcArgs(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof execute_args) - return this.equals((execute_args)that); - return false; - } - - public boolean equals(execute_args that) { - if (that == null) - return false; - - boolean this_present_functionName = true && this.is_set_functionName(); - boolean that_present_functionName = true && that.is_set_functionName(); - if (this_present_functionName || that_present_functionName) { - if (!(this_present_functionName && that_present_functionName)) - return false; - if (!this.functionName.equals(that.functionName)) - return false; - } - - boolean this_present_funcArgs = true && this.is_set_funcArgs(); - boolean that_present_funcArgs = true && that.is_set_funcArgs(); - if (this_present_funcArgs || that_present_funcArgs) { - if (!(this_present_funcArgs && that_present_funcArgs)) - return false; - if (!this.funcArgs.equals(that.funcArgs)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_functionName = true && (is_set_functionName()); - builder.append(present_functionName); - if (present_functionName) - builder.append(functionName); - - boolean present_funcArgs = true && (is_set_funcArgs()); - builder.append(present_funcArgs); - if (present_funcArgs) - builder.append(funcArgs); - - return builder.toHashCode(); - } - - public int compareTo(execute_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - execute_args typedOther = (execute_args)other; - - lastComparison = Boolean.valueOf(is_set_functionName()).compareTo(typedOther.is_set_functionName()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_functionName()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.functionName, typedOther.functionName); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_funcArgs()).compareTo(typedOther.is_set_funcArgs()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_funcArgs()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.funcArgs, typedOther.funcArgs); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // FUNCTION_NAME - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.functionName = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // FUNC_ARGS - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.funcArgs = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.functionName != null) { - oprot.writeFieldBegin(FUNCTION_NAME_FIELD_DESC); - oprot.writeString(this.functionName); - oprot.writeFieldEnd(); - } - if (this.funcArgs != null) { - oprot.writeFieldBegin(FUNC_ARGS_FIELD_DESC); - oprot.writeString(this.funcArgs); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("execute_args("); - boolean first = true; - - sb.append("functionName:"); - if (this.functionName == null) { - sb.append("null"); - } else { - sb.append(this.functionName); - } - first = false; - if (!first) sb.append(", "); - sb.append("funcArgs:"); - if (this.funcArgs == null) { - sb.append("null"); - } else { - sb.append(this.funcArgs); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class execute_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("execute_result"); - - private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRING, (short)0); - private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - - private String success; // required - private DRPCExecutionException e; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - SUCCESS((short)0, "success"), - E((short)1, "e"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - case 1: // E - return E; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(execute_result.class, metaDataMap); - } - - public execute_result() { - } - - public execute_result( - String success, - DRPCExecutionException e) - { - this(); - this.success = success; - this.e = e; - } - - /** - * Performs a deep copy on other. - */ - public execute_result(execute_result other) { - if (other.is_set_success()) { - this.success = other.success; - } - if (other.is_set_e()) { - this.e = new DRPCExecutionException(other.e); - } - } - - public execute_result deepCopy() { - return new execute_result(this); - } - - @Override - public void clear() { - this.success = null; - this.e = null; - } - - public String get_success() { - return this.success; - } - - public void set_success(String success) { - this.success = success; - } - - public void unset_success() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean is_set_success() { - return this.success != null; - } - - public void set_success_isSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public DRPCExecutionException get_e() { - return this.e; - } - - public void set_e(DRPCExecutionException e) { - this.e = e; - } - - public void unset_e() { - this.e = null; - } - - /** Returns true if field e is set (has been assigned a value) and false otherwise */ - public boolean is_set_e() { - return this.e != null; - } - - public void set_e_isSet(boolean value) { - if (!value) { - this.e = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unset_success(); - } else { - set_success((String)value); - } - break; - - case E: - if (value == null) { - unset_e(); - } else { - set_e((DRPCExecutionException)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return get_success(); - - case E: - return get_e(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return is_set_success(); - case E: - return is_set_e(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof execute_result) - return this.equals((execute_result)that); - return false; - } - - public boolean equals(execute_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.is_set_success(); - boolean that_present_success = true && that.is_set_success(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - boolean this_present_e = true && this.is_set_e(); - boolean that_present_e = true && that.is_set_e(); - if (this_present_e || that_present_e) { - if (!(this_present_e && that_present_e)) - return false; - if (!this.e.equals(that.e)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (is_set_success()); - builder.append(present_success); - if (present_success) - builder.append(success); - - boolean present_e = true && (is_set_e()); - builder.append(present_e); - if (present_e) - builder.append(e); - - return builder.toHashCode(); - } - - public int compareTo(execute_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - execute_result typedOther = (execute_result)other; - - lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_success()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_e()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 0: // SUCCESS - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.success = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 1: // E - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.e = new DRPCExecutionException(); - this.e.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_success()) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeString(this.success); - oprot.writeFieldEnd(); - } else if (this.is_set_e()) { - oprot.writeFieldBegin(E_FIELD_DESC); - this.e.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("execute_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - if (!first) sb.append(", "); - sb.append("e:"); - if (this.e == null) { - sb.append("null"); - } else { - sb.append(this.e); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/generated/DistributedRPCInvocations.java b/jstorm-client/src/main/java/backtype/storm/generated/DistributedRPCInvocations.java deleted file mode 100644 index 86bec9116..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/DistributedRPCInvocations.java +++ /dev/null @@ -1,2015 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class DistributedRPCInvocations { - - public interface Iface { - - public void result(String id, String result) throws org.apache.thrift7.TException; - - public DRPCRequest fetchRequest(String functionName) throws org.apache.thrift7.TException; - - public void failRequest(String id) throws org.apache.thrift7.TException; - - } - - public interface AsyncIface { - - public void result(String id, String result, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void fetchRequest(String functionName, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void failRequest(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - } - - public static class Client extends org.apache.thrift7.TServiceClient implements Iface { - public static class Factory implements org.apache.thrift7.TServiceClientFactory { - public Factory() {} - public Client getClient(org.apache.thrift7.protocol.TProtocol prot) { - return new Client(prot); - } - public Client getClient(org.apache.thrift7.protocol.TProtocol iprot, org.apache.thrift7.protocol.TProtocol oprot) { - return new Client(iprot, oprot); - } - } - - public Client(org.apache.thrift7.protocol.TProtocol prot) - { - super(prot, prot); - } - - public Client(org.apache.thrift7.protocol.TProtocol iprot, org.apache.thrift7.protocol.TProtocol oprot) { - super(iprot, oprot); - } - - public void result(String id, String result) throws org.apache.thrift7.TException - { - send_result(id, result); - recv_result(); - } - - public void send_result(String id, String result) throws org.apache.thrift7.TException - { - result_args args = new result_args(); - args.set_id(id); - args.set_result(result); - sendBase("result", args); - } - - public void recv_result() throws org.apache.thrift7.TException - { - result_result result = new result_result(); - receiveBase(result, "result"); - return; - } - - public DRPCRequest fetchRequest(String functionName) throws org.apache.thrift7.TException - { - send_fetchRequest(functionName); - return recv_fetchRequest(); - } - - public void send_fetchRequest(String functionName) throws org.apache.thrift7.TException - { - fetchRequest_args args = new fetchRequest_args(); - args.set_functionName(functionName); - sendBase("fetchRequest", args); - } - - public DRPCRequest recv_fetchRequest() throws org.apache.thrift7.TException - { - fetchRequest_result result = new fetchRequest_result(); - receiveBase(result, "fetchRequest"); - if (result.is_set_success()) { - return result.success; - } - throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "fetchRequest failed: unknown result"); - } - - public void failRequest(String id) throws org.apache.thrift7.TException - { - send_failRequest(id); - recv_failRequest(); - } - - public void send_failRequest(String id) throws org.apache.thrift7.TException - { - failRequest_args args = new failRequest_args(); - args.set_id(id); - sendBase("failRequest", args); - } - - public void recv_failRequest() throws org.apache.thrift7.TException - { - failRequest_result result = new failRequest_result(); - receiveBase(result, "failRequest"); - return; - } - - } - public static class AsyncClient extends org.apache.thrift7.async.TAsyncClient implements AsyncIface { - public static class Factory implements org.apache.thrift7.async.TAsyncClientFactory { - private org.apache.thrift7.async.TAsyncClientManager clientManager; - private org.apache.thrift7.protocol.TProtocolFactory protocolFactory; - public Factory(org.apache.thrift7.async.TAsyncClientManager clientManager, org.apache.thrift7.protocol.TProtocolFactory protocolFactory) { - this.clientManager = clientManager; - this.protocolFactory = protocolFactory; - } - public AsyncClient getAsyncClient(org.apache.thrift7.transport.TNonblockingTransport transport) { - return new AsyncClient(protocolFactory, clientManager, transport); - } - } - - public AsyncClient(org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.async.TAsyncClientManager clientManager, org.apache.thrift7.transport.TNonblockingTransport transport) { - super(protocolFactory, clientManager, transport); - } - - public void result(String id, String result, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - result_call method_call = new result_call(id, result, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class result_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String id; - private String result; - public result_call(String id, String result, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.id = id; - this.result = result; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("result", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - result_args args = new result_args(); - args.set_id(id); - args.set_result(result); - args.write(prot); - prot.writeMessageEnd(); - } - - public void getResult() throws org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - (new Client(prot)).recv_result(); - } - } - - public void fetchRequest(String functionName, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - fetchRequest_call method_call = new fetchRequest_call(functionName, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class fetchRequest_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String functionName; - public fetchRequest_call(String functionName, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.functionName = functionName; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("fetchRequest", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - fetchRequest_args args = new fetchRequest_args(); - args.set_functionName(functionName); - args.write(prot); - prot.writeMessageEnd(); - } - - public DRPCRequest getResult() throws org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_fetchRequest(); - } - } - - public void failRequest(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - failRequest_call method_call = new failRequest_call(id, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class failRequest_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String id; - public failRequest_call(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.id = id; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("failRequest", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - failRequest_args args = new failRequest_args(); - args.set_id(id); - args.write(prot); - prot.writeMessageEnd(); - } - - public void getResult() throws org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - (new Client(prot)).recv_failRequest(); - } - } - - } - - public static class Processor extends org.apache.thrift7.TBaseProcessor implements org.apache.thrift7.TProcessor { - private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName()); - public Processor(I iface) { - super(iface, getProcessMap(new HashMap>())); - } - - protected Processor(I iface, Map> processMap) { - super(iface, getProcessMap(processMap)); - } - - private static Map> getProcessMap(Map> processMap) { - processMap.put("result", new result()); - processMap.put("fetchRequest", new fetchRequest()); - processMap.put("failRequest", new failRequest()); - return processMap; - } - - private static class result extends org.apache.thrift7.ProcessFunction { - public result() { - super("result"); - } - - protected result_args getEmptyArgsInstance() { - return new result_args(); - } - - protected result_result getResult(I iface, result_args args) throws org.apache.thrift7.TException { - result_result result = new result_result(); - iface.result(args.id, args.result); - return result; - } - } - - private static class fetchRequest extends org.apache.thrift7.ProcessFunction { - public fetchRequest() { - super("fetchRequest"); - } - - protected fetchRequest_args getEmptyArgsInstance() { - return new fetchRequest_args(); - } - - protected fetchRequest_result getResult(I iface, fetchRequest_args args) throws org.apache.thrift7.TException { - fetchRequest_result result = new fetchRequest_result(); - result.success = iface.fetchRequest(args.functionName); - return result; - } - } - - private static class failRequest extends org.apache.thrift7.ProcessFunction { - public failRequest() { - super("failRequest"); - } - - protected failRequest_args getEmptyArgsInstance() { - return new failRequest_args(); - } - - protected failRequest_result getResult(I iface, failRequest_args args) throws org.apache.thrift7.TException { - failRequest_result result = new failRequest_result(); - iface.failRequest(args.id); - return result; - } - } - - } - - public static class result_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("result_args"); - - private static final org.apache.thrift7.protocol.TField ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("id", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField RESULT_FIELD_DESC = new org.apache.thrift7.protocol.TField("result", org.apache.thrift7.protocol.TType.STRING, (short)2); - - private String id; // required - private String result; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - ID((short)1, "id"), - RESULT((short)2, "result"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // ID - return ID; - case 2: // RESULT - return RESULT; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.ID, new org.apache.thrift7.meta_data.FieldMetaData("id", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.RESULT, new org.apache.thrift7.meta_data.FieldMetaData("result", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(result_args.class, metaDataMap); - } - - public result_args() { - } - - public result_args( - String id, - String result) - { - this(); - this.id = id; - this.result = result; - } - - /** - * Performs a deep copy on other. - */ - public result_args(result_args other) { - if (other.is_set_id()) { - this.id = other.id; - } - if (other.is_set_result()) { - this.result = other.result; - } - } - - public result_args deepCopy() { - return new result_args(this); - } - - @Override - public void clear() { - this.id = null; - this.result = null; - } - - public String get_id() { - return this.id; - } - - public void set_id(String id) { - this.id = id; - } - - public void unset_id() { - this.id = null; - } - - /** Returns true if field id is set (has been assigned a value) and false otherwise */ - public boolean is_set_id() { - return this.id != null; - } - - public void set_id_isSet(boolean value) { - if (!value) { - this.id = null; - } - } - - public String get_result() { - return this.result; - } - - public void set_result(String result) { - this.result = result; - } - - public void unset_result() { - this.result = null; - } - - /** Returns true if field result is set (has been assigned a value) and false otherwise */ - public boolean is_set_result() { - return this.result != null; - } - - public void set_result_isSet(boolean value) { - if (!value) { - this.result = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case ID: - if (value == null) { - unset_id(); - } else { - set_id((String)value); - } - break; - - case RESULT: - if (value == null) { - unset_result(); - } else { - set_result((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case ID: - return get_id(); - - case RESULT: - return get_result(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case ID: - return is_set_id(); - case RESULT: - return is_set_result(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof result_args) - return this.equals((result_args)that); - return false; - } - - public boolean equals(result_args that) { - if (that == null) - return false; - - boolean this_present_id = true && this.is_set_id(); - boolean that_present_id = true && that.is_set_id(); - if (this_present_id || that_present_id) { - if (!(this_present_id && that_present_id)) - return false; - if (!this.id.equals(that.id)) - return false; - } - - boolean this_present_result = true && this.is_set_result(); - boolean that_present_result = true && that.is_set_result(); - if (this_present_result || that_present_result) { - if (!(this_present_result && that_present_result)) - return false; - if (!this.result.equals(that.result)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_id = true && (is_set_id()); - builder.append(present_id); - if (present_id) - builder.append(id); - - boolean present_result = true && (is_set_result()); - builder.append(present_result); - if (present_result) - builder.append(result); - - return builder.toHashCode(); - } - - public int compareTo(result_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - result_args typedOther = (result_args)other; - - lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_id()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.id, typedOther.id); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_result()).compareTo(typedOther.is_set_result()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_result()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.result, typedOther.result); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // ID - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.id = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // RESULT - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.result = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.id != null) { - oprot.writeFieldBegin(ID_FIELD_DESC); - oprot.writeString(this.id); - oprot.writeFieldEnd(); - } - if (this.result != null) { - oprot.writeFieldBegin(RESULT_FIELD_DESC); - oprot.writeString(this.result); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("result_args("); - boolean first = true; - - sb.append("id:"); - if (this.id == null) { - sb.append("null"); - } else { - sb.append(this.id); - } - first = false; - if (!first) sb.append(", "); - sb.append("result:"); - if (this.result == null) { - sb.append("null"); - } else { - sb.append(this.result); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class result_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("result_result"); - - - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { -; - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(result_result.class, metaDataMap); - } - - public result_result() { - } - - /** - * Performs a deep copy on other. - */ - public result_result(result_result other) { - } - - public result_result deepCopy() { - return new result_result(this); - } - - @Override - public void clear() { - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof result_result) - return this.equals((result_result)that); - return false; - } - - public boolean equals(result_result that) { - if (that == null) - return false; - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - return builder.toHashCode(); - } - - public int compareTo(result_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - result_result typedOther = (result_result)other; - - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("result_result("); - boolean first = true; - - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class fetchRequest_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("fetchRequest_args"); - - private static final org.apache.thrift7.protocol.TField FUNCTION_NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("functionName", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String functionName; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - FUNCTION_NAME((short)1, "functionName"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // FUNCTION_NAME - return FUNCTION_NAME; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.FUNCTION_NAME, new org.apache.thrift7.meta_data.FieldMetaData("functionName", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(fetchRequest_args.class, metaDataMap); - } - - public fetchRequest_args() { - } - - public fetchRequest_args( - String functionName) - { - this(); - this.functionName = functionName; - } - - /** - * Performs a deep copy on other. - */ - public fetchRequest_args(fetchRequest_args other) { - if (other.is_set_functionName()) { - this.functionName = other.functionName; - } - } - - public fetchRequest_args deepCopy() { - return new fetchRequest_args(this); - } - - @Override - public void clear() { - this.functionName = null; - } - - public String get_functionName() { - return this.functionName; - } - - public void set_functionName(String functionName) { - this.functionName = functionName; - } - - public void unset_functionName() { - this.functionName = null; - } - - /** Returns true if field functionName is set (has been assigned a value) and false otherwise */ - public boolean is_set_functionName() { - return this.functionName != null; - } - - public void set_functionName_isSet(boolean value) { - if (!value) { - this.functionName = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case FUNCTION_NAME: - if (value == null) { - unset_functionName(); - } else { - set_functionName((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case FUNCTION_NAME: - return get_functionName(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case FUNCTION_NAME: - return is_set_functionName(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof fetchRequest_args) - return this.equals((fetchRequest_args)that); - return false; - } - - public boolean equals(fetchRequest_args that) { - if (that == null) - return false; - - boolean this_present_functionName = true && this.is_set_functionName(); - boolean that_present_functionName = true && that.is_set_functionName(); - if (this_present_functionName || that_present_functionName) { - if (!(this_present_functionName && that_present_functionName)) - return false; - if (!this.functionName.equals(that.functionName)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_functionName = true && (is_set_functionName()); - builder.append(present_functionName); - if (present_functionName) - builder.append(functionName); - - return builder.toHashCode(); - } - - public int compareTo(fetchRequest_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - fetchRequest_args typedOther = (fetchRequest_args)other; - - lastComparison = Boolean.valueOf(is_set_functionName()).compareTo(typedOther.is_set_functionName()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_functionName()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.functionName, typedOther.functionName); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // FUNCTION_NAME - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.functionName = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.functionName != null) { - oprot.writeFieldBegin(FUNCTION_NAME_FIELD_DESC); - oprot.writeString(this.functionName); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("fetchRequest_args("); - boolean first = true; - - sb.append("functionName:"); - if (this.functionName == null) { - sb.append("null"); - } else { - sb.append(this.functionName); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class fetchRequest_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("fetchRequest_result"); - - private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRUCT, (short)0); - - private DRPCRequest success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, DRPCRequest.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(fetchRequest_result.class, metaDataMap); - } - - public fetchRequest_result() { - } - - public fetchRequest_result( - DRPCRequest success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public fetchRequest_result(fetchRequest_result other) { - if (other.is_set_success()) { - this.success = new DRPCRequest(other.success); - } - } - - public fetchRequest_result deepCopy() { - return new fetchRequest_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public DRPCRequest get_success() { - return this.success; - } - - public void set_success(DRPCRequest success) { - this.success = success; - } - - public void unset_success() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean is_set_success() { - return this.success != null; - } - - public void set_success_isSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unset_success(); - } else { - set_success((DRPCRequest)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return get_success(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return is_set_success(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof fetchRequest_result) - return this.equals((fetchRequest_result)that); - return false; - } - - public boolean equals(fetchRequest_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.is_set_success(); - boolean that_present_success = true && that.is_set_success(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (is_set_success()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(fetchRequest_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - fetchRequest_result typedOther = (fetchRequest_result)other; - - lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_success()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 0: // SUCCESS - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.success = new DRPCRequest(); - this.success.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_success()) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - this.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("fetchRequest_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class failRequest_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("failRequest_args"); - - private static final org.apache.thrift7.protocol.TField ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("id", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String id; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - ID((short)1, "id"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // ID - return ID; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.ID, new org.apache.thrift7.meta_data.FieldMetaData("id", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(failRequest_args.class, metaDataMap); - } - - public failRequest_args() { - } - - public failRequest_args( - String id) - { - this(); - this.id = id; - } - - /** - * Performs a deep copy on other. - */ - public failRequest_args(failRequest_args other) { - if (other.is_set_id()) { - this.id = other.id; - } - } - - public failRequest_args deepCopy() { - return new failRequest_args(this); - } - - @Override - public void clear() { - this.id = null; - } - - public String get_id() { - return this.id; - } - - public void set_id(String id) { - this.id = id; - } - - public void unset_id() { - this.id = null; - } - - /** Returns true if field id is set (has been assigned a value) and false otherwise */ - public boolean is_set_id() { - return this.id != null; - } - - public void set_id_isSet(boolean value) { - if (!value) { - this.id = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case ID: - if (value == null) { - unset_id(); - } else { - set_id((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case ID: - return get_id(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case ID: - return is_set_id(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof failRequest_args) - return this.equals((failRequest_args)that); - return false; - } - - public boolean equals(failRequest_args that) { - if (that == null) - return false; - - boolean this_present_id = true && this.is_set_id(); - boolean that_present_id = true && that.is_set_id(); - if (this_present_id || that_present_id) { - if (!(this_present_id && that_present_id)) - return false; - if (!this.id.equals(that.id)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_id = true && (is_set_id()); - builder.append(present_id); - if (present_id) - builder.append(id); - - return builder.toHashCode(); - } - - public int compareTo(failRequest_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - failRequest_args typedOther = (failRequest_args)other; - - lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_id()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.id, typedOther.id); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // ID - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.id = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.id != null) { - oprot.writeFieldBegin(ID_FIELD_DESC); - oprot.writeString(this.id); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("failRequest_args("); - boolean first = true; - - sb.append("id:"); - if (this.id == null) { - sb.append("null"); - } else { - sb.append(this.id); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class failRequest_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("failRequest_result"); - - - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { -; - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(failRequest_result.class, metaDataMap); - } - - public failRequest_result() { - } - - /** - * Performs a deep copy on other. - */ - public failRequest_result(failRequest_result other) { - } - - public failRequest_result deepCopy() { - return new failRequest_result(this); - } - - @Override - public void clear() { - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof failRequest_result) - return this.equals((failRequest_result)that); - return false; - } - - public boolean equals(failRequest_result that) { - if (that == null) - return false; - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - return builder.toHashCode(); - } - - public int compareTo(failRequest_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - failRequest_result typedOther = (failRequest_result)other; - - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("failRequest_result("); - boolean first = true; - - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/generated/ErrorInfo.java b/jstorm-client/src/main/java/backtype/storm/generated/ErrorInfo.java deleted file mode 100644 index 5e3bb6ed6..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/ErrorInfo.java +++ /dev/null @@ -1,425 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class ErrorInfo implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("ErrorInfo"); - - private static final org.apache.thrift7.protocol.TField ERROR_FIELD_DESC = new org.apache.thrift7.protocol.TField("error", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField ERROR_TIME_SECS_FIELD_DESC = new org.apache.thrift7.protocol.TField("error_time_secs", org.apache.thrift7.protocol.TType.I32, (short)2); - - private String error; // required - private int error_time_secs; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - ERROR((short)1, "error"), - ERROR_TIME_SECS((short)2, "error_time_secs"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // ERROR - return ERROR; - case 2: // ERROR_TIME_SECS - return ERROR_TIME_SECS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __ERROR_TIME_SECS_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.ERROR, new org.apache.thrift7.meta_data.FieldMetaData("error", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.ERROR_TIME_SECS, new org.apache.thrift7.meta_data.FieldMetaData("error_time_secs", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(ErrorInfo.class, metaDataMap); - } - - public ErrorInfo() { - } - - public ErrorInfo( - String error, - int error_time_secs) - { - this(); - this.error = error; - this.error_time_secs = error_time_secs; - set_error_time_secs_isSet(true); - } - - /** - * Performs a deep copy on other. - */ - public ErrorInfo(ErrorInfo other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); - if (other.is_set_error()) { - this.error = other.error; - } - this.error_time_secs = other.error_time_secs; - } - - public ErrorInfo deepCopy() { - return new ErrorInfo(this); - } - - @Override - public void clear() { - this.error = null; - set_error_time_secs_isSet(false); - this.error_time_secs = 0; - } - - public String get_error() { - return this.error; - } - - public void set_error(String error) { - this.error = error; - } - - public void unset_error() { - this.error = null; - } - - /** Returns true if field error is set (has been assigned a value) and false otherwise */ - public boolean is_set_error() { - return this.error != null; - } - - public void set_error_isSet(boolean value) { - if (!value) { - this.error = null; - } - } - - public int get_error_time_secs() { - return this.error_time_secs; - } - - public void set_error_time_secs(int error_time_secs) { - this.error_time_secs = error_time_secs; - set_error_time_secs_isSet(true); - } - - public void unset_error_time_secs() { - __isset_bit_vector.clear(__ERROR_TIME_SECS_ISSET_ID); - } - - /** Returns true if field error_time_secs is set (has been assigned a value) and false otherwise */ - public boolean is_set_error_time_secs() { - return __isset_bit_vector.get(__ERROR_TIME_SECS_ISSET_ID); - } - - public void set_error_time_secs_isSet(boolean value) { - __isset_bit_vector.set(__ERROR_TIME_SECS_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case ERROR: - if (value == null) { - unset_error(); - } else { - set_error((String)value); - } - break; - - case ERROR_TIME_SECS: - if (value == null) { - unset_error_time_secs(); - } else { - set_error_time_secs((Integer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case ERROR: - return get_error(); - - case ERROR_TIME_SECS: - return Integer.valueOf(get_error_time_secs()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case ERROR: - return is_set_error(); - case ERROR_TIME_SECS: - return is_set_error_time_secs(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof ErrorInfo) - return this.equals((ErrorInfo)that); - return false; - } - - public boolean equals(ErrorInfo that) { - if (that == null) - return false; - - boolean this_present_error = true && this.is_set_error(); - boolean that_present_error = true && that.is_set_error(); - if (this_present_error || that_present_error) { - if (!(this_present_error && that_present_error)) - return false; - if (!this.error.equals(that.error)) - return false; - } - - boolean this_present_error_time_secs = true; - boolean that_present_error_time_secs = true; - if (this_present_error_time_secs || that_present_error_time_secs) { - if (!(this_present_error_time_secs && that_present_error_time_secs)) - return false; - if (this.error_time_secs != that.error_time_secs) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_error = true && (is_set_error()); - builder.append(present_error); - if (present_error) - builder.append(error); - - boolean present_error_time_secs = true; - builder.append(present_error_time_secs); - if (present_error_time_secs) - builder.append(error_time_secs); - - return builder.toHashCode(); - } - - public int compareTo(ErrorInfo other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - ErrorInfo typedOther = (ErrorInfo)other; - - lastComparison = Boolean.valueOf(is_set_error()).compareTo(typedOther.is_set_error()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_error()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.error, typedOther.error); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_error_time_secs()).compareTo(typedOther.is_set_error_time_secs()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_error_time_secs()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.error_time_secs, typedOther.error_time_secs); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // ERROR - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.error = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // ERROR_TIME_SECS - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.error_time_secs = iprot.readI32(); - set_error_time_secs_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.error != null) { - oprot.writeFieldBegin(ERROR_FIELD_DESC); - oprot.writeString(this.error); - oprot.writeFieldEnd(); - } - oprot.writeFieldBegin(ERROR_TIME_SECS_FIELD_DESC); - oprot.writeI32(this.error_time_secs); - oprot.writeFieldEnd(); - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("ErrorInfo("); - boolean first = true; - - sb.append("error:"); - if (this.error == null) { - sb.append("null"); - } else { - sb.append(this.error); - } - first = false; - if (!first) sb.append(", "); - sb.append("error_time_secs:"); - sb.append(this.error_time_secs); - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_error()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'error' is unset! Struct:" + toString()); - } - - if (!is_set_error_time_secs()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'error_time_secs' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bit_vector = new BitSet(1); - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/ExecutorInfo.java b/jstorm-client/src/main/java/backtype/storm/generated/ExecutorInfo.java deleted file mode 100644 index 2904b4ad4..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/ExecutorInfo.java +++ /dev/null @@ -1,420 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class ExecutorInfo implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("ExecutorInfo"); - - private static final org.apache.thrift7.protocol.TField TASK_START_FIELD_DESC = new org.apache.thrift7.protocol.TField("task_start", org.apache.thrift7.protocol.TType.I32, (short)1); - private static final org.apache.thrift7.protocol.TField TASK_END_FIELD_DESC = new org.apache.thrift7.protocol.TField("task_end", org.apache.thrift7.protocol.TType.I32, (short)2); - - private int task_start; // required - private int task_end; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - TASK_START((short)1, "task_start"), - TASK_END((short)2, "task_end"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // TASK_START - return TASK_START; - case 2: // TASK_END - return TASK_END; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __TASK_START_ISSET_ID = 0; - private static final int __TASK_END_ISSET_ID = 1; - private BitSet __isset_bit_vector = new BitSet(2); - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.TASK_START, new org.apache.thrift7.meta_data.FieldMetaData("task_start", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - tmpMap.put(_Fields.TASK_END, new org.apache.thrift7.meta_data.FieldMetaData("task_end", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(ExecutorInfo.class, metaDataMap); - } - - public ExecutorInfo() { - } - - public ExecutorInfo( - int task_start, - int task_end) - { - this(); - this.task_start = task_start; - set_task_start_isSet(true); - this.task_end = task_end; - set_task_end_isSet(true); - } - - /** - * Performs a deep copy on other. - */ - public ExecutorInfo(ExecutorInfo other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); - this.task_start = other.task_start; - this.task_end = other.task_end; - } - - public ExecutorInfo deepCopy() { - return new ExecutorInfo(this); - } - - @Override - public void clear() { - set_task_start_isSet(false); - this.task_start = 0; - set_task_end_isSet(false); - this.task_end = 0; - } - - public int get_task_start() { - return this.task_start; - } - - public void set_task_start(int task_start) { - this.task_start = task_start; - set_task_start_isSet(true); - } - - public void unset_task_start() { - __isset_bit_vector.clear(__TASK_START_ISSET_ID); - } - - /** Returns true if field task_start is set (has been assigned a value) and false otherwise */ - public boolean is_set_task_start() { - return __isset_bit_vector.get(__TASK_START_ISSET_ID); - } - - public void set_task_start_isSet(boolean value) { - __isset_bit_vector.set(__TASK_START_ISSET_ID, value); - } - - public int get_task_end() { - return this.task_end; - } - - public void set_task_end(int task_end) { - this.task_end = task_end; - set_task_end_isSet(true); - } - - public void unset_task_end() { - __isset_bit_vector.clear(__TASK_END_ISSET_ID); - } - - /** Returns true if field task_end is set (has been assigned a value) and false otherwise */ - public boolean is_set_task_end() { - return __isset_bit_vector.get(__TASK_END_ISSET_ID); - } - - public void set_task_end_isSet(boolean value) { - __isset_bit_vector.set(__TASK_END_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case TASK_START: - if (value == null) { - unset_task_start(); - } else { - set_task_start((Integer)value); - } - break; - - case TASK_END: - if (value == null) { - unset_task_end(); - } else { - set_task_end((Integer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case TASK_START: - return Integer.valueOf(get_task_start()); - - case TASK_END: - return Integer.valueOf(get_task_end()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case TASK_START: - return is_set_task_start(); - case TASK_END: - return is_set_task_end(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof ExecutorInfo) - return this.equals((ExecutorInfo)that); - return false; - } - - public boolean equals(ExecutorInfo that) { - if (that == null) - return false; - - boolean this_present_task_start = true; - boolean that_present_task_start = true; - if (this_present_task_start || that_present_task_start) { - if (!(this_present_task_start && that_present_task_start)) - return false; - if (this.task_start != that.task_start) - return false; - } - - boolean this_present_task_end = true; - boolean that_present_task_end = true; - if (this_present_task_end || that_present_task_end) { - if (!(this_present_task_end && that_present_task_end)) - return false; - if (this.task_end != that.task_end) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_task_start = true; - builder.append(present_task_start); - if (present_task_start) - builder.append(task_start); - - boolean present_task_end = true; - builder.append(present_task_end); - if (present_task_end) - builder.append(task_end); - - return builder.toHashCode(); - } - - public int compareTo(ExecutorInfo other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - ExecutorInfo typedOther = (ExecutorInfo)other; - - lastComparison = Boolean.valueOf(is_set_task_start()).compareTo(typedOther.is_set_task_start()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_task_start()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.task_start, typedOther.task_start); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_task_end()).compareTo(typedOther.is_set_task_end()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_task_end()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.task_end, typedOther.task_end); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // TASK_START - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.task_start = iprot.readI32(); - set_task_start_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // TASK_END - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.task_end = iprot.readI32(); - set_task_end_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(TASK_START_FIELD_DESC); - oprot.writeI32(this.task_start); - oprot.writeFieldEnd(); - oprot.writeFieldBegin(TASK_END_FIELD_DESC); - oprot.writeI32(this.task_end); - oprot.writeFieldEnd(); - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("ExecutorInfo("); - boolean first = true; - - sb.append("task_start:"); - sb.append(this.task_start); - first = false; - if (!first) sb.append(", "); - sb.append("task_end:"); - sb.append(this.task_end); - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_task_start()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'task_start' is unset! Struct:" + toString()); - } - - if (!is_set_task_end()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'task_end' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bit_vector = new BitSet(1); - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/ExecutorSpecificStats.java b/jstorm-client/src/main/java/backtype/storm/generated/ExecutorSpecificStats.java deleted file mode 100644 index fc2108f0b..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/ExecutorSpecificStats.java +++ /dev/null @@ -1,318 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class ExecutorSpecificStats extends org.apache.thrift7.TUnion { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("ExecutorSpecificStats"); - private static final org.apache.thrift7.protocol.TField BOLT_FIELD_DESC = new org.apache.thrift7.protocol.TField("bolt", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift7.protocol.TField SPOUT_FIELD_DESC = new org.apache.thrift7.protocol.TField("spout", org.apache.thrift7.protocol.TType.STRUCT, (short)2); - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - BOLT((short)1, "bolt"), - SPOUT((short)2, "spout"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // BOLT - return BOLT; - case 2: // SPOUT - return SPOUT; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.BOLT, new org.apache.thrift7.meta_data.FieldMetaData("bolt", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, BoltStats.class))); - tmpMap.put(_Fields.SPOUT, new org.apache.thrift7.meta_data.FieldMetaData("spout", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, SpoutStats.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(ExecutorSpecificStats.class, metaDataMap); - } - - public ExecutorSpecificStats() { - super(); - } - - public ExecutorSpecificStats(_Fields setField, Object value) { - super(setField, value); - } - - public ExecutorSpecificStats(ExecutorSpecificStats other) { - super(other); - } - public ExecutorSpecificStats deepCopy() { - return new ExecutorSpecificStats(this); - } - - public static ExecutorSpecificStats bolt(BoltStats value) { - ExecutorSpecificStats x = new ExecutorSpecificStats(); - x.set_bolt(value); - return x; - } - - public static ExecutorSpecificStats spout(SpoutStats value) { - ExecutorSpecificStats x = new ExecutorSpecificStats(); - x.set_spout(value); - return x; - } - - - @Override - protected void checkType(_Fields setField, Object value) throws ClassCastException { - switch (setField) { - case BOLT: - if (value instanceof BoltStats) { - break; - } - throw new ClassCastException("Was expecting value of type BoltStats for field 'bolt', but got " + value.getClass().getSimpleName()); - case SPOUT: - if (value instanceof SpoutStats) { - break; - } - throw new ClassCastException("Was expecting value of type SpoutStats for field 'spout', but got " + value.getClass().getSimpleName()); - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected Object readValue(org.apache.thrift7.protocol.TProtocol iprot, org.apache.thrift7.protocol.TField field) throws org.apache.thrift7.TException { - _Fields setField = _Fields.findByThriftId(field.id); - if (setField != null) { - switch (setField) { - case BOLT: - if (field.type == BOLT_FIELD_DESC.type) { - BoltStats bolt; - bolt = new BoltStats(); - bolt.read(iprot); - return bolt; - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case SPOUT: - if (field.type == SPOUT_FIELD_DESC.type) { - SpoutStats spout; - spout = new SpoutStats(); - spout.read(iprot); - return spout; - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - } - - @Override - protected void writeValue(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - switch (setField_) { - case BOLT: - BoltStats bolt = (BoltStats)value_; - bolt.write(oprot); - return; - case SPOUT: - SpoutStats spout = (SpoutStats)value_; - spout.write(oprot); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected org.apache.thrift7.protocol.TField getFieldDesc(_Fields setField) { - switch (setField) { - case BOLT: - return BOLT_FIELD_DESC; - case SPOUT: - return SPOUT_FIELD_DESC; - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected org.apache.thrift7.protocol.TStruct getStructDesc() { - return STRUCT_DESC; - } - - @Override - protected _Fields enumForId(short id) { - return _Fields.findByThriftIdOrThrow(id); - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - - public BoltStats get_bolt() { - if (getSetField() == _Fields.BOLT) { - return (BoltStats)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'bolt' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void set_bolt(BoltStats value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.BOLT; - value_ = value; - } - - public SpoutStats get_spout() { - if (getSetField() == _Fields.SPOUT) { - return (SpoutStats)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'spout' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void set_spout(SpoutStats value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.SPOUT; - value_ = value; - } - - public boolean is_set_bolt() { - return setField_ == _Fields.BOLT; - } - - - public boolean is_set_spout() { - return setField_ == _Fields.SPOUT; - } - - - public boolean equals(Object other) { - if (other instanceof ExecutorSpecificStats) { - return equals((ExecutorSpecificStats)other); - } else { - return false; - } - } - - public boolean equals(ExecutorSpecificStats other) { - return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); - } - - @Override - public int compareTo(ExecutorSpecificStats other) { - int lastComparison = org.apache.thrift7.TBaseHelper.compareTo(getSetField(), other.getSetField()); - if (lastComparison == 0) { - return org.apache.thrift7.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); - } - return lastComparison; - } - - - @Override - public int hashCode() { - HashCodeBuilder hcb = new HashCodeBuilder(); - hcb.append(this.getClass().getName()); - org.apache.thrift7.TFieldIdEnum setField = getSetField(); - if (setField != null) { - hcb.append(setField.getThriftFieldId()); - Object value = getFieldValue(); - if (value instanceof org.apache.thrift7.TEnum) { - hcb.append(((org.apache.thrift7.TEnum)getFieldValue()).getValue()); - } else { - hcb.append(value); - } - } - return hcb.toHashCode(); - } - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - -} diff --git a/jstorm-client/src/main/java/backtype/storm/generated/GlobalStreamId.java b/jstorm-client/src/main/java/backtype/storm/generated/GlobalStreamId.java deleted file mode 100644 index f443dcfc2..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/GlobalStreamId.java +++ /dev/null @@ -1,425 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class GlobalStreamId implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("GlobalStreamId"); - - private static final org.apache.thrift7.protocol.TField COMPONENT_ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("componentId", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField STREAM_ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("streamId", org.apache.thrift7.protocol.TType.STRING, (short)2); - - private String componentId; // required - private String streamId; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - COMPONENT_ID((short)1, "componentId"), - STREAM_ID((short)2, "streamId"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // COMPONENT_ID - return COMPONENT_ID; - case 2: // STREAM_ID - return STREAM_ID; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.COMPONENT_ID, new org.apache.thrift7.meta_data.FieldMetaData("componentId", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.STREAM_ID, new org.apache.thrift7.meta_data.FieldMetaData("streamId", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(GlobalStreamId.class, metaDataMap); - } - - public GlobalStreamId() { - } - - public GlobalStreamId( - String componentId, - String streamId) - { - this(); - this.componentId = componentId; - this.streamId = streamId; - } - - /** - * Performs a deep copy on other. - */ - public GlobalStreamId(GlobalStreamId other) { - if (other.is_set_componentId()) { - this.componentId = other.componentId; - } - if (other.is_set_streamId()) { - this.streamId = other.streamId; - } - } - - public GlobalStreamId deepCopy() { - return new GlobalStreamId(this); - } - - @Override - public void clear() { - this.componentId = null; - this.streamId = null; - } - - public String get_componentId() { - return this.componentId; - } - - public void set_componentId(String componentId) { - this.componentId = componentId; - } - - public void unset_componentId() { - this.componentId = null; - } - - /** Returns true if field componentId is set (has been assigned a value) and false otherwise */ - public boolean is_set_componentId() { - return this.componentId != null; - } - - public void set_componentId_isSet(boolean value) { - if (!value) { - this.componentId = null; - } - } - - public String get_streamId() { - return this.streamId; - } - - public void set_streamId(String streamId) { - this.streamId = streamId; - } - - public void unset_streamId() { - this.streamId = null; - } - - /** Returns true if field streamId is set (has been assigned a value) and false otherwise */ - public boolean is_set_streamId() { - return this.streamId != null; - } - - public void set_streamId_isSet(boolean value) { - if (!value) { - this.streamId = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case COMPONENT_ID: - if (value == null) { - unset_componentId(); - } else { - set_componentId((String)value); - } - break; - - case STREAM_ID: - if (value == null) { - unset_streamId(); - } else { - set_streamId((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case COMPONENT_ID: - return get_componentId(); - - case STREAM_ID: - return get_streamId(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case COMPONENT_ID: - return is_set_componentId(); - case STREAM_ID: - return is_set_streamId(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof GlobalStreamId) - return this.equals((GlobalStreamId)that); - return false; - } - - public boolean equals(GlobalStreamId that) { - if (that == null) - return false; - - boolean this_present_componentId = true && this.is_set_componentId(); - boolean that_present_componentId = true && that.is_set_componentId(); - if (this_present_componentId || that_present_componentId) { - if (!(this_present_componentId && that_present_componentId)) - return false; - if (!this.componentId.equals(that.componentId)) - return false; - } - - boolean this_present_streamId = true && this.is_set_streamId(); - boolean that_present_streamId = true && that.is_set_streamId(); - if (this_present_streamId || that_present_streamId) { - if (!(this_present_streamId && that_present_streamId)) - return false; - if (!this.streamId.equals(that.streamId)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_componentId = true && (is_set_componentId()); - builder.append(present_componentId); - if (present_componentId) - builder.append(componentId); - - boolean present_streamId = true && (is_set_streamId()); - builder.append(present_streamId); - if (present_streamId) - builder.append(streamId); - - return builder.toHashCode(); - } - - public int compareTo(GlobalStreamId other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - GlobalStreamId typedOther = (GlobalStreamId)other; - - lastComparison = Boolean.valueOf(is_set_componentId()).compareTo(typedOther.is_set_componentId()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_componentId()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.componentId, typedOther.componentId); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_streamId()).compareTo(typedOther.is_set_streamId()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_streamId()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.streamId, typedOther.streamId); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // COMPONENT_ID - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.componentId = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // STREAM_ID - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.streamId = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.componentId != null) { - oprot.writeFieldBegin(COMPONENT_ID_FIELD_DESC); - oprot.writeString(this.componentId); - oprot.writeFieldEnd(); - } - if (this.streamId != null) { - oprot.writeFieldBegin(STREAM_ID_FIELD_DESC); - oprot.writeString(this.streamId); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("GlobalStreamId("); - boolean first = true; - - sb.append("componentId:"); - if (this.componentId == null) { - sb.append("null"); - } else { - sb.append(this.componentId); - } - first = false; - if (!first) sb.append(", "); - sb.append("streamId:"); - if (this.streamId == null) { - sb.append("null"); - } else { - sb.append(this.streamId); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_componentId()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'componentId' is unset! Struct:" + toString()); - } - - if (!is_set_streamId()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'streamId' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/Grouping.java b/jstorm-client/src/main/java/backtype/storm/generated/Grouping.java deleted file mode 100644 index 45a477c30..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/Grouping.java +++ /dev/null @@ -1,714 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class Grouping extends org.apache.thrift7.TUnion { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("Grouping"); - private static final org.apache.thrift7.protocol.TField FIELDS_FIELD_DESC = new org.apache.thrift7.protocol.TField("fields", org.apache.thrift7.protocol.TType.LIST, (short)1); - private static final org.apache.thrift7.protocol.TField SHUFFLE_FIELD_DESC = new org.apache.thrift7.protocol.TField("shuffle", org.apache.thrift7.protocol.TType.STRUCT, (short)2); - private static final org.apache.thrift7.protocol.TField ALL_FIELD_DESC = new org.apache.thrift7.protocol.TField("all", org.apache.thrift7.protocol.TType.STRUCT, (short)3); - private static final org.apache.thrift7.protocol.TField NONE_FIELD_DESC = new org.apache.thrift7.protocol.TField("none", org.apache.thrift7.protocol.TType.STRUCT, (short)4); - private static final org.apache.thrift7.protocol.TField DIRECT_FIELD_DESC = new org.apache.thrift7.protocol.TField("direct", org.apache.thrift7.protocol.TType.STRUCT, (short)5); - private static final org.apache.thrift7.protocol.TField CUSTOM_OBJECT_FIELD_DESC = new org.apache.thrift7.protocol.TField("custom_object", org.apache.thrift7.protocol.TType.STRUCT, (short)6); - private static final org.apache.thrift7.protocol.TField CUSTOM_SERIALIZED_FIELD_DESC = new org.apache.thrift7.protocol.TField("custom_serialized", org.apache.thrift7.protocol.TType.STRING, (short)7); - private static final org.apache.thrift7.protocol.TField LOCAL_OR_SHUFFLE_FIELD_DESC = new org.apache.thrift7.protocol.TField("local_or_shuffle", org.apache.thrift7.protocol.TType.STRUCT, (short)8); - private static final org.apache.thrift7.protocol.TField LOCAL_FIRST_FIELD_DESC = new org.apache.thrift7.protocol.TField("localFirst", org.apache.thrift7.protocol.TType.STRUCT, (short)9); - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - FIELDS((short)1, "fields"), - SHUFFLE((short)2, "shuffle"), - ALL((short)3, "all"), - NONE((short)4, "none"), - DIRECT((short)5, "direct"), - CUSTOM_OBJECT((short)6, "custom_object"), - CUSTOM_SERIALIZED((short)7, "custom_serialized"), - LOCAL_OR_SHUFFLE((short)8, "local_or_shuffle"), - LOCAL_FIRST((short)9, "localFirst"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // FIELDS - return FIELDS; - case 2: // SHUFFLE - return SHUFFLE; - case 3: // ALL - return ALL; - case 4: // NONE - return NONE; - case 5: // DIRECT - return DIRECT; - case 6: // CUSTOM_OBJECT - return CUSTOM_OBJECT; - case 7: // CUSTOM_SERIALIZED - return CUSTOM_SERIALIZED; - case 8: // LOCAL_OR_SHUFFLE - return LOCAL_OR_SHUFFLE; - case 9: // LOCAL_FIRST - return LOCAL_FIRST; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.FIELDS, new org.apache.thrift7.meta_data.FieldMetaData("fields", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.ListMetaData(org.apache.thrift7.protocol.TType.LIST, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING)))); - tmpMap.put(_Fields.SHUFFLE, new org.apache.thrift7.meta_data.FieldMetaData("shuffle", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, NullStruct.class))); - tmpMap.put(_Fields.ALL, new org.apache.thrift7.meta_data.FieldMetaData("all", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, NullStruct.class))); - tmpMap.put(_Fields.NONE, new org.apache.thrift7.meta_data.FieldMetaData("none", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, NullStruct.class))); - tmpMap.put(_Fields.DIRECT, new org.apache.thrift7.meta_data.FieldMetaData("direct", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, NullStruct.class))); - tmpMap.put(_Fields.CUSTOM_OBJECT, new org.apache.thrift7.meta_data.FieldMetaData("custom_object", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, JavaObject.class))); - tmpMap.put(_Fields.CUSTOM_SERIALIZED, new org.apache.thrift7.meta_data.FieldMetaData("custom_serialized", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING , true))); - tmpMap.put(_Fields.LOCAL_OR_SHUFFLE, new org.apache.thrift7.meta_data.FieldMetaData("local_or_shuffle", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, NullStruct.class))); - tmpMap.put(_Fields.LOCAL_FIRST, new org.apache.thrift7.meta_data.FieldMetaData("localFirst", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, NullStruct.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(Grouping.class, metaDataMap); - } - - public Grouping() { - super(); - } - - public Grouping(_Fields setField, Object value) { - super(setField, value); - } - - public Grouping(Grouping other) { - super(other); - } - public Grouping deepCopy() { - return new Grouping(this); - } - - public static Grouping fields(List value) { - Grouping x = new Grouping(); - x.set_fields(value); - return x; - } - - public static Grouping shuffle(NullStruct value) { - Grouping x = new Grouping(); - x.set_shuffle(value); - return x; - } - - public static Grouping all(NullStruct value) { - Grouping x = new Grouping(); - x.set_all(value); - return x; - } - - public static Grouping none(NullStruct value) { - Grouping x = new Grouping(); - x.set_none(value); - return x; - } - - public static Grouping direct(NullStruct value) { - Grouping x = new Grouping(); - x.set_direct(value); - return x; - } - - public static Grouping custom_object(JavaObject value) { - Grouping x = new Grouping(); - x.set_custom_object(value); - return x; - } - - public static Grouping custom_serialized(ByteBuffer value) { - Grouping x = new Grouping(); - x.set_custom_serialized(value); - return x; - } - - public static Grouping custom_serialized(byte[] value) { - Grouping x = new Grouping(); - x.set_custom_serialized(ByteBuffer.wrap(value)); - return x; - } - - public static Grouping local_or_shuffle(NullStruct value) { - Grouping x = new Grouping(); - x.set_local_or_shuffle(value); - return x; - } - - public static Grouping localFirst(NullStruct value) { - Grouping x = new Grouping(); - x.set_localFirst(value); - return x; - } - - - @Override - protected void checkType(_Fields setField, Object value) throws ClassCastException { - switch (setField) { - case FIELDS: - if (value instanceof List) { - break; - } - throw new ClassCastException("Was expecting value of type List for field 'fields', but got " + value.getClass().getSimpleName()); - case SHUFFLE: - if (value instanceof NullStruct) { - break; - } - throw new ClassCastException("Was expecting value of type NullStruct for field 'shuffle', but got " + value.getClass().getSimpleName()); - case ALL: - if (value instanceof NullStruct) { - break; - } - throw new ClassCastException("Was expecting value of type NullStruct for field 'all', but got " + value.getClass().getSimpleName()); - case NONE: - if (value instanceof NullStruct) { - break; - } - throw new ClassCastException("Was expecting value of type NullStruct for field 'none', but got " + value.getClass().getSimpleName()); - case DIRECT: - if (value instanceof NullStruct) { - break; - } - throw new ClassCastException("Was expecting value of type NullStruct for field 'direct', but got " + value.getClass().getSimpleName()); - case CUSTOM_OBJECT: - if (value instanceof JavaObject) { - break; - } - throw new ClassCastException("Was expecting value of type JavaObject for field 'custom_object', but got " + value.getClass().getSimpleName()); - case CUSTOM_SERIALIZED: - if (value instanceof ByteBuffer) { - break; - } - throw new ClassCastException("Was expecting value of type ByteBuffer for field 'custom_serialized', but got " + value.getClass().getSimpleName()); - case LOCAL_OR_SHUFFLE: - if (value instanceof NullStruct) { - break; - } - throw new ClassCastException("Was expecting value of type NullStruct for field 'local_or_shuffle', but got " + value.getClass().getSimpleName()); - case LOCAL_FIRST: - if (value instanceof NullStruct) { - break; - } - throw new ClassCastException("Was expecting value of type NullStruct for field 'localFirst', but got " + value.getClass().getSimpleName()); - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected Object readValue(org.apache.thrift7.protocol.TProtocol iprot, org.apache.thrift7.protocol.TField field) throws org.apache.thrift7.TException { - _Fields setField = _Fields.findByThriftId(field.id); - if (setField != null) { - switch (setField) { - case FIELDS: - if (field.type == FIELDS_FIELD_DESC.type) { - List fields; - { - org.apache.thrift7.protocol.TList _list4 = iprot.readListBegin(); - fields = new ArrayList(_list4.size); - for (int _i5 = 0; _i5 < _list4.size; ++_i5) - { - String _elem6; // required - _elem6 = iprot.readString(); - fields.add(_elem6); - } - iprot.readListEnd(); - } - return fields; - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case SHUFFLE: - if (field.type == SHUFFLE_FIELD_DESC.type) { - NullStruct shuffle; - shuffle = new NullStruct(); - shuffle.read(iprot); - return shuffle; - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case ALL: - if (field.type == ALL_FIELD_DESC.type) { - NullStruct all; - all = new NullStruct(); - all.read(iprot); - return all; - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case NONE: - if (field.type == NONE_FIELD_DESC.type) { - NullStruct none; - none = new NullStruct(); - none.read(iprot); - return none; - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case DIRECT: - if (field.type == DIRECT_FIELD_DESC.type) { - NullStruct direct; - direct = new NullStruct(); - direct.read(iprot); - return direct; - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case CUSTOM_OBJECT: - if (field.type == CUSTOM_OBJECT_FIELD_DESC.type) { - JavaObject custom_object; - custom_object = new JavaObject(); - custom_object.read(iprot); - return custom_object; - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case CUSTOM_SERIALIZED: - if (field.type == CUSTOM_SERIALIZED_FIELD_DESC.type) { - ByteBuffer custom_serialized; - custom_serialized = iprot.readBinary(); - return custom_serialized; - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case LOCAL_OR_SHUFFLE: - if (field.type == LOCAL_OR_SHUFFLE_FIELD_DESC.type) { - NullStruct local_or_shuffle; - local_or_shuffle = new NullStruct(); - local_or_shuffle.read(iprot); - return local_or_shuffle; - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case LOCAL_FIRST: - if (field.type == LOCAL_FIRST_FIELD_DESC.type) { - NullStruct localFirst; - localFirst = new NullStruct(); - localFirst.read(iprot); - return localFirst; - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - } - - @Override - protected void writeValue(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - switch (setField_) { - case FIELDS: - List fields = (List)value_; - { - oprot.writeListBegin(new org.apache.thrift7.protocol.TList(org.apache.thrift7.protocol.TType.STRING, fields.size())); - for (String _iter7 : fields) - { - oprot.writeString(_iter7); - } - oprot.writeListEnd(); - } - return; - case SHUFFLE: - NullStruct shuffle = (NullStruct)value_; - shuffle.write(oprot); - return; - case ALL: - NullStruct all = (NullStruct)value_; - all.write(oprot); - return; - case NONE: - NullStruct none = (NullStruct)value_; - none.write(oprot); - return; - case DIRECT: - NullStruct direct = (NullStruct)value_; - direct.write(oprot); - return; - case CUSTOM_OBJECT: - JavaObject custom_object = (JavaObject)value_; - custom_object.write(oprot); - return; - case CUSTOM_SERIALIZED: - ByteBuffer custom_serialized = (ByteBuffer)value_; - oprot.writeBinary(custom_serialized); - return; - case LOCAL_OR_SHUFFLE: - NullStruct local_or_shuffle = (NullStruct)value_; - local_or_shuffle.write(oprot); - return; - case LOCAL_FIRST: - NullStruct localFirst = (NullStruct)value_; - localFirst.write(oprot); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected org.apache.thrift7.protocol.TField getFieldDesc(_Fields setField) { - switch (setField) { - case FIELDS: - return FIELDS_FIELD_DESC; - case SHUFFLE: - return SHUFFLE_FIELD_DESC; - case ALL: - return ALL_FIELD_DESC; - case NONE: - return NONE_FIELD_DESC; - case DIRECT: - return DIRECT_FIELD_DESC; - case CUSTOM_OBJECT: - return CUSTOM_OBJECT_FIELD_DESC; - case CUSTOM_SERIALIZED: - return CUSTOM_SERIALIZED_FIELD_DESC; - case LOCAL_OR_SHUFFLE: - return LOCAL_OR_SHUFFLE_FIELD_DESC; - case LOCAL_FIRST: - return LOCAL_FIRST_FIELD_DESC; - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected org.apache.thrift7.protocol.TStruct getStructDesc() { - return STRUCT_DESC; - } - - @Override - protected _Fields enumForId(short id) { - return _Fields.findByThriftIdOrThrow(id); - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - - public List get_fields() { - if (getSetField() == _Fields.FIELDS) { - return (List)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'fields' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void set_fields(List value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.FIELDS; - value_ = value; - } - - public NullStruct get_shuffle() { - if (getSetField() == _Fields.SHUFFLE) { - return (NullStruct)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'shuffle' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void set_shuffle(NullStruct value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.SHUFFLE; - value_ = value; - } - - public NullStruct get_all() { - if (getSetField() == _Fields.ALL) { - return (NullStruct)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'all' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void set_all(NullStruct value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.ALL; - value_ = value; - } - - public NullStruct get_none() { - if (getSetField() == _Fields.NONE) { - return (NullStruct)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'none' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void set_none(NullStruct value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.NONE; - value_ = value; - } - - public NullStruct get_direct() { - if (getSetField() == _Fields.DIRECT) { - return (NullStruct)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'direct' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void set_direct(NullStruct value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.DIRECT; - value_ = value; - } - - public JavaObject get_custom_object() { - if (getSetField() == _Fields.CUSTOM_OBJECT) { - return (JavaObject)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'custom_object' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void set_custom_object(JavaObject value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.CUSTOM_OBJECT; - value_ = value; - } - - public byte[] get_custom_serialized() { - set_custom_serialized(org.apache.thrift7.TBaseHelper.rightSize(buffer_for_custom_serialized())); - ByteBuffer b = buffer_for_custom_serialized(); - return b == null ? null : b.array(); - } - - public ByteBuffer buffer_for_custom_serialized() { - if (getSetField() == _Fields.CUSTOM_SERIALIZED) { - return (ByteBuffer)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'custom_serialized' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void set_custom_serialized(byte[] value) { - set_custom_serialized(ByteBuffer.wrap(value)); - } - - public void set_custom_serialized(ByteBuffer value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.CUSTOM_SERIALIZED; - value_ = value; - } - - public NullStruct get_local_or_shuffle() { - if (getSetField() == _Fields.LOCAL_OR_SHUFFLE) { - return (NullStruct)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'local_or_shuffle' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void set_local_or_shuffle(NullStruct value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.LOCAL_OR_SHUFFLE; - value_ = value; - } - - public NullStruct get_localFirst() { - if (getSetField() == _Fields.LOCAL_FIRST) { - return (NullStruct)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'localFirst' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void set_localFirst(NullStruct value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.LOCAL_FIRST; - value_ = value; - } - - public boolean is_set_fields() { - return setField_ == _Fields.FIELDS; - } - - - public boolean is_set_shuffle() { - return setField_ == _Fields.SHUFFLE; - } - - - public boolean is_set_all() { - return setField_ == _Fields.ALL; - } - - - public boolean is_set_none() { - return setField_ == _Fields.NONE; - } - - - public boolean is_set_direct() { - return setField_ == _Fields.DIRECT; - } - - - public boolean is_set_custom_object() { - return setField_ == _Fields.CUSTOM_OBJECT; - } - - - public boolean is_set_custom_serialized() { - return setField_ == _Fields.CUSTOM_SERIALIZED; - } - - - public boolean is_set_local_or_shuffle() { - return setField_ == _Fields.LOCAL_OR_SHUFFLE; - } - - - public boolean is_set_localFirst() { - return setField_ == _Fields.LOCAL_FIRST; - } - - - public boolean equals(Object other) { - if (other instanceof Grouping) { - return equals((Grouping)other); - } else { - return false; - } - } - - public boolean equals(Grouping other) { - return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); - } - - @Override - public int compareTo(Grouping other) { - int lastComparison = org.apache.thrift7.TBaseHelper.compareTo(getSetField(), other.getSetField()); - if (lastComparison == 0) { - return org.apache.thrift7.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); - } - return lastComparison; - } - - - @Override - public int hashCode() { - HashCodeBuilder hcb = new HashCodeBuilder(); - hcb.append(this.getClass().getName()); - org.apache.thrift7.TFieldIdEnum setField = getSetField(); - if (setField != null) { - hcb.append(setField.getThriftFieldId()); - Object value = getFieldValue(); - if (value instanceof org.apache.thrift7.TEnum) { - hcb.append(((org.apache.thrift7.TEnum)getFieldValue()).getValue()); - } else { - hcb.append(value); - } - } - return hcb.toHashCode(); - } - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - -} diff --git a/jstorm-client/src/main/java/backtype/storm/generated/InvalidTopologyException.java b/jstorm-client/src/main/java/backtype/storm/generated/InvalidTopologyException.java deleted file mode 100644 index a52fbfe22..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/InvalidTopologyException.java +++ /dev/null @@ -1,328 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class InvalidTopologyException extends Exception implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("InvalidTopologyException"); - - private static final org.apache.thrift7.protocol.TField MSG_FIELD_DESC = new org.apache.thrift7.protocol.TField("msg", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String msg; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - MSG((short)1, "msg"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // MSG - return MSG; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.MSG, new org.apache.thrift7.meta_data.FieldMetaData("msg", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(InvalidTopologyException.class, metaDataMap); - } - - public InvalidTopologyException() { - } - - public InvalidTopologyException( - String msg) - { - this(); - this.msg = msg; - } - - /** - * Performs a deep copy on other. - */ - public InvalidTopologyException(InvalidTopologyException other) { - if (other.is_set_msg()) { - this.msg = other.msg; - } - } - - public InvalidTopologyException deepCopy() { - return new InvalidTopologyException(this); - } - - @Override - public void clear() { - this.msg = null; - } - - public String get_msg() { - return this.msg; - } - - public void set_msg(String msg) { - this.msg = msg; - } - - public void unset_msg() { - this.msg = null; - } - - /** Returns true if field msg is set (has been assigned a value) and false otherwise */ - public boolean is_set_msg() { - return this.msg != null; - } - - public void set_msg_isSet(boolean value) { - if (!value) { - this.msg = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case MSG: - if (value == null) { - unset_msg(); - } else { - set_msg((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case MSG: - return get_msg(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case MSG: - return is_set_msg(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof InvalidTopologyException) - return this.equals((InvalidTopologyException)that); - return false; - } - - public boolean equals(InvalidTopologyException that) { - if (that == null) - return false; - - boolean this_present_msg = true && this.is_set_msg(); - boolean that_present_msg = true && that.is_set_msg(); - if (this_present_msg || that_present_msg) { - if (!(this_present_msg && that_present_msg)) - return false; - if (!this.msg.equals(that.msg)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_msg = true && (is_set_msg()); - builder.append(present_msg); - if (present_msg) - builder.append(msg); - - return builder.toHashCode(); - } - - public int compareTo(InvalidTopologyException other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - InvalidTopologyException typedOther = (InvalidTopologyException)other; - - lastComparison = Boolean.valueOf(is_set_msg()).compareTo(typedOther.is_set_msg()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_msg()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.msg, typedOther.msg); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // MSG - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.msg = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.msg != null) { - oprot.writeFieldBegin(MSG_FIELD_DESC); - oprot.writeString(this.msg); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("InvalidTopologyException("); - boolean first = true; - - sb.append("msg:"); - if (this.msg == null) { - sb.append("null"); - } else { - sb.append(this.msg); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_msg()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/JavaObject.java b/jstorm-client/src/main/java/backtype/storm/generated/JavaObject.java deleted file mode 100644 index f6fe43078..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/JavaObject.java +++ /dev/null @@ -1,463 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class JavaObject implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("JavaObject"); - - private static final org.apache.thrift7.protocol.TField FULL_CLASS_NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("full_class_name", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField ARGS_LIST_FIELD_DESC = new org.apache.thrift7.protocol.TField("args_list", org.apache.thrift7.protocol.TType.LIST, (short)2); - - private String full_class_name; // required - private List args_list; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - FULL_CLASS_NAME((short)1, "full_class_name"), - ARGS_LIST((short)2, "args_list"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // FULL_CLASS_NAME - return FULL_CLASS_NAME; - case 2: // ARGS_LIST - return ARGS_LIST; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.FULL_CLASS_NAME, new org.apache.thrift7.meta_data.FieldMetaData("full_class_name", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.ARGS_LIST, new org.apache.thrift7.meta_data.FieldMetaData("args_list", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.ListMetaData(org.apache.thrift7.protocol.TType.LIST, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, JavaObjectArg.class)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(JavaObject.class, metaDataMap); - } - - public JavaObject() { - } - - public JavaObject( - String full_class_name, - List args_list) - { - this(); - this.full_class_name = full_class_name; - this.args_list = args_list; - } - - /** - * Performs a deep copy on other. - */ - public JavaObject(JavaObject other) { - if (other.is_set_full_class_name()) { - this.full_class_name = other.full_class_name; - } - if (other.is_set_args_list()) { - List __this__args_list = new ArrayList(); - for (JavaObjectArg other_element : other.args_list) { - __this__args_list.add(new JavaObjectArg(other_element)); - } - this.args_list = __this__args_list; - } - } - - public JavaObject deepCopy() { - return new JavaObject(this); - } - - @Override - public void clear() { - this.full_class_name = null; - this.args_list = null; - } - - public String get_full_class_name() { - return this.full_class_name; - } - - public void set_full_class_name(String full_class_name) { - this.full_class_name = full_class_name; - } - - public void unset_full_class_name() { - this.full_class_name = null; - } - - /** Returns true if field full_class_name is set (has been assigned a value) and false otherwise */ - public boolean is_set_full_class_name() { - return this.full_class_name != null; - } - - public void set_full_class_name_isSet(boolean value) { - if (!value) { - this.full_class_name = null; - } - } - - public int get_args_list_size() { - return (this.args_list == null) ? 0 : this.args_list.size(); - } - - public java.util.Iterator get_args_list_iterator() { - return (this.args_list == null) ? null : this.args_list.iterator(); - } - - public void add_to_args_list(JavaObjectArg elem) { - if (this.args_list == null) { - this.args_list = new ArrayList(); - } - this.args_list.add(elem); - } - - public List get_args_list() { - return this.args_list; - } - - public void set_args_list(List args_list) { - this.args_list = args_list; - } - - public void unset_args_list() { - this.args_list = null; - } - - /** Returns true if field args_list is set (has been assigned a value) and false otherwise */ - public boolean is_set_args_list() { - return this.args_list != null; - } - - public void set_args_list_isSet(boolean value) { - if (!value) { - this.args_list = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case FULL_CLASS_NAME: - if (value == null) { - unset_full_class_name(); - } else { - set_full_class_name((String)value); - } - break; - - case ARGS_LIST: - if (value == null) { - unset_args_list(); - } else { - set_args_list((List)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case FULL_CLASS_NAME: - return get_full_class_name(); - - case ARGS_LIST: - return get_args_list(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case FULL_CLASS_NAME: - return is_set_full_class_name(); - case ARGS_LIST: - return is_set_args_list(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof JavaObject) - return this.equals((JavaObject)that); - return false; - } - - public boolean equals(JavaObject that) { - if (that == null) - return false; - - boolean this_present_full_class_name = true && this.is_set_full_class_name(); - boolean that_present_full_class_name = true && that.is_set_full_class_name(); - if (this_present_full_class_name || that_present_full_class_name) { - if (!(this_present_full_class_name && that_present_full_class_name)) - return false; - if (!this.full_class_name.equals(that.full_class_name)) - return false; - } - - boolean this_present_args_list = true && this.is_set_args_list(); - boolean that_present_args_list = true && that.is_set_args_list(); - if (this_present_args_list || that_present_args_list) { - if (!(this_present_args_list && that_present_args_list)) - return false; - if (!this.args_list.equals(that.args_list)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_full_class_name = true && (is_set_full_class_name()); - builder.append(present_full_class_name); - if (present_full_class_name) - builder.append(full_class_name); - - boolean present_args_list = true && (is_set_args_list()); - builder.append(present_args_list); - if (present_args_list) - builder.append(args_list); - - return builder.toHashCode(); - } - - public int compareTo(JavaObject other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - JavaObject typedOther = (JavaObject)other; - - lastComparison = Boolean.valueOf(is_set_full_class_name()).compareTo(typedOther.is_set_full_class_name()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_full_class_name()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.full_class_name, typedOther.full_class_name); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_args_list()).compareTo(typedOther.is_set_args_list()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_args_list()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.args_list, typedOther.args_list); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // FULL_CLASS_NAME - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.full_class_name = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // ARGS_LIST - if (field.type == org.apache.thrift7.protocol.TType.LIST) { - { - org.apache.thrift7.protocol.TList _list0 = iprot.readListBegin(); - this.args_list = new ArrayList(_list0.size); - for (int _i1 = 0; _i1 < _list0.size; ++_i1) - { - JavaObjectArg _elem2; // required - _elem2 = new JavaObjectArg(); - _elem2.read(iprot); - this.args_list.add(_elem2); - } - iprot.readListEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.full_class_name != null) { - oprot.writeFieldBegin(FULL_CLASS_NAME_FIELD_DESC); - oprot.writeString(this.full_class_name); - oprot.writeFieldEnd(); - } - if (this.args_list != null) { - oprot.writeFieldBegin(ARGS_LIST_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift7.protocol.TList(org.apache.thrift7.protocol.TType.STRUCT, this.args_list.size())); - for (JavaObjectArg _iter3 : this.args_list) - { - _iter3.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("JavaObject("); - boolean first = true; - - sb.append("full_class_name:"); - if (this.full_class_name == null) { - sb.append("null"); - } else { - sb.append(this.full_class_name); - } - first = false; - if (!first) sb.append(", "); - sb.append("args_list:"); - if (this.args_list == null) { - sb.append("null"); - } else { - sb.append(this.args_list); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_full_class_name()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'full_class_name' is unset! Struct:" + toString()); - } - - if (!is_set_args_list()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'args_list' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/JavaObjectArg.java b/jstorm-client/src/main/java/backtype/storm/generated/JavaObjectArg.java deleted file mode 100644 index c9f638139..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/JavaObjectArg.java +++ /dev/null @@ -1,532 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class JavaObjectArg extends org.apache.thrift7.TUnion { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("JavaObjectArg"); - private static final org.apache.thrift7.protocol.TField INT_ARG_FIELD_DESC = new org.apache.thrift7.protocol.TField("int_arg", org.apache.thrift7.protocol.TType.I32, (short)1); - private static final org.apache.thrift7.protocol.TField LONG_ARG_FIELD_DESC = new org.apache.thrift7.protocol.TField("long_arg", org.apache.thrift7.protocol.TType.I64, (short)2); - private static final org.apache.thrift7.protocol.TField STRING_ARG_FIELD_DESC = new org.apache.thrift7.protocol.TField("string_arg", org.apache.thrift7.protocol.TType.STRING, (short)3); - private static final org.apache.thrift7.protocol.TField BOOL_ARG_FIELD_DESC = new org.apache.thrift7.protocol.TField("bool_arg", org.apache.thrift7.protocol.TType.BOOL, (short)4); - private static final org.apache.thrift7.protocol.TField BINARY_ARG_FIELD_DESC = new org.apache.thrift7.protocol.TField("binary_arg", org.apache.thrift7.protocol.TType.STRING, (short)5); - private static final org.apache.thrift7.protocol.TField DOUBLE_ARG_FIELD_DESC = new org.apache.thrift7.protocol.TField("double_arg", org.apache.thrift7.protocol.TType.DOUBLE, (short)6); - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - INT_ARG((short)1, "int_arg"), - LONG_ARG((short)2, "long_arg"), - STRING_ARG((short)3, "string_arg"), - BOOL_ARG((short)4, "bool_arg"), - BINARY_ARG((short)5, "binary_arg"), - DOUBLE_ARG((short)6, "double_arg"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // INT_ARG - return INT_ARG; - case 2: // LONG_ARG - return LONG_ARG; - case 3: // STRING_ARG - return STRING_ARG; - case 4: // BOOL_ARG - return BOOL_ARG; - case 5: // BINARY_ARG - return BINARY_ARG; - case 6: // DOUBLE_ARG - return DOUBLE_ARG; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.INT_ARG, new org.apache.thrift7.meta_data.FieldMetaData("int_arg", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - tmpMap.put(_Fields.LONG_ARG, new org.apache.thrift7.meta_data.FieldMetaData("long_arg", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I64))); - tmpMap.put(_Fields.STRING_ARG, new org.apache.thrift7.meta_data.FieldMetaData("string_arg", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.BOOL_ARG, new org.apache.thrift7.meta_data.FieldMetaData("bool_arg", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.BOOL))); - tmpMap.put(_Fields.BINARY_ARG, new org.apache.thrift7.meta_data.FieldMetaData("binary_arg", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING , true))); - tmpMap.put(_Fields.DOUBLE_ARG, new org.apache.thrift7.meta_data.FieldMetaData("double_arg", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(JavaObjectArg.class, metaDataMap); - } - - public JavaObjectArg() { - super(); - } - - public JavaObjectArg(_Fields setField, Object value) { - super(setField, value); - } - - public JavaObjectArg(JavaObjectArg other) { - super(other); - } - public JavaObjectArg deepCopy() { - return new JavaObjectArg(this); - } - - public static JavaObjectArg int_arg(int value) { - JavaObjectArg x = new JavaObjectArg(); - x.set_int_arg(value); - return x; - } - - public static JavaObjectArg long_arg(long value) { - JavaObjectArg x = new JavaObjectArg(); - x.set_long_arg(value); - return x; - } - - public static JavaObjectArg string_arg(String value) { - JavaObjectArg x = new JavaObjectArg(); - x.set_string_arg(value); - return x; - } - - public static JavaObjectArg bool_arg(boolean value) { - JavaObjectArg x = new JavaObjectArg(); - x.set_bool_arg(value); - return x; - } - - public static JavaObjectArg binary_arg(ByteBuffer value) { - JavaObjectArg x = new JavaObjectArg(); - x.set_binary_arg(value); - return x; - } - - public static JavaObjectArg binary_arg(byte[] value) { - JavaObjectArg x = new JavaObjectArg(); - x.set_binary_arg(ByteBuffer.wrap(value)); - return x; - } - - public static JavaObjectArg double_arg(double value) { - JavaObjectArg x = new JavaObjectArg(); - x.set_double_arg(value); - return x; - } - - - @Override - protected void checkType(_Fields setField, Object value) throws ClassCastException { - switch (setField) { - case INT_ARG: - if (value instanceof Integer) { - break; - } - throw new ClassCastException("Was expecting value of type Integer for field 'int_arg', but got " + value.getClass().getSimpleName()); - case LONG_ARG: - if (value instanceof Long) { - break; - } - throw new ClassCastException("Was expecting value of type Long for field 'long_arg', but got " + value.getClass().getSimpleName()); - case STRING_ARG: - if (value instanceof String) { - break; - } - throw new ClassCastException("Was expecting value of type String for field 'string_arg', but got " + value.getClass().getSimpleName()); - case BOOL_ARG: - if (value instanceof Boolean) { - break; - } - throw new ClassCastException("Was expecting value of type Boolean for field 'bool_arg', but got " + value.getClass().getSimpleName()); - case BINARY_ARG: - if (value instanceof ByteBuffer) { - break; - } - throw new ClassCastException("Was expecting value of type ByteBuffer for field 'binary_arg', but got " + value.getClass().getSimpleName()); - case DOUBLE_ARG: - if (value instanceof Double) { - break; - } - throw new ClassCastException("Was expecting value of type Double for field 'double_arg', but got " + value.getClass().getSimpleName()); - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected Object readValue(org.apache.thrift7.protocol.TProtocol iprot, org.apache.thrift7.protocol.TField field) throws org.apache.thrift7.TException { - _Fields setField = _Fields.findByThriftId(field.id); - if (setField != null) { - switch (setField) { - case INT_ARG: - if (field.type == INT_ARG_FIELD_DESC.type) { - Integer int_arg; - int_arg = iprot.readI32(); - return int_arg; - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case LONG_ARG: - if (field.type == LONG_ARG_FIELD_DESC.type) { - Long long_arg; - long_arg = iprot.readI64(); - return long_arg; - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case STRING_ARG: - if (field.type == STRING_ARG_FIELD_DESC.type) { - String string_arg; - string_arg = iprot.readString(); - return string_arg; - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case BOOL_ARG: - if (field.type == BOOL_ARG_FIELD_DESC.type) { - Boolean bool_arg; - bool_arg = iprot.readBool(); - return bool_arg; - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case BINARY_ARG: - if (field.type == BINARY_ARG_FIELD_DESC.type) { - ByteBuffer binary_arg; - binary_arg = iprot.readBinary(); - return binary_arg; - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - case DOUBLE_ARG: - if (field.type == DOUBLE_ARG_FIELD_DESC.type) { - Double double_arg; - double_arg = iprot.readDouble(); - return double_arg; - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - default: - throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - return null; - } - } - - @Override - protected void writeValue(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - switch (setField_) { - case INT_ARG: - Integer int_arg = (Integer)value_; - oprot.writeI32(int_arg); - return; - case LONG_ARG: - Long long_arg = (Long)value_; - oprot.writeI64(long_arg); - return; - case STRING_ARG: - String string_arg = (String)value_; - oprot.writeString(string_arg); - return; - case BOOL_ARG: - Boolean bool_arg = (Boolean)value_; - oprot.writeBool(bool_arg); - return; - case BINARY_ARG: - ByteBuffer binary_arg = (ByteBuffer)value_; - oprot.writeBinary(binary_arg); - return; - case DOUBLE_ARG: - Double double_arg = (Double)value_; - oprot.writeDouble(double_arg); - return; - default: - throw new IllegalStateException("Cannot write union with unknown field " + setField_); - } - } - - @Override - protected org.apache.thrift7.protocol.TField getFieldDesc(_Fields setField) { - switch (setField) { - case INT_ARG: - return INT_ARG_FIELD_DESC; - case LONG_ARG: - return LONG_ARG_FIELD_DESC; - case STRING_ARG: - return STRING_ARG_FIELD_DESC; - case BOOL_ARG: - return BOOL_ARG_FIELD_DESC; - case BINARY_ARG: - return BINARY_ARG_FIELD_DESC; - case DOUBLE_ARG: - return DOUBLE_ARG_FIELD_DESC; - default: - throw new IllegalArgumentException("Unknown field id " + setField); - } - } - - @Override - protected org.apache.thrift7.protocol.TStruct getStructDesc() { - return STRUCT_DESC; - } - - @Override - protected _Fields enumForId(short id) { - return _Fields.findByThriftIdOrThrow(id); - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - - public int get_int_arg() { - if (getSetField() == _Fields.INT_ARG) { - return (Integer)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'int_arg' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void set_int_arg(int value) { - setField_ = _Fields.INT_ARG; - value_ = value; - } - - public long get_long_arg() { - if (getSetField() == _Fields.LONG_ARG) { - return (Long)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'long_arg' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void set_long_arg(long value) { - setField_ = _Fields.LONG_ARG; - value_ = value; - } - - public String get_string_arg() { - if (getSetField() == _Fields.STRING_ARG) { - return (String)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'string_arg' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void set_string_arg(String value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.STRING_ARG; - value_ = value; - } - - public boolean get_bool_arg() { - if (getSetField() == _Fields.BOOL_ARG) { - return (Boolean)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'bool_arg' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void set_bool_arg(boolean value) { - setField_ = _Fields.BOOL_ARG; - value_ = value; - } - - public byte[] get_binary_arg() { - set_binary_arg(org.apache.thrift7.TBaseHelper.rightSize(buffer_for_binary_arg())); - ByteBuffer b = buffer_for_binary_arg(); - return b == null ? null : b.array(); - } - - public ByteBuffer buffer_for_binary_arg() { - if (getSetField() == _Fields.BINARY_ARG) { - return (ByteBuffer)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'binary_arg' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void set_binary_arg(byte[] value) { - set_binary_arg(ByteBuffer.wrap(value)); - } - - public void set_binary_arg(ByteBuffer value) { - if (value == null) throw new NullPointerException(); - setField_ = _Fields.BINARY_ARG; - value_ = value; - } - - public double get_double_arg() { - if (getSetField() == _Fields.DOUBLE_ARG) { - return (Double)getFieldValue(); - } else { - throw new RuntimeException("Cannot get field 'double_arg' because union is currently set to " + getFieldDesc(getSetField()).name); - } - } - - public void set_double_arg(double value) { - setField_ = _Fields.DOUBLE_ARG; - value_ = value; - } - - public boolean is_set_int_arg() { - return setField_ == _Fields.INT_ARG; - } - - - public boolean is_set_long_arg() { - return setField_ == _Fields.LONG_ARG; - } - - - public boolean is_set_string_arg() { - return setField_ == _Fields.STRING_ARG; - } - - - public boolean is_set_bool_arg() { - return setField_ == _Fields.BOOL_ARG; - } - - - public boolean is_set_binary_arg() { - return setField_ == _Fields.BINARY_ARG; - } - - - public boolean is_set_double_arg() { - return setField_ == _Fields.DOUBLE_ARG; - } - - - public boolean equals(Object other) { - if (other instanceof JavaObjectArg) { - return equals((JavaObjectArg)other); - } else { - return false; - } - } - - public boolean equals(JavaObjectArg other) { - return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); - } - - @Override - public int compareTo(JavaObjectArg other) { - int lastComparison = org.apache.thrift7.TBaseHelper.compareTo(getSetField(), other.getSetField()); - if (lastComparison == 0) { - return org.apache.thrift7.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); - } - return lastComparison; - } - - - @Override - public int hashCode() { - HashCodeBuilder hcb = new HashCodeBuilder(); - hcb.append(this.getClass().getName()); - org.apache.thrift7.TFieldIdEnum setField = getSetField(); - if (setField != null) { - hcb.append(setField.getThriftFieldId()); - Object value = getFieldValue(); - if (value instanceof org.apache.thrift7.TEnum) { - hcb.append(((org.apache.thrift7.TEnum)getFieldValue()).getValue()); - } else { - hcb.append(value); - } - } - return hcb.toHashCode(); - } - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - -} diff --git a/jstorm-client/src/main/java/backtype/storm/generated/KillOptions.java b/jstorm-client/src/main/java/backtype/storm/generated/KillOptions.java deleted file mode 100644 index cf07150e9..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/KillOptions.java +++ /dev/null @@ -1,320 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class KillOptions implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("KillOptions"); - - private static final org.apache.thrift7.protocol.TField WAIT_SECS_FIELD_DESC = new org.apache.thrift7.protocol.TField("wait_secs", org.apache.thrift7.protocol.TType.I32, (short)1); - - private int wait_secs; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - WAIT_SECS((short)1, "wait_secs"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // WAIT_SECS - return WAIT_SECS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __WAIT_SECS_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.WAIT_SECS, new org.apache.thrift7.meta_data.FieldMetaData("wait_secs", org.apache.thrift7.TFieldRequirementType.OPTIONAL, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(KillOptions.class, metaDataMap); - } - - public KillOptions() { - } - - /** - * Performs a deep copy on other. - */ - public KillOptions(KillOptions other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); - this.wait_secs = other.wait_secs; - } - - public KillOptions deepCopy() { - return new KillOptions(this); - } - - @Override - public void clear() { - set_wait_secs_isSet(false); - this.wait_secs = 0; - } - - public int get_wait_secs() { - return this.wait_secs; - } - - public void set_wait_secs(int wait_secs) { - this.wait_secs = wait_secs; - set_wait_secs_isSet(true); - } - - public void unset_wait_secs() { - __isset_bit_vector.clear(__WAIT_SECS_ISSET_ID); - } - - /** Returns true if field wait_secs is set (has been assigned a value) and false otherwise */ - public boolean is_set_wait_secs() { - return __isset_bit_vector.get(__WAIT_SECS_ISSET_ID); - } - - public void set_wait_secs_isSet(boolean value) { - __isset_bit_vector.set(__WAIT_SECS_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case WAIT_SECS: - if (value == null) { - unset_wait_secs(); - } else { - set_wait_secs((Integer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case WAIT_SECS: - return Integer.valueOf(get_wait_secs()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case WAIT_SECS: - return is_set_wait_secs(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof KillOptions) - return this.equals((KillOptions)that); - return false; - } - - public boolean equals(KillOptions that) { - if (that == null) - return false; - - boolean this_present_wait_secs = true && this.is_set_wait_secs(); - boolean that_present_wait_secs = true && that.is_set_wait_secs(); - if (this_present_wait_secs || that_present_wait_secs) { - if (!(this_present_wait_secs && that_present_wait_secs)) - return false; - if (this.wait_secs != that.wait_secs) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_wait_secs = true && (is_set_wait_secs()); - builder.append(present_wait_secs); - if (present_wait_secs) - builder.append(wait_secs); - - return builder.toHashCode(); - } - - public int compareTo(KillOptions other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - KillOptions typedOther = (KillOptions)other; - - lastComparison = Boolean.valueOf(is_set_wait_secs()).compareTo(typedOther.is_set_wait_secs()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_wait_secs()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.wait_secs, typedOther.wait_secs); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // WAIT_SECS - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.wait_secs = iprot.readI32(); - set_wait_secs_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (is_set_wait_secs()) { - oprot.writeFieldBegin(WAIT_SECS_FIELD_DESC); - oprot.writeI32(this.wait_secs); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("KillOptions("); - boolean first = true; - - if (is_set_wait_secs()) { - sb.append("wait_secs:"); - sb.append(this.wait_secs); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bit_vector = new BitSet(1); - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/MonitorOptions.java b/jstorm-client/src/main/java/backtype/storm/generated/MonitorOptions.java deleted file mode 100644 index fa0adf35c..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/MonitorOptions.java +++ /dev/null @@ -1,320 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class MonitorOptions implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("MonitorOptions"); - - private static final org.apache.thrift7.protocol.TField IS_ENABLE_FIELD_DESC = new org.apache.thrift7.protocol.TField("isEnable", org.apache.thrift7.protocol.TType.BOOL, (short)1); - - private boolean isEnable; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - IS_ENABLE((short)1, "isEnable"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // IS_ENABLE - return IS_ENABLE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __ISENABLE_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.IS_ENABLE, new org.apache.thrift7.meta_data.FieldMetaData("isEnable", org.apache.thrift7.TFieldRequirementType.OPTIONAL, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.BOOL))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(MonitorOptions.class, metaDataMap); - } - - public MonitorOptions() { - } - - /** - * Performs a deep copy on other. - */ - public MonitorOptions(MonitorOptions other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); - this.isEnable = other.isEnable; - } - - public MonitorOptions deepCopy() { - return new MonitorOptions(this); - } - - @Override - public void clear() { - set_isEnable_isSet(false); - this.isEnable = false; - } - - public boolean is_isEnable() { - return this.isEnable; - } - - public void set_isEnable(boolean isEnable) { - this.isEnable = isEnable; - set_isEnable_isSet(true); - } - - public void unset_isEnable() { - __isset_bit_vector.clear(__ISENABLE_ISSET_ID); - } - - /** Returns true if field isEnable is set (has been assigned a value) and false otherwise */ - public boolean is_set_isEnable() { - return __isset_bit_vector.get(__ISENABLE_ISSET_ID); - } - - public void set_isEnable_isSet(boolean value) { - __isset_bit_vector.set(__ISENABLE_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case IS_ENABLE: - if (value == null) { - unset_isEnable(); - } else { - set_isEnable((Boolean)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case IS_ENABLE: - return Boolean.valueOf(is_isEnable()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case IS_ENABLE: - return is_set_isEnable(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof MonitorOptions) - return this.equals((MonitorOptions)that); - return false; - } - - public boolean equals(MonitorOptions that) { - if (that == null) - return false; - - boolean this_present_isEnable = true && this.is_set_isEnable(); - boolean that_present_isEnable = true && that.is_set_isEnable(); - if (this_present_isEnable || that_present_isEnable) { - if (!(this_present_isEnable && that_present_isEnable)) - return false; - if (this.isEnable != that.isEnable) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_isEnable = true && (is_set_isEnable()); - builder.append(present_isEnable); - if (present_isEnable) - builder.append(isEnable); - - return builder.toHashCode(); - } - - public int compareTo(MonitorOptions other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - MonitorOptions typedOther = (MonitorOptions)other; - - lastComparison = Boolean.valueOf(is_set_isEnable()).compareTo(typedOther.is_set_isEnable()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_isEnable()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.isEnable, typedOther.isEnable); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // IS_ENABLE - if (field.type == org.apache.thrift7.protocol.TType.BOOL) { - this.isEnable = iprot.readBool(); - set_isEnable_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (is_set_isEnable()) { - oprot.writeFieldBegin(IS_ENABLE_FIELD_DESC); - oprot.writeBool(this.isEnable); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("MonitorOptions("); - boolean first = true; - - if (is_set_isEnable()) { - sb.append("isEnable:"); - sb.append(this.isEnable); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bit_vector = new BitSet(1); - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/Nimbus.java b/jstorm-client/src/main/java/backtype/storm/generated/Nimbus.java deleted file mode 100644 index 6f63240d6..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/Nimbus.java +++ /dev/null @@ -1,18351 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class Nimbus { - - public interface Iface { - - public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift7.TException; - - public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws AlreadyAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift7.TException; - - public void killTopology(String name) throws NotAliveException, org.apache.thrift7.TException; - - public void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException, org.apache.thrift7.TException; - - public void activate(String name) throws NotAliveException, org.apache.thrift7.TException; - - public void deactivate(String name) throws NotAliveException, org.apache.thrift7.TException; - - public void rebalance(String name, RebalanceOptions options) throws NotAliveException, InvalidTopologyException, org.apache.thrift7.TException; - - public void metricMonitor(String name, MonitorOptions options) throws NotAliveException, org.apache.thrift7.TException; - - public void restart(String name, String jsonConf) throws NotAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift7.TException; - - public void beginLibUpload(String libName) throws org.apache.thrift7.TException; - - public String beginFileUpload() throws org.apache.thrift7.TException; - - public void uploadChunk(String location, ByteBuffer chunk) throws org.apache.thrift7.TException; - - public void finishFileUpload(String location) throws org.apache.thrift7.TException; - - public String beginFileDownload(String file) throws org.apache.thrift7.TException; - - public ByteBuffer downloadChunk(String id) throws org.apache.thrift7.TException; - - public String getNimbusConf() throws org.apache.thrift7.TException; - - public ClusterSummary getClusterInfo() throws org.apache.thrift7.TException; - - public TopologyInfo getTopologyInfo(String id) throws NotAliveException, org.apache.thrift7.TException; - - public TopologyInfo getTopologyInfoByName(String topologyName) throws NotAliveException, org.apache.thrift7.TException; - - public SupervisorWorkers getSupervisorWorkers(String host) throws NotAliveException, org.apache.thrift7.TException; - - public String getTopologyConf(String id) throws NotAliveException, org.apache.thrift7.TException; - - public StormTopology getTopology(String id) throws NotAliveException, org.apache.thrift7.TException; - - public StormTopology getUserTopology(String id) throws NotAliveException, org.apache.thrift7.TException; - - public TopologyMetricInfo getTopologyMetric(String id) throws NotAliveException, org.apache.thrift7.TException; - - } - - public interface AsyncIface { - - public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void killTopology(String name, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void killTopologyWithOpts(String name, KillOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void activate(String name, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void deactivate(String name, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void rebalance(String name, RebalanceOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void metricMonitor(String name, MonitorOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void restart(String name, String jsonConf, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void beginLibUpload(String libName, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void beginFileUpload(org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void uploadChunk(String location, ByteBuffer chunk, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void finishFileUpload(String location, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void beginFileDownload(String file, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void downloadChunk(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void getNimbusConf(org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void getClusterInfo(org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void getTopologyInfo(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void getTopologyInfoByName(String topologyName, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void getSupervisorWorkers(String host, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void getTopologyConf(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void getTopology(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void getUserTopology(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - public void getTopologyMetric(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException; - - } - - public static class Client extends org.apache.thrift7.TServiceClient implements Iface { - public static class Factory implements org.apache.thrift7.TServiceClientFactory { - public Factory() {} - public Client getClient(org.apache.thrift7.protocol.TProtocol prot) { - return new Client(prot); - } - public Client getClient(org.apache.thrift7.protocol.TProtocol iprot, org.apache.thrift7.protocol.TProtocol oprot) { - return new Client(iprot, oprot); - } - } - - public Client(org.apache.thrift7.protocol.TProtocol prot) - { - super(prot, prot); - } - - public Client(org.apache.thrift7.protocol.TProtocol iprot, org.apache.thrift7.protocol.TProtocol oprot) { - super(iprot, oprot); - } - - public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift7.TException - { - send_submitTopology(name, uploadedJarLocation, jsonConf, topology); - recv_submitTopology(); - } - - public void send_submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws org.apache.thrift7.TException - { - submitTopology_args args = new submitTopology_args(); - args.set_name(name); - args.set_uploadedJarLocation(uploadedJarLocation); - args.set_jsonConf(jsonConf); - args.set_topology(topology); - sendBase("submitTopology", args); - } - - public void recv_submitTopology() throws AlreadyAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift7.TException - { - submitTopology_result result = new submitTopology_result(); - receiveBase(result, "submitTopology"); - if (result.e != null) { - throw result.e; - } - if (result.ite != null) { - throw result.ite; - } - if (result.tae != null) { - throw result.tae; - } - return; - } - - public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws AlreadyAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift7.TException - { - send_submitTopologyWithOpts(name, uploadedJarLocation, jsonConf, topology, options); - recv_submitTopologyWithOpts(); - } - - public void send_submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws org.apache.thrift7.TException - { - submitTopologyWithOpts_args args = new submitTopologyWithOpts_args(); - args.set_name(name); - args.set_uploadedJarLocation(uploadedJarLocation); - args.set_jsonConf(jsonConf); - args.set_topology(topology); - args.set_options(options); - sendBase("submitTopologyWithOpts", args); - } - - public void recv_submitTopologyWithOpts() throws AlreadyAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift7.TException - { - submitTopologyWithOpts_result result = new submitTopologyWithOpts_result(); - receiveBase(result, "submitTopologyWithOpts"); - if (result.e != null) { - throw result.e; - } - if (result.ite != null) { - throw result.ite; - } - if (result.tae != null) { - throw result.tae; - } - return; - } - - public void killTopology(String name) throws NotAliveException, org.apache.thrift7.TException - { - send_killTopology(name); - recv_killTopology(); - } - - public void send_killTopology(String name) throws org.apache.thrift7.TException - { - killTopology_args args = new killTopology_args(); - args.set_name(name); - sendBase("killTopology", args); - } - - public void recv_killTopology() throws NotAliveException, org.apache.thrift7.TException - { - killTopology_result result = new killTopology_result(); - receiveBase(result, "killTopology"); - if (result.e != null) { - throw result.e; - } - return; - } - - public void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException, org.apache.thrift7.TException - { - send_killTopologyWithOpts(name, options); - recv_killTopologyWithOpts(); - } - - public void send_killTopologyWithOpts(String name, KillOptions options) throws org.apache.thrift7.TException - { - killTopologyWithOpts_args args = new killTopologyWithOpts_args(); - args.set_name(name); - args.set_options(options); - sendBase("killTopologyWithOpts", args); - } - - public void recv_killTopologyWithOpts() throws NotAliveException, org.apache.thrift7.TException - { - killTopologyWithOpts_result result = new killTopologyWithOpts_result(); - receiveBase(result, "killTopologyWithOpts"); - if (result.e != null) { - throw result.e; - } - return; - } - - public void activate(String name) throws NotAliveException, org.apache.thrift7.TException - { - send_activate(name); - recv_activate(); - } - - public void send_activate(String name) throws org.apache.thrift7.TException - { - activate_args args = new activate_args(); - args.set_name(name); - sendBase("activate", args); - } - - public void recv_activate() throws NotAliveException, org.apache.thrift7.TException - { - activate_result result = new activate_result(); - receiveBase(result, "activate"); - if (result.e != null) { - throw result.e; - } - return; - } - - public void deactivate(String name) throws NotAliveException, org.apache.thrift7.TException - { - send_deactivate(name); - recv_deactivate(); - } - - public void send_deactivate(String name) throws org.apache.thrift7.TException - { - deactivate_args args = new deactivate_args(); - args.set_name(name); - sendBase("deactivate", args); - } - - public void recv_deactivate() throws NotAliveException, org.apache.thrift7.TException - { - deactivate_result result = new deactivate_result(); - receiveBase(result, "deactivate"); - if (result.e != null) { - throw result.e; - } - return; - } - - public void rebalance(String name, RebalanceOptions options) throws NotAliveException, InvalidTopologyException, org.apache.thrift7.TException - { - send_rebalance(name, options); - recv_rebalance(); - } - - public void send_rebalance(String name, RebalanceOptions options) throws org.apache.thrift7.TException - { - rebalance_args args = new rebalance_args(); - args.set_name(name); - args.set_options(options); - sendBase("rebalance", args); - } - - public void recv_rebalance() throws NotAliveException, InvalidTopologyException, org.apache.thrift7.TException - { - rebalance_result result = new rebalance_result(); - receiveBase(result, "rebalance"); - if (result.e != null) { - throw result.e; - } - if (result.ite != null) { - throw result.ite; - } - return; - } - - public void metricMonitor(String name, MonitorOptions options) throws NotAliveException, org.apache.thrift7.TException - { - send_metricMonitor(name, options); - recv_metricMonitor(); - } - - public void send_metricMonitor(String name, MonitorOptions options) throws org.apache.thrift7.TException - { - metricMonitor_args args = new metricMonitor_args(); - args.set_name(name); - args.set_options(options); - sendBase("metricMonitor", args); - } - - public void recv_metricMonitor() throws NotAliveException, org.apache.thrift7.TException - { - metricMonitor_result result = new metricMonitor_result(); - receiveBase(result, "metricMonitor"); - if (result.e != null) { - throw result.e; - } - return; - } - - public void restart(String name, String jsonConf) throws NotAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift7.TException - { - send_restart(name, jsonConf); - recv_restart(); - } - - public void send_restart(String name, String jsonConf) throws org.apache.thrift7.TException - { - restart_args args = new restart_args(); - args.set_name(name); - args.set_jsonConf(jsonConf); - sendBase("restart", args); - } - - public void recv_restart() throws NotAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift7.TException - { - restart_result result = new restart_result(); - receiveBase(result, "restart"); - if (result.e != null) { - throw result.e; - } - if (result.ite != null) { - throw result.ite; - } - if (result.tae != null) { - throw result.tae; - } - return; - } - - public void beginLibUpload(String libName) throws org.apache.thrift7.TException - { - send_beginLibUpload(libName); - recv_beginLibUpload(); - } - - public void send_beginLibUpload(String libName) throws org.apache.thrift7.TException - { - beginLibUpload_args args = new beginLibUpload_args(); - args.set_libName(libName); - sendBase("beginLibUpload", args); - } - - public void recv_beginLibUpload() throws org.apache.thrift7.TException - { - beginLibUpload_result result = new beginLibUpload_result(); - receiveBase(result, "beginLibUpload"); - return; - } - - public String beginFileUpload() throws org.apache.thrift7.TException - { - send_beginFileUpload(); - return recv_beginFileUpload(); - } - - public void send_beginFileUpload() throws org.apache.thrift7.TException - { - beginFileUpload_args args = new beginFileUpload_args(); - sendBase("beginFileUpload", args); - } - - public String recv_beginFileUpload() throws org.apache.thrift7.TException - { - beginFileUpload_result result = new beginFileUpload_result(); - receiveBase(result, "beginFileUpload"); - if (result.is_set_success()) { - return result.success; - } - throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "beginFileUpload failed: unknown result"); - } - - public void uploadChunk(String location, ByteBuffer chunk) throws org.apache.thrift7.TException - { - send_uploadChunk(location, chunk); - recv_uploadChunk(); - } - - public void send_uploadChunk(String location, ByteBuffer chunk) throws org.apache.thrift7.TException - { - uploadChunk_args args = new uploadChunk_args(); - args.set_location(location); - args.set_chunk(chunk); - sendBase("uploadChunk", args); - } - - public void recv_uploadChunk() throws org.apache.thrift7.TException - { - uploadChunk_result result = new uploadChunk_result(); - receiveBase(result, "uploadChunk"); - return; - } - - public void finishFileUpload(String location) throws org.apache.thrift7.TException - { - send_finishFileUpload(location); - recv_finishFileUpload(); - } - - public void send_finishFileUpload(String location) throws org.apache.thrift7.TException - { - finishFileUpload_args args = new finishFileUpload_args(); - args.set_location(location); - sendBase("finishFileUpload", args); - } - - public void recv_finishFileUpload() throws org.apache.thrift7.TException - { - finishFileUpload_result result = new finishFileUpload_result(); - receiveBase(result, "finishFileUpload"); - return; - } - - public String beginFileDownload(String file) throws org.apache.thrift7.TException - { - send_beginFileDownload(file); - return recv_beginFileDownload(); - } - - public void send_beginFileDownload(String file) throws org.apache.thrift7.TException - { - beginFileDownload_args args = new beginFileDownload_args(); - args.set_file(file); - sendBase("beginFileDownload", args); - } - - public String recv_beginFileDownload() throws org.apache.thrift7.TException - { - beginFileDownload_result result = new beginFileDownload_result(); - receiveBase(result, "beginFileDownload"); - if (result.is_set_success()) { - return result.success; - } - throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "beginFileDownload failed: unknown result"); - } - - public ByteBuffer downloadChunk(String id) throws org.apache.thrift7.TException - { - send_downloadChunk(id); - return recv_downloadChunk(); - } - - public void send_downloadChunk(String id) throws org.apache.thrift7.TException - { - downloadChunk_args args = new downloadChunk_args(); - args.set_id(id); - sendBase("downloadChunk", args); - } - - public ByteBuffer recv_downloadChunk() throws org.apache.thrift7.TException - { - downloadChunk_result result = new downloadChunk_result(); - receiveBase(result, "downloadChunk"); - if (result.is_set_success()) { - return result.success; - } - throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "downloadChunk failed: unknown result"); - } - - public String getNimbusConf() throws org.apache.thrift7.TException - { - send_getNimbusConf(); - return recv_getNimbusConf(); - } - - public void send_getNimbusConf() throws org.apache.thrift7.TException - { - getNimbusConf_args args = new getNimbusConf_args(); - sendBase("getNimbusConf", args); - } - - public String recv_getNimbusConf() throws org.apache.thrift7.TException - { - getNimbusConf_result result = new getNimbusConf_result(); - receiveBase(result, "getNimbusConf"); - if (result.is_set_success()) { - return result.success; - } - throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "getNimbusConf failed: unknown result"); - } - - public ClusterSummary getClusterInfo() throws org.apache.thrift7.TException - { - send_getClusterInfo(); - return recv_getClusterInfo(); - } - - public void send_getClusterInfo() throws org.apache.thrift7.TException - { - getClusterInfo_args args = new getClusterInfo_args(); - sendBase("getClusterInfo", args); - } - - public ClusterSummary recv_getClusterInfo() throws org.apache.thrift7.TException - { - getClusterInfo_result result = new getClusterInfo_result(); - receiveBase(result, "getClusterInfo"); - if (result.is_set_success()) { - return result.success; - } - throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "getClusterInfo failed: unknown result"); - } - - public TopologyInfo getTopologyInfo(String id) throws NotAliveException, org.apache.thrift7.TException - { - send_getTopologyInfo(id); - return recv_getTopologyInfo(); - } - - public void send_getTopologyInfo(String id) throws org.apache.thrift7.TException - { - getTopologyInfo_args args = new getTopologyInfo_args(); - args.set_id(id); - sendBase("getTopologyInfo", args); - } - - public TopologyInfo recv_getTopologyInfo() throws NotAliveException, org.apache.thrift7.TException - { - getTopologyInfo_result result = new getTopologyInfo_result(); - receiveBase(result, "getTopologyInfo"); - if (result.is_set_success()) { - return result.success; - } - if (result.e != null) { - throw result.e; - } - throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "getTopologyInfo failed: unknown result"); - } - - public TopologyInfo getTopologyInfoByName(String topologyName) throws NotAliveException, org.apache.thrift7.TException - { - send_getTopologyInfoByName(topologyName); - return recv_getTopologyInfoByName(); - } - - public void send_getTopologyInfoByName(String topologyName) throws org.apache.thrift7.TException - { - getTopologyInfoByName_args args = new getTopologyInfoByName_args(); - args.set_topologyName(topologyName); - sendBase("getTopologyInfoByName", args); - } - - public TopologyInfo recv_getTopologyInfoByName() throws NotAliveException, org.apache.thrift7.TException - { - getTopologyInfoByName_result result = new getTopologyInfoByName_result(); - receiveBase(result, "getTopologyInfoByName"); - if (result.is_set_success()) { - return result.success; - } - if (result.e != null) { - throw result.e; - } - throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "getTopologyInfoByName failed: unknown result"); - } - - public SupervisorWorkers getSupervisorWorkers(String host) throws NotAliveException, org.apache.thrift7.TException - { - send_getSupervisorWorkers(host); - return recv_getSupervisorWorkers(); - } - - public void send_getSupervisorWorkers(String host) throws org.apache.thrift7.TException - { - getSupervisorWorkers_args args = new getSupervisorWorkers_args(); - args.set_host(host); - sendBase("getSupervisorWorkers", args); - } - - public SupervisorWorkers recv_getSupervisorWorkers() throws NotAliveException, org.apache.thrift7.TException - { - getSupervisorWorkers_result result = new getSupervisorWorkers_result(); - receiveBase(result, "getSupervisorWorkers"); - if (result.is_set_success()) { - return result.success; - } - if (result.e != null) { - throw result.e; - } - throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "getSupervisorWorkers failed: unknown result"); - } - - public String getTopologyConf(String id) throws NotAliveException, org.apache.thrift7.TException - { - send_getTopologyConf(id); - return recv_getTopologyConf(); - } - - public void send_getTopologyConf(String id) throws org.apache.thrift7.TException - { - getTopologyConf_args args = new getTopologyConf_args(); - args.set_id(id); - sendBase("getTopologyConf", args); - } - - public String recv_getTopologyConf() throws NotAliveException, org.apache.thrift7.TException - { - getTopologyConf_result result = new getTopologyConf_result(); - receiveBase(result, "getTopologyConf"); - if (result.is_set_success()) { - return result.success; - } - if (result.e != null) { - throw result.e; - } - throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "getTopologyConf failed: unknown result"); - } - - public StormTopology getTopology(String id) throws NotAliveException, org.apache.thrift7.TException - { - send_getTopology(id); - return recv_getTopology(); - } - - public void send_getTopology(String id) throws org.apache.thrift7.TException - { - getTopology_args args = new getTopology_args(); - args.set_id(id); - sendBase("getTopology", args); - } - - public StormTopology recv_getTopology() throws NotAliveException, org.apache.thrift7.TException - { - getTopology_result result = new getTopology_result(); - receiveBase(result, "getTopology"); - if (result.is_set_success()) { - return result.success; - } - if (result.e != null) { - throw result.e; - } - throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "getTopology failed: unknown result"); - } - - public StormTopology getUserTopology(String id) throws NotAliveException, org.apache.thrift7.TException - { - send_getUserTopology(id); - return recv_getUserTopology(); - } - - public void send_getUserTopology(String id) throws org.apache.thrift7.TException - { - getUserTopology_args args = new getUserTopology_args(); - args.set_id(id); - sendBase("getUserTopology", args); - } - - public StormTopology recv_getUserTopology() throws NotAliveException, org.apache.thrift7.TException - { - getUserTopology_result result = new getUserTopology_result(); - receiveBase(result, "getUserTopology"); - if (result.is_set_success()) { - return result.success; - } - if (result.e != null) { - throw result.e; - } - throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "getUserTopology failed: unknown result"); - } - - public TopologyMetricInfo getTopologyMetric(String id) throws NotAliveException, org.apache.thrift7.TException - { - send_getTopologyMetric(id); - return recv_getTopologyMetric(); - } - - public void send_getTopologyMetric(String id) throws org.apache.thrift7.TException - { - getTopologyMetric_args args = new getTopologyMetric_args(); - args.set_id(id); - sendBase("getTopologyMetric", args); - } - - public TopologyMetricInfo recv_getTopologyMetric() throws NotAliveException, org.apache.thrift7.TException - { - getTopologyMetric_result result = new getTopologyMetric_result(); - receiveBase(result, "getTopologyMetric"); - if (result.is_set_success()) { - return result.success; - } - if (result.e != null) { - throw result.e; - } - throw new org.apache.thrift7.TApplicationException(org.apache.thrift7.TApplicationException.MISSING_RESULT, "getTopologyMetric failed: unknown result"); - } - - } - public static class AsyncClient extends org.apache.thrift7.async.TAsyncClient implements AsyncIface { - public static class Factory implements org.apache.thrift7.async.TAsyncClientFactory { - private org.apache.thrift7.async.TAsyncClientManager clientManager; - private org.apache.thrift7.protocol.TProtocolFactory protocolFactory; - public Factory(org.apache.thrift7.async.TAsyncClientManager clientManager, org.apache.thrift7.protocol.TProtocolFactory protocolFactory) { - this.clientManager = clientManager; - this.protocolFactory = protocolFactory; - } - public AsyncClient getAsyncClient(org.apache.thrift7.transport.TNonblockingTransport transport) { - return new AsyncClient(protocolFactory, clientManager, transport); - } - } - - public AsyncClient(org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.async.TAsyncClientManager clientManager, org.apache.thrift7.transport.TNonblockingTransport transport) { - super(protocolFactory, clientManager, transport); - } - - public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - submitTopology_call method_call = new submitTopology_call(name, uploadedJarLocation, jsonConf, topology, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class submitTopology_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String name; - private String uploadedJarLocation; - private String jsonConf; - private StormTopology topology; - public submitTopology_call(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.name = name; - this.uploadedJarLocation = uploadedJarLocation; - this.jsonConf = jsonConf; - this.topology = topology; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("submitTopology", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - submitTopology_args args = new submitTopology_args(); - args.set_name(name); - args.set_uploadedJarLocation(uploadedJarLocation); - args.set_jsonConf(jsonConf); - args.set_topology(topology); - args.write(prot); - prot.writeMessageEnd(); - } - - public void getResult() throws AlreadyAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - (new Client(prot)).recv_submitTopology(); - } - } - - public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - submitTopologyWithOpts_call method_call = new submitTopologyWithOpts_call(name, uploadedJarLocation, jsonConf, topology, options, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class submitTopologyWithOpts_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String name; - private String uploadedJarLocation; - private String jsonConf; - private StormTopology topology; - private SubmitOptions options; - public submitTopologyWithOpts_call(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.name = name; - this.uploadedJarLocation = uploadedJarLocation; - this.jsonConf = jsonConf; - this.topology = topology; - this.options = options; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("submitTopologyWithOpts", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - submitTopologyWithOpts_args args = new submitTopologyWithOpts_args(); - args.set_name(name); - args.set_uploadedJarLocation(uploadedJarLocation); - args.set_jsonConf(jsonConf); - args.set_topology(topology); - args.set_options(options); - args.write(prot); - prot.writeMessageEnd(); - } - - public void getResult() throws AlreadyAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - (new Client(prot)).recv_submitTopologyWithOpts(); - } - } - - public void killTopology(String name, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - killTopology_call method_call = new killTopology_call(name, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class killTopology_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String name; - public killTopology_call(String name, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.name = name; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("killTopology", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - killTopology_args args = new killTopology_args(); - args.set_name(name); - args.write(prot); - prot.writeMessageEnd(); - } - - public void getResult() throws NotAliveException, org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - (new Client(prot)).recv_killTopology(); - } - } - - public void killTopologyWithOpts(String name, KillOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - killTopologyWithOpts_call method_call = new killTopologyWithOpts_call(name, options, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class killTopologyWithOpts_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String name; - private KillOptions options; - public killTopologyWithOpts_call(String name, KillOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.name = name; - this.options = options; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("killTopologyWithOpts", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - killTopologyWithOpts_args args = new killTopologyWithOpts_args(); - args.set_name(name); - args.set_options(options); - args.write(prot); - prot.writeMessageEnd(); - } - - public void getResult() throws NotAliveException, org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - (new Client(prot)).recv_killTopologyWithOpts(); - } - } - - public void activate(String name, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - activate_call method_call = new activate_call(name, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class activate_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String name; - public activate_call(String name, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.name = name; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("activate", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - activate_args args = new activate_args(); - args.set_name(name); - args.write(prot); - prot.writeMessageEnd(); - } - - public void getResult() throws NotAliveException, org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - (new Client(prot)).recv_activate(); - } - } - - public void deactivate(String name, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - deactivate_call method_call = new deactivate_call(name, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class deactivate_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String name; - public deactivate_call(String name, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.name = name; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("deactivate", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - deactivate_args args = new deactivate_args(); - args.set_name(name); - args.write(prot); - prot.writeMessageEnd(); - } - - public void getResult() throws NotAliveException, org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - (new Client(prot)).recv_deactivate(); - } - } - - public void rebalance(String name, RebalanceOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - rebalance_call method_call = new rebalance_call(name, options, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class rebalance_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String name; - private RebalanceOptions options; - public rebalance_call(String name, RebalanceOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.name = name; - this.options = options; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("rebalance", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - rebalance_args args = new rebalance_args(); - args.set_name(name); - args.set_options(options); - args.write(prot); - prot.writeMessageEnd(); - } - - public void getResult() throws NotAliveException, InvalidTopologyException, org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - (new Client(prot)).recv_rebalance(); - } - } - - public void metricMonitor(String name, MonitorOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - metricMonitor_call method_call = new metricMonitor_call(name, options, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class metricMonitor_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String name; - private MonitorOptions options; - public metricMonitor_call(String name, MonitorOptions options, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.name = name; - this.options = options; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("metricMonitor", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - metricMonitor_args args = new metricMonitor_args(); - args.set_name(name); - args.set_options(options); - args.write(prot); - prot.writeMessageEnd(); - } - - public void getResult() throws NotAliveException, org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - (new Client(prot)).recv_metricMonitor(); - } - } - - public void restart(String name, String jsonConf, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - restart_call method_call = new restart_call(name, jsonConf, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class restart_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String name; - private String jsonConf; - public restart_call(String name, String jsonConf, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.name = name; - this.jsonConf = jsonConf; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("restart", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - restart_args args = new restart_args(); - args.set_name(name); - args.set_jsonConf(jsonConf); - args.write(prot); - prot.writeMessageEnd(); - } - - public void getResult() throws NotAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - (new Client(prot)).recv_restart(); - } - } - - public void beginLibUpload(String libName, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - beginLibUpload_call method_call = new beginLibUpload_call(libName, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class beginLibUpload_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String libName; - public beginLibUpload_call(String libName, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.libName = libName; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("beginLibUpload", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - beginLibUpload_args args = new beginLibUpload_args(); - args.set_libName(libName); - args.write(prot); - prot.writeMessageEnd(); - } - - public void getResult() throws org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - (new Client(prot)).recv_beginLibUpload(); - } - } - - public void beginFileUpload(org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - beginFileUpload_call method_call = new beginFileUpload_call(resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class beginFileUpload_call extends org.apache.thrift7.async.TAsyncMethodCall { - public beginFileUpload_call(org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("beginFileUpload", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - beginFileUpload_args args = new beginFileUpload_args(); - args.write(prot); - prot.writeMessageEnd(); - } - - public String getResult() throws org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_beginFileUpload(); - } - } - - public void uploadChunk(String location, ByteBuffer chunk, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - uploadChunk_call method_call = new uploadChunk_call(location, chunk, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class uploadChunk_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String location; - private ByteBuffer chunk; - public uploadChunk_call(String location, ByteBuffer chunk, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.location = location; - this.chunk = chunk; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("uploadChunk", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - uploadChunk_args args = new uploadChunk_args(); - args.set_location(location); - args.set_chunk(chunk); - args.write(prot); - prot.writeMessageEnd(); - } - - public void getResult() throws org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - (new Client(prot)).recv_uploadChunk(); - } - } - - public void finishFileUpload(String location, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - finishFileUpload_call method_call = new finishFileUpload_call(location, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class finishFileUpload_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String location; - public finishFileUpload_call(String location, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.location = location; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("finishFileUpload", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - finishFileUpload_args args = new finishFileUpload_args(); - args.set_location(location); - args.write(prot); - prot.writeMessageEnd(); - } - - public void getResult() throws org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - (new Client(prot)).recv_finishFileUpload(); - } - } - - public void beginFileDownload(String file, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - beginFileDownload_call method_call = new beginFileDownload_call(file, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class beginFileDownload_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String file; - public beginFileDownload_call(String file, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.file = file; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("beginFileDownload", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - beginFileDownload_args args = new beginFileDownload_args(); - args.set_file(file); - args.write(prot); - prot.writeMessageEnd(); - } - - public String getResult() throws org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_beginFileDownload(); - } - } - - public void downloadChunk(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - downloadChunk_call method_call = new downloadChunk_call(id, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class downloadChunk_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String id; - public downloadChunk_call(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.id = id; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("downloadChunk", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - downloadChunk_args args = new downloadChunk_args(); - args.set_id(id); - args.write(prot); - prot.writeMessageEnd(); - } - - public ByteBuffer getResult() throws org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_downloadChunk(); - } - } - - public void getNimbusConf(org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - getNimbusConf_call method_call = new getNimbusConf_call(resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class getNimbusConf_call extends org.apache.thrift7.async.TAsyncMethodCall { - public getNimbusConf_call(org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("getNimbusConf", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - getNimbusConf_args args = new getNimbusConf_args(); - args.write(prot); - prot.writeMessageEnd(); - } - - public String getResult() throws org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_getNimbusConf(); - } - } - - public void getClusterInfo(org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - getClusterInfo_call method_call = new getClusterInfo_call(resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class getClusterInfo_call extends org.apache.thrift7.async.TAsyncMethodCall { - public getClusterInfo_call(org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("getClusterInfo", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - getClusterInfo_args args = new getClusterInfo_args(); - args.write(prot); - prot.writeMessageEnd(); - } - - public ClusterSummary getResult() throws org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_getClusterInfo(); - } - } - - public void getTopologyInfo(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - getTopologyInfo_call method_call = new getTopologyInfo_call(id, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class getTopologyInfo_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String id; - public getTopologyInfo_call(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.id = id; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("getTopologyInfo", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - getTopologyInfo_args args = new getTopologyInfo_args(); - args.set_id(id); - args.write(prot); - prot.writeMessageEnd(); - } - - public TopologyInfo getResult() throws NotAliveException, org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_getTopologyInfo(); - } - } - - public void getTopologyInfoByName(String topologyName, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - getTopologyInfoByName_call method_call = new getTopologyInfoByName_call(topologyName, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class getTopologyInfoByName_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String topologyName; - public getTopologyInfoByName_call(String topologyName, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.topologyName = topologyName; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("getTopologyInfoByName", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - getTopologyInfoByName_args args = new getTopologyInfoByName_args(); - args.set_topologyName(topologyName); - args.write(prot); - prot.writeMessageEnd(); - } - - public TopologyInfo getResult() throws NotAliveException, org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_getTopologyInfoByName(); - } - } - - public void getSupervisorWorkers(String host, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - getSupervisorWorkers_call method_call = new getSupervisorWorkers_call(host, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class getSupervisorWorkers_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String host; - public getSupervisorWorkers_call(String host, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.host = host; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("getSupervisorWorkers", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - getSupervisorWorkers_args args = new getSupervisorWorkers_args(); - args.set_host(host); - args.write(prot); - prot.writeMessageEnd(); - } - - public SupervisorWorkers getResult() throws NotAliveException, org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_getSupervisorWorkers(); - } - } - - public void getTopologyConf(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - getTopologyConf_call method_call = new getTopologyConf_call(id, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class getTopologyConf_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String id; - public getTopologyConf_call(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.id = id; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("getTopologyConf", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - getTopologyConf_args args = new getTopologyConf_args(); - args.set_id(id); - args.write(prot); - prot.writeMessageEnd(); - } - - public String getResult() throws NotAliveException, org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_getTopologyConf(); - } - } - - public void getTopology(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - getTopology_call method_call = new getTopology_call(id, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class getTopology_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String id; - public getTopology_call(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.id = id; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("getTopology", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - getTopology_args args = new getTopology_args(); - args.set_id(id); - args.write(prot); - prot.writeMessageEnd(); - } - - public StormTopology getResult() throws NotAliveException, org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_getTopology(); - } - } - - public void getUserTopology(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - getUserTopology_call method_call = new getUserTopology_call(id, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class getUserTopology_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String id; - public getUserTopology_call(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.id = id; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("getUserTopology", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - getUserTopology_args args = new getUserTopology_args(); - args.set_id(id); - args.write(prot); - prot.writeMessageEnd(); - } - - public StormTopology getResult() throws NotAliveException, org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_getUserTopology(); - } - } - - public void getTopologyMetric(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler) throws org.apache.thrift7.TException { - checkReady(); - getTopologyMetric_call method_call = new getTopologyMetric_call(id, resultHandler, this, ___protocolFactory, ___transport); - this.___currentMethod = method_call; - ___manager.call(method_call); - } - - public static class getTopologyMetric_call extends org.apache.thrift7.async.TAsyncMethodCall { - private String id; - public getTopologyMetric_call(String id, org.apache.thrift7.async.AsyncMethodCallback resultHandler, org.apache.thrift7.async.TAsyncClient client, org.apache.thrift7.protocol.TProtocolFactory protocolFactory, org.apache.thrift7.transport.TNonblockingTransport transport) throws org.apache.thrift7.TException { - super(client, protocolFactory, transport, resultHandler, false); - this.id = id; - } - - public void write_args(org.apache.thrift7.protocol.TProtocol prot) throws org.apache.thrift7.TException { - prot.writeMessageBegin(new org.apache.thrift7.protocol.TMessage("getTopologyMetric", org.apache.thrift7.protocol.TMessageType.CALL, 0)); - getTopologyMetric_args args = new getTopologyMetric_args(); - args.set_id(id); - args.write(prot); - prot.writeMessageEnd(); - } - - public TopologyMetricInfo getResult() throws NotAliveException, org.apache.thrift7.TException { - if (getState() != org.apache.thrift7.async.TAsyncMethodCall.State.RESPONSE_READ) { - throw new IllegalStateException("Method call not finished!"); - } - org.apache.thrift7.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift7.transport.TMemoryInputTransport(getFrameBuffer().array()); - org.apache.thrift7.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); - return (new Client(prot)).recv_getTopologyMetric(); - } - } - - } - - public static class Processor extends org.apache.thrift7.TBaseProcessor implements org.apache.thrift7.TProcessor { - private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName()); - public Processor(I iface) { - super(iface, getProcessMap(new HashMap>())); - } - - protected Processor(I iface, Map> processMap) { - super(iface, getProcessMap(processMap)); - } - - private static Map> getProcessMap(Map> processMap) { - processMap.put("submitTopology", new submitTopology()); - processMap.put("submitTopologyWithOpts", new submitTopologyWithOpts()); - processMap.put("killTopology", new killTopology()); - processMap.put("killTopologyWithOpts", new killTopologyWithOpts()); - processMap.put("activate", new activate()); - processMap.put("deactivate", new deactivate()); - processMap.put("rebalance", new rebalance()); - processMap.put("metricMonitor", new metricMonitor()); - processMap.put("restart", new restart()); - processMap.put("beginLibUpload", new beginLibUpload()); - processMap.put("beginFileUpload", new beginFileUpload()); - processMap.put("uploadChunk", new uploadChunk()); - processMap.put("finishFileUpload", new finishFileUpload()); - processMap.put("beginFileDownload", new beginFileDownload()); - processMap.put("downloadChunk", new downloadChunk()); - processMap.put("getNimbusConf", new getNimbusConf()); - processMap.put("getClusterInfo", new getClusterInfo()); - processMap.put("getTopologyInfo", new getTopologyInfo()); - processMap.put("getTopologyInfoByName", new getTopologyInfoByName()); - processMap.put("getSupervisorWorkers", new getSupervisorWorkers()); - processMap.put("getTopologyConf", new getTopologyConf()); - processMap.put("getTopology", new getTopology()); - processMap.put("getUserTopology", new getUserTopology()); - processMap.put("getTopologyMetric", new getTopologyMetric()); - return processMap; - } - - private static class submitTopology extends org.apache.thrift7.ProcessFunction { - public submitTopology() { - super("submitTopology"); - } - - protected submitTopology_args getEmptyArgsInstance() { - return new submitTopology_args(); - } - - protected submitTopology_result getResult(I iface, submitTopology_args args) throws org.apache.thrift7.TException { - submitTopology_result result = new submitTopology_result(); - try { - iface.submitTopology(args.name, args.uploadedJarLocation, args.jsonConf, args.topology); - } catch (AlreadyAliveException e) { - result.e = e; - } catch (InvalidTopologyException ite) { - result.ite = ite; - } catch (TopologyAssignException tae) { - result.tae = tae; - } - return result; - } - } - - private static class submitTopologyWithOpts extends org.apache.thrift7.ProcessFunction { - public submitTopologyWithOpts() { - super("submitTopologyWithOpts"); - } - - protected submitTopologyWithOpts_args getEmptyArgsInstance() { - return new submitTopologyWithOpts_args(); - } - - protected submitTopologyWithOpts_result getResult(I iface, submitTopologyWithOpts_args args) throws org.apache.thrift7.TException { - submitTopologyWithOpts_result result = new submitTopologyWithOpts_result(); - try { - iface.submitTopologyWithOpts(args.name, args.uploadedJarLocation, args.jsonConf, args.topology, args.options); - } catch (AlreadyAliveException e) { - result.e = e; - } catch (InvalidTopologyException ite) { - result.ite = ite; - } catch (TopologyAssignException tae) { - result.tae = tae; - } - return result; - } - } - - private static class killTopology extends org.apache.thrift7.ProcessFunction { - public killTopology() { - super("killTopology"); - } - - protected killTopology_args getEmptyArgsInstance() { - return new killTopology_args(); - } - - protected killTopology_result getResult(I iface, killTopology_args args) throws org.apache.thrift7.TException { - killTopology_result result = new killTopology_result(); - try { - iface.killTopology(args.name); - } catch (NotAliveException e) { - result.e = e; - } - return result; - } - } - - private static class killTopologyWithOpts extends org.apache.thrift7.ProcessFunction { - public killTopologyWithOpts() { - super("killTopologyWithOpts"); - } - - protected killTopologyWithOpts_args getEmptyArgsInstance() { - return new killTopologyWithOpts_args(); - } - - protected killTopologyWithOpts_result getResult(I iface, killTopologyWithOpts_args args) throws org.apache.thrift7.TException { - killTopologyWithOpts_result result = new killTopologyWithOpts_result(); - try { - iface.killTopologyWithOpts(args.name, args.options); - } catch (NotAliveException e) { - result.e = e; - } - return result; - } - } - - private static class activate extends org.apache.thrift7.ProcessFunction { - public activate() { - super("activate"); - } - - protected activate_args getEmptyArgsInstance() { - return new activate_args(); - } - - protected activate_result getResult(I iface, activate_args args) throws org.apache.thrift7.TException { - activate_result result = new activate_result(); - try { - iface.activate(args.name); - } catch (NotAliveException e) { - result.e = e; - } - return result; - } - } - - private static class deactivate extends org.apache.thrift7.ProcessFunction { - public deactivate() { - super("deactivate"); - } - - protected deactivate_args getEmptyArgsInstance() { - return new deactivate_args(); - } - - protected deactivate_result getResult(I iface, deactivate_args args) throws org.apache.thrift7.TException { - deactivate_result result = new deactivate_result(); - try { - iface.deactivate(args.name); - } catch (NotAliveException e) { - result.e = e; - } - return result; - } - } - - private static class rebalance extends org.apache.thrift7.ProcessFunction { - public rebalance() { - super("rebalance"); - } - - protected rebalance_args getEmptyArgsInstance() { - return new rebalance_args(); - } - - protected rebalance_result getResult(I iface, rebalance_args args) throws org.apache.thrift7.TException { - rebalance_result result = new rebalance_result(); - try { - iface.rebalance(args.name, args.options); - } catch (NotAliveException e) { - result.e = e; - } catch (InvalidTopologyException ite) { - result.ite = ite; - } - return result; - } - } - - private static class metricMonitor extends org.apache.thrift7.ProcessFunction { - public metricMonitor() { - super("metricMonitor"); - } - - protected metricMonitor_args getEmptyArgsInstance() { - return new metricMonitor_args(); - } - - protected metricMonitor_result getResult(I iface, metricMonitor_args args) throws org.apache.thrift7.TException { - metricMonitor_result result = new metricMonitor_result(); - try { - iface.metricMonitor(args.name, args.options); - } catch (NotAliveException e) { - result.e = e; - } - return result; - } - } - - private static class restart extends org.apache.thrift7.ProcessFunction { - public restart() { - super("restart"); - } - - protected restart_args getEmptyArgsInstance() { - return new restart_args(); - } - - protected restart_result getResult(I iface, restart_args args) throws org.apache.thrift7.TException { - restart_result result = new restart_result(); - try { - iface.restart(args.name, args.jsonConf); - } catch (NotAliveException e) { - result.e = e; - } catch (InvalidTopologyException ite) { - result.ite = ite; - } catch (TopologyAssignException tae) { - result.tae = tae; - } - return result; - } - } - - private static class beginLibUpload extends org.apache.thrift7.ProcessFunction { - public beginLibUpload() { - super("beginLibUpload"); - } - - protected beginLibUpload_args getEmptyArgsInstance() { - return new beginLibUpload_args(); - } - - protected beginLibUpload_result getResult(I iface, beginLibUpload_args args) throws org.apache.thrift7.TException { - beginLibUpload_result result = new beginLibUpload_result(); - iface.beginLibUpload(args.libName); - return result; - } - } - - private static class beginFileUpload extends org.apache.thrift7.ProcessFunction { - public beginFileUpload() { - super("beginFileUpload"); - } - - protected beginFileUpload_args getEmptyArgsInstance() { - return new beginFileUpload_args(); - } - - protected beginFileUpload_result getResult(I iface, beginFileUpload_args args) throws org.apache.thrift7.TException { - beginFileUpload_result result = new beginFileUpload_result(); - result.success = iface.beginFileUpload(); - return result; - } - } - - private static class uploadChunk extends org.apache.thrift7.ProcessFunction { - public uploadChunk() { - super("uploadChunk"); - } - - protected uploadChunk_args getEmptyArgsInstance() { - return new uploadChunk_args(); - } - - protected uploadChunk_result getResult(I iface, uploadChunk_args args) throws org.apache.thrift7.TException { - uploadChunk_result result = new uploadChunk_result(); - iface.uploadChunk(args.location, args.chunk); - return result; - } - } - - private static class finishFileUpload extends org.apache.thrift7.ProcessFunction { - public finishFileUpload() { - super("finishFileUpload"); - } - - protected finishFileUpload_args getEmptyArgsInstance() { - return new finishFileUpload_args(); - } - - protected finishFileUpload_result getResult(I iface, finishFileUpload_args args) throws org.apache.thrift7.TException { - finishFileUpload_result result = new finishFileUpload_result(); - iface.finishFileUpload(args.location); - return result; - } - } - - private static class beginFileDownload extends org.apache.thrift7.ProcessFunction { - public beginFileDownload() { - super("beginFileDownload"); - } - - protected beginFileDownload_args getEmptyArgsInstance() { - return new beginFileDownload_args(); - } - - protected beginFileDownload_result getResult(I iface, beginFileDownload_args args) throws org.apache.thrift7.TException { - beginFileDownload_result result = new beginFileDownload_result(); - result.success = iface.beginFileDownload(args.file); - return result; - } - } - - private static class downloadChunk extends org.apache.thrift7.ProcessFunction { - public downloadChunk() { - super("downloadChunk"); - } - - protected downloadChunk_args getEmptyArgsInstance() { - return new downloadChunk_args(); - } - - protected downloadChunk_result getResult(I iface, downloadChunk_args args) throws org.apache.thrift7.TException { - downloadChunk_result result = new downloadChunk_result(); - result.success = iface.downloadChunk(args.id); - return result; - } - } - - private static class getNimbusConf extends org.apache.thrift7.ProcessFunction { - public getNimbusConf() { - super("getNimbusConf"); - } - - protected getNimbusConf_args getEmptyArgsInstance() { - return new getNimbusConf_args(); - } - - protected getNimbusConf_result getResult(I iface, getNimbusConf_args args) throws org.apache.thrift7.TException { - getNimbusConf_result result = new getNimbusConf_result(); - result.success = iface.getNimbusConf(); - return result; - } - } - - private static class getClusterInfo extends org.apache.thrift7.ProcessFunction { - public getClusterInfo() { - super("getClusterInfo"); - } - - protected getClusterInfo_args getEmptyArgsInstance() { - return new getClusterInfo_args(); - } - - protected getClusterInfo_result getResult(I iface, getClusterInfo_args args) throws org.apache.thrift7.TException { - getClusterInfo_result result = new getClusterInfo_result(); - result.success = iface.getClusterInfo(); - return result; - } - } - - private static class getTopologyInfo extends org.apache.thrift7.ProcessFunction { - public getTopologyInfo() { - super("getTopologyInfo"); - } - - protected getTopologyInfo_args getEmptyArgsInstance() { - return new getTopologyInfo_args(); - } - - protected getTopologyInfo_result getResult(I iface, getTopologyInfo_args args) throws org.apache.thrift7.TException { - getTopologyInfo_result result = new getTopologyInfo_result(); - try { - result.success = iface.getTopologyInfo(args.id); - } catch (NotAliveException e) { - result.e = e; - } - return result; - } - } - - private static class getTopologyInfoByName extends org.apache.thrift7.ProcessFunction { - public getTopologyInfoByName() { - super("getTopologyInfoByName"); - } - - protected getTopologyInfoByName_args getEmptyArgsInstance() { - return new getTopologyInfoByName_args(); - } - - protected getTopologyInfoByName_result getResult(I iface, getTopologyInfoByName_args args) throws org.apache.thrift7.TException { - getTopologyInfoByName_result result = new getTopologyInfoByName_result(); - try { - result.success = iface.getTopologyInfoByName(args.topologyName); - } catch (NotAliveException e) { - result.e = e; - } - return result; - } - } - - private static class getSupervisorWorkers extends org.apache.thrift7.ProcessFunction { - public getSupervisorWorkers() { - super("getSupervisorWorkers"); - } - - protected getSupervisorWorkers_args getEmptyArgsInstance() { - return new getSupervisorWorkers_args(); - } - - protected getSupervisorWorkers_result getResult(I iface, getSupervisorWorkers_args args) throws org.apache.thrift7.TException { - getSupervisorWorkers_result result = new getSupervisorWorkers_result(); - try { - result.success = iface.getSupervisorWorkers(args.host); - } catch (NotAliveException e) { - result.e = e; - } - return result; - } - } - - private static class getTopologyConf extends org.apache.thrift7.ProcessFunction { - public getTopologyConf() { - super("getTopologyConf"); - } - - protected getTopologyConf_args getEmptyArgsInstance() { - return new getTopologyConf_args(); - } - - protected getTopologyConf_result getResult(I iface, getTopologyConf_args args) throws org.apache.thrift7.TException { - getTopologyConf_result result = new getTopologyConf_result(); - try { - result.success = iface.getTopologyConf(args.id); - } catch (NotAliveException e) { - result.e = e; - } - return result; - } - } - - private static class getTopology extends org.apache.thrift7.ProcessFunction { - public getTopology() { - super("getTopology"); - } - - protected getTopology_args getEmptyArgsInstance() { - return new getTopology_args(); - } - - protected getTopology_result getResult(I iface, getTopology_args args) throws org.apache.thrift7.TException { - getTopology_result result = new getTopology_result(); - try { - result.success = iface.getTopology(args.id); - } catch (NotAliveException e) { - result.e = e; - } - return result; - } - } - - private static class getUserTopology extends org.apache.thrift7.ProcessFunction { - public getUserTopology() { - super("getUserTopology"); - } - - protected getUserTopology_args getEmptyArgsInstance() { - return new getUserTopology_args(); - } - - protected getUserTopology_result getResult(I iface, getUserTopology_args args) throws org.apache.thrift7.TException { - getUserTopology_result result = new getUserTopology_result(); - try { - result.success = iface.getUserTopology(args.id); - } catch (NotAliveException e) { - result.e = e; - } - return result; - } - } - - private static class getTopologyMetric extends org.apache.thrift7.ProcessFunction { - public getTopologyMetric() { - super("getTopologyMetric"); - } - - protected getTopologyMetric_args getEmptyArgsInstance() { - return new getTopologyMetric_args(); - } - - protected getTopologyMetric_result getResult(I iface, getTopologyMetric_args args) throws org.apache.thrift7.TException { - getTopologyMetric_result result = new getTopologyMetric_result(); - try { - result.success = iface.getTopologyMetric(args.id); - } catch (NotAliveException e) { - result.e = e; - } - return result; - } - } - - } - - public static class submitTopology_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("submitTopology_args"); - - private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField UPLOADED_JAR_LOCATION_FIELD_DESC = new org.apache.thrift7.protocol.TField("uploadedJarLocation", org.apache.thrift7.protocol.TType.STRING, (short)2); - private static final org.apache.thrift7.protocol.TField JSON_CONF_FIELD_DESC = new org.apache.thrift7.protocol.TField("jsonConf", org.apache.thrift7.protocol.TType.STRING, (short)3); - private static final org.apache.thrift7.protocol.TField TOPOLOGY_FIELD_DESC = new org.apache.thrift7.protocol.TField("topology", org.apache.thrift7.protocol.TType.STRUCT, (short)4); - - private String name; // required - private String uploadedJarLocation; // required - private String jsonConf; // required - private StormTopology topology; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - NAME((short)1, "name"), - UPLOADED_JAR_LOCATION((short)2, "uploadedJarLocation"), - JSON_CONF((short)3, "jsonConf"), - TOPOLOGY((short)4, "topology"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // NAME - return NAME; - case 2: // UPLOADED_JAR_LOCATION - return UPLOADED_JAR_LOCATION; - case 3: // JSON_CONF - return JSON_CONF; - case 4: // TOPOLOGY - return TOPOLOGY; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.UPLOADED_JAR_LOCATION, new org.apache.thrift7.meta_data.FieldMetaData("uploadedJarLocation", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.JSON_CONF, new org.apache.thrift7.meta_data.FieldMetaData("jsonConf", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.TOPOLOGY, new org.apache.thrift7.meta_data.FieldMetaData("topology", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, StormTopology.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(submitTopology_args.class, metaDataMap); - } - - public submitTopology_args() { - } - - public submitTopology_args( - String name, - String uploadedJarLocation, - String jsonConf, - StormTopology topology) - { - this(); - this.name = name; - this.uploadedJarLocation = uploadedJarLocation; - this.jsonConf = jsonConf; - this.topology = topology; - } - - /** - * Performs a deep copy on other. - */ - public submitTopology_args(submitTopology_args other) { - if (other.is_set_name()) { - this.name = other.name; - } - if (other.is_set_uploadedJarLocation()) { - this.uploadedJarLocation = other.uploadedJarLocation; - } - if (other.is_set_jsonConf()) { - this.jsonConf = other.jsonConf; - } - if (other.is_set_topology()) { - this.topology = new StormTopology(other.topology); - } - } - - public submitTopology_args deepCopy() { - return new submitTopology_args(this); - } - - @Override - public void clear() { - this.name = null; - this.uploadedJarLocation = null; - this.jsonConf = null; - this.topology = null; - } - - public String get_name() { - return this.name; - } - - public void set_name(String name) { - this.name = name; - } - - public void unset_name() { - this.name = null; - } - - /** Returns true if field name is set (has been assigned a value) and false otherwise */ - public boolean is_set_name() { - return this.name != null; - } - - public void set_name_isSet(boolean value) { - if (!value) { - this.name = null; - } - } - - public String get_uploadedJarLocation() { - return this.uploadedJarLocation; - } - - public void set_uploadedJarLocation(String uploadedJarLocation) { - this.uploadedJarLocation = uploadedJarLocation; - } - - public void unset_uploadedJarLocation() { - this.uploadedJarLocation = null; - } - - /** Returns true if field uploadedJarLocation is set (has been assigned a value) and false otherwise */ - public boolean is_set_uploadedJarLocation() { - return this.uploadedJarLocation != null; - } - - public void set_uploadedJarLocation_isSet(boolean value) { - if (!value) { - this.uploadedJarLocation = null; - } - } - - public String get_jsonConf() { - return this.jsonConf; - } - - public void set_jsonConf(String jsonConf) { - this.jsonConf = jsonConf; - } - - public void unset_jsonConf() { - this.jsonConf = null; - } - - /** Returns true if field jsonConf is set (has been assigned a value) and false otherwise */ - public boolean is_set_jsonConf() { - return this.jsonConf != null; - } - - public void set_jsonConf_isSet(boolean value) { - if (!value) { - this.jsonConf = null; - } - } - - public StormTopology get_topology() { - return this.topology; - } - - public void set_topology(StormTopology topology) { - this.topology = topology; - } - - public void unset_topology() { - this.topology = null; - } - - /** Returns true if field topology is set (has been assigned a value) and false otherwise */ - public boolean is_set_topology() { - return this.topology != null; - } - - public void set_topology_isSet(boolean value) { - if (!value) { - this.topology = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case NAME: - if (value == null) { - unset_name(); - } else { - set_name((String)value); - } - break; - - case UPLOADED_JAR_LOCATION: - if (value == null) { - unset_uploadedJarLocation(); - } else { - set_uploadedJarLocation((String)value); - } - break; - - case JSON_CONF: - if (value == null) { - unset_jsonConf(); - } else { - set_jsonConf((String)value); - } - break; - - case TOPOLOGY: - if (value == null) { - unset_topology(); - } else { - set_topology((StormTopology)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case NAME: - return get_name(); - - case UPLOADED_JAR_LOCATION: - return get_uploadedJarLocation(); - - case JSON_CONF: - return get_jsonConf(); - - case TOPOLOGY: - return get_topology(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case NAME: - return is_set_name(); - case UPLOADED_JAR_LOCATION: - return is_set_uploadedJarLocation(); - case JSON_CONF: - return is_set_jsonConf(); - case TOPOLOGY: - return is_set_topology(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof submitTopology_args) - return this.equals((submitTopology_args)that); - return false; - } - - public boolean equals(submitTopology_args that) { - if (that == null) - return false; - - boolean this_present_name = true && this.is_set_name(); - boolean that_present_name = true && that.is_set_name(); - if (this_present_name || that_present_name) { - if (!(this_present_name && that_present_name)) - return false; - if (!this.name.equals(that.name)) - return false; - } - - boolean this_present_uploadedJarLocation = true && this.is_set_uploadedJarLocation(); - boolean that_present_uploadedJarLocation = true && that.is_set_uploadedJarLocation(); - if (this_present_uploadedJarLocation || that_present_uploadedJarLocation) { - if (!(this_present_uploadedJarLocation && that_present_uploadedJarLocation)) - return false; - if (!this.uploadedJarLocation.equals(that.uploadedJarLocation)) - return false; - } - - boolean this_present_jsonConf = true && this.is_set_jsonConf(); - boolean that_present_jsonConf = true && that.is_set_jsonConf(); - if (this_present_jsonConf || that_present_jsonConf) { - if (!(this_present_jsonConf && that_present_jsonConf)) - return false; - if (!this.jsonConf.equals(that.jsonConf)) - return false; - } - - boolean this_present_topology = true && this.is_set_topology(); - boolean that_present_topology = true && that.is_set_topology(); - if (this_present_topology || that_present_topology) { - if (!(this_present_topology && that_present_topology)) - return false; - if (!this.topology.equals(that.topology)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_name = true && (is_set_name()); - builder.append(present_name); - if (present_name) - builder.append(name); - - boolean present_uploadedJarLocation = true && (is_set_uploadedJarLocation()); - builder.append(present_uploadedJarLocation); - if (present_uploadedJarLocation) - builder.append(uploadedJarLocation); - - boolean present_jsonConf = true && (is_set_jsonConf()); - builder.append(present_jsonConf); - if (present_jsonConf) - builder.append(jsonConf); - - boolean present_topology = true && (is_set_topology()); - builder.append(present_topology); - if (present_topology) - builder.append(topology); - - return builder.toHashCode(); - } - - public int compareTo(submitTopology_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - submitTopology_args typedOther = (submitTopology_args)other; - - lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_name()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_uploadedJarLocation()).compareTo(typedOther.is_set_uploadedJarLocation()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_uploadedJarLocation()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.uploadedJarLocation, typedOther.uploadedJarLocation); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_jsonConf()).compareTo(typedOther.is_set_jsonConf()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_jsonConf()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.jsonConf, typedOther.jsonConf); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_topology()).compareTo(typedOther.is_set_topology()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_topology()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.topology, typedOther.topology); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // NAME - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.name = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // UPLOADED_JAR_LOCATION - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.uploadedJarLocation = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 3: // JSON_CONF - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.jsonConf = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 4: // TOPOLOGY - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.topology = new StormTopology(); - this.topology.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.name != null) { - oprot.writeFieldBegin(NAME_FIELD_DESC); - oprot.writeString(this.name); - oprot.writeFieldEnd(); - } - if (this.uploadedJarLocation != null) { - oprot.writeFieldBegin(UPLOADED_JAR_LOCATION_FIELD_DESC); - oprot.writeString(this.uploadedJarLocation); - oprot.writeFieldEnd(); - } - if (this.jsonConf != null) { - oprot.writeFieldBegin(JSON_CONF_FIELD_DESC); - oprot.writeString(this.jsonConf); - oprot.writeFieldEnd(); - } - if (this.topology != null) { - oprot.writeFieldBegin(TOPOLOGY_FIELD_DESC); - this.topology.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("submitTopology_args("); - boolean first = true; - - sb.append("name:"); - if (this.name == null) { - sb.append("null"); - } else { - sb.append(this.name); - } - first = false; - if (!first) sb.append(", "); - sb.append("uploadedJarLocation:"); - if (this.uploadedJarLocation == null) { - sb.append("null"); - } else { - sb.append(this.uploadedJarLocation); - } - first = false; - if (!first) sb.append(", "); - sb.append("jsonConf:"); - if (this.jsonConf == null) { - sb.append("null"); - } else { - sb.append(this.jsonConf); - } - first = false; - if (!first) sb.append(", "); - sb.append("topology:"); - if (this.topology == null) { - sb.append("null"); - } else { - sb.append(this.topology); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class submitTopology_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("submitTopology_result"); - - private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift7.protocol.TField ITE_FIELD_DESC = new org.apache.thrift7.protocol.TField("ite", org.apache.thrift7.protocol.TType.STRUCT, (short)2); - private static final org.apache.thrift7.protocol.TField TAE_FIELD_DESC = new org.apache.thrift7.protocol.TField("tae", org.apache.thrift7.protocol.TType.STRUCT, (short)3); - - private AlreadyAliveException e; // required - private InvalidTopologyException ite; // required - private TopologyAssignException tae; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - E((short)1, "e"), - ITE((short)2, "ite"), - TAE((short)3, "tae"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // E - return E; - case 2: // ITE - return ITE; - case 3: // TAE - return TAE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - tmpMap.put(_Fields.ITE, new org.apache.thrift7.meta_data.FieldMetaData("ite", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - tmpMap.put(_Fields.TAE, new org.apache.thrift7.meta_data.FieldMetaData("tae", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(submitTopology_result.class, metaDataMap); - } - - public submitTopology_result() { - } - - public submitTopology_result( - AlreadyAliveException e, - InvalidTopologyException ite, - TopologyAssignException tae) - { - this(); - this.e = e; - this.ite = ite; - this.tae = tae; - } - - /** - * Performs a deep copy on other. - */ - public submitTopology_result(submitTopology_result other) { - if (other.is_set_e()) { - this.e = new AlreadyAliveException(other.e); - } - if (other.is_set_ite()) { - this.ite = new InvalidTopologyException(other.ite); - } - if (other.is_set_tae()) { - this.tae = new TopologyAssignException(other.tae); - } - } - - public submitTopology_result deepCopy() { - return new submitTopology_result(this); - } - - @Override - public void clear() { - this.e = null; - this.ite = null; - this.tae = null; - } - - public AlreadyAliveException get_e() { - return this.e; - } - - public void set_e(AlreadyAliveException e) { - this.e = e; - } - - public void unset_e() { - this.e = null; - } - - /** Returns true if field e is set (has been assigned a value) and false otherwise */ - public boolean is_set_e() { - return this.e != null; - } - - public void set_e_isSet(boolean value) { - if (!value) { - this.e = null; - } - } - - public InvalidTopologyException get_ite() { - return this.ite; - } - - public void set_ite(InvalidTopologyException ite) { - this.ite = ite; - } - - public void unset_ite() { - this.ite = null; - } - - /** Returns true if field ite is set (has been assigned a value) and false otherwise */ - public boolean is_set_ite() { - return this.ite != null; - } - - public void set_ite_isSet(boolean value) { - if (!value) { - this.ite = null; - } - } - - public TopologyAssignException get_tae() { - return this.tae; - } - - public void set_tae(TopologyAssignException tae) { - this.tae = tae; - } - - public void unset_tae() { - this.tae = null; - } - - /** Returns true if field tae is set (has been assigned a value) and false otherwise */ - public boolean is_set_tae() { - return this.tae != null; - } - - public void set_tae_isSet(boolean value) { - if (!value) { - this.tae = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case E: - if (value == null) { - unset_e(); - } else { - set_e((AlreadyAliveException)value); - } - break; - - case ITE: - if (value == null) { - unset_ite(); - } else { - set_ite((InvalidTopologyException)value); - } - break; - - case TAE: - if (value == null) { - unset_tae(); - } else { - set_tae((TopologyAssignException)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case E: - return get_e(); - - case ITE: - return get_ite(); - - case TAE: - return get_tae(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case E: - return is_set_e(); - case ITE: - return is_set_ite(); - case TAE: - return is_set_tae(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof submitTopology_result) - return this.equals((submitTopology_result)that); - return false; - } - - public boolean equals(submitTopology_result that) { - if (that == null) - return false; - - boolean this_present_e = true && this.is_set_e(); - boolean that_present_e = true && that.is_set_e(); - if (this_present_e || that_present_e) { - if (!(this_present_e && that_present_e)) - return false; - if (!this.e.equals(that.e)) - return false; - } - - boolean this_present_ite = true && this.is_set_ite(); - boolean that_present_ite = true && that.is_set_ite(); - if (this_present_ite || that_present_ite) { - if (!(this_present_ite && that_present_ite)) - return false; - if (!this.ite.equals(that.ite)) - return false; - } - - boolean this_present_tae = true && this.is_set_tae(); - boolean that_present_tae = true && that.is_set_tae(); - if (this_present_tae || that_present_tae) { - if (!(this_present_tae && that_present_tae)) - return false; - if (!this.tae.equals(that.tae)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_e = true && (is_set_e()); - builder.append(present_e); - if (present_e) - builder.append(e); - - boolean present_ite = true && (is_set_ite()); - builder.append(present_ite); - if (present_ite) - builder.append(ite); - - boolean present_tae = true && (is_set_tae()); - builder.append(present_tae); - if (present_tae) - builder.append(tae); - - return builder.toHashCode(); - } - - public int compareTo(submitTopology_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - submitTopology_result typedOther = (submitTopology_result)other; - - lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_e()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_ite()).compareTo(typedOther.is_set_ite()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_ite()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.ite, typedOther.ite); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_tae()).compareTo(typedOther.is_set_tae()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_tae()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.tae, typedOther.tae); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // E - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.e = new AlreadyAliveException(); - this.e.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // ITE - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.ite = new InvalidTopologyException(); - this.ite.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 3: // TAE - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.tae = new TopologyAssignException(); - this.tae.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_e()) { - oprot.writeFieldBegin(E_FIELD_DESC); - this.e.write(oprot); - oprot.writeFieldEnd(); - } else if (this.is_set_ite()) { - oprot.writeFieldBegin(ITE_FIELD_DESC); - this.ite.write(oprot); - oprot.writeFieldEnd(); - } else if (this.is_set_tae()) { - oprot.writeFieldBegin(TAE_FIELD_DESC); - this.tae.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("submitTopology_result("); - boolean first = true; - - sb.append("e:"); - if (this.e == null) { - sb.append("null"); - } else { - sb.append(this.e); - } - first = false; - if (!first) sb.append(", "); - sb.append("ite:"); - if (this.ite == null) { - sb.append("null"); - } else { - sb.append(this.ite); - } - first = false; - if (!first) sb.append(", "); - sb.append("tae:"); - if (this.tae == null) { - sb.append("null"); - } else { - sb.append(this.tae); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class submitTopologyWithOpts_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("submitTopologyWithOpts_args"); - - private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField UPLOADED_JAR_LOCATION_FIELD_DESC = new org.apache.thrift7.protocol.TField("uploadedJarLocation", org.apache.thrift7.protocol.TType.STRING, (short)2); - private static final org.apache.thrift7.protocol.TField JSON_CONF_FIELD_DESC = new org.apache.thrift7.protocol.TField("jsonConf", org.apache.thrift7.protocol.TType.STRING, (short)3); - private static final org.apache.thrift7.protocol.TField TOPOLOGY_FIELD_DESC = new org.apache.thrift7.protocol.TField("topology", org.apache.thrift7.protocol.TType.STRUCT, (short)4); - private static final org.apache.thrift7.protocol.TField OPTIONS_FIELD_DESC = new org.apache.thrift7.protocol.TField("options", org.apache.thrift7.protocol.TType.STRUCT, (short)5); - - private String name; // required - private String uploadedJarLocation; // required - private String jsonConf; // required - private StormTopology topology; // required - private SubmitOptions options; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - NAME((short)1, "name"), - UPLOADED_JAR_LOCATION((short)2, "uploadedJarLocation"), - JSON_CONF((short)3, "jsonConf"), - TOPOLOGY((short)4, "topology"), - OPTIONS((short)5, "options"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // NAME - return NAME; - case 2: // UPLOADED_JAR_LOCATION - return UPLOADED_JAR_LOCATION; - case 3: // JSON_CONF - return JSON_CONF; - case 4: // TOPOLOGY - return TOPOLOGY; - case 5: // OPTIONS - return OPTIONS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.UPLOADED_JAR_LOCATION, new org.apache.thrift7.meta_data.FieldMetaData("uploadedJarLocation", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.JSON_CONF, new org.apache.thrift7.meta_data.FieldMetaData("jsonConf", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.TOPOLOGY, new org.apache.thrift7.meta_data.FieldMetaData("topology", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, StormTopology.class))); - tmpMap.put(_Fields.OPTIONS, new org.apache.thrift7.meta_data.FieldMetaData("options", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, SubmitOptions.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(submitTopologyWithOpts_args.class, metaDataMap); - } - - public submitTopologyWithOpts_args() { - } - - public submitTopologyWithOpts_args( - String name, - String uploadedJarLocation, - String jsonConf, - StormTopology topology, - SubmitOptions options) - { - this(); - this.name = name; - this.uploadedJarLocation = uploadedJarLocation; - this.jsonConf = jsonConf; - this.topology = topology; - this.options = options; - } - - /** - * Performs a deep copy on other. - */ - public submitTopologyWithOpts_args(submitTopologyWithOpts_args other) { - if (other.is_set_name()) { - this.name = other.name; - } - if (other.is_set_uploadedJarLocation()) { - this.uploadedJarLocation = other.uploadedJarLocation; - } - if (other.is_set_jsonConf()) { - this.jsonConf = other.jsonConf; - } - if (other.is_set_topology()) { - this.topology = new StormTopology(other.topology); - } - if (other.is_set_options()) { - this.options = new SubmitOptions(other.options); - } - } - - public submitTopologyWithOpts_args deepCopy() { - return new submitTopologyWithOpts_args(this); - } - - @Override - public void clear() { - this.name = null; - this.uploadedJarLocation = null; - this.jsonConf = null; - this.topology = null; - this.options = null; - } - - public String get_name() { - return this.name; - } - - public void set_name(String name) { - this.name = name; - } - - public void unset_name() { - this.name = null; - } - - /** Returns true if field name is set (has been assigned a value) and false otherwise */ - public boolean is_set_name() { - return this.name != null; - } - - public void set_name_isSet(boolean value) { - if (!value) { - this.name = null; - } - } - - public String get_uploadedJarLocation() { - return this.uploadedJarLocation; - } - - public void set_uploadedJarLocation(String uploadedJarLocation) { - this.uploadedJarLocation = uploadedJarLocation; - } - - public void unset_uploadedJarLocation() { - this.uploadedJarLocation = null; - } - - /** Returns true if field uploadedJarLocation is set (has been assigned a value) and false otherwise */ - public boolean is_set_uploadedJarLocation() { - return this.uploadedJarLocation != null; - } - - public void set_uploadedJarLocation_isSet(boolean value) { - if (!value) { - this.uploadedJarLocation = null; - } - } - - public String get_jsonConf() { - return this.jsonConf; - } - - public void set_jsonConf(String jsonConf) { - this.jsonConf = jsonConf; - } - - public void unset_jsonConf() { - this.jsonConf = null; - } - - /** Returns true if field jsonConf is set (has been assigned a value) and false otherwise */ - public boolean is_set_jsonConf() { - return this.jsonConf != null; - } - - public void set_jsonConf_isSet(boolean value) { - if (!value) { - this.jsonConf = null; - } - } - - public StormTopology get_topology() { - return this.topology; - } - - public void set_topology(StormTopology topology) { - this.topology = topology; - } - - public void unset_topology() { - this.topology = null; - } - - /** Returns true if field topology is set (has been assigned a value) and false otherwise */ - public boolean is_set_topology() { - return this.topology != null; - } - - public void set_topology_isSet(boolean value) { - if (!value) { - this.topology = null; - } - } - - public SubmitOptions get_options() { - return this.options; - } - - public void set_options(SubmitOptions options) { - this.options = options; - } - - public void unset_options() { - this.options = null; - } - - /** Returns true if field options is set (has been assigned a value) and false otherwise */ - public boolean is_set_options() { - return this.options != null; - } - - public void set_options_isSet(boolean value) { - if (!value) { - this.options = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case NAME: - if (value == null) { - unset_name(); - } else { - set_name((String)value); - } - break; - - case UPLOADED_JAR_LOCATION: - if (value == null) { - unset_uploadedJarLocation(); - } else { - set_uploadedJarLocation((String)value); - } - break; - - case JSON_CONF: - if (value == null) { - unset_jsonConf(); - } else { - set_jsonConf((String)value); - } - break; - - case TOPOLOGY: - if (value == null) { - unset_topology(); - } else { - set_topology((StormTopology)value); - } - break; - - case OPTIONS: - if (value == null) { - unset_options(); - } else { - set_options((SubmitOptions)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case NAME: - return get_name(); - - case UPLOADED_JAR_LOCATION: - return get_uploadedJarLocation(); - - case JSON_CONF: - return get_jsonConf(); - - case TOPOLOGY: - return get_topology(); - - case OPTIONS: - return get_options(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case NAME: - return is_set_name(); - case UPLOADED_JAR_LOCATION: - return is_set_uploadedJarLocation(); - case JSON_CONF: - return is_set_jsonConf(); - case TOPOLOGY: - return is_set_topology(); - case OPTIONS: - return is_set_options(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof submitTopologyWithOpts_args) - return this.equals((submitTopologyWithOpts_args)that); - return false; - } - - public boolean equals(submitTopologyWithOpts_args that) { - if (that == null) - return false; - - boolean this_present_name = true && this.is_set_name(); - boolean that_present_name = true && that.is_set_name(); - if (this_present_name || that_present_name) { - if (!(this_present_name && that_present_name)) - return false; - if (!this.name.equals(that.name)) - return false; - } - - boolean this_present_uploadedJarLocation = true && this.is_set_uploadedJarLocation(); - boolean that_present_uploadedJarLocation = true && that.is_set_uploadedJarLocation(); - if (this_present_uploadedJarLocation || that_present_uploadedJarLocation) { - if (!(this_present_uploadedJarLocation && that_present_uploadedJarLocation)) - return false; - if (!this.uploadedJarLocation.equals(that.uploadedJarLocation)) - return false; - } - - boolean this_present_jsonConf = true && this.is_set_jsonConf(); - boolean that_present_jsonConf = true && that.is_set_jsonConf(); - if (this_present_jsonConf || that_present_jsonConf) { - if (!(this_present_jsonConf && that_present_jsonConf)) - return false; - if (!this.jsonConf.equals(that.jsonConf)) - return false; - } - - boolean this_present_topology = true && this.is_set_topology(); - boolean that_present_topology = true && that.is_set_topology(); - if (this_present_topology || that_present_topology) { - if (!(this_present_topology && that_present_topology)) - return false; - if (!this.topology.equals(that.topology)) - return false; - } - - boolean this_present_options = true && this.is_set_options(); - boolean that_present_options = true && that.is_set_options(); - if (this_present_options || that_present_options) { - if (!(this_present_options && that_present_options)) - return false; - if (!this.options.equals(that.options)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_name = true && (is_set_name()); - builder.append(present_name); - if (present_name) - builder.append(name); - - boolean present_uploadedJarLocation = true && (is_set_uploadedJarLocation()); - builder.append(present_uploadedJarLocation); - if (present_uploadedJarLocation) - builder.append(uploadedJarLocation); - - boolean present_jsonConf = true && (is_set_jsonConf()); - builder.append(present_jsonConf); - if (present_jsonConf) - builder.append(jsonConf); - - boolean present_topology = true && (is_set_topology()); - builder.append(present_topology); - if (present_topology) - builder.append(topology); - - boolean present_options = true && (is_set_options()); - builder.append(present_options); - if (present_options) - builder.append(options); - - return builder.toHashCode(); - } - - public int compareTo(submitTopologyWithOpts_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - submitTopologyWithOpts_args typedOther = (submitTopologyWithOpts_args)other; - - lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_name()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_uploadedJarLocation()).compareTo(typedOther.is_set_uploadedJarLocation()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_uploadedJarLocation()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.uploadedJarLocation, typedOther.uploadedJarLocation); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_jsonConf()).compareTo(typedOther.is_set_jsonConf()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_jsonConf()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.jsonConf, typedOther.jsonConf); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_topology()).compareTo(typedOther.is_set_topology()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_topology()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.topology, typedOther.topology); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_options()).compareTo(typedOther.is_set_options()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_options()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.options, typedOther.options); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // NAME - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.name = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // UPLOADED_JAR_LOCATION - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.uploadedJarLocation = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 3: // JSON_CONF - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.jsonConf = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 4: // TOPOLOGY - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.topology = new StormTopology(); - this.topology.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 5: // OPTIONS - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.options = new SubmitOptions(); - this.options.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.name != null) { - oprot.writeFieldBegin(NAME_FIELD_DESC); - oprot.writeString(this.name); - oprot.writeFieldEnd(); - } - if (this.uploadedJarLocation != null) { - oprot.writeFieldBegin(UPLOADED_JAR_LOCATION_FIELD_DESC); - oprot.writeString(this.uploadedJarLocation); - oprot.writeFieldEnd(); - } - if (this.jsonConf != null) { - oprot.writeFieldBegin(JSON_CONF_FIELD_DESC); - oprot.writeString(this.jsonConf); - oprot.writeFieldEnd(); - } - if (this.topology != null) { - oprot.writeFieldBegin(TOPOLOGY_FIELD_DESC); - this.topology.write(oprot); - oprot.writeFieldEnd(); - } - if (this.options != null) { - oprot.writeFieldBegin(OPTIONS_FIELD_DESC); - this.options.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("submitTopologyWithOpts_args("); - boolean first = true; - - sb.append("name:"); - if (this.name == null) { - sb.append("null"); - } else { - sb.append(this.name); - } - first = false; - if (!first) sb.append(", "); - sb.append("uploadedJarLocation:"); - if (this.uploadedJarLocation == null) { - sb.append("null"); - } else { - sb.append(this.uploadedJarLocation); - } - first = false; - if (!first) sb.append(", "); - sb.append("jsonConf:"); - if (this.jsonConf == null) { - sb.append("null"); - } else { - sb.append(this.jsonConf); - } - first = false; - if (!first) sb.append(", "); - sb.append("topology:"); - if (this.topology == null) { - sb.append("null"); - } else { - sb.append(this.topology); - } - first = false; - if (!first) sb.append(", "); - sb.append("options:"); - if (this.options == null) { - sb.append("null"); - } else { - sb.append(this.options); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class submitTopologyWithOpts_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("submitTopologyWithOpts_result"); - - private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift7.protocol.TField ITE_FIELD_DESC = new org.apache.thrift7.protocol.TField("ite", org.apache.thrift7.protocol.TType.STRUCT, (short)2); - private static final org.apache.thrift7.protocol.TField TAE_FIELD_DESC = new org.apache.thrift7.protocol.TField("tae", org.apache.thrift7.protocol.TType.STRUCT, (short)3); - - private AlreadyAliveException e; // required - private InvalidTopologyException ite; // required - private TopologyAssignException tae; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - E((short)1, "e"), - ITE((short)2, "ite"), - TAE((short)3, "tae"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // E - return E; - case 2: // ITE - return ITE; - case 3: // TAE - return TAE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - tmpMap.put(_Fields.ITE, new org.apache.thrift7.meta_data.FieldMetaData("ite", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - tmpMap.put(_Fields.TAE, new org.apache.thrift7.meta_data.FieldMetaData("tae", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(submitTopologyWithOpts_result.class, metaDataMap); - } - - public submitTopologyWithOpts_result() { - } - - public submitTopologyWithOpts_result( - AlreadyAliveException e, - InvalidTopologyException ite, - TopologyAssignException tae) - { - this(); - this.e = e; - this.ite = ite; - this.tae = tae; - } - - /** - * Performs a deep copy on other. - */ - public submitTopologyWithOpts_result(submitTopologyWithOpts_result other) { - if (other.is_set_e()) { - this.e = new AlreadyAliveException(other.e); - } - if (other.is_set_ite()) { - this.ite = new InvalidTopologyException(other.ite); - } - if (other.is_set_tae()) { - this.tae = new TopologyAssignException(other.tae); - } - } - - public submitTopologyWithOpts_result deepCopy() { - return new submitTopologyWithOpts_result(this); - } - - @Override - public void clear() { - this.e = null; - this.ite = null; - this.tae = null; - } - - public AlreadyAliveException get_e() { - return this.e; - } - - public void set_e(AlreadyAliveException e) { - this.e = e; - } - - public void unset_e() { - this.e = null; - } - - /** Returns true if field e is set (has been assigned a value) and false otherwise */ - public boolean is_set_e() { - return this.e != null; - } - - public void set_e_isSet(boolean value) { - if (!value) { - this.e = null; - } - } - - public InvalidTopologyException get_ite() { - return this.ite; - } - - public void set_ite(InvalidTopologyException ite) { - this.ite = ite; - } - - public void unset_ite() { - this.ite = null; - } - - /** Returns true if field ite is set (has been assigned a value) and false otherwise */ - public boolean is_set_ite() { - return this.ite != null; - } - - public void set_ite_isSet(boolean value) { - if (!value) { - this.ite = null; - } - } - - public TopologyAssignException get_tae() { - return this.tae; - } - - public void set_tae(TopologyAssignException tae) { - this.tae = tae; - } - - public void unset_tae() { - this.tae = null; - } - - /** Returns true if field tae is set (has been assigned a value) and false otherwise */ - public boolean is_set_tae() { - return this.tae != null; - } - - public void set_tae_isSet(boolean value) { - if (!value) { - this.tae = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case E: - if (value == null) { - unset_e(); - } else { - set_e((AlreadyAliveException)value); - } - break; - - case ITE: - if (value == null) { - unset_ite(); - } else { - set_ite((InvalidTopologyException)value); - } - break; - - case TAE: - if (value == null) { - unset_tae(); - } else { - set_tae((TopologyAssignException)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case E: - return get_e(); - - case ITE: - return get_ite(); - - case TAE: - return get_tae(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case E: - return is_set_e(); - case ITE: - return is_set_ite(); - case TAE: - return is_set_tae(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof submitTopologyWithOpts_result) - return this.equals((submitTopologyWithOpts_result)that); - return false; - } - - public boolean equals(submitTopologyWithOpts_result that) { - if (that == null) - return false; - - boolean this_present_e = true && this.is_set_e(); - boolean that_present_e = true && that.is_set_e(); - if (this_present_e || that_present_e) { - if (!(this_present_e && that_present_e)) - return false; - if (!this.e.equals(that.e)) - return false; - } - - boolean this_present_ite = true && this.is_set_ite(); - boolean that_present_ite = true && that.is_set_ite(); - if (this_present_ite || that_present_ite) { - if (!(this_present_ite && that_present_ite)) - return false; - if (!this.ite.equals(that.ite)) - return false; - } - - boolean this_present_tae = true && this.is_set_tae(); - boolean that_present_tae = true && that.is_set_tae(); - if (this_present_tae || that_present_tae) { - if (!(this_present_tae && that_present_tae)) - return false; - if (!this.tae.equals(that.tae)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_e = true && (is_set_e()); - builder.append(present_e); - if (present_e) - builder.append(e); - - boolean present_ite = true && (is_set_ite()); - builder.append(present_ite); - if (present_ite) - builder.append(ite); - - boolean present_tae = true && (is_set_tae()); - builder.append(present_tae); - if (present_tae) - builder.append(tae); - - return builder.toHashCode(); - } - - public int compareTo(submitTopologyWithOpts_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - submitTopologyWithOpts_result typedOther = (submitTopologyWithOpts_result)other; - - lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_e()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_ite()).compareTo(typedOther.is_set_ite()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_ite()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.ite, typedOther.ite); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_tae()).compareTo(typedOther.is_set_tae()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_tae()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.tae, typedOther.tae); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // E - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.e = new AlreadyAliveException(); - this.e.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // ITE - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.ite = new InvalidTopologyException(); - this.ite.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 3: // TAE - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.tae = new TopologyAssignException(); - this.tae.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_e()) { - oprot.writeFieldBegin(E_FIELD_DESC); - this.e.write(oprot); - oprot.writeFieldEnd(); - } else if (this.is_set_ite()) { - oprot.writeFieldBegin(ITE_FIELD_DESC); - this.ite.write(oprot); - oprot.writeFieldEnd(); - } else if (this.is_set_tae()) { - oprot.writeFieldBegin(TAE_FIELD_DESC); - this.tae.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("submitTopologyWithOpts_result("); - boolean first = true; - - sb.append("e:"); - if (this.e == null) { - sb.append("null"); - } else { - sb.append(this.e); - } - first = false; - if (!first) sb.append(", "); - sb.append("ite:"); - if (this.ite == null) { - sb.append("null"); - } else { - sb.append(this.ite); - } - first = false; - if (!first) sb.append(", "); - sb.append("tae:"); - if (this.tae == null) { - sb.append("null"); - } else { - sb.append(this.tae); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class killTopology_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("killTopology_args"); - - private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String name; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - NAME((short)1, "name"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // NAME - return NAME; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(killTopology_args.class, metaDataMap); - } - - public killTopology_args() { - } - - public killTopology_args( - String name) - { - this(); - this.name = name; - } - - /** - * Performs a deep copy on other. - */ - public killTopology_args(killTopology_args other) { - if (other.is_set_name()) { - this.name = other.name; - } - } - - public killTopology_args deepCopy() { - return new killTopology_args(this); - } - - @Override - public void clear() { - this.name = null; - } - - public String get_name() { - return this.name; - } - - public void set_name(String name) { - this.name = name; - } - - public void unset_name() { - this.name = null; - } - - /** Returns true if field name is set (has been assigned a value) and false otherwise */ - public boolean is_set_name() { - return this.name != null; - } - - public void set_name_isSet(boolean value) { - if (!value) { - this.name = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case NAME: - if (value == null) { - unset_name(); - } else { - set_name((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case NAME: - return get_name(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case NAME: - return is_set_name(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof killTopology_args) - return this.equals((killTopology_args)that); - return false; - } - - public boolean equals(killTopology_args that) { - if (that == null) - return false; - - boolean this_present_name = true && this.is_set_name(); - boolean that_present_name = true && that.is_set_name(); - if (this_present_name || that_present_name) { - if (!(this_present_name && that_present_name)) - return false; - if (!this.name.equals(that.name)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_name = true && (is_set_name()); - builder.append(present_name); - if (present_name) - builder.append(name); - - return builder.toHashCode(); - } - - public int compareTo(killTopology_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - killTopology_args typedOther = (killTopology_args)other; - - lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_name()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // NAME - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.name = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.name != null) { - oprot.writeFieldBegin(NAME_FIELD_DESC); - oprot.writeString(this.name); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("killTopology_args("); - boolean first = true; - - sb.append("name:"); - if (this.name == null) { - sb.append("null"); - } else { - sb.append(this.name); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class killTopology_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("killTopology_result"); - - private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - - private NotAliveException e; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - E((short)1, "e"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // E - return E; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(killTopology_result.class, metaDataMap); - } - - public killTopology_result() { - } - - public killTopology_result( - NotAliveException e) - { - this(); - this.e = e; - } - - /** - * Performs a deep copy on other. - */ - public killTopology_result(killTopology_result other) { - if (other.is_set_e()) { - this.e = new NotAliveException(other.e); - } - } - - public killTopology_result deepCopy() { - return new killTopology_result(this); - } - - @Override - public void clear() { - this.e = null; - } - - public NotAliveException get_e() { - return this.e; - } - - public void set_e(NotAliveException e) { - this.e = e; - } - - public void unset_e() { - this.e = null; - } - - /** Returns true if field e is set (has been assigned a value) and false otherwise */ - public boolean is_set_e() { - return this.e != null; - } - - public void set_e_isSet(boolean value) { - if (!value) { - this.e = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case E: - if (value == null) { - unset_e(); - } else { - set_e((NotAliveException)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case E: - return get_e(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case E: - return is_set_e(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof killTopology_result) - return this.equals((killTopology_result)that); - return false; - } - - public boolean equals(killTopology_result that) { - if (that == null) - return false; - - boolean this_present_e = true && this.is_set_e(); - boolean that_present_e = true && that.is_set_e(); - if (this_present_e || that_present_e) { - if (!(this_present_e && that_present_e)) - return false; - if (!this.e.equals(that.e)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_e = true && (is_set_e()); - builder.append(present_e); - if (present_e) - builder.append(e); - - return builder.toHashCode(); - } - - public int compareTo(killTopology_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - killTopology_result typedOther = (killTopology_result)other; - - lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_e()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // E - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.e = new NotAliveException(); - this.e.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_e()) { - oprot.writeFieldBegin(E_FIELD_DESC); - this.e.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("killTopology_result("); - boolean first = true; - - sb.append("e:"); - if (this.e == null) { - sb.append("null"); - } else { - sb.append(this.e); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class killTopologyWithOpts_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("killTopologyWithOpts_args"); - - private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField OPTIONS_FIELD_DESC = new org.apache.thrift7.protocol.TField("options", org.apache.thrift7.protocol.TType.STRUCT, (short)2); - - private String name; // required - private KillOptions options; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - NAME((short)1, "name"), - OPTIONS((short)2, "options"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // NAME - return NAME; - case 2: // OPTIONS - return OPTIONS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.OPTIONS, new org.apache.thrift7.meta_data.FieldMetaData("options", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, KillOptions.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(killTopologyWithOpts_args.class, metaDataMap); - } - - public killTopologyWithOpts_args() { - } - - public killTopologyWithOpts_args( - String name, - KillOptions options) - { - this(); - this.name = name; - this.options = options; - } - - /** - * Performs a deep copy on other. - */ - public killTopologyWithOpts_args(killTopologyWithOpts_args other) { - if (other.is_set_name()) { - this.name = other.name; - } - if (other.is_set_options()) { - this.options = new KillOptions(other.options); - } - } - - public killTopologyWithOpts_args deepCopy() { - return new killTopologyWithOpts_args(this); - } - - @Override - public void clear() { - this.name = null; - this.options = null; - } - - public String get_name() { - return this.name; - } - - public void set_name(String name) { - this.name = name; - } - - public void unset_name() { - this.name = null; - } - - /** Returns true if field name is set (has been assigned a value) and false otherwise */ - public boolean is_set_name() { - return this.name != null; - } - - public void set_name_isSet(boolean value) { - if (!value) { - this.name = null; - } - } - - public KillOptions get_options() { - return this.options; - } - - public void set_options(KillOptions options) { - this.options = options; - } - - public void unset_options() { - this.options = null; - } - - /** Returns true if field options is set (has been assigned a value) and false otherwise */ - public boolean is_set_options() { - return this.options != null; - } - - public void set_options_isSet(boolean value) { - if (!value) { - this.options = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case NAME: - if (value == null) { - unset_name(); - } else { - set_name((String)value); - } - break; - - case OPTIONS: - if (value == null) { - unset_options(); - } else { - set_options((KillOptions)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case NAME: - return get_name(); - - case OPTIONS: - return get_options(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case NAME: - return is_set_name(); - case OPTIONS: - return is_set_options(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof killTopologyWithOpts_args) - return this.equals((killTopologyWithOpts_args)that); - return false; - } - - public boolean equals(killTopologyWithOpts_args that) { - if (that == null) - return false; - - boolean this_present_name = true && this.is_set_name(); - boolean that_present_name = true && that.is_set_name(); - if (this_present_name || that_present_name) { - if (!(this_present_name && that_present_name)) - return false; - if (!this.name.equals(that.name)) - return false; - } - - boolean this_present_options = true && this.is_set_options(); - boolean that_present_options = true && that.is_set_options(); - if (this_present_options || that_present_options) { - if (!(this_present_options && that_present_options)) - return false; - if (!this.options.equals(that.options)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_name = true && (is_set_name()); - builder.append(present_name); - if (present_name) - builder.append(name); - - boolean present_options = true && (is_set_options()); - builder.append(present_options); - if (present_options) - builder.append(options); - - return builder.toHashCode(); - } - - public int compareTo(killTopologyWithOpts_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - killTopologyWithOpts_args typedOther = (killTopologyWithOpts_args)other; - - lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_name()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_options()).compareTo(typedOther.is_set_options()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_options()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.options, typedOther.options); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // NAME - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.name = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // OPTIONS - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.options = new KillOptions(); - this.options.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.name != null) { - oprot.writeFieldBegin(NAME_FIELD_DESC); - oprot.writeString(this.name); - oprot.writeFieldEnd(); - } - if (this.options != null) { - oprot.writeFieldBegin(OPTIONS_FIELD_DESC); - this.options.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("killTopologyWithOpts_args("); - boolean first = true; - - sb.append("name:"); - if (this.name == null) { - sb.append("null"); - } else { - sb.append(this.name); - } - first = false; - if (!first) sb.append(", "); - sb.append("options:"); - if (this.options == null) { - sb.append("null"); - } else { - sb.append(this.options); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class killTopologyWithOpts_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("killTopologyWithOpts_result"); - - private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - - private NotAliveException e; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - E((short)1, "e"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // E - return E; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(killTopologyWithOpts_result.class, metaDataMap); - } - - public killTopologyWithOpts_result() { - } - - public killTopologyWithOpts_result( - NotAliveException e) - { - this(); - this.e = e; - } - - /** - * Performs a deep copy on other. - */ - public killTopologyWithOpts_result(killTopologyWithOpts_result other) { - if (other.is_set_e()) { - this.e = new NotAliveException(other.e); - } - } - - public killTopologyWithOpts_result deepCopy() { - return new killTopologyWithOpts_result(this); - } - - @Override - public void clear() { - this.e = null; - } - - public NotAliveException get_e() { - return this.e; - } - - public void set_e(NotAliveException e) { - this.e = e; - } - - public void unset_e() { - this.e = null; - } - - /** Returns true if field e is set (has been assigned a value) and false otherwise */ - public boolean is_set_e() { - return this.e != null; - } - - public void set_e_isSet(boolean value) { - if (!value) { - this.e = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case E: - if (value == null) { - unset_e(); - } else { - set_e((NotAliveException)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case E: - return get_e(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case E: - return is_set_e(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof killTopologyWithOpts_result) - return this.equals((killTopologyWithOpts_result)that); - return false; - } - - public boolean equals(killTopologyWithOpts_result that) { - if (that == null) - return false; - - boolean this_present_e = true && this.is_set_e(); - boolean that_present_e = true && that.is_set_e(); - if (this_present_e || that_present_e) { - if (!(this_present_e && that_present_e)) - return false; - if (!this.e.equals(that.e)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_e = true && (is_set_e()); - builder.append(present_e); - if (present_e) - builder.append(e); - - return builder.toHashCode(); - } - - public int compareTo(killTopologyWithOpts_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - killTopologyWithOpts_result typedOther = (killTopologyWithOpts_result)other; - - lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_e()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // E - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.e = new NotAliveException(); - this.e.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_e()) { - oprot.writeFieldBegin(E_FIELD_DESC); - this.e.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("killTopologyWithOpts_result("); - boolean first = true; - - sb.append("e:"); - if (this.e == null) { - sb.append("null"); - } else { - sb.append(this.e); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class activate_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("activate_args"); - - private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String name; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - NAME((short)1, "name"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // NAME - return NAME; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(activate_args.class, metaDataMap); - } - - public activate_args() { - } - - public activate_args( - String name) - { - this(); - this.name = name; - } - - /** - * Performs a deep copy on other. - */ - public activate_args(activate_args other) { - if (other.is_set_name()) { - this.name = other.name; - } - } - - public activate_args deepCopy() { - return new activate_args(this); - } - - @Override - public void clear() { - this.name = null; - } - - public String get_name() { - return this.name; - } - - public void set_name(String name) { - this.name = name; - } - - public void unset_name() { - this.name = null; - } - - /** Returns true if field name is set (has been assigned a value) and false otherwise */ - public boolean is_set_name() { - return this.name != null; - } - - public void set_name_isSet(boolean value) { - if (!value) { - this.name = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case NAME: - if (value == null) { - unset_name(); - } else { - set_name((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case NAME: - return get_name(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case NAME: - return is_set_name(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof activate_args) - return this.equals((activate_args)that); - return false; - } - - public boolean equals(activate_args that) { - if (that == null) - return false; - - boolean this_present_name = true && this.is_set_name(); - boolean that_present_name = true && that.is_set_name(); - if (this_present_name || that_present_name) { - if (!(this_present_name && that_present_name)) - return false; - if (!this.name.equals(that.name)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_name = true && (is_set_name()); - builder.append(present_name); - if (present_name) - builder.append(name); - - return builder.toHashCode(); - } - - public int compareTo(activate_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - activate_args typedOther = (activate_args)other; - - lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_name()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // NAME - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.name = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.name != null) { - oprot.writeFieldBegin(NAME_FIELD_DESC); - oprot.writeString(this.name); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("activate_args("); - boolean first = true; - - sb.append("name:"); - if (this.name == null) { - sb.append("null"); - } else { - sb.append(this.name); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class activate_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("activate_result"); - - private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - - private NotAliveException e; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - E((short)1, "e"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // E - return E; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(activate_result.class, metaDataMap); - } - - public activate_result() { - } - - public activate_result( - NotAliveException e) - { - this(); - this.e = e; - } - - /** - * Performs a deep copy on other. - */ - public activate_result(activate_result other) { - if (other.is_set_e()) { - this.e = new NotAliveException(other.e); - } - } - - public activate_result deepCopy() { - return new activate_result(this); - } - - @Override - public void clear() { - this.e = null; - } - - public NotAliveException get_e() { - return this.e; - } - - public void set_e(NotAliveException e) { - this.e = e; - } - - public void unset_e() { - this.e = null; - } - - /** Returns true if field e is set (has been assigned a value) and false otherwise */ - public boolean is_set_e() { - return this.e != null; - } - - public void set_e_isSet(boolean value) { - if (!value) { - this.e = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case E: - if (value == null) { - unset_e(); - } else { - set_e((NotAliveException)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case E: - return get_e(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case E: - return is_set_e(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof activate_result) - return this.equals((activate_result)that); - return false; - } - - public boolean equals(activate_result that) { - if (that == null) - return false; - - boolean this_present_e = true && this.is_set_e(); - boolean that_present_e = true && that.is_set_e(); - if (this_present_e || that_present_e) { - if (!(this_present_e && that_present_e)) - return false; - if (!this.e.equals(that.e)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_e = true && (is_set_e()); - builder.append(present_e); - if (present_e) - builder.append(e); - - return builder.toHashCode(); - } - - public int compareTo(activate_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - activate_result typedOther = (activate_result)other; - - lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_e()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // E - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.e = new NotAliveException(); - this.e.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_e()) { - oprot.writeFieldBegin(E_FIELD_DESC); - this.e.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("activate_result("); - boolean first = true; - - sb.append("e:"); - if (this.e == null) { - sb.append("null"); - } else { - sb.append(this.e); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class deactivate_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("deactivate_args"); - - private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String name; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - NAME((short)1, "name"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // NAME - return NAME; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(deactivate_args.class, metaDataMap); - } - - public deactivate_args() { - } - - public deactivate_args( - String name) - { - this(); - this.name = name; - } - - /** - * Performs a deep copy on other. - */ - public deactivate_args(deactivate_args other) { - if (other.is_set_name()) { - this.name = other.name; - } - } - - public deactivate_args deepCopy() { - return new deactivate_args(this); - } - - @Override - public void clear() { - this.name = null; - } - - public String get_name() { - return this.name; - } - - public void set_name(String name) { - this.name = name; - } - - public void unset_name() { - this.name = null; - } - - /** Returns true if field name is set (has been assigned a value) and false otherwise */ - public boolean is_set_name() { - return this.name != null; - } - - public void set_name_isSet(boolean value) { - if (!value) { - this.name = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case NAME: - if (value == null) { - unset_name(); - } else { - set_name((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case NAME: - return get_name(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case NAME: - return is_set_name(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof deactivate_args) - return this.equals((deactivate_args)that); - return false; - } - - public boolean equals(deactivate_args that) { - if (that == null) - return false; - - boolean this_present_name = true && this.is_set_name(); - boolean that_present_name = true && that.is_set_name(); - if (this_present_name || that_present_name) { - if (!(this_present_name && that_present_name)) - return false; - if (!this.name.equals(that.name)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_name = true && (is_set_name()); - builder.append(present_name); - if (present_name) - builder.append(name); - - return builder.toHashCode(); - } - - public int compareTo(deactivate_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - deactivate_args typedOther = (deactivate_args)other; - - lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_name()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // NAME - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.name = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.name != null) { - oprot.writeFieldBegin(NAME_FIELD_DESC); - oprot.writeString(this.name); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("deactivate_args("); - boolean first = true; - - sb.append("name:"); - if (this.name == null) { - sb.append("null"); - } else { - sb.append(this.name); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class deactivate_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("deactivate_result"); - - private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - - private NotAliveException e; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - E((short)1, "e"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // E - return E; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(deactivate_result.class, metaDataMap); - } - - public deactivate_result() { - } - - public deactivate_result( - NotAliveException e) - { - this(); - this.e = e; - } - - /** - * Performs a deep copy on other. - */ - public deactivate_result(deactivate_result other) { - if (other.is_set_e()) { - this.e = new NotAliveException(other.e); - } - } - - public deactivate_result deepCopy() { - return new deactivate_result(this); - } - - @Override - public void clear() { - this.e = null; - } - - public NotAliveException get_e() { - return this.e; - } - - public void set_e(NotAliveException e) { - this.e = e; - } - - public void unset_e() { - this.e = null; - } - - /** Returns true if field e is set (has been assigned a value) and false otherwise */ - public boolean is_set_e() { - return this.e != null; - } - - public void set_e_isSet(boolean value) { - if (!value) { - this.e = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case E: - if (value == null) { - unset_e(); - } else { - set_e((NotAliveException)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case E: - return get_e(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case E: - return is_set_e(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof deactivate_result) - return this.equals((deactivate_result)that); - return false; - } - - public boolean equals(deactivate_result that) { - if (that == null) - return false; - - boolean this_present_e = true && this.is_set_e(); - boolean that_present_e = true && that.is_set_e(); - if (this_present_e || that_present_e) { - if (!(this_present_e && that_present_e)) - return false; - if (!this.e.equals(that.e)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_e = true && (is_set_e()); - builder.append(present_e); - if (present_e) - builder.append(e); - - return builder.toHashCode(); - } - - public int compareTo(deactivate_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - deactivate_result typedOther = (deactivate_result)other; - - lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_e()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // E - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.e = new NotAliveException(); - this.e.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_e()) { - oprot.writeFieldBegin(E_FIELD_DESC); - this.e.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("deactivate_result("); - boolean first = true; - - sb.append("e:"); - if (this.e == null) { - sb.append("null"); - } else { - sb.append(this.e); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class rebalance_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("rebalance_args"); - - private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField OPTIONS_FIELD_DESC = new org.apache.thrift7.protocol.TField("options", org.apache.thrift7.protocol.TType.STRUCT, (short)2); - - private String name; // required - private RebalanceOptions options; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - NAME((short)1, "name"), - OPTIONS((short)2, "options"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // NAME - return NAME; - case 2: // OPTIONS - return OPTIONS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.OPTIONS, new org.apache.thrift7.meta_data.FieldMetaData("options", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, RebalanceOptions.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(rebalance_args.class, metaDataMap); - } - - public rebalance_args() { - } - - public rebalance_args( - String name, - RebalanceOptions options) - { - this(); - this.name = name; - this.options = options; - } - - /** - * Performs a deep copy on other. - */ - public rebalance_args(rebalance_args other) { - if (other.is_set_name()) { - this.name = other.name; - } - if (other.is_set_options()) { - this.options = new RebalanceOptions(other.options); - } - } - - public rebalance_args deepCopy() { - return new rebalance_args(this); - } - - @Override - public void clear() { - this.name = null; - this.options = null; - } - - public String get_name() { - return this.name; - } - - public void set_name(String name) { - this.name = name; - } - - public void unset_name() { - this.name = null; - } - - /** Returns true if field name is set (has been assigned a value) and false otherwise */ - public boolean is_set_name() { - return this.name != null; - } - - public void set_name_isSet(boolean value) { - if (!value) { - this.name = null; - } - } - - public RebalanceOptions get_options() { - return this.options; - } - - public void set_options(RebalanceOptions options) { - this.options = options; - } - - public void unset_options() { - this.options = null; - } - - /** Returns true if field options is set (has been assigned a value) and false otherwise */ - public boolean is_set_options() { - return this.options != null; - } - - public void set_options_isSet(boolean value) { - if (!value) { - this.options = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case NAME: - if (value == null) { - unset_name(); - } else { - set_name((String)value); - } - break; - - case OPTIONS: - if (value == null) { - unset_options(); - } else { - set_options((RebalanceOptions)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case NAME: - return get_name(); - - case OPTIONS: - return get_options(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case NAME: - return is_set_name(); - case OPTIONS: - return is_set_options(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof rebalance_args) - return this.equals((rebalance_args)that); - return false; - } - - public boolean equals(rebalance_args that) { - if (that == null) - return false; - - boolean this_present_name = true && this.is_set_name(); - boolean that_present_name = true && that.is_set_name(); - if (this_present_name || that_present_name) { - if (!(this_present_name && that_present_name)) - return false; - if (!this.name.equals(that.name)) - return false; - } - - boolean this_present_options = true && this.is_set_options(); - boolean that_present_options = true && that.is_set_options(); - if (this_present_options || that_present_options) { - if (!(this_present_options && that_present_options)) - return false; - if (!this.options.equals(that.options)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_name = true && (is_set_name()); - builder.append(present_name); - if (present_name) - builder.append(name); - - boolean present_options = true && (is_set_options()); - builder.append(present_options); - if (present_options) - builder.append(options); - - return builder.toHashCode(); - } - - public int compareTo(rebalance_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - rebalance_args typedOther = (rebalance_args)other; - - lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_name()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_options()).compareTo(typedOther.is_set_options()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_options()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.options, typedOther.options); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // NAME - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.name = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // OPTIONS - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.options = new RebalanceOptions(); - this.options.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.name != null) { - oprot.writeFieldBegin(NAME_FIELD_DESC); - oprot.writeString(this.name); - oprot.writeFieldEnd(); - } - if (this.options != null) { - oprot.writeFieldBegin(OPTIONS_FIELD_DESC); - this.options.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("rebalance_args("); - boolean first = true; - - sb.append("name:"); - if (this.name == null) { - sb.append("null"); - } else { - sb.append(this.name); - } - first = false; - if (!first) sb.append(", "); - sb.append("options:"); - if (this.options == null) { - sb.append("null"); - } else { - sb.append(this.options); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class rebalance_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("rebalance_result"); - - private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift7.protocol.TField ITE_FIELD_DESC = new org.apache.thrift7.protocol.TField("ite", org.apache.thrift7.protocol.TType.STRUCT, (short)2); - - private NotAliveException e; // required - private InvalidTopologyException ite; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - E((short)1, "e"), - ITE((short)2, "ite"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // E - return E; - case 2: // ITE - return ITE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - tmpMap.put(_Fields.ITE, new org.apache.thrift7.meta_data.FieldMetaData("ite", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(rebalance_result.class, metaDataMap); - } - - public rebalance_result() { - } - - public rebalance_result( - NotAliveException e, - InvalidTopologyException ite) - { - this(); - this.e = e; - this.ite = ite; - } - - /** - * Performs a deep copy on other. - */ - public rebalance_result(rebalance_result other) { - if (other.is_set_e()) { - this.e = new NotAliveException(other.e); - } - if (other.is_set_ite()) { - this.ite = new InvalidTopologyException(other.ite); - } - } - - public rebalance_result deepCopy() { - return new rebalance_result(this); - } - - @Override - public void clear() { - this.e = null; - this.ite = null; - } - - public NotAliveException get_e() { - return this.e; - } - - public void set_e(NotAliveException e) { - this.e = e; - } - - public void unset_e() { - this.e = null; - } - - /** Returns true if field e is set (has been assigned a value) and false otherwise */ - public boolean is_set_e() { - return this.e != null; - } - - public void set_e_isSet(boolean value) { - if (!value) { - this.e = null; - } - } - - public InvalidTopologyException get_ite() { - return this.ite; - } - - public void set_ite(InvalidTopologyException ite) { - this.ite = ite; - } - - public void unset_ite() { - this.ite = null; - } - - /** Returns true if field ite is set (has been assigned a value) and false otherwise */ - public boolean is_set_ite() { - return this.ite != null; - } - - public void set_ite_isSet(boolean value) { - if (!value) { - this.ite = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case E: - if (value == null) { - unset_e(); - } else { - set_e((NotAliveException)value); - } - break; - - case ITE: - if (value == null) { - unset_ite(); - } else { - set_ite((InvalidTopologyException)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case E: - return get_e(); - - case ITE: - return get_ite(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case E: - return is_set_e(); - case ITE: - return is_set_ite(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof rebalance_result) - return this.equals((rebalance_result)that); - return false; - } - - public boolean equals(rebalance_result that) { - if (that == null) - return false; - - boolean this_present_e = true && this.is_set_e(); - boolean that_present_e = true && that.is_set_e(); - if (this_present_e || that_present_e) { - if (!(this_present_e && that_present_e)) - return false; - if (!this.e.equals(that.e)) - return false; - } - - boolean this_present_ite = true && this.is_set_ite(); - boolean that_present_ite = true && that.is_set_ite(); - if (this_present_ite || that_present_ite) { - if (!(this_present_ite && that_present_ite)) - return false; - if (!this.ite.equals(that.ite)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_e = true && (is_set_e()); - builder.append(present_e); - if (present_e) - builder.append(e); - - boolean present_ite = true && (is_set_ite()); - builder.append(present_ite); - if (present_ite) - builder.append(ite); - - return builder.toHashCode(); - } - - public int compareTo(rebalance_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - rebalance_result typedOther = (rebalance_result)other; - - lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_e()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_ite()).compareTo(typedOther.is_set_ite()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_ite()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.ite, typedOther.ite); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // E - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.e = new NotAliveException(); - this.e.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // ITE - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.ite = new InvalidTopologyException(); - this.ite.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_e()) { - oprot.writeFieldBegin(E_FIELD_DESC); - this.e.write(oprot); - oprot.writeFieldEnd(); - } else if (this.is_set_ite()) { - oprot.writeFieldBegin(ITE_FIELD_DESC); - this.ite.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("rebalance_result("); - boolean first = true; - - sb.append("e:"); - if (this.e == null) { - sb.append("null"); - } else { - sb.append(this.e); - } - first = false; - if (!first) sb.append(", "); - sb.append("ite:"); - if (this.ite == null) { - sb.append("null"); - } else { - sb.append(this.ite); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class metricMonitor_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("metricMonitor_args"); - - private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField OPTIONS_FIELD_DESC = new org.apache.thrift7.protocol.TField("options", org.apache.thrift7.protocol.TType.STRUCT, (short)2); - - private String name; // required - private MonitorOptions options; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - NAME((short)1, "name"), - OPTIONS((short)2, "options"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // NAME - return NAME; - case 2: // OPTIONS - return OPTIONS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.OPTIONS, new org.apache.thrift7.meta_data.FieldMetaData("options", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, MonitorOptions.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(metricMonitor_args.class, metaDataMap); - } - - public metricMonitor_args() { - } - - public metricMonitor_args( - String name, - MonitorOptions options) - { - this(); - this.name = name; - this.options = options; - } - - /** - * Performs a deep copy on other. - */ - public metricMonitor_args(metricMonitor_args other) { - if (other.is_set_name()) { - this.name = other.name; - } - if (other.is_set_options()) { - this.options = new MonitorOptions(other.options); - } - } - - public metricMonitor_args deepCopy() { - return new metricMonitor_args(this); - } - - @Override - public void clear() { - this.name = null; - this.options = null; - } - - public String get_name() { - return this.name; - } - - public void set_name(String name) { - this.name = name; - } - - public void unset_name() { - this.name = null; - } - - /** Returns true if field name is set (has been assigned a value) and false otherwise */ - public boolean is_set_name() { - return this.name != null; - } - - public void set_name_isSet(boolean value) { - if (!value) { - this.name = null; - } - } - - public MonitorOptions get_options() { - return this.options; - } - - public void set_options(MonitorOptions options) { - this.options = options; - } - - public void unset_options() { - this.options = null; - } - - /** Returns true if field options is set (has been assigned a value) and false otherwise */ - public boolean is_set_options() { - return this.options != null; - } - - public void set_options_isSet(boolean value) { - if (!value) { - this.options = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case NAME: - if (value == null) { - unset_name(); - } else { - set_name((String)value); - } - break; - - case OPTIONS: - if (value == null) { - unset_options(); - } else { - set_options((MonitorOptions)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case NAME: - return get_name(); - - case OPTIONS: - return get_options(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case NAME: - return is_set_name(); - case OPTIONS: - return is_set_options(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof metricMonitor_args) - return this.equals((metricMonitor_args)that); - return false; - } - - public boolean equals(metricMonitor_args that) { - if (that == null) - return false; - - boolean this_present_name = true && this.is_set_name(); - boolean that_present_name = true && that.is_set_name(); - if (this_present_name || that_present_name) { - if (!(this_present_name && that_present_name)) - return false; - if (!this.name.equals(that.name)) - return false; - } - - boolean this_present_options = true && this.is_set_options(); - boolean that_present_options = true && that.is_set_options(); - if (this_present_options || that_present_options) { - if (!(this_present_options && that_present_options)) - return false; - if (!this.options.equals(that.options)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_name = true && (is_set_name()); - builder.append(present_name); - if (present_name) - builder.append(name); - - boolean present_options = true && (is_set_options()); - builder.append(present_options); - if (present_options) - builder.append(options); - - return builder.toHashCode(); - } - - public int compareTo(metricMonitor_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - metricMonitor_args typedOther = (metricMonitor_args)other; - - lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_name()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_options()).compareTo(typedOther.is_set_options()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_options()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.options, typedOther.options); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // NAME - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.name = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // OPTIONS - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.options = new MonitorOptions(); - this.options.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.name != null) { - oprot.writeFieldBegin(NAME_FIELD_DESC); - oprot.writeString(this.name); - oprot.writeFieldEnd(); - } - if (this.options != null) { - oprot.writeFieldBegin(OPTIONS_FIELD_DESC); - this.options.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("metricMonitor_args("); - boolean first = true; - - sb.append("name:"); - if (this.name == null) { - sb.append("null"); - } else { - sb.append(this.name); - } - first = false; - if (!first) sb.append(", "); - sb.append("options:"); - if (this.options == null) { - sb.append("null"); - } else { - sb.append(this.options); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class metricMonitor_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("metricMonitor_result"); - - private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - - private NotAliveException e; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - E((short)1, "e"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // E - return E; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(metricMonitor_result.class, metaDataMap); - } - - public metricMonitor_result() { - } - - public metricMonitor_result( - NotAliveException e) - { - this(); - this.e = e; - } - - /** - * Performs a deep copy on other. - */ - public metricMonitor_result(metricMonitor_result other) { - if (other.is_set_e()) { - this.e = new NotAliveException(other.e); - } - } - - public metricMonitor_result deepCopy() { - return new metricMonitor_result(this); - } - - @Override - public void clear() { - this.e = null; - } - - public NotAliveException get_e() { - return this.e; - } - - public void set_e(NotAliveException e) { - this.e = e; - } - - public void unset_e() { - this.e = null; - } - - /** Returns true if field e is set (has been assigned a value) and false otherwise */ - public boolean is_set_e() { - return this.e != null; - } - - public void set_e_isSet(boolean value) { - if (!value) { - this.e = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case E: - if (value == null) { - unset_e(); - } else { - set_e((NotAliveException)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case E: - return get_e(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case E: - return is_set_e(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof metricMonitor_result) - return this.equals((metricMonitor_result)that); - return false; - } - - public boolean equals(metricMonitor_result that) { - if (that == null) - return false; - - boolean this_present_e = true && this.is_set_e(); - boolean that_present_e = true && that.is_set_e(); - if (this_present_e || that_present_e) { - if (!(this_present_e && that_present_e)) - return false; - if (!this.e.equals(that.e)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_e = true && (is_set_e()); - builder.append(present_e); - if (present_e) - builder.append(e); - - return builder.toHashCode(); - } - - public int compareTo(metricMonitor_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - metricMonitor_result typedOther = (metricMonitor_result)other; - - lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_e()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // E - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.e = new NotAliveException(); - this.e.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_e()) { - oprot.writeFieldBegin(E_FIELD_DESC); - this.e.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("metricMonitor_result("); - boolean first = true; - - sb.append("e:"); - if (this.e == null) { - sb.append("null"); - } else { - sb.append(this.e); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class restart_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("restart_args"); - - private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField JSON_CONF_FIELD_DESC = new org.apache.thrift7.protocol.TField("jsonConf", org.apache.thrift7.protocol.TType.STRING, (short)2); - - private String name; // required - private String jsonConf; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - NAME((short)1, "name"), - JSON_CONF((short)2, "jsonConf"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // NAME - return NAME; - case 2: // JSON_CONF - return JSON_CONF; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.JSON_CONF, new org.apache.thrift7.meta_data.FieldMetaData("jsonConf", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(restart_args.class, metaDataMap); - } - - public restart_args() { - } - - public restart_args( - String name, - String jsonConf) - { - this(); - this.name = name; - this.jsonConf = jsonConf; - } - - /** - * Performs a deep copy on other. - */ - public restart_args(restart_args other) { - if (other.is_set_name()) { - this.name = other.name; - } - if (other.is_set_jsonConf()) { - this.jsonConf = other.jsonConf; - } - } - - public restart_args deepCopy() { - return new restart_args(this); - } - - @Override - public void clear() { - this.name = null; - this.jsonConf = null; - } - - public String get_name() { - return this.name; - } - - public void set_name(String name) { - this.name = name; - } - - public void unset_name() { - this.name = null; - } - - /** Returns true if field name is set (has been assigned a value) and false otherwise */ - public boolean is_set_name() { - return this.name != null; - } - - public void set_name_isSet(boolean value) { - if (!value) { - this.name = null; - } - } - - public String get_jsonConf() { - return this.jsonConf; - } - - public void set_jsonConf(String jsonConf) { - this.jsonConf = jsonConf; - } - - public void unset_jsonConf() { - this.jsonConf = null; - } - - /** Returns true if field jsonConf is set (has been assigned a value) and false otherwise */ - public boolean is_set_jsonConf() { - return this.jsonConf != null; - } - - public void set_jsonConf_isSet(boolean value) { - if (!value) { - this.jsonConf = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case NAME: - if (value == null) { - unset_name(); - } else { - set_name((String)value); - } - break; - - case JSON_CONF: - if (value == null) { - unset_jsonConf(); - } else { - set_jsonConf((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case NAME: - return get_name(); - - case JSON_CONF: - return get_jsonConf(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case NAME: - return is_set_name(); - case JSON_CONF: - return is_set_jsonConf(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof restart_args) - return this.equals((restart_args)that); - return false; - } - - public boolean equals(restart_args that) { - if (that == null) - return false; - - boolean this_present_name = true && this.is_set_name(); - boolean that_present_name = true && that.is_set_name(); - if (this_present_name || that_present_name) { - if (!(this_present_name && that_present_name)) - return false; - if (!this.name.equals(that.name)) - return false; - } - - boolean this_present_jsonConf = true && this.is_set_jsonConf(); - boolean that_present_jsonConf = true && that.is_set_jsonConf(); - if (this_present_jsonConf || that_present_jsonConf) { - if (!(this_present_jsonConf && that_present_jsonConf)) - return false; - if (!this.jsonConf.equals(that.jsonConf)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_name = true && (is_set_name()); - builder.append(present_name); - if (present_name) - builder.append(name); - - boolean present_jsonConf = true && (is_set_jsonConf()); - builder.append(present_jsonConf); - if (present_jsonConf) - builder.append(jsonConf); - - return builder.toHashCode(); - } - - public int compareTo(restart_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - restart_args typedOther = (restart_args)other; - - lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_name()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_jsonConf()).compareTo(typedOther.is_set_jsonConf()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_jsonConf()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.jsonConf, typedOther.jsonConf); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // NAME - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.name = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // JSON_CONF - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.jsonConf = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.name != null) { - oprot.writeFieldBegin(NAME_FIELD_DESC); - oprot.writeString(this.name); - oprot.writeFieldEnd(); - } - if (this.jsonConf != null) { - oprot.writeFieldBegin(JSON_CONF_FIELD_DESC); - oprot.writeString(this.jsonConf); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("restart_args("); - boolean first = true; - - sb.append("name:"); - if (this.name == null) { - sb.append("null"); - } else { - sb.append(this.name); - } - first = false; - if (!first) sb.append(", "); - sb.append("jsonConf:"); - if (this.jsonConf == null) { - sb.append("null"); - } else { - sb.append(this.jsonConf); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class restart_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("restart_result"); - - private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift7.protocol.TField ITE_FIELD_DESC = new org.apache.thrift7.protocol.TField("ite", org.apache.thrift7.protocol.TType.STRUCT, (short)2); - private static final org.apache.thrift7.protocol.TField TAE_FIELD_DESC = new org.apache.thrift7.protocol.TField("tae", org.apache.thrift7.protocol.TType.STRUCT, (short)3); - - private NotAliveException e; // required - private InvalidTopologyException ite; // required - private TopologyAssignException tae; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - E((short)1, "e"), - ITE((short)2, "ite"), - TAE((short)3, "tae"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // E - return E; - case 2: // ITE - return ITE; - case 3: // TAE - return TAE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - tmpMap.put(_Fields.ITE, new org.apache.thrift7.meta_data.FieldMetaData("ite", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - tmpMap.put(_Fields.TAE, new org.apache.thrift7.meta_data.FieldMetaData("tae", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(restart_result.class, metaDataMap); - } - - public restart_result() { - } - - public restart_result( - NotAliveException e, - InvalidTopologyException ite, - TopologyAssignException tae) - { - this(); - this.e = e; - this.ite = ite; - this.tae = tae; - } - - /** - * Performs a deep copy on other. - */ - public restart_result(restart_result other) { - if (other.is_set_e()) { - this.e = new NotAliveException(other.e); - } - if (other.is_set_ite()) { - this.ite = new InvalidTopologyException(other.ite); - } - if (other.is_set_tae()) { - this.tae = new TopologyAssignException(other.tae); - } - } - - public restart_result deepCopy() { - return new restart_result(this); - } - - @Override - public void clear() { - this.e = null; - this.ite = null; - this.tae = null; - } - - public NotAliveException get_e() { - return this.e; - } - - public void set_e(NotAliveException e) { - this.e = e; - } - - public void unset_e() { - this.e = null; - } - - /** Returns true if field e is set (has been assigned a value) and false otherwise */ - public boolean is_set_e() { - return this.e != null; - } - - public void set_e_isSet(boolean value) { - if (!value) { - this.e = null; - } - } - - public InvalidTopologyException get_ite() { - return this.ite; - } - - public void set_ite(InvalidTopologyException ite) { - this.ite = ite; - } - - public void unset_ite() { - this.ite = null; - } - - /** Returns true if field ite is set (has been assigned a value) and false otherwise */ - public boolean is_set_ite() { - return this.ite != null; - } - - public void set_ite_isSet(boolean value) { - if (!value) { - this.ite = null; - } - } - - public TopologyAssignException get_tae() { - return this.tae; - } - - public void set_tae(TopologyAssignException tae) { - this.tae = tae; - } - - public void unset_tae() { - this.tae = null; - } - - /** Returns true if field tae is set (has been assigned a value) and false otherwise */ - public boolean is_set_tae() { - return this.tae != null; - } - - public void set_tae_isSet(boolean value) { - if (!value) { - this.tae = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case E: - if (value == null) { - unset_e(); - } else { - set_e((NotAliveException)value); - } - break; - - case ITE: - if (value == null) { - unset_ite(); - } else { - set_ite((InvalidTopologyException)value); - } - break; - - case TAE: - if (value == null) { - unset_tae(); - } else { - set_tae((TopologyAssignException)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case E: - return get_e(); - - case ITE: - return get_ite(); - - case TAE: - return get_tae(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case E: - return is_set_e(); - case ITE: - return is_set_ite(); - case TAE: - return is_set_tae(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof restart_result) - return this.equals((restart_result)that); - return false; - } - - public boolean equals(restart_result that) { - if (that == null) - return false; - - boolean this_present_e = true && this.is_set_e(); - boolean that_present_e = true && that.is_set_e(); - if (this_present_e || that_present_e) { - if (!(this_present_e && that_present_e)) - return false; - if (!this.e.equals(that.e)) - return false; - } - - boolean this_present_ite = true && this.is_set_ite(); - boolean that_present_ite = true && that.is_set_ite(); - if (this_present_ite || that_present_ite) { - if (!(this_present_ite && that_present_ite)) - return false; - if (!this.ite.equals(that.ite)) - return false; - } - - boolean this_present_tae = true && this.is_set_tae(); - boolean that_present_tae = true && that.is_set_tae(); - if (this_present_tae || that_present_tae) { - if (!(this_present_tae && that_present_tae)) - return false; - if (!this.tae.equals(that.tae)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_e = true && (is_set_e()); - builder.append(present_e); - if (present_e) - builder.append(e); - - boolean present_ite = true && (is_set_ite()); - builder.append(present_ite); - if (present_ite) - builder.append(ite); - - boolean present_tae = true && (is_set_tae()); - builder.append(present_tae); - if (present_tae) - builder.append(tae); - - return builder.toHashCode(); - } - - public int compareTo(restart_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - restart_result typedOther = (restart_result)other; - - lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_e()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_ite()).compareTo(typedOther.is_set_ite()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_ite()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.ite, typedOther.ite); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_tae()).compareTo(typedOther.is_set_tae()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_tae()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.tae, typedOther.tae); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // E - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.e = new NotAliveException(); - this.e.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // ITE - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.ite = new InvalidTopologyException(); - this.ite.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 3: // TAE - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.tae = new TopologyAssignException(); - this.tae.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_e()) { - oprot.writeFieldBegin(E_FIELD_DESC); - this.e.write(oprot); - oprot.writeFieldEnd(); - } else if (this.is_set_ite()) { - oprot.writeFieldBegin(ITE_FIELD_DESC); - this.ite.write(oprot); - oprot.writeFieldEnd(); - } else if (this.is_set_tae()) { - oprot.writeFieldBegin(TAE_FIELD_DESC); - this.tae.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("restart_result("); - boolean first = true; - - sb.append("e:"); - if (this.e == null) { - sb.append("null"); - } else { - sb.append(this.e); - } - first = false; - if (!first) sb.append(", "); - sb.append("ite:"); - if (this.ite == null) { - sb.append("null"); - } else { - sb.append(this.ite); - } - first = false; - if (!first) sb.append(", "); - sb.append("tae:"); - if (this.tae == null) { - sb.append("null"); - } else { - sb.append(this.tae); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class beginLibUpload_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("beginLibUpload_args"); - - private static final org.apache.thrift7.protocol.TField LIB_NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("libName", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String libName; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - LIB_NAME((short)1, "libName"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // LIB_NAME - return LIB_NAME; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.LIB_NAME, new org.apache.thrift7.meta_data.FieldMetaData("libName", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(beginLibUpload_args.class, metaDataMap); - } - - public beginLibUpload_args() { - } - - public beginLibUpload_args( - String libName) - { - this(); - this.libName = libName; - } - - /** - * Performs a deep copy on other. - */ - public beginLibUpload_args(beginLibUpload_args other) { - if (other.is_set_libName()) { - this.libName = other.libName; - } - } - - public beginLibUpload_args deepCopy() { - return new beginLibUpload_args(this); - } - - @Override - public void clear() { - this.libName = null; - } - - public String get_libName() { - return this.libName; - } - - public void set_libName(String libName) { - this.libName = libName; - } - - public void unset_libName() { - this.libName = null; - } - - /** Returns true if field libName is set (has been assigned a value) and false otherwise */ - public boolean is_set_libName() { - return this.libName != null; - } - - public void set_libName_isSet(boolean value) { - if (!value) { - this.libName = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case LIB_NAME: - if (value == null) { - unset_libName(); - } else { - set_libName((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case LIB_NAME: - return get_libName(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case LIB_NAME: - return is_set_libName(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof beginLibUpload_args) - return this.equals((beginLibUpload_args)that); - return false; - } - - public boolean equals(beginLibUpload_args that) { - if (that == null) - return false; - - boolean this_present_libName = true && this.is_set_libName(); - boolean that_present_libName = true && that.is_set_libName(); - if (this_present_libName || that_present_libName) { - if (!(this_present_libName && that_present_libName)) - return false; - if (!this.libName.equals(that.libName)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_libName = true && (is_set_libName()); - builder.append(present_libName); - if (present_libName) - builder.append(libName); - - return builder.toHashCode(); - } - - public int compareTo(beginLibUpload_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - beginLibUpload_args typedOther = (beginLibUpload_args)other; - - lastComparison = Boolean.valueOf(is_set_libName()).compareTo(typedOther.is_set_libName()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_libName()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.libName, typedOther.libName); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // LIB_NAME - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.libName = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.libName != null) { - oprot.writeFieldBegin(LIB_NAME_FIELD_DESC); - oprot.writeString(this.libName); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("beginLibUpload_args("); - boolean first = true; - - sb.append("libName:"); - if (this.libName == null) { - sb.append("null"); - } else { - sb.append(this.libName); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class beginLibUpload_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("beginLibUpload_result"); - - - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { -; - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(beginLibUpload_result.class, metaDataMap); - } - - public beginLibUpload_result() { - } - - /** - * Performs a deep copy on other. - */ - public beginLibUpload_result(beginLibUpload_result other) { - } - - public beginLibUpload_result deepCopy() { - return new beginLibUpload_result(this); - } - - @Override - public void clear() { - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof beginLibUpload_result) - return this.equals((beginLibUpload_result)that); - return false; - } - - public boolean equals(beginLibUpload_result that) { - if (that == null) - return false; - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - return builder.toHashCode(); - } - - public int compareTo(beginLibUpload_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - beginLibUpload_result typedOther = (beginLibUpload_result)other; - - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("beginLibUpload_result("); - boolean first = true; - - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class beginFileUpload_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("beginFileUpload_args"); - - - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { -; - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(beginFileUpload_args.class, metaDataMap); - } - - public beginFileUpload_args() { - } - - /** - * Performs a deep copy on other. - */ - public beginFileUpload_args(beginFileUpload_args other) { - } - - public beginFileUpload_args deepCopy() { - return new beginFileUpload_args(this); - } - - @Override - public void clear() { - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof beginFileUpload_args) - return this.equals((beginFileUpload_args)that); - return false; - } - - public boolean equals(beginFileUpload_args that) { - if (that == null) - return false; - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - return builder.toHashCode(); - } - - public int compareTo(beginFileUpload_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - beginFileUpload_args typedOther = (beginFileUpload_args)other; - - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("beginFileUpload_args("); - boolean first = true; - - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class beginFileUpload_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("beginFileUpload_result"); - - private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRING, (short)0); - - private String success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(beginFileUpload_result.class, metaDataMap); - } - - public beginFileUpload_result() { - } - - public beginFileUpload_result( - String success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public beginFileUpload_result(beginFileUpload_result other) { - if (other.is_set_success()) { - this.success = other.success; - } - } - - public beginFileUpload_result deepCopy() { - return new beginFileUpload_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public String get_success() { - return this.success; - } - - public void set_success(String success) { - this.success = success; - } - - public void unset_success() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean is_set_success() { - return this.success != null; - } - - public void set_success_isSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unset_success(); - } else { - set_success((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return get_success(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return is_set_success(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof beginFileUpload_result) - return this.equals((beginFileUpload_result)that); - return false; - } - - public boolean equals(beginFileUpload_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.is_set_success(); - boolean that_present_success = true && that.is_set_success(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (is_set_success()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(beginFileUpload_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - beginFileUpload_result typedOther = (beginFileUpload_result)other; - - lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_success()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 0: // SUCCESS - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.success = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_success()) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeString(this.success); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("beginFileUpload_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class uploadChunk_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("uploadChunk_args"); - - private static final org.apache.thrift7.protocol.TField LOCATION_FIELD_DESC = new org.apache.thrift7.protocol.TField("location", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField CHUNK_FIELD_DESC = new org.apache.thrift7.protocol.TField("chunk", org.apache.thrift7.protocol.TType.STRING, (short)2); - - private String location; // required - private ByteBuffer chunk; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - LOCATION((short)1, "location"), - CHUNK((short)2, "chunk"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // LOCATION - return LOCATION; - case 2: // CHUNK - return CHUNK; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.LOCATION, new org.apache.thrift7.meta_data.FieldMetaData("location", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.CHUNK, new org.apache.thrift7.meta_data.FieldMetaData("chunk", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING , true))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(uploadChunk_args.class, metaDataMap); - } - - public uploadChunk_args() { - } - - public uploadChunk_args( - String location, - ByteBuffer chunk) - { - this(); - this.location = location; - this.chunk = chunk; - } - - /** - * Performs a deep copy on other. - */ - public uploadChunk_args(uploadChunk_args other) { - if (other.is_set_location()) { - this.location = other.location; - } - if (other.is_set_chunk()) { - this.chunk = org.apache.thrift7.TBaseHelper.copyBinary(other.chunk); -; - } - } - - public uploadChunk_args deepCopy() { - return new uploadChunk_args(this); - } - - @Override - public void clear() { - this.location = null; - this.chunk = null; - } - - public String get_location() { - return this.location; - } - - public void set_location(String location) { - this.location = location; - } - - public void unset_location() { - this.location = null; - } - - /** Returns true if field location is set (has been assigned a value) and false otherwise */ - public boolean is_set_location() { - return this.location != null; - } - - public void set_location_isSet(boolean value) { - if (!value) { - this.location = null; - } - } - - public byte[] get_chunk() { - set_chunk(org.apache.thrift7.TBaseHelper.rightSize(chunk)); - return chunk == null ? null : chunk.array(); - } - - public ByteBuffer buffer_for_chunk() { - return chunk; - } - - public void set_chunk(byte[] chunk) { - set_chunk(chunk == null ? (ByteBuffer)null : ByteBuffer.wrap(chunk)); - } - - public void set_chunk(ByteBuffer chunk) { - this.chunk = chunk; - } - - public void unset_chunk() { - this.chunk = null; - } - - /** Returns true if field chunk is set (has been assigned a value) and false otherwise */ - public boolean is_set_chunk() { - return this.chunk != null; - } - - public void set_chunk_isSet(boolean value) { - if (!value) { - this.chunk = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case LOCATION: - if (value == null) { - unset_location(); - } else { - set_location((String)value); - } - break; - - case CHUNK: - if (value == null) { - unset_chunk(); - } else { - set_chunk((ByteBuffer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case LOCATION: - return get_location(); - - case CHUNK: - return get_chunk(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case LOCATION: - return is_set_location(); - case CHUNK: - return is_set_chunk(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof uploadChunk_args) - return this.equals((uploadChunk_args)that); - return false; - } - - public boolean equals(uploadChunk_args that) { - if (that == null) - return false; - - boolean this_present_location = true && this.is_set_location(); - boolean that_present_location = true && that.is_set_location(); - if (this_present_location || that_present_location) { - if (!(this_present_location && that_present_location)) - return false; - if (!this.location.equals(that.location)) - return false; - } - - boolean this_present_chunk = true && this.is_set_chunk(); - boolean that_present_chunk = true && that.is_set_chunk(); - if (this_present_chunk || that_present_chunk) { - if (!(this_present_chunk && that_present_chunk)) - return false; - if (!this.chunk.equals(that.chunk)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_location = true && (is_set_location()); - builder.append(present_location); - if (present_location) - builder.append(location); - - boolean present_chunk = true && (is_set_chunk()); - builder.append(present_chunk); - if (present_chunk) - builder.append(chunk); - - return builder.toHashCode(); - } - - public int compareTo(uploadChunk_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - uploadChunk_args typedOther = (uploadChunk_args)other; - - lastComparison = Boolean.valueOf(is_set_location()).compareTo(typedOther.is_set_location()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_location()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.location, typedOther.location); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_chunk()).compareTo(typedOther.is_set_chunk()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_chunk()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.chunk, typedOther.chunk); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // LOCATION - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.location = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // CHUNK - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.chunk = iprot.readBinary(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.location != null) { - oprot.writeFieldBegin(LOCATION_FIELD_DESC); - oprot.writeString(this.location); - oprot.writeFieldEnd(); - } - if (this.chunk != null) { - oprot.writeFieldBegin(CHUNK_FIELD_DESC); - oprot.writeBinary(this.chunk); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("uploadChunk_args("); - boolean first = true; - - sb.append("location:"); - if (this.location == null) { - sb.append("null"); - } else { - sb.append(this.location); - } - first = false; - if (!first) sb.append(", "); - sb.append("chunk:"); - if (this.chunk == null) { - sb.append("null"); - } else { - org.apache.thrift7.TBaseHelper.toString(this.chunk, sb); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class uploadChunk_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("uploadChunk_result"); - - - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { -; - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(uploadChunk_result.class, metaDataMap); - } - - public uploadChunk_result() { - } - - /** - * Performs a deep copy on other. - */ - public uploadChunk_result(uploadChunk_result other) { - } - - public uploadChunk_result deepCopy() { - return new uploadChunk_result(this); - } - - @Override - public void clear() { - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof uploadChunk_result) - return this.equals((uploadChunk_result)that); - return false; - } - - public boolean equals(uploadChunk_result that) { - if (that == null) - return false; - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - return builder.toHashCode(); - } - - public int compareTo(uploadChunk_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - uploadChunk_result typedOther = (uploadChunk_result)other; - - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("uploadChunk_result("); - boolean first = true; - - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class finishFileUpload_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("finishFileUpload_args"); - - private static final org.apache.thrift7.protocol.TField LOCATION_FIELD_DESC = new org.apache.thrift7.protocol.TField("location", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String location; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - LOCATION((short)1, "location"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // LOCATION - return LOCATION; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.LOCATION, new org.apache.thrift7.meta_data.FieldMetaData("location", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(finishFileUpload_args.class, metaDataMap); - } - - public finishFileUpload_args() { - } - - public finishFileUpload_args( - String location) - { - this(); - this.location = location; - } - - /** - * Performs a deep copy on other. - */ - public finishFileUpload_args(finishFileUpload_args other) { - if (other.is_set_location()) { - this.location = other.location; - } - } - - public finishFileUpload_args deepCopy() { - return new finishFileUpload_args(this); - } - - @Override - public void clear() { - this.location = null; - } - - public String get_location() { - return this.location; - } - - public void set_location(String location) { - this.location = location; - } - - public void unset_location() { - this.location = null; - } - - /** Returns true if field location is set (has been assigned a value) and false otherwise */ - public boolean is_set_location() { - return this.location != null; - } - - public void set_location_isSet(boolean value) { - if (!value) { - this.location = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case LOCATION: - if (value == null) { - unset_location(); - } else { - set_location((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case LOCATION: - return get_location(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case LOCATION: - return is_set_location(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof finishFileUpload_args) - return this.equals((finishFileUpload_args)that); - return false; - } - - public boolean equals(finishFileUpload_args that) { - if (that == null) - return false; - - boolean this_present_location = true && this.is_set_location(); - boolean that_present_location = true && that.is_set_location(); - if (this_present_location || that_present_location) { - if (!(this_present_location && that_present_location)) - return false; - if (!this.location.equals(that.location)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_location = true && (is_set_location()); - builder.append(present_location); - if (present_location) - builder.append(location); - - return builder.toHashCode(); - } - - public int compareTo(finishFileUpload_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - finishFileUpload_args typedOther = (finishFileUpload_args)other; - - lastComparison = Boolean.valueOf(is_set_location()).compareTo(typedOther.is_set_location()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_location()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.location, typedOther.location); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // LOCATION - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.location = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.location != null) { - oprot.writeFieldBegin(LOCATION_FIELD_DESC); - oprot.writeString(this.location); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("finishFileUpload_args("); - boolean first = true; - - sb.append("location:"); - if (this.location == null) { - sb.append("null"); - } else { - sb.append(this.location); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class finishFileUpload_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("finishFileUpload_result"); - - - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { -; - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(finishFileUpload_result.class, metaDataMap); - } - - public finishFileUpload_result() { - } - - /** - * Performs a deep copy on other. - */ - public finishFileUpload_result(finishFileUpload_result other) { - } - - public finishFileUpload_result deepCopy() { - return new finishFileUpload_result(this); - } - - @Override - public void clear() { - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof finishFileUpload_result) - return this.equals((finishFileUpload_result)that); - return false; - } - - public boolean equals(finishFileUpload_result that) { - if (that == null) - return false; - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - return builder.toHashCode(); - } - - public int compareTo(finishFileUpload_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - finishFileUpload_result typedOther = (finishFileUpload_result)other; - - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("finishFileUpload_result("); - boolean first = true; - - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class beginFileDownload_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("beginFileDownload_args"); - - private static final org.apache.thrift7.protocol.TField FILE_FIELD_DESC = new org.apache.thrift7.protocol.TField("file", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String file; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - FILE((short)1, "file"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // FILE - return FILE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.FILE, new org.apache.thrift7.meta_data.FieldMetaData("file", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(beginFileDownload_args.class, metaDataMap); - } - - public beginFileDownload_args() { - } - - public beginFileDownload_args( - String file) - { - this(); - this.file = file; - } - - /** - * Performs a deep copy on other. - */ - public beginFileDownload_args(beginFileDownload_args other) { - if (other.is_set_file()) { - this.file = other.file; - } - } - - public beginFileDownload_args deepCopy() { - return new beginFileDownload_args(this); - } - - @Override - public void clear() { - this.file = null; - } - - public String get_file() { - return this.file; - } - - public void set_file(String file) { - this.file = file; - } - - public void unset_file() { - this.file = null; - } - - /** Returns true if field file is set (has been assigned a value) and false otherwise */ - public boolean is_set_file() { - return this.file != null; - } - - public void set_file_isSet(boolean value) { - if (!value) { - this.file = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case FILE: - if (value == null) { - unset_file(); - } else { - set_file((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case FILE: - return get_file(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case FILE: - return is_set_file(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof beginFileDownload_args) - return this.equals((beginFileDownload_args)that); - return false; - } - - public boolean equals(beginFileDownload_args that) { - if (that == null) - return false; - - boolean this_present_file = true && this.is_set_file(); - boolean that_present_file = true && that.is_set_file(); - if (this_present_file || that_present_file) { - if (!(this_present_file && that_present_file)) - return false; - if (!this.file.equals(that.file)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_file = true && (is_set_file()); - builder.append(present_file); - if (present_file) - builder.append(file); - - return builder.toHashCode(); - } - - public int compareTo(beginFileDownload_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - beginFileDownload_args typedOther = (beginFileDownload_args)other; - - lastComparison = Boolean.valueOf(is_set_file()).compareTo(typedOther.is_set_file()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_file()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.file, typedOther.file); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // FILE - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.file = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.file != null) { - oprot.writeFieldBegin(FILE_FIELD_DESC); - oprot.writeString(this.file); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("beginFileDownload_args("); - boolean first = true; - - sb.append("file:"); - if (this.file == null) { - sb.append("null"); - } else { - sb.append(this.file); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class beginFileDownload_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("beginFileDownload_result"); - - private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRING, (short)0); - - private String success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(beginFileDownload_result.class, metaDataMap); - } - - public beginFileDownload_result() { - } - - public beginFileDownload_result( - String success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public beginFileDownload_result(beginFileDownload_result other) { - if (other.is_set_success()) { - this.success = other.success; - } - } - - public beginFileDownload_result deepCopy() { - return new beginFileDownload_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public String get_success() { - return this.success; - } - - public void set_success(String success) { - this.success = success; - } - - public void unset_success() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean is_set_success() { - return this.success != null; - } - - public void set_success_isSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unset_success(); - } else { - set_success((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return get_success(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return is_set_success(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof beginFileDownload_result) - return this.equals((beginFileDownload_result)that); - return false; - } - - public boolean equals(beginFileDownload_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.is_set_success(); - boolean that_present_success = true && that.is_set_success(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (is_set_success()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(beginFileDownload_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - beginFileDownload_result typedOther = (beginFileDownload_result)other; - - lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_success()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 0: // SUCCESS - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.success = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_success()) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeString(this.success); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("beginFileDownload_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class downloadChunk_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("downloadChunk_args"); - - private static final org.apache.thrift7.protocol.TField ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("id", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String id; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - ID((short)1, "id"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // ID - return ID; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.ID, new org.apache.thrift7.meta_data.FieldMetaData("id", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(downloadChunk_args.class, metaDataMap); - } - - public downloadChunk_args() { - } - - public downloadChunk_args( - String id) - { - this(); - this.id = id; - } - - /** - * Performs a deep copy on other. - */ - public downloadChunk_args(downloadChunk_args other) { - if (other.is_set_id()) { - this.id = other.id; - } - } - - public downloadChunk_args deepCopy() { - return new downloadChunk_args(this); - } - - @Override - public void clear() { - this.id = null; - } - - public String get_id() { - return this.id; - } - - public void set_id(String id) { - this.id = id; - } - - public void unset_id() { - this.id = null; - } - - /** Returns true if field id is set (has been assigned a value) and false otherwise */ - public boolean is_set_id() { - return this.id != null; - } - - public void set_id_isSet(boolean value) { - if (!value) { - this.id = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case ID: - if (value == null) { - unset_id(); - } else { - set_id((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case ID: - return get_id(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case ID: - return is_set_id(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof downloadChunk_args) - return this.equals((downloadChunk_args)that); - return false; - } - - public boolean equals(downloadChunk_args that) { - if (that == null) - return false; - - boolean this_present_id = true && this.is_set_id(); - boolean that_present_id = true && that.is_set_id(); - if (this_present_id || that_present_id) { - if (!(this_present_id && that_present_id)) - return false; - if (!this.id.equals(that.id)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_id = true && (is_set_id()); - builder.append(present_id); - if (present_id) - builder.append(id); - - return builder.toHashCode(); - } - - public int compareTo(downloadChunk_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - downloadChunk_args typedOther = (downloadChunk_args)other; - - lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_id()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.id, typedOther.id); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // ID - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.id = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.id != null) { - oprot.writeFieldBegin(ID_FIELD_DESC); - oprot.writeString(this.id); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("downloadChunk_args("); - boolean first = true; - - sb.append("id:"); - if (this.id == null) { - sb.append("null"); - } else { - sb.append(this.id); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class downloadChunk_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("downloadChunk_result"); - - private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRING, (short)0); - - private ByteBuffer success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING , true))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(downloadChunk_result.class, metaDataMap); - } - - public downloadChunk_result() { - } - - public downloadChunk_result( - ByteBuffer success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public downloadChunk_result(downloadChunk_result other) { - if (other.is_set_success()) { - this.success = org.apache.thrift7.TBaseHelper.copyBinary(other.success); -; - } - } - - public downloadChunk_result deepCopy() { - return new downloadChunk_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public byte[] get_success() { - set_success(org.apache.thrift7.TBaseHelper.rightSize(success)); - return success == null ? null : success.array(); - } - - public ByteBuffer buffer_for_success() { - return success; - } - - public void set_success(byte[] success) { - set_success(success == null ? (ByteBuffer)null : ByteBuffer.wrap(success)); - } - - public void set_success(ByteBuffer success) { - this.success = success; - } - - public void unset_success() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean is_set_success() { - return this.success != null; - } - - public void set_success_isSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unset_success(); - } else { - set_success((ByteBuffer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return get_success(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return is_set_success(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof downloadChunk_result) - return this.equals((downloadChunk_result)that); - return false; - } - - public boolean equals(downloadChunk_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.is_set_success(); - boolean that_present_success = true && that.is_set_success(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (is_set_success()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(downloadChunk_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - downloadChunk_result typedOther = (downloadChunk_result)other; - - lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_success()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 0: // SUCCESS - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.success = iprot.readBinary(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_success()) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeBinary(this.success); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("downloadChunk_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - org.apache.thrift7.TBaseHelper.toString(this.success, sb); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class getNimbusConf_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getNimbusConf_args"); - - - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { -; - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getNimbusConf_args.class, metaDataMap); - } - - public getNimbusConf_args() { - } - - /** - * Performs a deep copy on other. - */ - public getNimbusConf_args(getNimbusConf_args other) { - } - - public getNimbusConf_args deepCopy() { - return new getNimbusConf_args(this); - } - - @Override - public void clear() { - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof getNimbusConf_args) - return this.equals((getNimbusConf_args)that); - return false; - } - - public boolean equals(getNimbusConf_args that) { - if (that == null) - return false; - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - return builder.toHashCode(); - } - - public int compareTo(getNimbusConf_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - getNimbusConf_args typedOther = (getNimbusConf_args)other; - - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("getNimbusConf_args("); - boolean first = true; - - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class getNimbusConf_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getNimbusConf_result"); - - private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRING, (short)0); - - private String success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getNimbusConf_result.class, metaDataMap); - } - - public getNimbusConf_result() { - } - - public getNimbusConf_result( - String success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public getNimbusConf_result(getNimbusConf_result other) { - if (other.is_set_success()) { - this.success = other.success; - } - } - - public getNimbusConf_result deepCopy() { - return new getNimbusConf_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public String get_success() { - return this.success; - } - - public void set_success(String success) { - this.success = success; - } - - public void unset_success() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean is_set_success() { - return this.success != null; - } - - public void set_success_isSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unset_success(); - } else { - set_success((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return get_success(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return is_set_success(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof getNimbusConf_result) - return this.equals((getNimbusConf_result)that); - return false; - } - - public boolean equals(getNimbusConf_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.is_set_success(); - boolean that_present_success = true && that.is_set_success(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (is_set_success()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(getNimbusConf_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - getNimbusConf_result typedOther = (getNimbusConf_result)other; - - lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_success()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 0: // SUCCESS - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.success = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_success()) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeString(this.success); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("getNimbusConf_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class getClusterInfo_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getClusterInfo_args"); - - - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { -; - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getClusterInfo_args.class, metaDataMap); - } - - public getClusterInfo_args() { - } - - /** - * Performs a deep copy on other. - */ - public getClusterInfo_args(getClusterInfo_args other) { - } - - public getClusterInfo_args deepCopy() { - return new getClusterInfo_args(this); - } - - @Override - public void clear() { - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof getClusterInfo_args) - return this.equals((getClusterInfo_args)that); - return false; - } - - public boolean equals(getClusterInfo_args that) { - if (that == null) - return false; - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - return builder.toHashCode(); - } - - public int compareTo(getClusterInfo_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - getClusterInfo_args typedOther = (getClusterInfo_args)other; - - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("getClusterInfo_args("); - boolean first = true; - - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class getClusterInfo_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getClusterInfo_result"); - - private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRUCT, (short)0); - - private ClusterSummary success; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - SUCCESS((short)0, "success"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ClusterSummary.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getClusterInfo_result.class, metaDataMap); - } - - public getClusterInfo_result() { - } - - public getClusterInfo_result( - ClusterSummary success) - { - this(); - this.success = success; - } - - /** - * Performs a deep copy on other. - */ - public getClusterInfo_result(getClusterInfo_result other) { - if (other.is_set_success()) { - this.success = new ClusterSummary(other.success); - } - } - - public getClusterInfo_result deepCopy() { - return new getClusterInfo_result(this); - } - - @Override - public void clear() { - this.success = null; - } - - public ClusterSummary get_success() { - return this.success; - } - - public void set_success(ClusterSummary success) { - this.success = success; - } - - public void unset_success() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean is_set_success() { - return this.success != null; - } - - public void set_success_isSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unset_success(); - } else { - set_success((ClusterSummary)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return get_success(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return is_set_success(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof getClusterInfo_result) - return this.equals((getClusterInfo_result)that); - return false; - } - - public boolean equals(getClusterInfo_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.is_set_success(); - boolean that_present_success = true && that.is_set_success(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (is_set_success()); - builder.append(present_success); - if (present_success) - builder.append(success); - - return builder.toHashCode(); - } - - public int compareTo(getClusterInfo_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - getClusterInfo_result typedOther = (getClusterInfo_result)other; - - lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_success()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 0: // SUCCESS - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.success = new ClusterSummary(); - this.success.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_success()) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - this.success.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("getClusterInfo_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class getTopologyInfo_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getTopologyInfo_args"); - - private static final org.apache.thrift7.protocol.TField ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("id", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String id; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - ID((short)1, "id"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // ID - return ID; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.ID, new org.apache.thrift7.meta_data.FieldMetaData("id", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyInfo_args.class, metaDataMap); - } - - public getTopologyInfo_args() { - } - - public getTopologyInfo_args( - String id) - { - this(); - this.id = id; - } - - /** - * Performs a deep copy on other. - */ - public getTopologyInfo_args(getTopologyInfo_args other) { - if (other.is_set_id()) { - this.id = other.id; - } - } - - public getTopologyInfo_args deepCopy() { - return new getTopologyInfo_args(this); - } - - @Override - public void clear() { - this.id = null; - } - - public String get_id() { - return this.id; - } - - public void set_id(String id) { - this.id = id; - } - - public void unset_id() { - this.id = null; - } - - /** Returns true if field id is set (has been assigned a value) and false otherwise */ - public boolean is_set_id() { - return this.id != null; - } - - public void set_id_isSet(boolean value) { - if (!value) { - this.id = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case ID: - if (value == null) { - unset_id(); - } else { - set_id((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case ID: - return get_id(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case ID: - return is_set_id(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof getTopologyInfo_args) - return this.equals((getTopologyInfo_args)that); - return false; - } - - public boolean equals(getTopologyInfo_args that) { - if (that == null) - return false; - - boolean this_present_id = true && this.is_set_id(); - boolean that_present_id = true && that.is_set_id(); - if (this_present_id || that_present_id) { - if (!(this_present_id && that_present_id)) - return false; - if (!this.id.equals(that.id)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_id = true && (is_set_id()); - builder.append(present_id); - if (present_id) - builder.append(id); - - return builder.toHashCode(); - } - - public int compareTo(getTopologyInfo_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - getTopologyInfo_args typedOther = (getTopologyInfo_args)other; - - lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_id()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.id, typedOther.id); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // ID - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.id = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.id != null) { - oprot.writeFieldBegin(ID_FIELD_DESC); - oprot.writeString(this.id); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("getTopologyInfo_args("); - boolean first = true; - - sb.append("id:"); - if (this.id == null) { - sb.append("null"); - } else { - sb.append(this.id); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class getTopologyInfo_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getTopologyInfo_result"); - - private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRUCT, (short)0); - private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - - private TopologyInfo success; // required - private NotAliveException e; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - SUCCESS((short)0, "success"), - E((short)1, "e"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - case 1: // E - return E; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, TopologyInfo.class))); - tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyInfo_result.class, metaDataMap); - } - - public getTopologyInfo_result() { - } - - public getTopologyInfo_result( - TopologyInfo success, - NotAliveException e) - { - this(); - this.success = success; - this.e = e; - } - - /** - * Performs a deep copy on other. - */ - public getTopologyInfo_result(getTopologyInfo_result other) { - if (other.is_set_success()) { - this.success = new TopologyInfo(other.success); - } - if (other.is_set_e()) { - this.e = new NotAliveException(other.e); - } - } - - public getTopologyInfo_result deepCopy() { - return new getTopologyInfo_result(this); - } - - @Override - public void clear() { - this.success = null; - this.e = null; - } - - public TopologyInfo get_success() { - return this.success; - } - - public void set_success(TopologyInfo success) { - this.success = success; - } - - public void unset_success() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean is_set_success() { - return this.success != null; - } - - public void set_success_isSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public NotAliveException get_e() { - return this.e; - } - - public void set_e(NotAliveException e) { - this.e = e; - } - - public void unset_e() { - this.e = null; - } - - /** Returns true if field e is set (has been assigned a value) and false otherwise */ - public boolean is_set_e() { - return this.e != null; - } - - public void set_e_isSet(boolean value) { - if (!value) { - this.e = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unset_success(); - } else { - set_success((TopologyInfo)value); - } - break; - - case E: - if (value == null) { - unset_e(); - } else { - set_e((NotAliveException)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return get_success(); - - case E: - return get_e(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return is_set_success(); - case E: - return is_set_e(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof getTopologyInfo_result) - return this.equals((getTopologyInfo_result)that); - return false; - } - - public boolean equals(getTopologyInfo_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.is_set_success(); - boolean that_present_success = true && that.is_set_success(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - boolean this_present_e = true && this.is_set_e(); - boolean that_present_e = true && that.is_set_e(); - if (this_present_e || that_present_e) { - if (!(this_present_e && that_present_e)) - return false; - if (!this.e.equals(that.e)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (is_set_success()); - builder.append(present_success); - if (present_success) - builder.append(success); - - boolean present_e = true && (is_set_e()); - builder.append(present_e); - if (present_e) - builder.append(e); - - return builder.toHashCode(); - } - - public int compareTo(getTopologyInfo_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - getTopologyInfo_result typedOther = (getTopologyInfo_result)other; - - lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_success()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_e()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 0: // SUCCESS - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.success = new TopologyInfo(); - this.success.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 1: // E - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.e = new NotAliveException(); - this.e.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_success()) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - this.success.write(oprot); - oprot.writeFieldEnd(); - } else if (this.is_set_e()) { - oprot.writeFieldBegin(E_FIELD_DESC); - this.e.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("getTopologyInfo_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - if (!first) sb.append(", "); - sb.append("e:"); - if (this.e == null) { - sb.append("null"); - } else { - sb.append(this.e); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class getTopologyInfoByName_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getTopologyInfoByName_args"); - - private static final org.apache.thrift7.protocol.TField TOPOLOGY_NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("topologyName", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String topologyName; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - TOPOLOGY_NAME((short)1, "topologyName"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // TOPOLOGY_NAME - return TOPOLOGY_NAME; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.TOPOLOGY_NAME, new org.apache.thrift7.meta_data.FieldMetaData("topologyName", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyInfoByName_args.class, metaDataMap); - } - - public getTopologyInfoByName_args() { - } - - public getTopologyInfoByName_args( - String topologyName) - { - this(); - this.topologyName = topologyName; - } - - /** - * Performs a deep copy on other. - */ - public getTopologyInfoByName_args(getTopologyInfoByName_args other) { - if (other.is_set_topologyName()) { - this.topologyName = other.topologyName; - } - } - - public getTopologyInfoByName_args deepCopy() { - return new getTopologyInfoByName_args(this); - } - - @Override - public void clear() { - this.topologyName = null; - } - - public String get_topologyName() { - return this.topologyName; - } - - public void set_topologyName(String topologyName) { - this.topologyName = topologyName; - } - - public void unset_topologyName() { - this.topologyName = null; - } - - /** Returns true if field topologyName is set (has been assigned a value) and false otherwise */ - public boolean is_set_topologyName() { - return this.topologyName != null; - } - - public void set_topologyName_isSet(boolean value) { - if (!value) { - this.topologyName = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case TOPOLOGY_NAME: - if (value == null) { - unset_topologyName(); - } else { - set_topologyName((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case TOPOLOGY_NAME: - return get_topologyName(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case TOPOLOGY_NAME: - return is_set_topologyName(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof getTopologyInfoByName_args) - return this.equals((getTopologyInfoByName_args)that); - return false; - } - - public boolean equals(getTopologyInfoByName_args that) { - if (that == null) - return false; - - boolean this_present_topologyName = true && this.is_set_topologyName(); - boolean that_present_topologyName = true && that.is_set_topologyName(); - if (this_present_topologyName || that_present_topologyName) { - if (!(this_present_topologyName && that_present_topologyName)) - return false; - if (!this.topologyName.equals(that.topologyName)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_topologyName = true && (is_set_topologyName()); - builder.append(present_topologyName); - if (present_topologyName) - builder.append(topologyName); - - return builder.toHashCode(); - } - - public int compareTo(getTopologyInfoByName_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - getTopologyInfoByName_args typedOther = (getTopologyInfoByName_args)other; - - lastComparison = Boolean.valueOf(is_set_topologyName()).compareTo(typedOther.is_set_topologyName()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_topologyName()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.topologyName, typedOther.topologyName); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // TOPOLOGY_NAME - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.topologyName = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.topologyName != null) { - oprot.writeFieldBegin(TOPOLOGY_NAME_FIELD_DESC); - oprot.writeString(this.topologyName); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("getTopologyInfoByName_args("); - boolean first = true; - - sb.append("topologyName:"); - if (this.topologyName == null) { - sb.append("null"); - } else { - sb.append(this.topologyName); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class getTopologyInfoByName_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getTopologyInfoByName_result"); - - private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRUCT, (short)0); - private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - - private TopologyInfo success; // required - private NotAliveException e; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - SUCCESS((short)0, "success"), - E((short)1, "e"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - case 1: // E - return E; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, TopologyInfo.class))); - tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyInfoByName_result.class, metaDataMap); - } - - public getTopologyInfoByName_result() { - } - - public getTopologyInfoByName_result( - TopologyInfo success, - NotAliveException e) - { - this(); - this.success = success; - this.e = e; - } - - /** - * Performs a deep copy on other. - */ - public getTopologyInfoByName_result(getTopologyInfoByName_result other) { - if (other.is_set_success()) { - this.success = new TopologyInfo(other.success); - } - if (other.is_set_e()) { - this.e = new NotAliveException(other.e); - } - } - - public getTopologyInfoByName_result deepCopy() { - return new getTopologyInfoByName_result(this); - } - - @Override - public void clear() { - this.success = null; - this.e = null; - } - - public TopologyInfo get_success() { - return this.success; - } - - public void set_success(TopologyInfo success) { - this.success = success; - } - - public void unset_success() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean is_set_success() { - return this.success != null; - } - - public void set_success_isSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public NotAliveException get_e() { - return this.e; - } - - public void set_e(NotAliveException e) { - this.e = e; - } - - public void unset_e() { - this.e = null; - } - - /** Returns true if field e is set (has been assigned a value) and false otherwise */ - public boolean is_set_e() { - return this.e != null; - } - - public void set_e_isSet(boolean value) { - if (!value) { - this.e = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unset_success(); - } else { - set_success((TopologyInfo)value); - } - break; - - case E: - if (value == null) { - unset_e(); - } else { - set_e((NotAliveException)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return get_success(); - - case E: - return get_e(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return is_set_success(); - case E: - return is_set_e(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof getTopologyInfoByName_result) - return this.equals((getTopologyInfoByName_result)that); - return false; - } - - public boolean equals(getTopologyInfoByName_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.is_set_success(); - boolean that_present_success = true && that.is_set_success(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - boolean this_present_e = true && this.is_set_e(); - boolean that_present_e = true && that.is_set_e(); - if (this_present_e || that_present_e) { - if (!(this_present_e && that_present_e)) - return false; - if (!this.e.equals(that.e)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (is_set_success()); - builder.append(present_success); - if (present_success) - builder.append(success); - - boolean present_e = true && (is_set_e()); - builder.append(present_e); - if (present_e) - builder.append(e); - - return builder.toHashCode(); - } - - public int compareTo(getTopologyInfoByName_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - getTopologyInfoByName_result typedOther = (getTopologyInfoByName_result)other; - - lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_success()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_e()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 0: // SUCCESS - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.success = new TopologyInfo(); - this.success.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 1: // E - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.e = new NotAliveException(); - this.e.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_success()) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - this.success.write(oprot); - oprot.writeFieldEnd(); - } else if (this.is_set_e()) { - oprot.writeFieldBegin(E_FIELD_DESC); - this.e.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("getTopologyInfoByName_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - if (!first) sb.append(", "); - sb.append("e:"); - if (this.e == null) { - sb.append("null"); - } else { - sb.append(this.e); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class getSupervisorWorkers_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getSupervisorWorkers_args"); - - private static final org.apache.thrift7.protocol.TField HOST_FIELD_DESC = new org.apache.thrift7.protocol.TField("host", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String host; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - HOST((short)1, "host"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // HOST - return HOST; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.HOST, new org.apache.thrift7.meta_data.FieldMetaData("host", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getSupervisorWorkers_args.class, metaDataMap); - } - - public getSupervisorWorkers_args() { - } - - public getSupervisorWorkers_args( - String host) - { - this(); - this.host = host; - } - - /** - * Performs a deep copy on other. - */ - public getSupervisorWorkers_args(getSupervisorWorkers_args other) { - if (other.is_set_host()) { - this.host = other.host; - } - } - - public getSupervisorWorkers_args deepCopy() { - return new getSupervisorWorkers_args(this); - } - - @Override - public void clear() { - this.host = null; - } - - public String get_host() { - return this.host; - } - - public void set_host(String host) { - this.host = host; - } - - public void unset_host() { - this.host = null; - } - - /** Returns true if field host is set (has been assigned a value) and false otherwise */ - public boolean is_set_host() { - return this.host != null; - } - - public void set_host_isSet(boolean value) { - if (!value) { - this.host = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case HOST: - if (value == null) { - unset_host(); - } else { - set_host((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case HOST: - return get_host(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case HOST: - return is_set_host(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof getSupervisorWorkers_args) - return this.equals((getSupervisorWorkers_args)that); - return false; - } - - public boolean equals(getSupervisorWorkers_args that) { - if (that == null) - return false; - - boolean this_present_host = true && this.is_set_host(); - boolean that_present_host = true && that.is_set_host(); - if (this_present_host || that_present_host) { - if (!(this_present_host && that_present_host)) - return false; - if (!this.host.equals(that.host)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_host = true && (is_set_host()); - builder.append(present_host); - if (present_host) - builder.append(host); - - return builder.toHashCode(); - } - - public int compareTo(getSupervisorWorkers_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - getSupervisorWorkers_args typedOther = (getSupervisorWorkers_args)other; - - lastComparison = Boolean.valueOf(is_set_host()).compareTo(typedOther.is_set_host()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_host()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.host, typedOther.host); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // HOST - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.host = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.host != null) { - oprot.writeFieldBegin(HOST_FIELD_DESC); - oprot.writeString(this.host); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("getSupervisorWorkers_args("); - boolean first = true; - - sb.append("host:"); - if (this.host == null) { - sb.append("null"); - } else { - sb.append(this.host); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class getSupervisorWorkers_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getSupervisorWorkers_result"); - - private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRUCT, (short)0); - private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - - private SupervisorWorkers success; // required - private NotAliveException e; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - SUCCESS((short)0, "success"), - E((short)1, "e"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - case 1: // E - return E; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, SupervisorWorkers.class))); - tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getSupervisorWorkers_result.class, metaDataMap); - } - - public getSupervisorWorkers_result() { - } - - public getSupervisorWorkers_result( - SupervisorWorkers success, - NotAliveException e) - { - this(); - this.success = success; - this.e = e; - } - - /** - * Performs a deep copy on other. - */ - public getSupervisorWorkers_result(getSupervisorWorkers_result other) { - if (other.is_set_success()) { - this.success = new SupervisorWorkers(other.success); - } - if (other.is_set_e()) { - this.e = new NotAliveException(other.e); - } - } - - public getSupervisorWorkers_result deepCopy() { - return new getSupervisorWorkers_result(this); - } - - @Override - public void clear() { - this.success = null; - this.e = null; - } - - public SupervisorWorkers get_success() { - return this.success; - } - - public void set_success(SupervisorWorkers success) { - this.success = success; - } - - public void unset_success() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean is_set_success() { - return this.success != null; - } - - public void set_success_isSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public NotAliveException get_e() { - return this.e; - } - - public void set_e(NotAliveException e) { - this.e = e; - } - - public void unset_e() { - this.e = null; - } - - /** Returns true if field e is set (has been assigned a value) and false otherwise */ - public boolean is_set_e() { - return this.e != null; - } - - public void set_e_isSet(boolean value) { - if (!value) { - this.e = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unset_success(); - } else { - set_success((SupervisorWorkers)value); - } - break; - - case E: - if (value == null) { - unset_e(); - } else { - set_e((NotAliveException)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return get_success(); - - case E: - return get_e(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return is_set_success(); - case E: - return is_set_e(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof getSupervisorWorkers_result) - return this.equals((getSupervisorWorkers_result)that); - return false; - } - - public boolean equals(getSupervisorWorkers_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.is_set_success(); - boolean that_present_success = true && that.is_set_success(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - boolean this_present_e = true && this.is_set_e(); - boolean that_present_e = true && that.is_set_e(); - if (this_present_e || that_present_e) { - if (!(this_present_e && that_present_e)) - return false; - if (!this.e.equals(that.e)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (is_set_success()); - builder.append(present_success); - if (present_success) - builder.append(success); - - boolean present_e = true && (is_set_e()); - builder.append(present_e); - if (present_e) - builder.append(e); - - return builder.toHashCode(); - } - - public int compareTo(getSupervisorWorkers_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - getSupervisorWorkers_result typedOther = (getSupervisorWorkers_result)other; - - lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_success()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_e()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 0: // SUCCESS - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.success = new SupervisorWorkers(); - this.success.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 1: // E - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.e = new NotAliveException(); - this.e.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_success()) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - this.success.write(oprot); - oprot.writeFieldEnd(); - } else if (this.is_set_e()) { - oprot.writeFieldBegin(E_FIELD_DESC); - this.e.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("getSupervisorWorkers_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - if (!first) sb.append(", "); - sb.append("e:"); - if (this.e == null) { - sb.append("null"); - } else { - sb.append(this.e); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class getTopologyConf_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getTopologyConf_args"); - - private static final org.apache.thrift7.protocol.TField ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("id", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String id; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - ID((short)1, "id"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // ID - return ID; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.ID, new org.apache.thrift7.meta_data.FieldMetaData("id", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyConf_args.class, metaDataMap); - } - - public getTopologyConf_args() { - } - - public getTopologyConf_args( - String id) - { - this(); - this.id = id; - } - - /** - * Performs a deep copy on other. - */ - public getTopologyConf_args(getTopologyConf_args other) { - if (other.is_set_id()) { - this.id = other.id; - } - } - - public getTopologyConf_args deepCopy() { - return new getTopologyConf_args(this); - } - - @Override - public void clear() { - this.id = null; - } - - public String get_id() { - return this.id; - } - - public void set_id(String id) { - this.id = id; - } - - public void unset_id() { - this.id = null; - } - - /** Returns true if field id is set (has been assigned a value) and false otherwise */ - public boolean is_set_id() { - return this.id != null; - } - - public void set_id_isSet(boolean value) { - if (!value) { - this.id = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case ID: - if (value == null) { - unset_id(); - } else { - set_id((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case ID: - return get_id(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case ID: - return is_set_id(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof getTopologyConf_args) - return this.equals((getTopologyConf_args)that); - return false; - } - - public boolean equals(getTopologyConf_args that) { - if (that == null) - return false; - - boolean this_present_id = true && this.is_set_id(); - boolean that_present_id = true && that.is_set_id(); - if (this_present_id || that_present_id) { - if (!(this_present_id && that_present_id)) - return false; - if (!this.id.equals(that.id)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_id = true && (is_set_id()); - builder.append(present_id); - if (present_id) - builder.append(id); - - return builder.toHashCode(); - } - - public int compareTo(getTopologyConf_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - getTopologyConf_args typedOther = (getTopologyConf_args)other; - - lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_id()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.id, typedOther.id); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // ID - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.id = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.id != null) { - oprot.writeFieldBegin(ID_FIELD_DESC); - oprot.writeString(this.id); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("getTopologyConf_args("); - boolean first = true; - - sb.append("id:"); - if (this.id == null) { - sb.append("null"); - } else { - sb.append(this.id); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class getTopologyConf_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getTopologyConf_result"); - - private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRING, (short)0); - private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - - private String success; // required - private NotAliveException e; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - SUCCESS((short)0, "success"), - E((short)1, "e"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - case 1: // E - return E; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyConf_result.class, metaDataMap); - } - - public getTopologyConf_result() { - } - - public getTopologyConf_result( - String success, - NotAliveException e) - { - this(); - this.success = success; - this.e = e; - } - - /** - * Performs a deep copy on other. - */ - public getTopologyConf_result(getTopologyConf_result other) { - if (other.is_set_success()) { - this.success = other.success; - } - if (other.is_set_e()) { - this.e = new NotAliveException(other.e); - } - } - - public getTopologyConf_result deepCopy() { - return new getTopologyConf_result(this); - } - - @Override - public void clear() { - this.success = null; - this.e = null; - } - - public String get_success() { - return this.success; - } - - public void set_success(String success) { - this.success = success; - } - - public void unset_success() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean is_set_success() { - return this.success != null; - } - - public void set_success_isSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public NotAliveException get_e() { - return this.e; - } - - public void set_e(NotAliveException e) { - this.e = e; - } - - public void unset_e() { - this.e = null; - } - - /** Returns true if field e is set (has been assigned a value) and false otherwise */ - public boolean is_set_e() { - return this.e != null; - } - - public void set_e_isSet(boolean value) { - if (!value) { - this.e = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unset_success(); - } else { - set_success((String)value); - } - break; - - case E: - if (value == null) { - unset_e(); - } else { - set_e((NotAliveException)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return get_success(); - - case E: - return get_e(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return is_set_success(); - case E: - return is_set_e(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof getTopologyConf_result) - return this.equals((getTopologyConf_result)that); - return false; - } - - public boolean equals(getTopologyConf_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.is_set_success(); - boolean that_present_success = true && that.is_set_success(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - boolean this_present_e = true && this.is_set_e(); - boolean that_present_e = true && that.is_set_e(); - if (this_present_e || that_present_e) { - if (!(this_present_e && that_present_e)) - return false; - if (!this.e.equals(that.e)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (is_set_success()); - builder.append(present_success); - if (present_success) - builder.append(success); - - boolean present_e = true && (is_set_e()); - builder.append(present_e); - if (present_e) - builder.append(e); - - return builder.toHashCode(); - } - - public int compareTo(getTopologyConf_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - getTopologyConf_result typedOther = (getTopologyConf_result)other; - - lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_success()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_e()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 0: // SUCCESS - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.success = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 1: // E - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.e = new NotAliveException(); - this.e.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_success()) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - oprot.writeString(this.success); - oprot.writeFieldEnd(); - } else if (this.is_set_e()) { - oprot.writeFieldBegin(E_FIELD_DESC); - this.e.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("getTopologyConf_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - if (!first) sb.append(", "); - sb.append("e:"); - if (this.e == null) { - sb.append("null"); - } else { - sb.append(this.e); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class getTopology_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getTopology_args"); - - private static final org.apache.thrift7.protocol.TField ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("id", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String id; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - ID((short)1, "id"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // ID - return ID; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.ID, new org.apache.thrift7.meta_data.FieldMetaData("id", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getTopology_args.class, metaDataMap); - } - - public getTopology_args() { - } - - public getTopology_args( - String id) - { - this(); - this.id = id; - } - - /** - * Performs a deep copy on other. - */ - public getTopology_args(getTopology_args other) { - if (other.is_set_id()) { - this.id = other.id; - } - } - - public getTopology_args deepCopy() { - return new getTopology_args(this); - } - - @Override - public void clear() { - this.id = null; - } - - public String get_id() { - return this.id; - } - - public void set_id(String id) { - this.id = id; - } - - public void unset_id() { - this.id = null; - } - - /** Returns true if field id is set (has been assigned a value) and false otherwise */ - public boolean is_set_id() { - return this.id != null; - } - - public void set_id_isSet(boolean value) { - if (!value) { - this.id = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case ID: - if (value == null) { - unset_id(); - } else { - set_id((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case ID: - return get_id(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case ID: - return is_set_id(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof getTopology_args) - return this.equals((getTopology_args)that); - return false; - } - - public boolean equals(getTopology_args that) { - if (that == null) - return false; - - boolean this_present_id = true && this.is_set_id(); - boolean that_present_id = true && that.is_set_id(); - if (this_present_id || that_present_id) { - if (!(this_present_id && that_present_id)) - return false; - if (!this.id.equals(that.id)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_id = true && (is_set_id()); - builder.append(present_id); - if (present_id) - builder.append(id); - - return builder.toHashCode(); - } - - public int compareTo(getTopology_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - getTopology_args typedOther = (getTopology_args)other; - - lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_id()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.id, typedOther.id); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // ID - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.id = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.id != null) { - oprot.writeFieldBegin(ID_FIELD_DESC); - oprot.writeString(this.id); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("getTopology_args("); - boolean first = true; - - sb.append("id:"); - if (this.id == null) { - sb.append("null"); - } else { - sb.append(this.id); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class getTopology_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getTopology_result"); - - private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRUCT, (short)0); - private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - - private StormTopology success; // required - private NotAliveException e; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - SUCCESS((short)0, "success"), - E((short)1, "e"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - case 1: // E - return E; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, StormTopology.class))); - tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getTopology_result.class, metaDataMap); - } - - public getTopology_result() { - } - - public getTopology_result( - StormTopology success, - NotAliveException e) - { - this(); - this.success = success; - this.e = e; - } - - /** - * Performs a deep copy on other. - */ - public getTopology_result(getTopology_result other) { - if (other.is_set_success()) { - this.success = new StormTopology(other.success); - } - if (other.is_set_e()) { - this.e = new NotAliveException(other.e); - } - } - - public getTopology_result deepCopy() { - return new getTopology_result(this); - } - - @Override - public void clear() { - this.success = null; - this.e = null; - } - - public StormTopology get_success() { - return this.success; - } - - public void set_success(StormTopology success) { - this.success = success; - } - - public void unset_success() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean is_set_success() { - return this.success != null; - } - - public void set_success_isSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public NotAliveException get_e() { - return this.e; - } - - public void set_e(NotAliveException e) { - this.e = e; - } - - public void unset_e() { - this.e = null; - } - - /** Returns true if field e is set (has been assigned a value) and false otherwise */ - public boolean is_set_e() { - return this.e != null; - } - - public void set_e_isSet(boolean value) { - if (!value) { - this.e = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unset_success(); - } else { - set_success((StormTopology)value); - } - break; - - case E: - if (value == null) { - unset_e(); - } else { - set_e((NotAliveException)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return get_success(); - - case E: - return get_e(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return is_set_success(); - case E: - return is_set_e(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof getTopology_result) - return this.equals((getTopology_result)that); - return false; - } - - public boolean equals(getTopology_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.is_set_success(); - boolean that_present_success = true && that.is_set_success(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - boolean this_present_e = true && this.is_set_e(); - boolean that_present_e = true && that.is_set_e(); - if (this_present_e || that_present_e) { - if (!(this_present_e && that_present_e)) - return false; - if (!this.e.equals(that.e)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (is_set_success()); - builder.append(present_success); - if (present_success) - builder.append(success); - - boolean present_e = true && (is_set_e()); - builder.append(present_e); - if (present_e) - builder.append(e); - - return builder.toHashCode(); - } - - public int compareTo(getTopology_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - getTopology_result typedOther = (getTopology_result)other; - - lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_success()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_e()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 0: // SUCCESS - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.success = new StormTopology(); - this.success.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 1: // E - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.e = new NotAliveException(); - this.e.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_success()) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - this.success.write(oprot); - oprot.writeFieldEnd(); - } else if (this.is_set_e()) { - oprot.writeFieldBegin(E_FIELD_DESC); - this.e.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("getTopology_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - if (!first) sb.append(", "); - sb.append("e:"); - if (this.e == null) { - sb.append("null"); - } else { - sb.append(this.e); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class getUserTopology_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getUserTopology_args"); - - private static final org.apache.thrift7.protocol.TField ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("id", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String id; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - ID((short)1, "id"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // ID - return ID; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.ID, new org.apache.thrift7.meta_data.FieldMetaData("id", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getUserTopology_args.class, metaDataMap); - } - - public getUserTopology_args() { - } - - public getUserTopology_args( - String id) - { - this(); - this.id = id; - } - - /** - * Performs a deep copy on other. - */ - public getUserTopology_args(getUserTopology_args other) { - if (other.is_set_id()) { - this.id = other.id; - } - } - - public getUserTopology_args deepCopy() { - return new getUserTopology_args(this); - } - - @Override - public void clear() { - this.id = null; - } - - public String get_id() { - return this.id; - } - - public void set_id(String id) { - this.id = id; - } - - public void unset_id() { - this.id = null; - } - - /** Returns true if field id is set (has been assigned a value) and false otherwise */ - public boolean is_set_id() { - return this.id != null; - } - - public void set_id_isSet(boolean value) { - if (!value) { - this.id = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case ID: - if (value == null) { - unset_id(); - } else { - set_id((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case ID: - return get_id(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case ID: - return is_set_id(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof getUserTopology_args) - return this.equals((getUserTopology_args)that); - return false; - } - - public boolean equals(getUserTopology_args that) { - if (that == null) - return false; - - boolean this_present_id = true && this.is_set_id(); - boolean that_present_id = true && that.is_set_id(); - if (this_present_id || that_present_id) { - if (!(this_present_id && that_present_id)) - return false; - if (!this.id.equals(that.id)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_id = true && (is_set_id()); - builder.append(present_id); - if (present_id) - builder.append(id); - - return builder.toHashCode(); - } - - public int compareTo(getUserTopology_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - getUserTopology_args typedOther = (getUserTopology_args)other; - - lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_id()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.id, typedOther.id); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // ID - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.id = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.id != null) { - oprot.writeFieldBegin(ID_FIELD_DESC); - oprot.writeString(this.id); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("getUserTopology_args("); - boolean first = true; - - sb.append("id:"); - if (this.id == null) { - sb.append("null"); - } else { - sb.append(this.id); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class getUserTopology_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getUserTopology_result"); - - private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRUCT, (short)0); - private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - - private StormTopology success; // required - private NotAliveException e; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - SUCCESS((short)0, "success"), - E((short)1, "e"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - case 1: // E - return E; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, StormTopology.class))); - tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getUserTopology_result.class, metaDataMap); - } - - public getUserTopology_result() { - } - - public getUserTopology_result( - StormTopology success, - NotAliveException e) - { - this(); - this.success = success; - this.e = e; - } - - /** - * Performs a deep copy on other. - */ - public getUserTopology_result(getUserTopology_result other) { - if (other.is_set_success()) { - this.success = new StormTopology(other.success); - } - if (other.is_set_e()) { - this.e = new NotAliveException(other.e); - } - } - - public getUserTopology_result deepCopy() { - return new getUserTopology_result(this); - } - - @Override - public void clear() { - this.success = null; - this.e = null; - } - - public StormTopology get_success() { - return this.success; - } - - public void set_success(StormTopology success) { - this.success = success; - } - - public void unset_success() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean is_set_success() { - return this.success != null; - } - - public void set_success_isSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public NotAliveException get_e() { - return this.e; - } - - public void set_e(NotAliveException e) { - this.e = e; - } - - public void unset_e() { - this.e = null; - } - - /** Returns true if field e is set (has been assigned a value) and false otherwise */ - public boolean is_set_e() { - return this.e != null; - } - - public void set_e_isSet(boolean value) { - if (!value) { - this.e = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unset_success(); - } else { - set_success((StormTopology)value); - } - break; - - case E: - if (value == null) { - unset_e(); - } else { - set_e((NotAliveException)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return get_success(); - - case E: - return get_e(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return is_set_success(); - case E: - return is_set_e(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof getUserTopology_result) - return this.equals((getUserTopology_result)that); - return false; - } - - public boolean equals(getUserTopology_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.is_set_success(); - boolean that_present_success = true && that.is_set_success(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - boolean this_present_e = true && this.is_set_e(); - boolean that_present_e = true && that.is_set_e(); - if (this_present_e || that_present_e) { - if (!(this_present_e && that_present_e)) - return false; - if (!this.e.equals(that.e)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (is_set_success()); - builder.append(present_success); - if (present_success) - builder.append(success); - - boolean present_e = true && (is_set_e()); - builder.append(present_e); - if (present_e) - builder.append(e); - - return builder.toHashCode(); - } - - public int compareTo(getUserTopology_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - getUserTopology_result typedOther = (getUserTopology_result)other; - - lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_success()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_e()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 0: // SUCCESS - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.success = new StormTopology(); - this.success.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 1: // E - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.e = new NotAliveException(); - this.e.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_success()) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - this.success.write(oprot); - oprot.writeFieldEnd(); - } else if (this.is_set_e()) { - oprot.writeFieldBegin(E_FIELD_DESC); - this.e.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("getUserTopology_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - if (!first) sb.append(", "); - sb.append("e:"); - if (this.e == null) { - sb.append("null"); - } else { - sb.append(this.e); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class getTopologyMetric_args implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getTopologyMetric_args"); - - private static final org.apache.thrift7.protocol.TField ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("id", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String id; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - ID((short)1, "id"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // ID - return ID; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.ID, new org.apache.thrift7.meta_data.FieldMetaData("id", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyMetric_args.class, metaDataMap); - } - - public getTopologyMetric_args() { - } - - public getTopologyMetric_args( - String id) - { - this(); - this.id = id; - } - - /** - * Performs a deep copy on other. - */ - public getTopologyMetric_args(getTopologyMetric_args other) { - if (other.is_set_id()) { - this.id = other.id; - } - } - - public getTopologyMetric_args deepCopy() { - return new getTopologyMetric_args(this); - } - - @Override - public void clear() { - this.id = null; - } - - public String get_id() { - return this.id; - } - - public void set_id(String id) { - this.id = id; - } - - public void unset_id() { - this.id = null; - } - - /** Returns true if field id is set (has been assigned a value) and false otherwise */ - public boolean is_set_id() { - return this.id != null; - } - - public void set_id_isSet(boolean value) { - if (!value) { - this.id = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case ID: - if (value == null) { - unset_id(); - } else { - set_id((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case ID: - return get_id(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case ID: - return is_set_id(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof getTopologyMetric_args) - return this.equals((getTopologyMetric_args)that); - return false; - } - - public boolean equals(getTopologyMetric_args that) { - if (that == null) - return false; - - boolean this_present_id = true && this.is_set_id(); - boolean that_present_id = true && that.is_set_id(); - if (this_present_id || that_present_id) { - if (!(this_present_id && that_present_id)) - return false; - if (!this.id.equals(that.id)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_id = true && (is_set_id()); - builder.append(present_id); - if (present_id) - builder.append(id); - - return builder.toHashCode(); - } - - public int compareTo(getTopologyMetric_args other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - getTopologyMetric_args typedOther = (getTopologyMetric_args)other; - - lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_id()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.id, typedOther.id); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // ID - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.id = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.id != null) { - oprot.writeFieldBegin(ID_FIELD_DESC); - oprot.writeString(this.id); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("getTopologyMetric_args("); - boolean first = true; - - sb.append("id:"); - if (this.id == null) { - sb.append("null"); - } else { - sb.append(this.id); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - - public static class getTopologyMetric_result implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("getTopologyMetric_result"); - - private static final org.apache.thrift7.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift7.protocol.TField("success", org.apache.thrift7.protocol.TType.STRUCT, (short)0); - private static final org.apache.thrift7.protocol.TField E_FIELD_DESC = new org.apache.thrift7.protocol.TField("e", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - - private TopologyMetricInfo success; // required - private NotAliveException e; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - SUCCESS((short)0, "success"), - E((short)1, "e"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 0: // SUCCESS - return SUCCESS; - case 1: // E - return E; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUCCESS, new org.apache.thrift7.meta_data.FieldMetaData("success", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, TopologyMetricInfo.class))); - tmpMap.put(_Fields.E, new org.apache.thrift7.meta_data.FieldMetaData("e", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRUCT))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyMetric_result.class, metaDataMap); - } - - public getTopologyMetric_result() { - } - - public getTopologyMetric_result( - TopologyMetricInfo success, - NotAliveException e) - { - this(); - this.success = success; - this.e = e; - } - - /** - * Performs a deep copy on other. - */ - public getTopologyMetric_result(getTopologyMetric_result other) { - if (other.is_set_success()) { - this.success = new TopologyMetricInfo(other.success); - } - if (other.is_set_e()) { - this.e = new NotAliveException(other.e); - } - } - - public getTopologyMetric_result deepCopy() { - return new getTopologyMetric_result(this); - } - - @Override - public void clear() { - this.success = null; - this.e = null; - } - - public TopologyMetricInfo get_success() { - return this.success; - } - - public void set_success(TopologyMetricInfo success) { - this.success = success; - } - - public void unset_success() { - this.success = null; - } - - /** Returns true if field success is set (has been assigned a value) and false otherwise */ - public boolean is_set_success() { - return this.success != null; - } - - public void set_success_isSet(boolean value) { - if (!value) { - this.success = null; - } - } - - public NotAliveException get_e() { - return this.e; - } - - public void set_e(NotAliveException e) { - this.e = e; - } - - public void unset_e() { - this.e = null; - } - - /** Returns true if field e is set (has been assigned a value) and false otherwise */ - public boolean is_set_e() { - return this.e != null; - } - - public void set_e_isSet(boolean value) { - if (!value) { - this.e = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUCCESS: - if (value == null) { - unset_success(); - } else { - set_success((TopologyMetricInfo)value); - } - break; - - case E: - if (value == null) { - unset_e(); - } else { - set_e((NotAliveException)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUCCESS: - return get_success(); - - case E: - return get_e(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUCCESS: - return is_set_success(); - case E: - return is_set_e(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof getTopologyMetric_result) - return this.equals((getTopologyMetric_result)that); - return false; - } - - public boolean equals(getTopologyMetric_result that) { - if (that == null) - return false; - - boolean this_present_success = true && this.is_set_success(); - boolean that_present_success = true && that.is_set_success(); - if (this_present_success || that_present_success) { - if (!(this_present_success && that_present_success)) - return false; - if (!this.success.equals(that.success)) - return false; - } - - boolean this_present_e = true && this.is_set_e(); - boolean that_present_e = true && that.is_set_e(); - if (this_present_e || that_present_e) { - if (!(this_present_e && that_present_e)) - return false; - if (!this.e.equals(that.e)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_success = true && (is_set_success()); - builder.append(present_success); - if (present_success) - builder.append(success); - - boolean present_e = true && (is_set_e()); - builder.append(present_e); - if (present_e) - builder.append(e); - - return builder.toHashCode(); - } - - public int compareTo(getTopologyMetric_result other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - getTopologyMetric_result typedOther = (getTopologyMetric_result)other; - - lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_success()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.success, typedOther.success); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_e()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.e, typedOther.e); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 0: // SUCCESS - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.success = new TopologyMetricInfo(); - this.success.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 1: // E - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.e = new NotAliveException(); - this.e.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - oprot.writeStructBegin(STRUCT_DESC); - - if (this.is_set_success()) { - oprot.writeFieldBegin(SUCCESS_FIELD_DESC); - this.success.write(oprot); - oprot.writeFieldEnd(); - } else if (this.is_set_e()) { - oprot.writeFieldBegin(E_FIELD_DESC); - this.e.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("getTopologyMetric_result("); - boolean first = true; - - sb.append("success:"); - if (this.success == null) { - sb.append("null"); - } else { - sb.append(this.success); - } - first = false; - if (!first) sb.append(", "); - sb.append("e:"); - if (this.e == null) { - sb.append("null"); - } else { - sb.append(this.e); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/generated/NotAliveException.java b/jstorm-client/src/main/java/backtype/storm/generated/NotAliveException.java deleted file mode 100644 index 30d18ed2b..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/NotAliveException.java +++ /dev/null @@ -1,328 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class NotAliveException extends Exception implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("NotAliveException"); - - private static final org.apache.thrift7.protocol.TField MSG_FIELD_DESC = new org.apache.thrift7.protocol.TField("msg", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String msg; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - MSG((short)1, "msg"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // MSG - return MSG; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.MSG, new org.apache.thrift7.meta_data.FieldMetaData("msg", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(NotAliveException.class, metaDataMap); - } - - public NotAliveException() { - } - - public NotAliveException( - String msg) - { - this(); - this.msg = msg; - } - - /** - * Performs a deep copy on other. - */ - public NotAliveException(NotAliveException other) { - if (other.is_set_msg()) { - this.msg = other.msg; - } - } - - public NotAliveException deepCopy() { - return new NotAliveException(this); - } - - @Override - public void clear() { - this.msg = null; - } - - public String get_msg() { - return this.msg; - } - - public void set_msg(String msg) { - this.msg = msg; - } - - public void unset_msg() { - this.msg = null; - } - - /** Returns true if field msg is set (has been assigned a value) and false otherwise */ - public boolean is_set_msg() { - return this.msg != null; - } - - public void set_msg_isSet(boolean value) { - if (!value) { - this.msg = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case MSG: - if (value == null) { - unset_msg(); - } else { - set_msg((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case MSG: - return get_msg(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case MSG: - return is_set_msg(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof NotAliveException) - return this.equals((NotAliveException)that); - return false; - } - - public boolean equals(NotAliveException that) { - if (that == null) - return false; - - boolean this_present_msg = true && this.is_set_msg(); - boolean that_present_msg = true && that.is_set_msg(); - if (this_present_msg || that_present_msg) { - if (!(this_present_msg && that_present_msg)) - return false; - if (!this.msg.equals(that.msg)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_msg = true && (is_set_msg()); - builder.append(present_msg); - if (present_msg) - builder.append(msg); - - return builder.toHashCode(); - } - - public int compareTo(NotAliveException other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - NotAliveException typedOther = (NotAliveException)other; - - lastComparison = Boolean.valueOf(is_set_msg()).compareTo(typedOther.is_set_msg()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_msg()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.msg, typedOther.msg); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // MSG - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.msg = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.msg != null) { - oprot.writeFieldBegin(MSG_FIELD_DESC); - oprot.writeString(this.msg); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("NotAliveException("); - boolean first = true; - - sb.append("msg:"); - if (this.msg == null) { - sb.append("null"); - } else { - sb.append(this.msg); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_msg()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/NullStruct.java b/jstorm-client/src/main/java/backtype/storm/generated/NullStruct.java deleted file mode 100644 index ce89767dd..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/NullStruct.java +++ /dev/null @@ -1,225 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class NullStruct implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("NullStruct"); - - - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { -; - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(NullStruct.class, metaDataMap); - } - - public NullStruct() { - } - - /** - * Performs a deep copy on other. - */ - public NullStruct(NullStruct other) { - } - - public NullStruct deepCopy() { - return new NullStruct(this); - } - - @Override - public void clear() { - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof NullStruct) - return this.equals((NullStruct)that); - return false; - } - - public boolean equals(NullStruct that) { - if (that == null) - return false; - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - return builder.toHashCode(); - } - - public int compareTo(NullStruct other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - NullStruct typedOther = (NullStruct)other; - - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("NullStruct("); - boolean first = true; - - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/RebalanceOptions.java b/jstorm-client/src/main/java/backtype/storm/generated/RebalanceOptions.java deleted file mode 100644 index dde038d1a..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/RebalanceOptions.java +++ /dev/null @@ -1,409 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class RebalanceOptions implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("RebalanceOptions"); - - private static final org.apache.thrift7.protocol.TField WAIT_SECS_FIELD_DESC = new org.apache.thrift7.protocol.TField("wait_secs", org.apache.thrift7.protocol.TType.I32, (short)1); - private static final org.apache.thrift7.protocol.TField NUM_WORKERS_FIELD_DESC = new org.apache.thrift7.protocol.TField("num_workers", org.apache.thrift7.protocol.TType.I32, (short)2); - - private int wait_secs; // required - private int num_workers; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - WAIT_SECS((short)1, "wait_secs"), - NUM_WORKERS((short)2, "num_workers"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // WAIT_SECS - return WAIT_SECS; - case 2: // NUM_WORKERS - return NUM_WORKERS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __WAIT_SECS_ISSET_ID = 0; - private static final int __NUM_WORKERS_ISSET_ID = 1; - private BitSet __isset_bit_vector = new BitSet(2); - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.WAIT_SECS, new org.apache.thrift7.meta_data.FieldMetaData("wait_secs", org.apache.thrift7.TFieldRequirementType.OPTIONAL, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - tmpMap.put(_Fields.NUM_WORKERS, new org.apache.thrift7.meta_data.FieldMetaData("num_workers", org.apache.thrift7.TFieldRequirementType.OPTIONAL, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(RebalanceOptions.class, metaDataMap); - } - - public RebalanceOptions() { - } - - /** - * Performs a deep copy on other. - */ - public RebalanceOptions(RebalanceOptions other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); - this.wait_secs = other.wait_secs; - this.num_workers = other.num_workers; - } - - public RebalanceOptions deepCopy() { - return new RebalanceOptions(this); - } - - @Override - public void clear() { - set_wait_secs_isSet(false); - this.wait_secs = 0; - set_num_workers_isSet(false); - this.num_workers = 0; - } - - public int get_wait_secs() { - return this.wait_secs; - } - - public void set_wait_secs(int wait_secs) { - this.wait_secs = wait_secs; - set_wait_secs_isSet(true); - } - - public void unset_wait_secs() { - __isset_bit_vector.clear(__WAIT_SECS_ISSET_ID); - } - - /** Returns true if field wait_secs is set (has been assigned a value) and false otherwise */ - public boolean is_set_wait_secs() { - return __isset_bit_vector.get(__WAIT_SECS_ISSET_ID); - } - - public void set_wait_secs_isSet(boolean value) { - __isset_bit_vector.set(__WAIT_SECS_ISSET_ID, value); - } - - public int get_num_workers() { - return this.num_workers; - } - - public void set_num_workers(int num_workers) { - this.num_workers = num_workers; - set_num_workers_isSet(true); - } - - public void unset_num_workers() { - __isset_bit_vector.clear(__NUM_WORKERS_ISSET_ID); - } - - /** Returns true if field num_workers is set (has been assigned a value) and false otherwise */ - public boolean is_set_num_workers() { - return __isset_bit_vector.get(__NUM_WORKERS_ISSET_ID); - } - - public void set_num_workers_isSet(boolean value) { - __isset_bit_vector.set(__NUM_WORKERS_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case WAIT_SECS: - if (value == null) { - unset_wait_secs(); - } else { - set_wait_secs((Integer)value); - } - break; - - case NUM_WORKERS: - if (value == null) { - unset_num_workers(); - } else { - set_num_workers((Integer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case WAIT_SECS: - return Integer.valueOf(get_wait_secs()); - - case NUM_WORKERS: - return Integer.valueOf(get_num_workers()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case WAIT_SECS: - return is_set_wait_secs(); - case NUM_WORKERS: - return is_set_num_workers(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof RebalanceOptions) - return this.equals((RebalanceOptions)that); - return false; - } - - public boolean equals(RebalanceOptions that) { - if (that == null) - return false; - - boolean this_present_wait_secs = true && this.is_set_wait_secs(); - boolean that_present_wait_secs = true && that.is_set_wait_secs(); - if (this_present_wait_secs || that_present_wait_secs) { - if (!(this_present_wait_secs && that_present_wait_secs)) - return false; - if (this.wait_secs != that.wait_secs) - return false; - } - - boolean this_present_num_workers = true && this.is_set_num_workers(); - boolean that_present_num_workers = true && that.is_set_num_workers(); - if (this_present_num_workers || that_present_num_workers) { - if (!(this_present_num_workers && that_present_num_workers)) - return false; - if (this.num_workers != that.num_workers) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_wait_secs = true && (is_set_wait_secs()); - builder.append(present_wait_secs); - if (present_wait_secs) - builder.append(wait_secs); - - boolean present_num_workers = true && (is_set_num_workers()); - builder.append(present_num_workers); - if (present_num_workers) - builder.append(num_workers); - - return builder.toHashCode(); - } - - public int compareTo(RebalanceOptions other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - RebalanceOptions typedOther = (RebalanceOptions)other; - - lastComparison = Boolean.valueOf(is_set_wait_secs()).compareTo(typedOther.is_set_wait_secs()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_wait_secs()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.wait_secs, typedOther.wait_secs); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_num_workers()).compareTo(typedOther.is_set_num_workers()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_num_workers()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.num_workers, typedOther.num_workers); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // WAIT_SECS - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.wait_secs = iprot.readI32(); - set_wait_secs_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // NUM_WORKERS - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.num_workers = iprot.readI32(); - set_num_workers_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (is_set_wait_secs()) { - oprot.writeFieldBegin(WAIT_SECS_FIELD_DESC); - oprot.writeI32(this.wait_secs); - oprot.writeFieldEnd(); - } - if (is_set_num_workers()) { - oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC); - oprot.writeI32(this.num_workers); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("RebalanceOptions("); - boolean first = true; - - if (is_set_wait_secs()) { - sb.append("wait_secs:"); - sb.append(this.wait_secs); - first = false; - } - if (is_set_num_workers()) { - if (!first) sb.append(", "); - sb.append("num_workers:"); - sb.append(this.num_workers); - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bit_vector = new BitSet(1); - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/ShellComponent.java b/jstorm-client/src/main/java/backtype/storm/generated/ShellComponent.java deleted file mode 100644 index 58fef88d7..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/ShellComponent.java +++ /dev/null @@ -1,417 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class ShellComponent implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("ShellComponent"); - - private static final org.apache.thrift7.protocol.TField EXECUTION_COMMAND_FIELD_DESC = new org.apache.thrift7.protocol.TField("execution_command", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField SCRIPT_FIELD_DESC = new org.apache.thrift7.protocol.TField("script", org.apache.thrift7.protocol.TType.STRING, (short)2); - - private String execution_command; // required - private String script; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - EXECUTION_COMMAND((short)1, "execution_command"), - SCRIPT((short)2, "script"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // EXECUTION_COMMAND - return EXECUTION_COMMAND; - case 2: // SCRIPT - return SCRIPT; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.EXECUTION_COMMAND, new org.apache.thrift7.meta_data.FieldMetaData("execution_command", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.SCRIPT, new org.apache.thrift7.meta_data.FieldMetaData("script", org.apache.thrift7.TFieldRequirementType.DEFAULT, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(ShellComponent.class, metaDataMap); - } - - public ShellComponent() { - } - - public ShellComponent( - String execution_command, - String script) - { - this(); - this.execution_command = execution_command; - this.script = script; - } - - /** - * Performs a deep copy on other. - */ - public ShellComponent(ShellComponent other) { - if (other.is_set_execution_command()) { - this.execution_command = other.execution_command; - } - if (other.is_set_script()) { - this.script = other.script; - } - } - - public ShellComponent deepCopy() { - return new ShellComponent(this); - } - - @Override - public void clear() { - this.execution_command = null; - this.script = null; - } - - public String get_execution_command() { - return this.execution_command; - } - - public void set_execution_command(String execution_command) { - this.execution_command = execution_command; - } - - public void unset_execution_command() { - this.execution_command = null; - } - - /** Returns true if field execution_command is set (has been assigned a value) and false otherwise */ - public boolean is_set_execution_command() { - return this.execution_command != null; - } - - public void set_execution_command_isSet(boolean value) { - if (!value) { - this.execution_command = null; - } - } - - public String get_script() { - return this.script; - } - - public void set_script(String script) { - this.script = script; - } - - public void unset_script() { - this.script = null; - } - - /** Returns true if field script is set (has been assigned a value) and false otherwise */ - public boolean is_set_script() { - return this.script != null; - } - - public void set_script_isSet(boolean value) { - if (!value) { - this.script = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case EXECUTION_COMMAND: - if (value == null) { - unset_execution_command(); - } else { - set_execution_command((String)value); - } - break; - - case SCRIPT: - if (value == null) { - unset_script(); - } else { - set_script((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case EXECUTION_COMMAND: - return get_execution_command(); - - case SCRIPT: - return get_script(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case EXECUTION_COMMAND: - return is_set_execution_command(); - case SCRIPT: - return is_set_script(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof ShellComponent) - return this.equals((ShellComponent)that); - return false; - } - - public boolean equals(ShellComponent that) { - if (that == null) - return false; - - boolean this_present_execution_command = true && this.is_set_execution_command(); - boolean that_present_execution_command = true && that.is_set_execution_command(); - if (this_present_execution_command || that_present_execution_command) { - if (!(this_present_execution_command && that_present_execution_command)) - return false; - if (!this.execution_command.equals(that.execution_command)) - return false; - } - - boolean this_present_script = true && this.is_set_script(); - boolean that_present_script = true && that.is_set_script(); - if (this_present_script || that_present_script) { - if (!(this_present_script && that_present_script)) - return false; - if (!this.script.equals(that.script)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_execution_command = true && (is_set_execution_command()); - builder.append(present_execution_command); - if (present_execution_command) - builder.append(execution_command); - - boolean present_script = true && (is_set_script()); - builder.append(present_script); - if (present_script) - builder.append(script); - - return builder.toHashCode(); - } - - public int compareTo(ShellComponent other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - ShellComponent typedOther = (ShellComponent)other; - - lastComparison = Boolean.valueOf(is_set_execution_command()).compareTo(typedOther.is_set_execution_command()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_execution_command()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.execution_command, typedOther.execution_command); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_script()).compareTo(typedOther.is_set_script()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_script()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.script, typedOther.script); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // EXECUTION_COMMAND - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.execution_command = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // SCRIPT - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.script = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.execution_command != null) { - oprot.writeFieldBegin(EXECUTION_COMMAND_FIELD_DESC); - oprot.writeString(this.execution_command); - oprot.writeFieldEnd(); - } - if (this.script != null) { - oprot.writeFieldBegin(SCRIPT_FIELD_DESC); - oprot.writeString(this.script); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("ShellComponent("); - boolean first = true; - - sb.append("execution_command:"); - if (this.execution_command == null) { - sb.append("null"); - } else { - sb.append(this.execution_command); - } - first = false; - if (!first) sb.append(", "); - sb.append("script:"); - if (this.script == null) { - sb.append("null"); - } else { - sb.append(this.script); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/SpoutSpec.java b/jstorm-client/src/main/java/backtype/storm/generated/SpoutSpec.java deleted file mode 100644 index 60f6d4f0b..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/SpoutSpec.java +++ /dev/null @@ -1,427 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class SpoutSpec implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("SpoutSpec"); - - private static final org.apache.thrift7.protocol.TField SPOUT_OBJECT_FIELD_DESC = new org.apache.thrift7.protocol.TField("spout_object", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift7.protocol.TField COMMON_FIELD_DESC = new org.apache.thrift7.protocol.TField("common", org.apache.thrift7.protocol.TType.STRUCT, (short)2); - - private ComponentObject spout_object; // required - private ComponentCommon common; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - SPOUT_OBJECT((short)1, "spout_object"), - COMMON((short)2, "common"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SPOUT_OBJECT - return SPOUT_OBJECT; - case 2: // COMMON - return COMMON; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SPOUT_OBJECT, new org.apache.thrift7.meta_data.FieldMetaData("spout_object", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ComponentObject.class))); - tmpMap.put(_Fields.COMMON, new org.apache.thrift7.meta_data.FieldMetaData("common", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ComponentCommon.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(SpoutSpec.class, metaDataMap); - } - - public SpoutSpec() { - } - - public SpoutSpec( - ComponentObject spout_object, - ComponentCommon common) - { - this(); - this.spout_object = spout_object; - this.common = common; - } - - /** - * Performs a deep copy on other. - */ - public SpoutSpec(SpoutSpec other) { - if (other.is_set_spout_object()) { - this.spout_object = new ComponentObject(other.spout_object); - } - if (other.is_set_common()) { - this.common = new ComponentCommon(other.common); - } - } - - public SpoutSpec deepCopy() { - return new SpoutSpec(this); - } - - @Override - public void clear() { - this.spout_object = null; - this.common = null; - } - - public ComponentObject get_spout_object() { - return this.spout_object; - } - - public void set_spout_object(ComponentObject spout_object) { - this.spout_object = spout_object; - } - - public void unset_spout_object() { - this.spout_object = null; - } - - /** Returns true if field spout_object is set (has been assigned a value) and false otherwise */ - public boolean is_set_spout_object() { - return this.spout_object != null; - } - - public void set_spout_object_isSet(boolean value) { - if (!value) { - this.spout_object = null; - } - } - - public ComponentCommon get_common() { - return this.common; - } - - public void set_common(ComponentCommon common) { - this.common = common; - } - - public void unset_common() { - this.common = null; - } - - /** Returns true if field common is set (has been assigned a value) and false otherwise */ - public boolean is_set_common() { - return this.common != null; - } - - public void set_common_isSet(boolean value) { - if (!value) { - this.common = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SPOUT_OBJECT: - if (value == null) { - unset_spout_object(); - } else { - set_spout_object((ComponentObject)value); - } - break; - - case COMMON: - if (value == null) { - unset_common(); - } else { - set_common((ComponentCommon)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SPOUT_OBJECT: - return get_spout_object(); - - case COMMON: - return get_common(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SPOUT_OBJECT: - return is_set_spout_object(); - case COMMON: - return is_set_common(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof SpoutSpec) - return this.equals((SpoutSpec)that); - return false; - } - - public boolean equals(SpoutSpec that) { - if (that == null) - return false; - - boolean this_present_spout_object = true && this.is_set_spout_object(); - boolean that_present_spout_object = true && that.is_set_spout_object(); - if (this_present_spout_object || that_present_spout_object) { - if (!(this_present_spout_object && that_present_spout_object)) - return false; - if (!this.spout_object.equals(that.spout_object)) - return false; - } - - boolean this_present_common = true && this.is_set_common(); - boolean that_present_common = true && that.is_set_common(); - if (this_present_common || that_present_common) { - if (!(this_present_common && that_present_common)) - return false; - if (!this.common.equals(that.common)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_spout_object = true && (is_set_spout_object()); - builder.append(present_spout_object); - if (present_spout_object) - builder.append(spout_object); - - boolean present_common = true && (is_set_common()); - builder.append(present_common); - if (present_common) - builder.append(common); - - return builder.toHashCode(); - } - - public int compareTo(SpoutSpec other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - SpoutSpec typedOther = (SpoutSpec)other; - - lastComparison = Boolean.valueOf(is_set_spout_object()).compareTo(typedOther.is_set_spout_object()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_spout_object()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.spout_object, typedOther.spout_object); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_common()).compareTo(typedOther.is_set_common()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_common()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.common, typedOther.common); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // SPOUT_OBJECT - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.spout_object = new ComponentObject(); - this.spout_object.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // COMMON - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.common = new ComponentCommon(); - this.common.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.spout_object != null) { - oprot.writeFieldBegin(SPOUT_OBJECT_FIELD_DESC); - this.spout_object.write(oprot); - oprot.writeFieldEnd(); - } - if (this.common != null) { - oprot.writeFieldBegin(COMMON_FIELD_DESC); - this.common.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("SpoutSpec("); - boolean first = true; - - sb.append("spout_object:"); - if (this.spout_object == null) { - sb.append("null"); - } else { - sb.append(this.spout_object); - } - first = false; - if (!first) sb.append(", "); - sb.append("common:"); - if (this.common == null) { - sb.append("null"); - } else { - sb.append(this.common); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_spout_object()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'spout_object' is unset! Struct:" + toString()); - } - - if (!is_set_common()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'common' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/SpoutStats.java b/jstorm-client/src/main/java/backtype/storm/generated/SpoutStats.java deleted file mode 100644 index cad95c1d6..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/SpoutStats.java +++ /dev/null @@ -1,756 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class SpoutStats implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("SpoutStats"); - - private static final org.apache.thrift7.protocol.TField ACKED_FIELD_DESC = new org.apache.thrift7.protocol.TField("acked", org.apache.thrift7.protocol.TType.MAP, (short)1); - private static final org.apache.thrift7.protocol.TField FAILED_FIELD_DESC = new org.apache.thrift7.protocol.TField("failed", org.apache.thrift7.protocol.TType.MAP, (short)2); - private static final org.apache.thrift7.protocol.TField COMPLETE_MS_AVG_FIELD_DESC = new org.apache.thrift7.protocol.TField("complete_ms_avg", org.apache.thrift7.protocol.TType.MAP, (short)3); - - private Map> acked; // required - private Map> failed; // required - private Map> complete_ms_avg; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - ACKED((short)1, "acked"), - FAILED((short)2, "failed"), - COMPLETE_MS_AVG((short)3, "complete_ms_avg"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // ACKED - return ACKED; - case 2: // FAILED - return FAILED; - case 3: // COMPLETE_MS_AVG - return COMPLETE_MS_AVG; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.ACKED, new org.apache.thrift7.meta_data.FieldMetaData("acked", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I64))))); - tmpMap.put(_Fields.FAILED, new org.apache.thrift7.meta_data.FieldMetaData("failed", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I64))))); - tmpMap.put(_Fields.COMPLETE_MS_AVG, new org.apache.thrift7.meta_data.FieldMetaData("complete_ms_avg", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE))))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(SpoutStats.class, metaDataMap); - } - - public SpoutStats() { - } - - public SpoutStats( - Map> acked, - Map> failed, - Map> complete_ms_avg) - { - this(); - this.acked = acked; - this.failed = failed; - this.complete_ms_avg = complete_ms_avg; - } - - /** - * Performs a deep copy on other. - */ - public SpoutStats(SpoutStats other) { - if (other.is_set_acked()) { - Map> __this__acked = new HashMap>(); - for (Map.Entry> other_element : other.acked.entrySet()) { - - String other_element_key = other_element.getKey(); - Map other_element_value = other_element.getValue(); - - String __this__acked_copy_key = other_element_key; - - Map __this__acked_copy_value = new HashMap(); - for (Map.Entry other_element_value_element : other_element_value.entrySet()) { - - String other_element_value_element_key = other_element_value_element.getKey(); - Long other_element_value_element_value = other_element_value_element.getValue(); - - String __this__acked_copy_value_copy_key = other_element_value_element_key; - - Long __this__acked_copy_value_copy_value = other_element_value_element_value; - - __this__acked_copy_value.put(__this__acked_copy_value_copy_key, __this__acked_copy_value_copy_value); - } - - __this__acked.put(__this__acked_copy_key, __this__acked_copy_value); - } - this.acked = __this__acked; - } - if (other.is_set_failed()) { - Map> __this__failed = new HashMap>(); - for (Map.Entry> other_element : other.failed.entrySet()) { - - String other_element_key = other_element.getKey(); - Map other_element_value = other_element.getValue(); - - String __this__failed_copy_key = other_element_key; - - Map __this__failed_copy_value = new HashMap(); - for (Map.Entry other_element_value_element : other_element_value.entrySet()) { - - String other_element_value_element_key = other_element_value_element.getKey(); - Long other_element_value_element_value = other_element_value_element.getValue(); - - String __this__failed_copy_value_copy_key = other_element_value_element_key; - - Long __this__failed_copy_value_copy_value = other_element_value_element_value; - - __this__failed_copy_value.put(__this__failed_copy_value_copy_key, __this__failed_copy_value_copy_value); - } - - __this__failed.put(__this__failed_copy_key, __this__failed_copy_value); - } - this.failed = __this__failed; - } - if (other.is_set_complete_ms_avg()) { - Map> __this__complete_ms_avg = new HashMap>(); - for (Map.Entry> other_element : other.complete_ms_avg.entrySet()) { - - String other_element_key = other_element.getKey(); - Map other_element_value = other_element.getValue(); - - String __this__complete_ms_avg_copy_key = other_element_key; - - Map __this__complete_ms_avg_copy_value = new HashMap(); - for (Map.Entry other_element_value_element : other_element_value.entrySet()) { - - String other_element_value_element_key = other_element_value_element.getKey(); - Double other_element_value_element_value = other_element_value_element.getValue(); - - String __this__complete_ms_avg_copy_value_copy_key = other_element_value_element_key; - - Double __this__complete_ms_avg_copy_value_copy_value = other_element_value_element_value; - - __this__complete_ms_avg_copy_value.put(__this__complete_ms_avg_copy_value_copy_key, __this__complete_ms_avg_copy_value_copy_value); - } - - __this__complete_ms_avg.put(__this__complete_ms_avg_copy_key, __this__complete_ms_avg_copy_value); - } - this.complete_ms_avg = __this__complete_ms_avg; - } - } - - public SpoutStats deepCopy() { - return new SpoutStats(this); - } - - @Override - public void clear() { - this.acked = null; - this.failed = null; - this.complete_ms_avg = null; - } - - public int get_acked_size() { - return (this.acked == null) ? 0 : this.acked.size(); - } - - public void put_to_acked(String key, Map val) { - if (this.acked == null) { - this.acked = new HashMap>(); - } - this.acked.put(key, val); - } - - public Map> get_acked() { - return this.acked; - } - - public void set_acked(Map> acked) { - this.acked = acked; - } - - public void unset_acked() { - this.acked = null; - } - - /** Returns true if field acked is set (has been assigned a value) and false otherwise */ - public boolean is_set_acked() { - return this.acked != null; - } - - public void set_acked_isSet(boolean value) { - if (!value) { - this.acked = null; - } - } - - public int get_failed_size() { - return (this.failed == null) ? 0 : this.failed.size(); - } - - public void put_to_failed(String key, Map val) { - if (this.failed == null) { - this.failed = new HashMap>(); - } - this.failed.put(key, val); - } - - public Map> get_failed() { - return this.failed; - } - - public void set_failed(Map> failed) { - this.failed = failed; - } - - public void unset_failed() { - this.failed = null; - } - - /** Returns true if field failed is set (has been assigned a value) and false otherwise */ - public boolean is_set_failed() { - return this.failed != null; - } - - public void set_failed_isSet(boolean value) { - if (!value) { - this.failed = null; - } - } - - public int get_complete_ms_avg_size() { - return (this.complete_ms_avg == null) ? 0 : this.complete_ms_avg.size(); - } - - public void put_to_complete_ms_avg(String key, Map val) { - if (this.complete_ms_avg == null) { - this.complete_ms_avg = new HashMap>(); - } - this.complete_ms_avg.put(key, val); - } - - public Map> get_complete_ms_avg() { - return this.complete_ms_avg; - } - - public void set_complete_ms_avg(Map> complete_ms_avg) { - this.complete_ms_avg = complete_ms_avg; - } - - public void unset_complete_ms_avg() { - this.complete_ms_avg = null; - } - - /** Returns true if field complete_ms_avg is set (has been assigned a value) and false otherwise */ - public boolean is_set_complete_ms_avg() { - return this.complete_ms_avg != null; - } - - public void set_complete_ms_avg_isSet(boolean value) { - if (!value) { - this.complete_ms_avg = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case ACKED: - if (value == null) { - unset_acked(); - } else { - set_acked((Map>)value); - } - break; - - case FAILED: - if (value == null) { - unset_failed(); - } else { - set_failed((Map>)value); - } - break; - - case COMPLETE_MS_AVG: - if (value == null) { - unset_complete_ms_avg(); - } else { - set_complete_ms_avg((Map>)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case ACKED: - return get_acked(); - - case FAILED: - return get_failed(); - - case COMPLETE_MS_AVG: - return get_complete_ms_avg(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case ACKED: - return is_set_acked(); - case FAILED: - return is_set_failed(); - case COMPLETE_MS_AVG: - return is_set_complete_ms_avg(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof SpoutStats) - return this.equals((SpoutStats)that); - return false; - } - - public boolean equals(SpoutStats that) { - if (that == null) - return false; - - boolean this_present_acked = true && this.is_set_acked(); - boolean that_present_acked = true && that.is_set_acked(); - if (this_present_acked || that_present_acked) { - if (!(this_present_acked && that_present_acked)) - return false; - if (!this.acked.equals(that.acked)) - return false; - } - - boolean this_present_failed = true && this.is_set_failed(); - boolean that_present_failed = true && that.is_set_failed(); - if (this_present_failed || that_present_failed) { - if (!(this_present_failed && that_present_failed)) - return false; - if (!this.failed.equals(that.failed)) - return false; - } - - boolean this_present_complete_ms_avg = true && this.is_set_complete_ms_avg(); - boolean that_present_complete_ms_avg = true && that.is_set_complete_ms_avg(); - if (this_present_complete_ms_avg || that_present_complete_ms_avg) { - if (!(this_present_complete_ms_avg && that_present_complete_ms_avg)) - return false; - if (!this.complete_ms_avg.equals(that.complete_ms_avg)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_acked = true && (is_set_acked()); - builder.append(present_acked); - if (present_acked) - builder.append(acked); - - boolean present_failed = true && (is_set_failed()); - builder.append(present_failed); - if (present_failed) - builder.append(failed); - - boolean present_complete_ms_avg = true && (is_set_complete_ms_avg()); - builder.append(present_complete_ms_avg); - if (present_complete_ms_avg) - builder.append(complete_ms_avg); - - return builder.toHashCode(); - } - - public int compareTo(SpoutStats other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - SpoutStats typedOther = (SpoutStats)other; - - lastComparison = Boolean.valueOf(is_set_acked()).compareTo(typedOther.is_set_acked()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_acked()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.acked, typedOther.acked); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_failed()).compareTo(typedOther.is_set_failed()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_failed()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.failed, typedOther.failed); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_complete_ms_avg()).compareTo(typedOther.is_set_complete_ms_avg()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_complete_ms_avg()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.complete_ms_avg, typedOther.complete_ms_avg); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // ACKED - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map95 = iprot.readMapBegin(); - this.acked = new HashMap>(2*_map95.size); - for (int _i96 = 0; _i96 < _map95.size; ++_i96) - { - String _key97; // required - Map _val98; // required - _key97 = iprot.readString(); - { - org.apache.thrift7.protocol.TMap _map99 = iprot.readMapBegin(); - _val98 = new HashMap(2*_map99.size); - for (int _i100 = 0; _i100 < _map99.size; ++_i100) - { - String _key101; // required - long _val102; // required - _key101 = iprot.readString(); - _val102 = iprot.readI64(); - _val98.put(_key101, _val102); - } - iprot.readMapEnd(); - } - this.acked.put(_key97, _val98); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // FAILED - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map103 = iprot.readMapBegin(); - this.failed = new HashMap>(2*_map103.size); - for (int _i104 = 0; _i104 < _map103.size; ++_i104) - { - String _key105; // required - Map _val106; // required - _key105 = iprot.readString(); - { - org.apache.thrift7.protocol.TMap _map107 = iprot.readMapBegin(); - _val106 = new HashMap(2*_map107.size); - for (int _i108 = 0; _i108 < _map107.size; ++_i108) - { - String _key109; // required - long _val110; // required - _key109 = iprot.readString(); - _val110 = iprot.readI64(); - _val106.put(_key109, _val110); - } - iprot.readMapEnd(); - } - this.failed.put(_key105, _val106); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 3: // COMPLETE_MS_AVG - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map111 = iprot.readMapBegin(); - this.complete_ms_avg = new HashMap>(2*_map111.size); - for (int _i112 = 0; _i112 < _map111.size; ++_i112) - { - String _key113; // required - Map _val114; // required - _key113 = iprot.readString(); - { - org.apache.thrift7.protocol.TMap _map115 = iprot.readMapBegin(); - _val114 = new HashMap(2*_map115.size); - for (int _i116 = 0; _i116 < _map115.size; ++_i116) - { - String _key117; // required - double _val118; // required - _key117 = iprot.readString(); - _val118 = iprot.readDouble(); - _val114.put(_key117, _val118); - } - iprot.readMapEnd(); - } - this.complete_ms_avg.put(_key113, _val114); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.acked != null) { - oprot.writeFieldBegin(ACKED_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.acked.size())); - for (Map.Entry> _iter119 : this.acked.entrySet()) - { - oprot.writeString(_iter119.getKey()); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.I64, _iter119.getValue().size())); - for (Map.Entry _iter120 : _iter119.getValue().entrySet()) - { - oprot.writeString(_iter120.getKey()); - oprot.writeI64(_iter120.getValue()); - } - oprot.writeMapEnd(); - } - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.failed != null) { - oprot.writeFieldBegin(FAILED_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.failed.size())); - for (Map.Entry> _iter121 : this.failed.entrySet()) - { - oprot.writeString(_iter121.getKey()); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.I64, _iter121.getValue().size())); - for (Map.Entry _iter122 : _iter121.getValue().entrySet()) - { - oprot.writeString(_iter122.getKey()); - oprot.writeI64(_iter122.getValue()); - } - oprot.writeMapEnd(); - } - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.complete_ms_avg != null) { - oprot.writeFieldBegin(COMPLETE_MS_AVG_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.complete_ms_avg.size())); - for (Map.Entry> _iter123 : this.complete_ms_avg.entrySet()) - { - oprot.writeString(_iter123.getKey()); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.DOUBLE, _iter123.getValue().size())); - for (Map.Entry _iter124 : _iter123.getValue().entrySet()) - { - oprot.writeString(_iter124.getKey()); - oprot.writeDouble(_iter124.getValue()); - } - oprot.writeMapEnd(); - } - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("SpoutStats("); - boolean first = true; - - sb.append("acked:"); - if (this.acked == null) { - sb.append("null"); - } else { - sb.append(this.acked); - } - first = false; - if (!first) sb.append(", "); - sb.append("failed:"); - if (this.failed == null) { - sb.append("null"); - } else { - sb.append(this.failed); - } - first = false; - if (!first) sb.append(", "); - sb.append("complete_ms_avg:"); - if (this.complete_ms_avg == null) { - sb.append("null"); - } else { - sb.append(this.complete_ms_avg); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_acked()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'acked' is unset! Struct:" + toString()); - } - - if (!is_set_failed()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'failed' is unset! Struct:" + toString()); - } - - if (!is_set_complete_ms_avg()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'complete_ms_avg' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/StateSpoutSpec.java b/jstorm-client/src/main/java/backtype/storm/generated/StateSpoutSpec.java deleted file mode 100644 index 4d1baf3b3..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/StateSpoutSpec.java +++ /dev/null @@ -1,427 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class StateSpoutSpec implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("StateSpoutSpec"); - - private static final org.apache.thrift7.protocol.TField STATE_SPOUT_OBJECT_FIELD_DESC = new org.apache.thrift7.protocol.TField("state_spout_object", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift7.protocol.TField COMMON_FIELD_DESC = new org.apache.thrift7.protocol.TField("common", org.apache.thrift7.protocol.TType.STRUCT, (short)2); - - private ComponentObject state_spout_object; // required - private ComponentCommon common; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - STATE_SPOUT_OBJECT((short)1, "state_spout_object"), - COMMON((short)2, "common"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // STATE_SPOUT_OBJECT - return STATE_SPOUT_OBJECT; - case 2: // COMMON - return COMMON; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.STATE_SPOUT_OBJECT, new org.apache.thrift7.meta_data.FieldMetaData("state_spout_object", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ComponentObject.class))); - tmpMap.put(_Fields.COMMON, new org.apache.thrift7.meta_data.FieldMetaData("common", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ComponentCommon.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(StateSpoutSpec.class, metaDataMap); - } - - public StateSpoutSpec() { - } - - public StateSpoutSpec( - ComponentObject state_spout_object, - ComponentCommon common) - { - this(); - this.state_spout_object = state_spout_object; - this.common = common; - } - - /** - * Performs a deep copy on other. - */ - public StateSpoutSpec(StateSpoutSpec other) { - if (other.is_set_state_spout_object()) { - this.state_spout_object = new ComponentObject(other.state_spout_object); - } - if (other.is_set_common()) { - this.common = new ComponentCommon(other.common); - } - } - - public StateSpoutSpec deepCopy() { - return new StateSpoutSpec(this); - } - - @Override - public void clear() { - this.state_spout_object = null; - this.common = null; - } - - public ComponentObject get_state_spout_object() { - return this.state_spout_object; - } - - public void set_state_spout_object(ComponentObject state_spout_object) { - this.state_spout_object = state_spout_object; - } - - public void unset_state_spout_object() { - this.state_spout_object = null; - } - - /** Returns true if field state_spout_object is set (has been assigned a value) and false otherwise */ - public boolean is_set_state_spout_object() { - return this.state_spout_object != null; - } - - public void set_state_spout_object_isSet(boolean value) { - if (!value) { - this.state_spout_object = null; - } - } - - public ComponentCommon get_common() { - return this.common; - } - - public void set_common(ComponentCommon common) { - this.common = common; - } - - public void unset_common() { - this.common = null; - } - - /** Returns true if field common is set (has been assigned a value) and false otherwise */ - public boolean is_set_common() { - return this.common != null; - } - - public void set_common_isSet(boolean value) { - if (!value) { - this.common = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case STATE_SPOUT_OBJECT: - if (value == null) { - unset_state_spout_object(); - } else { - set_state_spout_object((ComponentObject)value); - } - break; - - case COMMON: - if (value == null) { - unset_common(); - } else { - set_common((ComponentCommon)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case STATE_SPOUT_OBJECT: - return get_state_spout_object(); - - case COMMON: - return get_common(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case STATE_SPOUT_OBJECT: - return is_set_state_spout_object(); - case COMMON: - return is_set_common(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof StateSpoutSpec) - return this.equals((StateSpoutSpec)that); - return false; - } - - public boolean equals(StateSpoutSpec that) { - if (that == null) - return false; - - boolean this_present_state_spout_object = true && this.is_set_state_spout_object(); - boolean that_present_state_spout_object = true && that.is_set_state_spout_object(); - if (this_present_state_spout_object || that_present_state_spout_object) { - if (!(this_present_state_spout_object && that_present_state_spout_object)) - return false; - if (!this.state_spout_object.equals(that.state_spout_object)) - return false; - } - - boolean this_present_common = true && this.is_set_common(); - boolean that_present_common = true && that.is_set_common(); - if (this_present_common || that_present_common) { - if (!(this_present_common && that_present_common)) - return false; - if (!this.common.equals(that.common)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_state_spout_object = true && (is_set_state_spout_object()); - builder.append(present_state_spout_object); - if (present_state_spout_object) - builder.append(state_spout_object); - - boolean present_common = true && (is_set_common()); - builder.append(present_common); - if (present_common) - builder.append(common); - - return builder.toHashCode(); - } - - public int compareTo(StateSpoutSpec other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - StateSpoutSpec typedOther = (StateSpoutSpec)other; - - lastComparison = Boolean.valueOf(is_set_state_spout_object()).compareTo(typedOther.is_set_state_spout_object()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_state_spout_object()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.state_spout_object, typedOther.state_spout_object); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_common()).compareTo(typedOther.is_set_common()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_common()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.common, typedOther.common); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // STATE_SPOUT_OBJECT - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.state_spout_object = new ComponentObject(); - this.state_spout_object.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // COMMON - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.common = new ComponentCommon(); - this.common.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.state_spout_object != null) { - oprot.writeFieldBegin(STATE_SPOUT_OBJECT_FIELD_DESC); - this.state_spout_object.write(oprot); - oprot.writeFieldEnd(); - } - if (this.common != null) { - oprot.writeFieldBegin(COMMON_FIELD_DESC); - this.common.write(oprot); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("StateSpoutSpec("); - boolean first = true; - - sb.append("state_spout_object:"); - if (this.state_spout_object == null) { - sb.append("null"); - } else { - sb.append(this.state_spout_object); - } - first = false; - if (!first) sb.append(", "); - sb.append("common:"); - if (this.common == null) { - sb.append("null"); - } else { - sb.append(this.common); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_state_spout_object()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'state_spout_object' is unset! Struct:" + toString()); - } - - if (!is_set_common()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'common' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/StormTopology.java b/jstorm-client/src/main/java/backtype/storm/generated/StormTopology.java deleted file mode 100644 index bfe298483..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/StormTopology.java +++ /dev/null @@ -1,660 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class StormTopology implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("StormTopology"); - - private static final org.apache.thrift7.protocol.TField SPOUTS_FIELD_DESC = new org.apache.thrift7.protocol.TField("spouts", org.apache.thrift7.protocol.TType.MAP, (short)1); - private static final org.apache.thrift7.protocol.TField BOLTS_FIELD_DESC = new org.apache.thrift7.protocol.TField("bolts", org.apache.thrift7.protocol.TType.MAP, (short)2); - private static final org.apache.thrift7.protocol.TField STATE_SPOUTS_FIELD_DESC = new org.apache.thrift7.protocol.TField("state_spouts", org.apache.thrift7.protocol.TType.MAP, (short)3); - - private Map spouts; // required - private Map bolts; // required - private Map state_spouts; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - SPOUTS((short)1, "spouts"), - BOLTS((short)2, "bolts"), - STATE_SPOUTS((short)3, "state_spouts"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SPOUTS - return SPOUTS; - case 2: // BOLTS - return BOLTS; - case 3: // STATE_SPOUTS - return STATE_SPOUTS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SPOUTS, new org.apache.thrift7.meta_data.FieldMetaData("spouts", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, SpoutSpec.class)))); - tmpMap.put(_Fields.BOLTS, new org.apache.thrift7.meta_data.FieldMetaData("bolts", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, Bolt.class)))); - tmpMap.put(_Fields.STATE_SPOUTS, new org.apache.thrift7.meta_data.FieldMetaData("state_spouts", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, StateSpoutSpec.class)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(StormTopology.class, metaDataMap); - } - - public StormTopology() { - } - - public StormTopology( - Map spouts, - Map bolts, - Map state_spouts) - { - this(); - this.spouts = spouts; - this.bolts = bolts; - this.state_spouts = state_spouts; - } - - /** - * Performs a deep copy on other. - */ - public StormTopology(StormTopology other) { - if (other.is_set_spouts()) { - Map __this__spouts = new HashMap(); - for (Map.Entry other_element : other.spouts.entrySet()) { - - String other_element_key = other_element.getKey(); - SpoutSpec other_element_value = other_element.getValue(); - - String __this__spouts_copy_key = other_element_key; - - SpoutSpec __this__spouts_copy_value = new SpoutSpec(other_element_value); - - __this__spouts.put(__this__spouts_copy_key, __this__spouts_copy_value); - } - this.spouts = __this__spouts; - } - if (other.is_set_bolts()) { - Map __this__bolts = new HashMap(); - for (Map.Entry other_element : other.bolts.entrySet()) { - - String other_element_key = other_element.getKey(); - Bolt other_element_value = other_element.getValue(); - - String __this__bolts_copy_key = other_element_key; - - Bolt __this__bolts_copy_value = new Bolt(other_element_value); - - __this__bolts.put(__this__bolts_copy_key, __this__bolts_copy_value); - } - this.bolts = __this__bolts; - } - if (other.is_set_state_spouts()) { - Map __this__state_spouts = new HashMap(); - for (Map.Entry other_element : other.state_spouts.entrySet()) { - - String other_element_key = other_element.getKey(); - StateSpoutSpec other_element_value = other_element.getValue(); - - String __this__state_spouts_copy_key = other_element_key; - - StateSpoutSpec __this__state_spouts_copy_value = new StateSpoutSpec(other_element_value); - - __this__state_spouts.put(__this__state_spouts_copy_key, __this__state_spouts_copy_value); - } - this.state_spouts = __this__state_spouts; - } - } - - public StormTopology deepCopy() { - return new StormTopology(this); - } - - @Override - public void clear() { - this.spouts = null; - this.bolts = null; - this.state_spouts = null; - } - - public int get_spouts_size() { - return (this.spouts == null) ? 0 : this.spouts.size(); - } - - public void put_to_spouts(String key, SpoutSpec val) { - if (this.spouts == null) { - this.spouts = new HashMap(); - } - this.spouts.put(key, val); - } - - public Map get_spouts() { - return this.spouts; - } - - public void set_spouts(Map spouts) { - this.spouts = spouts; - } - - public void unset_spouts() { - this.spouts = null; - } - - /** Returns true if field spouts is set (has been assigned a value) and false otherwise */ - public boolean is_set_spouts() { - return this.spouts != null; - } - - public void set_spouts_isSet(boolean value) { - if (!value) { - this.spouts = null; - } - } - - public int get_bolts_size() { - return (this.bolts == null) ? 0 : this.bolts.size(); - } - - public void put_to_bolts(String key, Bolt val) { - if (this.bolts == null) { - this.bolts = new HashMap(); - } - this.bolts.put(key, val); - } - - public Map get_bolts() { - return this.bolts; - } - - public void set_bolts(Map bolts) { - this.bolts = bolts; - } - - public void unset_bolts() { - this.bolts = null; - } - - /** Returns true if field bolts is set (has been assigned a value) and false otherwise */ - public boolean is_set_bolts() { - return this.bolts != null; - } - - public void set_bolts_isSet(boolean value) { - if (!value) { - this.bolts = null; - } - } - - public int get_state_spouts_size() { - return (this.state_spouts == null) ? 0 : this.state_spouts.size(); - } - - public void put_to_state_spouts(String key, StateSpoutSpec val) { - if (this.state_spouts == null) { - this.state_spouts = new HashMap(); - } - this.state_spouts.put(key, val); - } - - public Map get_state_spouts() { - return this.state_spouts; - } - - public void set_state_spouts(Map state_spouts) { - this.state_spouts = state_spouts; - } - - public void unset_state_spouts() { - this.state_spouts = null; - } - - /** Returns true if field state_spouts is set (has been assigned a value) and false otherwise */ - public boolean is_set_state_spouts() { - return this.state_spouts != null; - } - - public void set_state_spouts_isSet(boolean value) { - if (!value) { - this.state_spouts = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SPOUTS: - if (value == null) { - unset_spouts(); - } else { - set_spouts((Map)value); - } - break; - - case BOLTS: - if (value == null) { - unset_bolts(); - } else { - set_bolts((Map)value); - } - break; - - case STATE_SPOUTS: - if (value == null) { - unset_state_spouts(); - } else { - set_state_spouts((Map)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SPOUTS: - return get_spouts(); - - case BOLTS: - return get_bolts(); - - case STATE_SPOUTS: - return get_state_spouts(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SPOUTS: - return is_set_spouts(); - case BOLTS: - return is_set_bolts(); - case STATE_SPOUTS: - return is_set_state_spouts(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof StormTopology) - return this.equals((StormTopology)that); - return false; - } - - public boolean equals(StormTopology that) { - if (that == null) - return false; - - boolean this_present_spouts = true && this.is_set_spouts(); - boolean that_present_spouts = true && that.is_set_spouts(); - if (this_present_spouts || that_present_spouts) { - if (!(this_present_spouts && that_present_spouts)) - return false; - if (!this.spouts.equals(that.spouts)) - return false; - } - - boolean this_present_bolts = true && this.is_set_bolts(); - boolean that_present_bolts = true && that.is_set_bolts(); - if (this_present_bolts || that_present_bolts) { - if (!(this_present_bolts && that_present_bolts)) - return false; - if (!this.bolts.equals(that.bolts)) - return false; - } - - boolean this_present_state_spouts = true && this.is_set_state_spouts(); - boolean that_present_state_spouts = true && that.is_set_state_spouts(); - if (this_present_state_spouts || that_present_state_spouts) { - if (!(this_present_state_spouts && that_present_state_spouts)) - return false; - if (!this.state_spouts.equals(that.state_spouts)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_spouts = true && (is_set_spouts()); - builder.append(present_spouts); - if (present_spouts) - builder.append(spouts); - - boolean present_bolts = true && (is_set_bolts()); - builder.append(present_bolts); - if (present_bolts) - builder.append(bolts); - - boolean present_state_spouts = true && (is_set_state_spouts()); - builder.append(present_state_spouts); - if (present_state_spouts) - builder.append(state_spouts); - - return builder.toHashCode(); - } - - public int compareTo(StormTopology other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - StormTopology typedOther = (StormTopology)other; - - lastComparison = Boolean.valueOf(is_set_spouts()).compareTo(typedOther.is_set_spouts()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_spouts()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.spouts, typedOther.spouts); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_bolts()).compareTo(typedOther.is_set_bolts()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_bolts()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.bolts, typedOther.bolts); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_state_spouts()).compareTo(typedOther.is_set_state_spouts()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_state_spouts()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.state_spouts, typedOther.state_spouts); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // SPOUTS - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map22 = iprot.readMapBegin(); - this.spouts = new HashMap(2*_map22.size); - for (int _i23 = 0; _i23 < _map22.size; ++_i23) - { - String _key24; // required - SpoutSpec _val25; // required - _key24 = iprot.readString(); - _val25 = new SpoutSpec(); - _val25.read(iprot); - this.spouts.put(_key24, _val25); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // BOLTS - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map26 = iprot.readMapBegin(); - this.bolts = new HashMap(2*_map26.size); - for (int _i27 = 0; _i27 < _map26.size; ++_i27) - { - String _key28; // required - Bolt _val29; // required - _key28 = iprot.readString(); - _val29 = new Bolt(); - _val29.read(iprot); - this.bolts.put(_key28, _val29); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 3: // STATE_SPOUTS - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map30 = iprot.readMapBegin(); - this.state_spouts = new HashMap(2*_map30.size); - for (int _i31 = 0; _i31 < _map30.size; ++_i31) - { - String _key32; // required - StateSpoutSpec _val33; // required - _key32 = iprot.readString(); - _val33 = new StateSpoutSpec(); - _val33.read(iprot); - this.state_spouts.put(_key32, _val33); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.spouts != null) { - oprot.writeFieldBegin(SPOUTS_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.STRUCT, this.spouts.size())); - for (Map.Entry _iter34 : this.spouts.entrySet()) - { - oprot.writeString(_iter34.getKey()); - _iter34.getValue().write(oprot); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.bolts != null) { - oprot.writeFieldBegin(BOLTS_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.STRUCT, this.bolts.size())); - for (Map.Entry _iter35 : this.bolts.entrySet()) - { - oprot.writeString(_iter35.getKey()); - _iter35.getValue().write(oprot); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.state_spouts != null) { - oprot.writeFieldBegin(STATE_SPOUTS_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.STRUCT, this.state_spouts.size())); - for (Map.Entry _iter36 : this.state_spouts.entrySet()) - { - oprot.writeString(_iter36.getKey()); - _iter36.getValue().write(oprot); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("StormTopology("); - boolean first = true; - - sb.append("spouts:"); - if (this.spouts == null) { - sb.append("null"); - } else { - sb.append(this.spouts); - } - first = false; - if (!first) sb.append(", "); - sb.append("bolts:"); - if (this.bolts == null) { - sb.append("null"); - } else { - sb.append(this.bolts); - } - first = false; - if (!first) sb.append(", "); - sb.append("state_spouts:"); - if (this.state_spouts == null) { - sb.append("null"); - } else { - sb.append(this.state_spouts); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_spouts()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'spouts' is unset! Struct:" + toString()); - } - - if (!is_set_bolts()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'bolts' is unset! Struct:" + toString()); - } - - if (!is_set_state_spouts()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'state_spouts' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/StreamInfo.java b/jstorm-client/src/main/java/backtype/storm/generated/StreamInfo.java deleted file mode 100644 index 83a23dfc5..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/StreamInfo.java +++ /dev/null @@ -1,462 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class StreamInfo implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("StreamInfo"); - - private static final org.apache.thrift7.protocol.TField OUTPUT_FIELDS_FIELD_DESC = new org.apache.thrift7.protocol.TField("output_fields", org.apache.thrift7.protocol.TType.LIST, (short)1); - private static final org.apache.thrift7.protocol.TField DIRECT_FIELD_DESC = new org.apache.thrift7.protocol.TField("direct", org.apache.thrift7.protocol.TType.BOOL, (short)2); - - private List output_fields; // required - private boolean direct; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - OUTPUT_FIELDS((short)1, "output_fields"), - DIRECT((short)2, "direct"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // OUTPUT_FIELDS - return OUTPUT_FIELDS; - case 2: // DIRECT - return DIRECT; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __DIRECT_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.OUTPUT_FIELDS, new org.apache.thrift7.meta_data.FieldMetaData("output_fields", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.ListMetaData(org.apache.thrift7.protocol.TType.LIST, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING)))); - tmpMap.put(_Fields.DIRECT, new org.apache.thrift7.meta_data.FieldMetaData("direct", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.BOOL))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(StreamInfo.class, metaDataMap); - } - - public StreamInfo() { - } - - public StreamInfo( - List output_fields, - boolean direct) - { - this(); - this.output_fields = output_fields; - this.direct = direct; - set_direct_isSet(true); - } - - /** - * Performs a deep copy on other. - */ - public StreamInfo(StreamInfo other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); - if (other.is_set_output_fields()) { - List __this__output_fields = new ArrayList(); - for (String other_element : other.output_fields) { - __this__output_fields.add(other_element); - } - this.output_fields = __this__output_fields; - } - this.direct = other.direct; - } - - public StreamInfo deepCopy() { - return new StreamInfo(this); - } - - @Override - public void clear() { - this.output_fields = null; - set_direct_isSet(false); - this.direct = false; - } - - public int get_output_fields_size() { - return (this.output_fields == null) ? 0 : this.output_fields.size(); - } - - public java.util.Iterator get_output_fields_iterator() { - return (this.output_fields == null) ? null : this.output_fields.iterator(); - } - - public void add_to_output_fields(String elem) { - if (this.output_fields == null) { - this.output_fields = new ArrayList(); - } - this.output_fields.add(elem); - } - - public List get_output_fields() { - return this.output_fields; - } - - public void set_output_fields(List output_fields) { - this.output_fields = output_fields; - } - - public void unset_output_fields() { - this.output_fields = null; - } - - /** Returns true if field output_fields is set (has been assigned a value) and false otherwise */ - public boolean is_set_output_fields() { - return this.output_fields != null; - } - - public void set_output_fields_isSet(boolean value) { - if (!value) { - this.output_fields = null; - } - } - - public boolean is_direct() { - return this.direct; - } - - public void set_direct(boolean direct) { - this.direct = direct; - set_direct_isSet(true); - } - - public void unset_direct() { - __isset_bit_vector.clear(__DIRECT_ISSET_ID); - } - - /** Returns true if field direct is set (has been assigned a value) and false otherwise */ - public boolean is_set_direct() { - return __isset_bit_vector.get(__DIRECT_ISSET_ID); - } - - public void set_direct_isSet(boolean value) { - __isset_bit_vector.set(__DIRECT_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case OUTPUT_FIELDS: - if (value == null) { - unset_output_fields(); - } else { - set_output_fields((List)value); - } - break; - - case DIRECT: - if (value == null) { - unset_direct(); - } else { - set_direct((Boolean)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case OUTPUT_FIELDS: - return get_output_fields(); - - case DIRECT: - return Boolean.valueOf(is_direct()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case OUTPUT_FIELDS: - return is_set_output_fields(); - case DIRECT: - return is_set_direct(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof StreamInfo) - return this.equals((StreamInfo)that); - return false; - } - - public boolean equals(StreamInfo that) { - if (that == null) - return false; - - boolean this_present_output_fields = true && this.is_set_output_fields(); - boolean that_present_output_fields = true && that.is_set_output_fields(); - if (this_present_output_fields || that_present_output_fields) { - if (!(this_present_output_fields && that_present_output_fields)) - return false; - if (!this.output_fields.equals(that.output_fields)) - return false; - } - - boolean this_present_direct = true; - boolean that_present_direct = true; - if (this_present_direct || that_present_direct) { - if (!(this_present_direct && that_present_direct)) - return false; - if (this.direct != that.direct) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_output_fields = true && (is_set_output_fields()); - builder.append(present_output_fields); - if (present_output_fields) - builder.append(output_fields); - - boolean present_direct = true; - builder.append(present_direct); - if (present_direct) - builder.append(direct); - - return builder.toHashCode(); - } - - public int compareTo(StreamInfo other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - StreamInfo typedOther = (StreamInfo)other; - - lastComparison = Boolean.valueOf(is_set_output_fields()).compareTo(typedOther.is_set_output_fields()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_output_fields()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.output_fields, typedOther.output_fields); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_direct()).compareTo(typedOther.is_set_direct()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_direct()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.direct, typedOther.direct); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // OUTPUT_FIELDS - if (field.type == org.apache.thrift7.protocol.TType.LIST) { - { - org.apache.thrift7.protocol.TList _list8 = iprot.readListBegin(); - this.output_fields = new ArrayList(_list8.size); - for (int _i9 = 0; _i9 < _list8.size; ++_i9) - { - String _elem10; // required - _elem10 = iprot.readString(); - this.output_fields.add(_elem10); - } - iprot.readListEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // DIRECT - if (field.type == org.apache.thrift7.protocol.TType.BOOL) { - this.direct = iprot.readBool(); - set_direct_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.output_fields != null) { - oprot.writeFieldBegin(OUTPUT_FIELDS_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift7.protocol.TList(org.apache.thrift7.protocol.TType.STRING, this.output_fields.size())); - for (String _iter11 : this.output_fields) - { - oprot.writeString(_iter11); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldBegin(DIRECT_FIELD_DESC); - oprot.writeBool(this.direct); - oprot.writeFieldEnd(); - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("StreamInfo("); - boolean first = true; - - sb.append("output_fields:"); - if (this.output_fields == null) { - sb.append("null"); - } else { - sb.append(this.output_fields); - } - first = false; - if (!first) sb.append(", "); - sb.append("direct:"); - sb.append(this.direct); - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_output_fields()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'output_fields' is unset! Struct:" + toString()); - } - - if (!is_set_direct()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'direct' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bit_vector = new BitSet(1); - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/SubmitOptions.java b/jstorm-client/src/main/java/backtype/storm/generated/SubmitOptions.java deleted file mode 100644 index 8b4ffc716..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/SubmitOptions.java +++ /dev/null @@ -1,340 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class SubmitOptions implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("SubmitOptions"); - - private static final org.apache.thrift7.protocol.TField INITIAL_STATUS_FIELD_DESC = new org.apache.thrift7.protocol.TField("initial_status", org.apache.thrift7.protocol.TType.I32, (short)1); - - private TopologyInitialStatus initial_status; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - /** - * - * @see TopologyInitialStatus - */ - INITIAL_STATUS((short)1, "initial_status"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // INITIAL_STATUS - return INITIAL_STATUS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.INITIAL_STATUS, new org.apache.thrift7.meta_data.FieldMetaData("initial_status", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.EnumMetaData(org.apache.thrift7.protocol.TType.ENUM, TopologyInitialStatus.class))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(SubmitOptions.class, metaDataMap); - } - - public SubmitOptions() { - } - - public SubmitOptions( - TopologyInitialStatus initial_status) - { - this(); - this.initial_status = initial_status; - } - - /** - * Performs a deep copy on other. - */ - public SubmitOptions(SubmitOptions other) { - if (other.is_set_initial_status()) { - this.initial_status = other.initial_status; - } - } - - public SubmitOptions deepCopy() { - return new SubmitOptions(this); - } - - @Override - public void clear() { - this.initial_status = null; - } - - /** - * - * @see TopologyInitialStatus - */ - public TopologyInitialStatus get_initial_status() { - return this.initial_status; - } - - /** - * - * @see TopologyInitialStatus - */ - public void set_initial_status(TopologyInitialStatus initial_status) { - this.initial_status = initial_status; - } - - public void unset_initial_status() { - this.initial_status = null; - } - - /** Returns true if field initial_status is set (has been assigned a value) and false otherwise */ - public boolean is_set_initial_status() { - return this.initial_status != null; - } - - public void set_initial_status_isSet(boolean value) { - if (!value) { - this.initial_status = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case INITIAL_STATUS: - if (value == null) { - unset_initial_status(); - } else { - set_initial_status((TopologyInitialStatus)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case INITIAL_STATUS: - return get_initial_status(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case INITIAL_STATUS: - return is_set_initial_status(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof SubmitOptions) - return this.equals((SubmitOptions)that); - return false; - } - - public boolean equals(SubmitOptions that) { - if (that == null) - return false; - - boolean this_present_initial_status = true && this.is_set_initial_status(); - boolean that_present_initial_status = true && that.is_set_initial_status(); - if (this_present_initial_status || that_present_initial_status) { - if (!(this_present_initial_status && that_present_initial_status)) - return false; - if (!this.initial_status.equals(that.initial_status)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_initial_status = true && (is_set_initial_status()); - builder.append(present_initial_status); - if (present_initial_status) - builder.append(initial_status.getValue()); - - return builder.toHashCode(); - } - - public int compareTo(SubmitOptions other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - SubmitOptions typedOther = (SubmitOptions)other; - - lastComparison = Boolean.valueOf(is_set_initial_status()).compareTo(typedOther.is_set_initial_status()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_initial_status()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.initial_status, typedOther.initial_status); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // INITIAL_STATUS - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.initial_status = TopologyInitialStatus.findByValue(iprot.readI32()); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.initial_status != null) { - oprot.writeFieldBegin(INITIAL_STATUS_FIELD_DESC); - oprot.writeI32(this.initial_status.getValue()); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("SubmitOptions("); - boolean first = true; - - sb.append("initial_status:"); - if (this.initial_status == null) { - sb.append("null"); - } else { - sb.append(this.initial_status); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_initial_status()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'initial_status' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/SupervisorSummary.java b/jstorm-client/src/main/java/backtype/storm/generated/SupervisorSummary.java deleted file mode 100644 index 4f7cb8acb..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/SupervisorSummary.java +++ /dev/null @@ -1,706 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class SupervisorSummary implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("SupervisorSummary"); - - private static final org.apache.thrift7.protocol.TField HOST_FIELD_DESC = new org.apache.thrift7.protocol.TField("host", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField SUPERVISOR_ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("supervisor_id", org.apache.thrift7.protocol.TType.STRING, (short)2); - private static final org.apache.thrift7.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift7.protocol.TField("uptime_secs", org.apache.thrift7.protocol.TType.I32, (short)3); - private static final org.apache.thrift7.protocol.TField NUM_WORKERS_FIELD_DESC = new org.apache.thrift7.protocol.TField("num_workers", org.apache.thrift7.protocol.TType.I32, (short)4); - private static final org.apache.thrift7.protocol.TField NUM_USED_WORKERS_FIELD_DESC = new org.apache.thrift7.protocol.TField("num_used_workers", org.apache.thrift7.protocol.TType.I32, (short)5); - - private String host; // required - private String supervisor_id; // required - private int uptime_secs; // required - private int num_workers; // required - private int num_used_workers; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - HOST((short)1, "host"), - SUPERVISOR_ID((short)2, "supervisor_id"), - UPTIME_SECS((short)3, "uptime_secs"), - NUM_WORKERS((short)4, "num_workers"), - NUM_USED_WORKERS((short)5, "num_used_workers"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // HOST - return HOST; - case 2: // SUPERVISOR_ID - return SUPERVISOR_ID; - case 3: // UPTIME_SECS - return UPTIME_SECS; - case 4: // NUM_WORKERS - return NUM_WORKERS; - case 5: // NUM_USED_WORKERS - return NUM_USED_WORKERS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __UPTIME_SECS_ISSET_ID = 0; - private static final int __NUM_WORKERS_ISSET_ID = 1; - private static final int __NUM_USED_WORKERS_ISSET_ID = 2; - private BitSet __isset_bit_vector = new BitSet(3); - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.HOST, new org.apache.thrift7.meta_data.FieldMetaData("host", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.SUPERVISOR_ID, new org.apache.thrift7.meta_data.FieldMetaData("supervisor_id", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift7.meta_data.FieldMetaData("uptime_secs", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - tmpMap.put(_Fields.NUM_WORKERS, new org.apache.thrift7.meta_data.FieldMetaData("num_workers", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - tmpMap.put(_Fields.NUM_USED_WORKERS, new org.apache.thrift7.meta_data.FieldMetaData("num_used_workers", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(SupervisorSummary.class, metaDataMap); - } - - public SupervisorSummary() { - } - - public SupervisorSummary( - String host, - String supervisor_id, - int uptime_secs, - int num_workers, - int num_used_workers) - { - this(); - this.host = host; - this.supervisor_id = supervisor_id; - this.uptime_secs = uptime_secs; - set_uptime_secs_isSet(true); - this.num_workers = num_workers; - set_num_workers_isSet(true); - this.num_used_workers = num_used_workers; - set_num_used_workers_isSet(true); - } - - /** - * Performs a deep copy on other. - */ - public SupervisorSummary(SupervisorSummary other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); - if (other.is_set_host()) { - this.host = other.host; - } - if (other.is_set_supervisor_id()) { - this.supervisor_id = other.supervisor_id; - } - this.uptime_secs = other.uptime_secs; - this.num_workers = other.num_workers; - this.num_used_workers = other.num_used_workers; - } - - public SupervisorSummary deepCopy() { - return new SupervisorSummary(this); - } - - @Override - public void clear() { - this.host = null; - this.supervisor_id = null; - set_uptime_secs_isSet(false); - this.uptime_secs = 0; - set_num_workers_isSet(false); - this.num_workers = 0; - set_num_used_workers_isSet(false); - this.num_used_workers = 0; - } - - public String get_host() { - return this.host; - } - - public void set_host(String host) { - this.host = host; - } - - public void unset_host() { - this.host = null; - } - - /** Returns true if field host is set (has been assigned a value) and false otherwise */ - public boolean is_set_host() { - return this.host != null; - } - - public void set_host_isSet(boolean value) { - if (!value) { - this.host = null; - } - } - - public String get_supervisor_id() { - return this.supervisor_id; - } - - public void set_supervisor_id(String supervisor_id) { - this.supervisor_id = supervisor_id; - } - - public void unset_supervisor_id() { - this.supervisor_id = null; - } - - /** Returns true if field supervisor_id is set (has been assigned a value) and false otherwise */ - public boolean is_set_supervisor_id() { - return this.supervisor_id != null; - } - - public void set_supervisor_id_isSet(boolean value) { - if (!value) { - this.supervisor_id = null; - } - } - - public int get_uptime_secs() { - return this.uptime_secs; - } - - public void set_uptime_secs(int uptime_secs) { - this.uptime_secs = uptime_secs; - set_uptime_secs_isSet(true); - } - - public void unset_uptime_secs() { - __isset_bit_vector.clear(__UPTIME_SECS_ISSET_ID); - } - - /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */ - public boolean is_set_uptime_secs() { - return __isset_bit_vector.get(__UPTIME_SECS_ISSET_ID); - } - - public void set_uptime_secs_isSet(boolean value) { - __isset_bit_vector.set(__UPTIME_SECS_ISSET_ID, value); - } - - public int get_num_workers() { - return this.num_workers; - } - - public void set_num_workers(int num_workers) { - this.num_workers = num_workers; - set_num_workers_isSet(true); - } - - public void unset_num_workers() { - __isset_bit_vector.clear(__NUM_WORKERS_ISSET_ID); - } - - /** Returns true if field num_workers is set (has been assigned a value) and false otherwise */ - public boolean is_set_num_workers() { - return __isset_bit_vector.get(__NUM_WORKERS_ISSET_ID); - } - - public void set_num_workers_isSet(boolean value) { - __isset_bit_vector.set(__NUM_WORKERS_ISSET_ID, value); - } - - public int get_num_used_workers() { - return this.num_used_workers; - } - - public void set_num_used_workers(int num_used_workers) { - this.num_used_workers = num_used_workers; - set_num_used_workers_isSet(true); - } - - public void unset_num_used_workers() { - __isset_bit_vector.clear(__NUM_USED_WORKERS_ISSET_ID); - } - - /** Returns true if field num_used_workers is set (has been assigned a value) and false otherwise */ - public boolean is_set_num_used_workers() { - return __isset_bit_vector.get(__NUM_USED_WORKERS_ISSET_ID); - } - - public void set_num_used_workers_isSet(boolean value) { - __isset_bit_vector.set(__NUM_USED_WORKERS_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case HOST: - if (value == null) { - unset_host(); - } else { - set_host((String)value); - } - break; - - case SUPERVISOR_ID: - if (value == null) { - unset_supervisor_id(); - } else { - set_supervisor_id((String)value); - } - break; - - case UPTIME_SECS: - if (value == null) { - unset_uptime_secs(); - } else { - set_uptime_secs((Integer)value); - } - break; - - case NUM_WORKERS: - if (value == null) { - unset_num_workers(); - } else { - set_num_workers((Integer)value); - } - break; - - case NUM_USED_WORKERS: - if (value == null) { - unset_num_used_workers(); - } else { - set_num_used_workers((Integer)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case HOST: - return get_host(); - - case SUPERVISOR_ID: - return get_supervisor_id(); - - case UPTIME_SECS: - return Integer.valueOf(get_uptime_secs()); - - case NUM_WORKERS: - return Integer.valueOf(get_num_workers()); - - case NUM_USED_WORKERS: - return Integer.valueOf(get_num_used_workers()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case HOST: - return is_set_host(); - case SUPERVISOR_ID: - return is_set_supervisor_id(); - case UPTIME_SECS: - return is_set_uptime_secs(); - case NUM_WORKERS: - return is_set_num_workers(); - case NUM_USED_WORKERS: - return is_set_num_used_workers(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof SupervisorSummary) - return this.equals((SupervisorSummary)that); - return false; - } - - public boolean equals(SupervisorSummary that) { - if (that == null) - return false; - - boolean this_present_host = true && this.is_set_host(); - boolean that_present_host = true && that.is_set_host(); - if (this_present_host || that_present_host) { - if (!(this_present_host && that_present_host)) - return false; - if (!this.host.equals(that.host)) - return false; - } - - boolean this_present_supervisor_id = true && this.is_set_supervisor_id(); - boolean that_present_supervisor_id = true && that.is_set_supervisor_id(); - if (this_present_supervisor_id || that_present_supervisor_id) { - if (!(this_present_supervisor_id && that_present_supervisor_id)) - return false; - if (!this.supervisor_id.equals(that.supervisor_id)) - return false; - } - - boolean this_present_uptime_secs = true; - boolean that_present_uptime_secs = true; - if (this_present_uptime_secs || that_present_uptime_secs) { - if (!(this_present_uptime_secs && that_present_uptime_secs)) - return false; - if (this.uptime_secs != that.uptime_secs) - return false; - } - - boolean this_present_num_workers = true; - boolean that_present_num_workers = true; - if (this_present_num_workers || that_present_num_workers) { - if (!(this_present_num_workers && that_present_num_workers)) - return false; - if (this.num_workers != that.num_workers) - return false; - } - - boolean this_present_num_used_workers = true; - boolean that_present_num_used_workers = true; - if (this_present_num_used_workers || that_present_num_used_workers) { - if (!(this_present_num_used_workers && that_present_num_used_workers)) - return false; - if (this.num_used_workers != that.num_used_workers) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_host = true && (is_set_host()); - builder.append(present_host); - if (present_host) - builder.append(host); - - boolean present_supervisor_id = true && (is_set_supervisor_id()); - builder.append(present_supervisor_id); - if (present_supervisor_id) - builder.append(supervisor_id); - - boolean present_uptime_secs = true; - builder.append(present_uptime_secs); - if (present_uptime_secs) - builder.append(uptime_secs); - - boolean present_num_workers = true; - builder.append(present_num_workers); - if (present_num_workers) - builder.append(num_workers); - - boolean present_num_used_workers = true; - builder.append(present_num_used_workers); - if (present_num_used_workers) - builder.append(num_used_workers); - - return builder.toHashCode(); - } - - public int compareTo(SupervisorSummary other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - SupervisorSummary typedOther = (SupervisorSummary)other; - - lastComparison = Boolean.valueOf(is_set_host()).compareTo(typedOther.is_set_host()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_host()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.host, typedOther.host); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_supervisor_id()).compareTo(typedOther.is_set_supervisor_id()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_supervisor_id()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.supervisor_id, typedOther.supervisor_id); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(typedOther.is_set_uptime_secs()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_uptime_secs()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.uptime_secs, typedOther.uptime_secs); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_num_workers()).compareTo(typedOther.is_set_num_workers()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_num_workers()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.num_workers, typedOther.num_workers); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_num_used_workers()).compareTo(typedOther.is_set_num_used_workers()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_num_used_workers()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.num_used_workers, typedOther.num_used_workers); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // HOST - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.host = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // SUPERVISOR_ID - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.supervisor_id = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 3: // UPTIME_SECS - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.uptime_secs = iprot.readI32(); - set_uptime_secs_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 4: // NUM_WORKERS - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.num_workers = iprot.readI32(); - set_num_workers_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 5: // NUM_USED_WORKERS - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.num_used_workers = iprot.readI32(); - set_num_used_workers_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.host != null) { - oprot.writeFieldBegin(HOST_FIELD_DESC); - oprot.writeString(this.host); - oprot.writeFieldEnd(); - } - if (this.supervisor_id != null) { - oprot.writeFieldBegin(SUPERVISOR_ID_FIELD_DESC); - oprot.writeString(this.supervisor_id); - oprot.writeFieldEnd(); - } - oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC); - oprot.writeI32(this.uptime_secs); - oprot.writeFieldEnd(); - oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC); - oprot.writeI32(this.num_workers); - oprot.writeFieldEnd(); - oprot.writeFieldBegin(NUM_USED_WORKERS_FIELD_DESC); - oprot.writeI32(this.num_used_workers); - oprot.writeFieldEnd(); - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("SupervisorSummary("); - boolean first = true; - - sb.append("host:"); - if (this.host == null) { - sb.append("null"); - } else { - sb.append(this.host); - } - first = false; - if (!first) sb.append(", "); - sb.append("supervisor_id:"); - if (this.supervisor_id == null) { - sb.append("null"); - } else { - sb.append(this.supervisor_id); - } - first = false; - if (!first) sb.append(", "); - sb.append("uptime_secs:"); - sb.append(this.uptime_secs); - first = false; - if (!first) sb.append(", "); - sb.append("num_workers:"); - sb.append(this.num_workers); - first = false; - if (!first) sb.append(", "); - sb.append("num_used_workers:"); - sb.append(this.num_used_workers); - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_host()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'host' is unset! Struct:" + toString()); - } - - if (!is_set_supervisor_id()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'supervisor_id' is unset! Struct:" + toString()); - } - - if (!is_set_uptime_secs()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString()); - } - - if (!is_set_num_workers()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'num_workers' is unset! Struct:" + toString()); - } - - if (!is_set_num_used_workers()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'num_used_workers' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bit_vector = new BitSet(1); - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/SupervisorWorkers.java b/jstorm-client/src/main/java/backtype/storm/generated/SupervisorWorkers.java deleted file mode 100644 index b8cb513c8..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/SupervisorWorkers.java +++ /dev/null @@ -1,464 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class SupervisorWorkers implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("SupervisorWorkers"); - - private static final org.apache.thrift7.protocol.TField SUPERVISOR_FIELD_DESC = new org.apache.thrift7.protocol.TField("supervisor", org.apache.thrift7.protocol.TType.STRUCT, (short)1); - private static final org.apache.thrift7.protocol.TField WORKERS_FIELD_DESC = new org.apache.thrift7.protocol.TField("workers", org.apache.thrift7.protocol.TType.LIST, (short)2); - - private SupervisorSummary supervisor; // required - private List workers; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - SUPERVISOR((short)1, "supervisor"), - WORKERS((short)2, "workers"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // SUPERVISOR - return SUPERVISOR; - case 2: // WORKERS - return WORKERS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.SUPERVISOR, new org.apache.thrift7.meta_data.FieldMetaData("supervisor", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, SupervisorSummary.class))); - tmpMap.put(_Fields.WORKERS, new org.apache.thrift7.meta_data.FieldMetaData("workers", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.ListMetaData(org.apache.thrift7.protocol.TType.LIST, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, WorkerSummary.class)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(SupervisorWorkers.class, metaDataMap); - } - - public SupervisorWorkers() { - } - - public SupervisorWorkers( - SupervisorSummary supervisor, - List workers) - { - this(); - this.supervisor = supervisor; - this.workers = workers; - } - - /** - * Performs a deep copy on other. - */ - public SupervisorWorkers(SupervisorWorkers other) { - if (other.is_set_supervisor()) { - this.supervisor = new SupervisorSummary(other.supervisor); - } - if (other.is_set_workers()) { - List __this__workers = new ArrayList(); - for (WorkerSummary other_element : other.workers) { - __this__workers.add(new WorkerSummary(other_element)); - } - this.workers = __this__workers; - } - } - - public SupervisorWorkers deepCopy() { - return new SupervisorWorkers(this); - } - - @Override - public void clear() { - this.supervisor = null; - this.workers = null; - } - - public SupervisorSummary get_supervisor() { - return this.supervisor; - } - - public void set_supervisor(SupervisorSummary supervisor) { - this.supervisor = supervisor; - } - - public void unset_supervisor() { - this.supervisor = null; - } - - /** Returns true if field supervisor is set (has been assigned a value) and false otherwise */ - public boolean is_set_supervisor() { - return this.supervisor != null; - } - - public void set_supervisor_isSet(boolean value) { - if (!value) { - this.supervisor = null; - } - } - - public int get_workers_size() { - return (this.workers == null) ? 0 : this.workers.size(); - } - - public java.util.Iterator get_workers_iterator() { - return (this.workers == null) ? null : this.workers.iterator(); - } - - public void add_to_workers(WorkerSummary elem) { - if (this.workers == null) { - this.workers = new ArrayList(); - } - this.workers.add(elem); - } - - public List get_workers() { - return this.workers; - } - - public void set_workers(List workers) { - this.workers = workers; - } - - public void unset_workers() { - this.workers = null; - } - - /** Returns true if field workers is set (has been assigned a value) and false otherwise */ - public boolean is_set_workers() { - return this.workers != null; - } - - public void set_workers_isSet(boolean value) { - if (!value) { - this.workers = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case SUPERVISOR: - if (value == null) { - unset_supervisor(); - } else { - set_supervisor((SupervisorSummary)value); - } - break; - - case WORKERS: - if (value == null) { - unset_workers(); - } else { - set_workers((List)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case SUPERVISOR: - return get_supervisor(); - - case WORKERS: - return get_workers(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case SUPERVISOR: - return is_set_supervisor(); - case WORKERS: - return is_set_workers(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof SupervisorWorkers) - return this.equals((SupervisorWorkers)that); - return false; - } - - public boolean equals(SupervisorWorkers that) { - if (that == null) - return false; - - boolean this_present_supervisor = true && this.is_set_supervisor(); - boolean that_present_supervisor = true && that.is_set_supervisor(); - if (this_present_supervisor || that_present_supervisor) { - if (!(this_present_supervisor && that_present_supervisor)) - return false; - if (!this.supervisor.equals(that.supervisor)) - return false; - } - - boolean this_present_workers = true && this.is_set_workers(); - boolean that_present_workers = true && that.is_set_workers(); - if (this_present_workers || that_present_workers) { - if (!(this_present_workers && that_present_workers)) - return false; - if (!this.workers.equals(that.workers)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_supervisor = true && (is_set_supervisor()); - builder.append(present_supervisor); - if (present_supervisor) - builder.append(supervisor); - - boolean present_workers = true && (is_set_workers()); - builder.append(present_workers); - if (present_workers) - builder.append(workers); - - return builder.toHashCode(); - } - - public int compareTo(SupervisorWorkers other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - SupervisorWorkers typedOther = (SupervisorWorkers)other; - - lastComparison = Boolean.valueOf(is_set_supervisor()).compareTo(typedOther.is_set_supervisor()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_supervisor()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.supervisor, typedOther.supervisor); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_workers()).compareTo(typedOther.is_set_workers()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_workers()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.workers, typedOther.workers); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // SUPERVISOR - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.supervisor = new SupervisorSummary(); - this.supervisor.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // WORKERS - if (field.type == org.apache.thrift7.protocol.TType.LIST) { - { - org.apache.thrift7.protocol.TList _list205 = iprot.readListBegin(); - this.workers = new ArrayList(_list205.size); - for (int _i206 = 0; _i206 < _list205.size; ++_i206) - { - WorkerSummary _elem207; // required - _elem207 = new WorkerSummary(); - _elem207.read(iprot); - this.workers.add(_elem207); - } - iprot.readListEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.supervisor != null) { - oprot.writeFieldBegin(SUPERVISOR_FIELD_DESC); - this.supervisor.write(oprot); - oprot.writeFieldEnd(); - } - if (this.workers != null) { - oprot.writeFieldBegin(WORKERS_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift7.protocol.TList(org.apache.thrift7.protocol.TType.STRUCT, this.workers.size())); - for (WorkerSummary _iter208 : this.workers) - { - _iter208.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("SupervisorWorkers("); - boolean first = true; - - sb.append("supervisor:"); - if (this.supervisor == null) { - sb.append("null"); - } else { - sb.append(this.supervisor); - } - first = false; - if (!first) sb.append(", "); - sb.append("workers:"); - if (this.workers == null) { - sb.append("null"); - } else { - sb.append(this.workers); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_supervisor()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'supervisor' is unset! Struct:" + toString()); - } - - if (!is_set_workers()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'workers' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/TaskMetricData.java b/jstorm-client/src/main/java/backtype/storm/generated/TaskMetricData.java deleted file mode 100644 index 8d89649f8..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/TaskMetricData.java +++ /dev/null @@ -1,1135 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TaskMetricData implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("TaskMetricData"); - - private static final org.apache.thrift7.protocol.TField TASK_ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("task_id", org.apache.thrift7.protocol.TType.I32, (short)1); - private static final org.apache.thrift7.protocol.TField COMPONENT_ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("component_id", org.apache.thrift7.protocol.TType.STRING, (short)2); - private static final org.apache.thrift7.protocol.TField GAUGE_FIELD_DESC = new org.apache.thrift7.protocol.TField("gauge", org.apache.thrift7.protocol.TType.MAP, (short)3); - private static final org.apache.thrift7.protocol.TField COUNTER_FIELD_DESC = new org.apache.thrift7.protocol.TField("counter", org.apache.thrift7.protocol.TType.MAP, (short)4); - private static final org.apache.thrift7.protocol.TField METER_FIELD_DESC = new org.apache.thrift7.protocol.TField("meter", org.apache.thrift7.protocol.TType.MAP, (short)5); - private static final org.apache.thrift7.protocol.TField TIMER_FIELD_DESC = new org.apache.thrift7.protocol.TField("timer", org.apache.thrift7.protocol.TType.MAP, (short)6); - private static final org.apache.thrift7.protocol.TField HISTOGRAM_FIELD_DESC = new org.apache.thrift7.protocol.TField("histogram", org.apache.thrift7.protocol.TType.MAP, (short)7); - - private int task_id; // required - private String component_id; // required - private Map gauge; // required - private Map counter; // required - private Map meter; // required - private Map timer; // required - private Map histogram; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - TASK_ID((short)1, "task_id"), - COMPONENT_ID((short)2, "component_id"), - GAUGE((short)3, "gauge"), - COUNTER((short)4, "counter"), - METER((short)5, "meter"), - TIMER((short)6, "timer"), - HISTOGRAM((short)7, "histogram"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // TASK_ID - return TASK_ID; - case 2: // COMPONENT_ID - return COMPONENT_ID; - case 3: // GAUGE - return GAUGE; - case 4: // COUNTER - return COUNTER; - case 5: // METER - return METER; - case 6: // TIMER - return TIMER; - case 7: // HISTOGRAM - return HISTOGRAM; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __TASK_ID_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.TASK_ID, new org.apache.thrift7.meta_data.FieldMetaData("task_id", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - tmpMap.put(_Fields.COMPONENT_ID, new org.apache.thrift7.meta_data.FieldMetaData("component_id", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.GAUGE, new org.apache.thrift7.meta_data.FieldMetaData("gauge", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE)))); - tmpMap.put(_Fields.COUNTER, new org.apache.thrift7.meta_data.FieldMetaData("counter", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE)))); - tmpMap.put(_Fields.METER, new org.apache.thrift7.meta_data.FieldMetaData("meter", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE)))); - tmpMap.put(_Fields.TIMER, new org.apache.thrift7.meta_data.FieldMetaData("timer", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE)))); - tmpMap.put(_Fields.HISTOGRAM, new org.apache.thrift7.meta_data.FieldMetaData("histogram", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(TaskMetricData.class, metaDataMap); - } - - public TaskMetricData() { - } - - public TaskMetricData( - int task_id, - String component_id, - Map gauge, - Map counter, - Map meter, - Map timer, - Map histogram) - { - this(); - this.task_id = task_id; - set_task_id_isSet(true); - this.component_id = component_id; - this.gauge = gauge; - this.counter = counter; - this.meter = meter; - this.timer = timer; - this.histogram = histogram; - } - - /** - * Performs a deep copy on other. - */ - public TaskMetricData(TaskMetricData other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); - this.task_id = other.task_id; - if (other.is_set_component_id()) { - this.component_id = other.component_id; - } - if (other.is_set_gauge()) { - Map __this__gauge = new HashMap(); - for (Map.Entry other_element : other.gauge.entrySet()) { - - String other_element_key = other_element.getKey(); - Double other_element_value = other_element.getValue(); - - String __this__gauge_copy_key = other_element_key; - - Double __this__gauge_copy_value = other_element_value; - - __this__gauge.put(__this__gauge_copy_key, __this__gauge_copy_value); - } - this.gauge = __this__gauge; - } - if (other.is_set_counter()) { - Map __this__counter = new HashMap(); - for (Map.Entry other_element : other.counter.entrySet()) { - - String other_element_key = other_element.getKey(); - Double other_element_value = other_element.getValue(); - - String __this__counter_copy_key = other_element_key; - - Double __this__counter_copy_value = other_element_value; - - __this__counter.put(__this__counter_copy_key, __this__counter_copy_value); - } - this.counter = __this__counter; - } - if (other.is_set_meter()) { - Map __this__meter = new HashMap(); - for (Map.Entry other_element : other.meter.entrySet()) { - - String other_element_key = other_element.getKey(); - Double other_element_value = other_element.getValue(); - - String __this__meter_copy_key = other_element_key; - - Double __this__meter_copy_value = other_element_value; - - __this__meter.put(__this__meter_copy_key, __this__meter_copy_value); - } - this.meter = __this__meter; - } - if (other.is_set_timer()) { - Map __this__timer = new HashMap(); - for (Map.Entry other_element : other.timer.entrySet()) { - - String other_element_key = other_element.getKey(); - Double other_element_value = other_element.getValue(); - - String __this__timer_copy_key = other_element_key; - - Double __this__timer_copy_value = other_element_value; - - __this__timer.put(__this__timer_copy_key, __this__timer_copy_value); - } - this.timer = __this__timer; - } - if (other.is_set_histogram()) { - Map __this__histogram = new HashMap(); - for (Map.Entry other_element : other.histogram.entrySet()) { - - String other_element_key = other_element.getKey(); - Double other_element_value = other_element.getValue(); - - String __this__histogram_copy_key = other_element_key; - - Double __this__histogram_copy_value = other_element_value; - - __this__histogram.put(__this__histogram_copy_key, __this__histogram_copy_value); - } - this.histogram = __this__histogram; - } - } - - public TaskMetricData deepCopy() { - return new TaskMetricData(this); - } - - @Override - public void clear() { - set_task_id_isSet(false); - this.task_id = 0; - this.component_id = null; - this.gauge = null; - this.counter = null; - this.meter = null; - this.timer = null; - this.histogram = null; - } - - public int get_task_id() { - return this.task_id; - } - - public void set_task_id(int task_id) { - this.task_id = task_id; - set_task_id_isSet(true); - } - - public void unset_task_id() { - __isset_bit_vector.clear(__TASK_ID_ISSET_ID); - } - - /** Returns true if field task_id is set (has been assigned a value) and false otherwise */ - public boolean is_set_task_id() { - return __isset_bit_vector.get(__TASK_ID_ISSET_ID); - } - - public void set_task_id_isSet(boolean value) { - __isset_bit_vector.set(__TASK_ID_ISSET_ID, value); - } - - public String get_component_id() { - return this.component_id; - } - - public void set_component_id(String component_id) { - this.component_id = component_id; - } - - public void unset_component_id() { - this.component_id = null; - } - - /** Returns true if field component_id is set (has been assigned a value) and false otherwise */ - public boolean is_set_component_id() { - return this.component_id != null; - } - - public void set_component_id_isSet(boolean value) { - if (!value) { - this.component_id = null; - } - } - - public int get_gauge_size() { - return (this.gauge == null) ? 0 : this.gauge.size(); - } - - public void put_to_gauge(String key, double val) { - if (this.gauge == null) { - this.gauge = new HashMap(); - } - this.gauge.put(key, val); - } - - public Map get_gauge() { - return this.gauge; - } - - public void set_gauge(Map gauge) { - this.gauge = gauge; - } - - public void unset_gauge() { - this.gauge = null; - } - - /** Returns true if field gauge is set (has been assigned a value) and false otherwise */ - public boolean is_set_gauge() { - return this.gauge != null; - } - - public void set_gauge_isSet(boolean value) { - if (!value) { - this.gauge = null; - } - } - - public int get_counter_size() { - return (this.counter == null) ? 0 : this.counter.size(); - } - - public void put_to_counter(String key, double val) { - if (this.counter == null) { - this.counter = new HashMap(); - } - this.counter.put(key, val); - } - - public Map get_counter() { - return this.counter; - } - - public void set_counter(Map counter) { - this.counter = counter; - } - - public void unset_counter() { - this.counter = null; - } - - /** Returns true if field counter is set (has been assigned a value) and false otherwise */ - public boolean is_set_counter() { - return this.counter != null; - } - - public void set_counter_isSet(boolean value) { - if (!value) { - this.counter = null; - } - } - - public int get_meter_size() { - return (this.meter == null) ? 0 : this.meter.size(); - } - - public void put_to_meter(String key, double val) { - if (this.meter == null) { - this.meter = new HashMap(); - } - this.meter.put(key, val); - } - - public Map get_meter() { - return this.meter; - } - - public void set_meter(Map meter) { - this.meter = meter; - } - - public void unset_meter() { - this.meter = null; - } - - /** Returns true if field meter is set (has been assigned a value) and false otherwise */ - public boolean is_set_meter() { - return this.meter != null; - } - - public void set_meter_isSet(boolean value) { - if (!value) { - this.meter = null; - } - } - - public int get_timer_size() { - return (this.timer == null) ? 0 : this.timer.size(); - } - - public void put_to_timer(String key, double val) { - if (this.timer == null) { - this.timer = new HashMap(); - } - this.timer.put(key, val); - } - - public Map get_timer() { - return this.timer; - } - - public void set_timer(Map timer) { - this.timer = timer; - } - - public void unset_timer() { - this.timer = null; - } - - /** Returns true if field timer is set (has been assigned a value) and false otherwise */ - public boolean is_set_timer() { - return this.timer != null; - } - - public void set_timer_isSet(boolean value) { - if (!value) { - this.timer = null; - } - } - - public int get_histogram_size() { - return (this.histogram == null) ? 0 : this.histogram.size(); - } - - public void put_to_histogram(String key, double val) { - if (this.histogram == null) { - this.histogram = new HashMap(); - } - this.histogram.put(key, val); - } - - public Map get_histogram() { - return this.histogram; - } - - public void set_histogram(Map histogram) { - this.histogram = histogram; - } - - public void unset_histogram() { - this.histogram = null; - } - - /** Returns true if field histogram is set (has been assigned a value) and false otherwise */ - public boolean is_set_histogram() { - return this.histogram != null; - } - - public void set_histogram_isSet(boolean value) { - if (!value) { - this.histogram = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case TASK_ID: - if (value == null) { - unset_task_id(); - } else { - set_task_id((Integer)value); - } - break; - - case COMPONENT_ID: - if (value == null) { - unset_component_id(); - } else { - set_component_id((String)value); - } - break; - - case GAUGE: - if (value == null) { - unset_gauge(); - } else { - set_gauge((Map)value); - } - break; - - case COUNTER: - if (value == null) { - unset_counter(); - } else { - set_counter((Map)value); - } - break; - - case METER: - if (value == null) { - unset_meter(); - } else { - set_meter((Map)value); - } - break; - - case TIMER: - if (value == null) { - unset_timer(); - } else { - set_timer((Map)value); - } - break; - - case HISTOGRAM: - if (value == null) { - unset_histogram(); - } else { - set_histogram((Map)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case TASK_ID: - return Integer.valueOf(get_task_id()); - - case COMPONENT_ID: - return get_component_id(); - - case GAUGE: - return get_gauge(); - - case COUNTER: - return get_counter(); - - case METER: - return get_meter(); - - case TIMER: - return get_timer(); - - case HISTOGRAM: - return get_histogram(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case TASK_ID: - return is_set_task_id(); - case COMPONENT_ID: - return is_set_component_id(); - case GAUGE: - return is_set_gauge(); - case COUNTER: - return is_set_counter(); - case METER: - return is_set_meter(); - case TIMER: - return is_set_timer(); - case HISTOGRAM: - return is_set_histogram(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TaskMetricData) - return this.equals((TaskMetricData)that); - return false; - } - - public boolean equals(TaskMetricData that) { - if (that == null) - return false; - - boolean this_present_task_id = true; - boolean that_present_task_id = true; - if (this_present_task_id || that_present_task_id) { - if (!(this_present_task_id && that_present_task_id)) - return false; - if (this.task_id != that.task_id) - return false; - } - - boolean this_present_component_id = true && this.is_set_component_id(); - boolean that_present_component_id = true && that.is_set_component_id(); - if (this_present_component_id || that_present_component_id) { - if (!(this_present_component_id && that_present_component_id)) - return false; - if (!this.component_id.equals(that.component_id)) - return false; - } - - boolean this_present_gauge = true && this.is_set_gauge(); - boolean that_present_gauge = true && that.is_set_gauge(); - if (this_present_gauge || that_present_gauge) { - if (!(this_present_gauge && that_present_gauge)) - return false; - if (!this.gauge.equals(that.gauge)) - return false; - } - - boolean this_present_counter = true && this.is_set_counter(); - boolean that_present_counter = true && that.is_set_counter(); - if (this_present_counter || that_present_counter) { - if (!(this_present_counter && that_present_counter)) - return false; - if (!this.counter.equals(that.counter)) - return false; - } - - boolean this_present_meter = true && this.is_set_meter(); - boolean that_present_meter = true && that.is_set_meter(); - if (this_present_meter || that_present_meter) { - if (!(this_present_meter && that_present_meter)) - return false; - if (!this.meter.equals(that.meter)) - return false; - } - - boolean this_present_timer = true && this.is_set_timer(); - boolean that_present_timer = true && that.is_set_timer(); - if (this_present_timer || that_present_timer) { - if (!(this_present_timer && that_present_timer)) - return false; - if (!this.timer.equals(that.timer)) - return false; - } - - boolean this_present_histogram = true && this.is_set_histogram(); - boolean that_present_histogram = true && that.is_set_histogram(); - if (this_present_histogram || that_present_histogram) { - if (!(this_present_histogram && that_present_histogram)) - return false; - if (!this.histogram.equals(that.histogram)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_task_id = true; - builder.append(present_task_id); - if (present_task_id) - builder.append(task_id); - - boolean present_component_id = true && (is_set_component_id()); - builder.append(present_component_id); - if (present_component_id) - builder.append(component_id); - - boolean present_gauge = true && (is_set_gauge()); - builder.append(present_gauge); - if (present_gauge) - builder.append(gauge); - - boolean present_counter = true && (is_set_counter()); - builder.append(present_counter); - if (present_counter) - builder.append(counter); - - boolean present_meter = true && (is_set_meter()); - builder.append(present_meter); - if (present_meter) - builder.append(meter); - - boolean present_timer = true && (is_set_timer()); - builder.append(present_timer); - if (present_timer) - builder.append(timer); - - boolean present_histogram = true && (is_set_histogram()); - builder.append(present_histogram); - if (present_histogram) - builder.append(histogram); - - return builder.toHashCode(); - } - - public int compareTo(TaskMetricData other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TaskMetricData typedOther = (TaskMetricData)other; - - lastComparison = Boolean.valueOf(is_set_task_id()).compareTo(typedOther.is_set_task_id()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_task_id()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.task_id, typedOther.task_id); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_component_id()).compareTo(typedOther.is_set_component_id()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_component_id()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.component_id, typedOther.component_id); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_gauge()).compareTo(typedOther.is_set_gauge()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_gauge()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.gauge, typedOther.gauge); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_counter()).compareTo(typedOther.is_set_counter()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_counter()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.counter, typedOther.counter); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_meter()).compareTo(typedOther.is_set_meter()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_meter()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.meter, typedOther.meter); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_timer()).compareTo(typedOther.is_set_timer()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_timer()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.timer, typedOther.timer); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_histogram()).compareTo(typedOther.is_set_histogram()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_histogram()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.histogram, typedOther.histogram); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // TASK_ID - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.task_id = iprot.readI32(); - set_task_id_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // COMPONENT_ID - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.component_id = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 3: // GAUGE - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map209 = iprot.readMapBegin(); - this.gauge = new HashMap(2*_map209.size); - for (int _i210 = 0; _i210 < _map209.size; ++_i210) - { - String _key211; // required - double _val212; // required - _key211 = iprot.readString(); - _val212 = iprot.readDouble(); - this.gauge.put(_key211, _val212); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 4: // COUNTER - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map213 = iprot.readMapBegin(); - this.counter = new HashMap(2*_map213.size); - for (int _i214 = 0; _i214 < _map213.size; ++_i214) - { - String _key215; // required - double _val216; // required - _key215 = iprot.readString(); - _val216 = iprot.readDouble(); - this.counter.put(_key215, _val216); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 5: // METER - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map217 = iprot.readMapBegin(); - this.meter = new HashMap(2*_map217.size); - for (int _i218 = 0; _i218 < _map217.size; ++_i218) - { - String _key219; // required - double _val220; // required - _key219 = iprot.readString(); - _val220 = iprot.readDouble(); - this.meter.put(_key219, _val220); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 6: // TIMER - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map221 = iprot.readMapBegin(); - this.timer = new HashMap(2*_map221.size); - for (int _i222 = 0; _i222 < _map221.size; ++_i222) - { - String _key223; // required - double _val224; // required - _key223 = iprot.readString(); - _val224 = iprot.readDouble(); - this.timer.put(_key223, _val224); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 7: // HISTOGRAM - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map225 = iprot.readMapBegin(); - this.histogram = new HashMap(2*_map225.size); - for (int _i226 = 0; _i226 < _map225.size; ++_i226) - { - String _key227; // required - double _val228; // required - _key227 = iprot.readString(); - _val228 = iprot.readDouble(); - this.histogram.put(_key227, _val228); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(TASK_ID_FIELD_DESC); - oprot.writeI32(this.task_id); - oprot.writeFieldEnd(); - if (this.component_id != null) { - oprot.writeFieldBegin(COMPONENT_ID_FIELD_DESC); - oprot.writeString(this.component_id); - oprot.writeFieldEnd(); - } - if (this.gauge != null) { - oprot.writeFieldBegin(GAUGE_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.DOUBLE, this.gauge.size())); - for (Map.Entry _iter229 : this.gauge.entrySet()) - { - oprot.writeString(_iter229.getKey()); - oprot.writeDouble(_iter229.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.counter != null) { - oprot.writeFieldBegin(COUNTER_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.DOUBLE, this.counter.size())); - for (Map.Entry _iter230 : this.counter.entrySet()) - { - oprot.writeString(_iter230.getKey()); - oprot.writeDouble(_iter230.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.meter != null) { - oprot.writeFieldBegin(METER_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.DOUBLE, this.meter.size())); - for (Map.Entry _iter231 : this.meter.entrySet()) - { - oprot.writeString(_iter231.getKey()); - oprot.writeDouble(_iter231.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.timer != null) { - oprot.writeFieldBegin(TIMER_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.DOUBLE, this.timer.size())); - for (Map.Entry _iter232 : this.timer.entrySet()) - { - oprot.writeString(_iter232.getKey()); - oprot.writeDouble(_iter232.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.histogram != null) { - oprot.writeFieldBegin(HISTOGRAM_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.DOUBLE, this.histogram.size())); - for (Map.Entry _iter233 : this.histogram.entrySet()) - { - oprot.writeString(_iter233.getKey()); - oprot.writeDouble(_iter233.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TaskMetricData("); - boolean first = true; - - sb.append("task_id:"); - sb.append(this.task_id); - first = false; - if (!first) sb.append(", "); - sb.append("component_id:"); - if (this.component_id == null) { - sb.append("null"); - } else { - sb.append(this.component_id); - } - first = false; - if (!first) sb.append(", "); - sb.append("gauge:"); - if (this.gauge == null) { - sb.append("null"); - } else { - sb.append(this.gauge); - } - first = false; - if (!first) sb.append(", "); - sb.append("counter:"); - if (this.counter == null) { - sb.append("null"); - } else { - sb.append(this.counter); - } - first = false; - if (!first) sb.append(", "); - sb.append("meter:"); - if (this.meter == null) { - sb.append("null"); - } else { - sb.append(this.meter); - } - first = false; - if (!first) sb.append(", "); - sb.append("timer:"); - if (this.timer == null) { - sb.append("null"); - } else { - sb.append(this.timer); - } - first = false; - if (!first) sb.append(", "); - sb.append("histogram:"); - if (this.histogram == null) { - sb.append("null"); - } else { - sb.append(this.histogram); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_task_id()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'task_id' is unset! Struct:" + toString()); - } - - if (!is_set_component_id()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'component_id' is unset! Struct:" + toString()); - } - - if (!is_set_gauge()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'gauge' is unset! Struct:" + toString()); - } - - if (!is_set_counter()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'counter' is unset! Struct:" + toString()); - } - - if (!is_set_meter()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'meter' is unset! Struct:" + toString()); - } - - if (!is_set_timer()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'timer' is unset! Struct:" + toString()); - } - - if (!is_set_histogram()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'histogram' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bit_vector = new BitSet(1); - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/TaskStats.java b/jstorm-client/src/main/java/backtype/storm/generated/TaskStats.java deleted file mode 100644 index da38eb0ba..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/TaskStats.java +++ /dev/null @@ -1,1285 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TaskStats implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("TaskStats"); - - private static final org.apache.thrift7.protocol.TField EMITTED_FIELD_DESC = new org.apache.thrift7.protocol.TField("emitted", org.apache.thrift7.protocol.TType.MAP, (short)1); - private static final org.apache.thrift7.protocol.TField SEND_TPS_FIELD_DESC = new org.apache.thrift7.protocol.TField("send_tps", org.apache.thrift7.protocol.TType.MAP, (short)2); - private static final org.apache.thrift7.protocol.TField RECV_TPS_FIELD_DESC = new org.apache.thrift7.protocol.TField("recv_tps", org.apache.thrift7.protocol.TType.MAP, (short)3); - private static final org.apache.thrift7.protocol.TField ACKED_FIELD_DESC = new org.apache.thrift7.protocol.TField("acked", org.apache.thrift7.protocol.TType.MAP, (short)4); - private static final org.apache.thrift7.protocol.TField FAILED_FIELD_DESC = new org.apache.thrift7.protocol.TField("failed", org.apache.thrift7.protocol.TType.MAP, (short)5); - private static final org.apache.thrift7.protocol.TField PROCESS_MS_AVG_FIELD_DESC = new org.apache.thrift7.protocol.TField("process_ms_avg", org.apache.thrift7.protocol.TType.MAP, (short)6); - - private Map> emitted; // required - private Map> send_tps; // required - private Map> recv_tps; // required - private Map> acked; // required - private Map> failed; // required - private Map> process_ms_avg; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - EMITTED((short)1, "emitted"), - SEND_TPS((short)2, "send_tps"), - RECV_TPS((short)3, "recv_tps"), - ACKED((short)4, "acked"), - FAILED((short)5, "failed"), - PROCESS_MS_AVG((short)6, "process_ms_avg"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // EMITTED - return EMITTED; - case 2: // SEND_TPS - return SEND_TPS; - case 3: // RECV_TPS - return RECV_TPS; - case 4: // ACKED - return ACKED; - case 5: // FAILED - return FAILED; - case 6: // PROCESS_MS_AVG - return PROCESS_MS_AVG; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.EMITTED, new org.apache.thrift7.meta_data.FieldMetaData("emitted", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I64))))); - tmpMap.put(_Fields.SEND_TPS, new org.apache.thrift7.meta_data.FieldMetaData("send_tps", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE))))); - tmpMap.put(_Fields.RECV_TPS, new org.apache.thrift7.meta_data.FieldMetaData("recv_tps", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, GlobalStreamId.class), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE))))); - tmpMap.put(_Fields.ACKED, new org.apache.thrift7.meta_data.FieldMetaData("acked", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, GlobalStreamId.class), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I64))))); - tmpMap.put(_Fields.FAILED, new org.apache.thrift7.meta_data.FieldMetaData("failed", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, GlobalStreamId.class), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I64))))); - tmpMap.put(_Fields.PROCESS_MS_AVG, new org.apache.thrift7.meta_data.FieldMetaData("process_ms_avg", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, GlobalStreamId.class), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE))))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(TaskStats.class, metaDataMap); - } - - public TaskStats() { - } - - public TaskStats( - Map> emitted, - Map> send_tps, - Map> recv_tps, - Map> acked, - Map> failed, - Map> process_ms_avg) - { - this(); - this.emitted = emitted; - this.send_tps = send_tps; - this.recv_tps = recv_tps; - this.acked = acked; - this.failed = failed; - this.process_ms_avg = process_ms_avg; - } - - /** - * Performs a deep copy on other. - */ - public TaskStats(TaskStats other) { - if (other.is_set_emitted()) { - Map> __this__emitted = new HashMap>(); - for (Map.Entry> other_element : other.emitted.entrySet()) { - - String other_element_key = other_element.getKey(); - Map other_element_value = other_element.getValue(); - - String __this__emitted_copy_key = other_element_key; - - Map __this__emitted_copy_value = new HashMap(); - for (Map.Entry other_element_value_element : other_element_value.entrySet()) { - - String other_element_value_element_key = other_element_value_element.getKey(); - Long other_element_value_element_value = other_element_value_element.getValue(); - - String __this__emitted_copy_value_copy_key = other_element_value_element_key; - - Long __this__emitted_copy_value_copy_value = other_element_value_element_value; - - __this__emitted_copy_value.put(__this__emitted_copy_value_copy_key, __this__emitted_copy_value_copy_value); - } - - __this__emitted.put(__this__emitted_copy_key, __this__emitted_copy_value); - } - this.emitted = __this__emitted; - } - if (other.is_set_send_tps()) { - Map> __this__send_tps = new HashMap>(); - for (Map.Entry> other_element : other.send_tps.entrySet()) { - - String other_element_key = other_element.getKey(); - Map other_element_value = other_element.getValue(); - - String __this__send_tps_copy_key = other_element_key; - - Map __this__send_tps_copy_value = new HashMap(); - for (Map.Entry other_element_value_element : other_element_value.entrySet()) { - - String other_element_value_element_key = other_element_value_element.getKey(); - Double other_element_value_element_value = other_element_value_element.getValue(); - - String __this__send_tps_copy_value_copy_key = other_element_value_element_key; - - Double __this__send_tps_copy_value_copy_value = other_element_value_element_value; - - __this__send_tps_copy_value.put(__this__send_tps_copy_value_copy_key, __this__send_tps_copy_value_copy_value); - } - - __this__send_tps.put(__this__send_tps_copy_key, __this__send_tps_copy_value); - } - this.send_tps = __this__send_tps; - } - if (other.is_set_recv_tps()) { - Map> __this__recv_tps = new HashMap>(); - for (Map.Entry> other_element : other.recv_tps.entrySet()) { - - String other_element_key = other_element.getKey(); - Map other_element_value = other_element.getValue(); - - String __this__recv_tps_copy_key = other_element_key; - - Map __this__recv_tps_copy_value = new HashMap(); - for (Map.Entry other_element_value_element : other_element_value.entrySet()) { - - GlobalStreamId other_element_value_element_key = other_element_value_element.getKey(); - Double other_element_value_element_value = other_element_value_element.getValue(); - - GlobalStreamId __this__recv_tps_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key); - - Double __this__recv_tps_copy_value_copy_value = other_element_value_element_value; - - __this__recv_tps_copy_value.put(__this__recv_tps_copy_value_copy_key, __this__recv_tps_copy_value_copy_value); - } - - __this__recv_tps.put(__this__recv_tps_copy_key, __this__recv_tps_copy_value); - } - this.recv_tps = __this__recv_tps; - } - if (other.is_set_acked()) { - Map> __this__acked = new HashMap>(); - for (Map.Entry> other_element : other.acked.entrySet()) { - - String other_element_key = other_element.getKey(); - Map other_element_value = other_element.getValue(); - - String __this__acked_copy_key = other_element_key; - - Map __this__acked_copy_value = new HashMap(); - for (Map.Entry other_element_value_element : other_element_value.entrySet()) { - - GlobalStreamId other_element_value_element_key = other_element_value_element.getKey(); - Long other_element_value_element_value = other_element_value_element.getValue(); - - GlobalStreamId __this__acked_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key); - - Long __this__acked_copy_value_copy_value = other_element_value_element_value; - - __this__acked_copy_value.put(__this__acked_copy_value_copy_key, __this__acked_copy_value_copy_value); - } - - __this__acked.put(__this__acked_copy_key, __this__acked_copy_value); - } - this.acked = __this__acked; - } - if (other.is_set_failed()) { - Map> __this__failed = new HashMap>(); - for (Map.Entry> other_element : other.failed.entrySet()) { - - String other_element_key = other_element.getKey(); - Map other_element_value = other_element.getValue(); - - String __this__failed_copy_key = other_element_key; - - Map __this__failed_copy_value = new HashMap(); - for (Map.Entry other_element_value_element : other_element_value.entrySet()) { - - GlobalStreamId other_element_value_element_key = other_element_value_element.getKey(); - Long other_element_value_element_value = other_element_value_element.getValue(); - - GlobalStreamId __this__failed_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key); - - Long __this__failed_copy_value_copy_value = other_element_value_element_value; - - __this__failed_copy_value.put(__this__failed_copy_value_copy_key, __this__failed_copy_value_copy_value); - } - - __this__failed.put(__this__failed_copy_key, __this__failed_copy_value); - } - this.failed = __this__failed; - } - if (other.is_set_process_ms_avg()) { - Map> __this__process_ms_avg = new HashMap>(); - for (Map.Entry> other_element : other.process_ms_avg.entrySet()) { - - String other_element_key = other_element.getKey(); - Map other_element_value = other_element.getValue(); - - String __this__process_ms_avg_copy_key = other_element_key; - - Map __this__process_ms_avg_copy_value = new HashMap(); - for (Map.Entry other_element_value_element : other_element_value.entrySet()) { - - GlobalStreamId other_element_value_element_key = other_element_value_element.getKey(); - Double other_element_value_element_value = other_element_value_element.getValue(); - - GlobalStreamId __this__process_ms_avg_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key); - - Double __this__process_ms_avg_copy_value_copy_value = other_element_value_element_value; - - __this__process_ms_avg_copy_value.put(__this__process_ms_avg_copy_value_copy_key, __this__process_ms_avg_copy_value_copy_value); - } - - __this__process_ms_avg.put(__this__process_ms_avg_copy_key, __this__process_ms_avg_copy_value); - } - this.process_ms_avg = __this__process_ms_avg; - } - } - - public TaskStats deepCopy() { - return new TaskStats(this); - } - - @Override - public void clear() { - this.emitted = null; - this.send_tps = null; - this.recv_tps = null; - this.acked = null; - this.failed = null; - this.process_ms_avg = null; - } - - public int get_emitted_size() { - return (this.emitted == null) ? 0 : this.emitted.size(); - } - - public void put_to_emitted(String key, Map val) { - if (this.emitted == null) { - this.emitted = new HashMap>(); - } - this.emitted.put(key, val); - } - - public Map> get_emitted() { - return this.emitted; - } - - public void set_emitted(Map> emitted) { - this.emitted = emitted; - } - - public void unset_emitted() { - this.emitted = null; - } - - /** Returns true if field emitted is set (has been assigned a value) and false otherwise */ - public boolean is_set_emitted() { - return this.emitted != null; - } - - public void set_emitted_isSet(boolean value) { - if (!value) { - this.emitted = null; - } - } - - public int get_send_tps_size() { - return (this.send_tps == null) ? 0 : this.send_tps.size(); - } - - public void put_to_send_tps(String key, Map val) { - if (this.send_tps == null) { - this.send_tps = new HashMap>(); - } - this.send_tps.put(key, val); - } - - public Map> get_send_tps() { - return this.send_tps; - } - - public void set_send_tps(Map> send_tps) { - this.send_tps = send_tps; - } - - public void unset_send_tps() { - this.send_tps = null; - } - - /** Returns true if field send_tps is set (has been assigned a value) and false otherwise */ - public boolean is_set_send_tps() { - return this.send_tps != null; - } - - public void set_send_tps_isSet(boolean value) { - if (!value) { - this.send_tps = null; - } - } - - public int get_recv_tps_size() { - return (this.recv_tps == null) ? 0 : this.recv_tps.size(); - } - - public void put_to_recv_tps(String key, Map val) { - if (this.recv_tps == null) { - this.recv_tps = new HashMap>(); - } - this.recv_tps.put(key, val); - } - - public Map> get_recv_tps() { - return this.recv_tps; - } - - public void set_recv_tps(Map> recv_tps) { - this.recv_tps = recv_tps; - } - - public void unset_recv_tps() { - this.recv_tps = null; - } - - /** Returns true if field recv_tps is set (has been assigned a value) and false otherwise */ - public boolean is_set_recv_tps() { - return this.recv_tps != null; - } - - public void set_recv_tps_isSet(boolean value) { - if (!value) { - this.recv_tps = null; - } - } - - public int get_acked_size() { - return (this.acked == null) ? 0 : this.acked.size(); - } - - public void put_to_acked(String key, Map val) { - if (this.acked == null) { - this.acked = new HashMap>(); - } - this.acked.put(key, val); - } - - public Map> get_acked() { - return this.acked; - } - - public void set_acked(Map> acked) { - this.acked = acked; - } - - public void unset_acked() { - this.acked = null; - } - - /** Returns true if field acked is set (has been assigned a value) and false otherwise */ - public boolean is_set_acked() { - return this.acked != null; - } - - public void set_acked_isSet(boolean value) { - if (!value) { - this.acked = null; - } - } - - public int get_failed_size() { - return (this.failed == null) ? 0 : this.failed.size(); - } - - public void put_to_failed(String key, Map val) { - if (this.failed == null) { - this.failed = new HashMap>(); - } - this.failed.put(key, val); - } - - public Map> get_failed() { - return this.failed; - } - - public void set_failed(Map> failed) { - this.failed = failed; - } - - public void unset_failed() { - this.failed = null; - } - - /** Returns true if field failed is set (has been assigned a value) and false otherwise */ - public boolean is_set_failed() { - return this.failed != null; - } - - public void set_failed_isSet(boolean value) { - if (!value) { - this.failed = null; - } - } - - public int get_process_ms_avg_size() { - return (this.process_ms_avg == null) ? 0 : this.process_ms_avg.size(); - } - - public void put_to_process_ms_avg(String key, Map val) { - if (this.process_ms_avg == null) { - this.process_ms_avg = new HashMap>(); - } - this.process_ms_avg.put(key, val); - } - - public Map> get_process_ms_avg() { - return this.process_ms_avg; - } - - public void set_process_ms_avg(Map> process_ms_avg) { - this.process_ms_avg = process_ms_avg; - } - - public void unset_process_ms_avg() { - this.process_ms_avg = null; - } - - /** Returns true if field process_ms_avg is set (has been assigned a value) and false otherwise */ - public boolean is_set_process_ms_avg() { - return this.process_ms_avg != null; - } - - public void set_process_ms_avg_isSet(boolean value) { - if (!value) { - this.process_ms_avg = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case EMITTED: - if (value == null) { - unset_emitted(); - } else { - set_emitted((Map>)value); - } - break; - - case SEND_TPS: - if (value == null) { - unset_send_tps(); - } else { - set_send_tps((Map>)value); - } - break; - - case RECV_TPS: - if (value == null) { - unset_recv_tps(); - } else { - set_recv_tps((Map>)value); - } - break; - - case ACKED: - if (value == null) { - unset_acked(); - } else { - set_acked((Map>)value); - } - break; - - case FAILED: - if (value == null) { - unset_failed(); - } else { - set_failed((Map>)value); - } - break; - - case PROCESS_MS_AVG: - if (value == null) { - unset_process_ms_avg(); - } else { - set_process_ms_avg((Map>)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case EMITTED: - return get_emitted(); - - case SEND_TPS: - return get_send_tps(); - - case RECV_TPS: - return get_recv_tps(); - - case ACKED: - return get_acked(); - - case FAILED: - return get_failed(); - - case PROCESS_MS_AVG: - return get_process_ms_avg(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case EMITTED: - return is_set_emitted(); - case SEND_TPS: - return is_set_send_tps(); - case RECV_TPS: - return is_set_recv_tps(); - case ACKED: - return is_set_acked(); - case FAILED: - return is_set_failed(); - case PROCESS_MS_AVG: - return is_set_process_ms_avg(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TaskStats) - return this.equals((TaskStats)that); - return false; - } - - public boolean equals(TaskStats that) { - if (that == null) - return false; - - boolean this_present_emitted = true && this.is_set_emitted(); - boolean that_present_emitted = true && that.is_set_emitted(); - if (this_present_emitted || that_present_emitted) { - if (!(this_present_emitted && that_present_emitted)) - return false; - if (!this.emitted.equals(that.emitted)) - return false; - } - - boolean this_present_send_tps = true && this.is_set_send_tps(); - boolean that_present_send_tps = true && that.is_set_send_tps(); - if (this_present_send_tps || that_present_send_tps) { - if (!(this_present_send_tps && that_present_send_tps)) - return false; - if (!this.send_tps.equals(that.send_tps)) - return false; - } - - boolean this_present_recv_tps = true && this.is_set_recv_tps(); - boolean that_present_recv_tps = true && that.is_set_recv_tps(); - if (this_present_recv_tps || that_present_recv_tps) { - if (!(this_present_recv_tps && that_present_recv_tps)) - return false; - if (!this.recv_tps.equals(that.recv_tps)) - return false; - } - - boolean this_present_acked = true && this.is_set_acked(); - boolean that_present_acked = true && that.is_set_acked(); - if (this_present_acked || that_present_acked) { - if (!(this_present_acked && that_present_acked)) - return false; - if (!this.acked.equals(that.acked)) - return false; - } - - boolean this_present_failed = true && this.is_set_failed(); - boolean that_present_failed = true && that.is_set_failed(); - if (this_present_failed || that_present_failed) { - if (!(this_present_failed && that_present_failed)) - return false; - if (!this.failed.equals(that.failed)) - return false; - } - - boolean this_present_process_ms_avg = true && this.is_set_process_ms_avg(); - boolean that_present_process_ms_avg = true && that.is_set_process_ms_avg(); - if (this_present_process_ms_avg || that_present_process_ms_avg) { - if (!(this_present_process_ms_avg && that_present_process_ms_avg)) - return false; - if (!this.process_ms_avg.equals(that.process_ms_avg)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_emitted = true && (is_set_emitted()); - builder.append(present_emitted); - if (present_emitted) - builder.append(emitted); - - boolean present_send_tps = true && (is_set_send_tps()); - builder.append(present_send_tps); - if (present_send_tps) - builder.append(send_tps); - - boolean present_recv_tps = true && (is_set_recv_tps()); - builder.append(present_recv_tps); - if (present_recv_tps) - builder.append(recv_tps); - - boolean present_acked = true && (is_set_acked()); - builder.append(present_acked); - if (present_acked) - builder.append(acked); - - boolean present_failed = true && (is_set_failed()); - builder.append(present_failed); - if (present_failed) - builder.append(failed); - - boolean present_process_ms_avg = true && (is_set_process_ms_avg()); - builder.append(present_process_ms_avg); - if (present_process_ms_avg) - builder.append(process_ms_avg); - - return builder.toHashCode(); - } - - public int compareTo(TaskStats other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TaskStats typedOther = (TaskStats)other; - - lastComparison = Boolean.valueOf(is_set_emitted()).compareTo(typedOther.is_set_emitted()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_emitted()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.emitted, typedOther.emitted); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_send_tps()).compareTo(typedOther.is_set_send_tps()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_send_tps()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.send_tps, typedOther.send_tps); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_recv_tps()).compareTo(typedOther.is_set_recv_tps()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_recv_tps()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.recv_tps, typedOther.recv_tps); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_acked()).compareTo(typedOther.is_set_acked()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_acked()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.acked, typedOther.acked); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_failed()).compareTo(typedOther.is_set_failed()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_failed()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.failed, typedOther.failed); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_process_ms_avg()).compareTo(typedOther.is_set_process_ms_avg()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_process_ms_avg()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.process_ms_avg, typedOther.process_ms_avg); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // EMITTED - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map125 = iprot.readMapBegin(); - this.emitted = new HashMap>(2*_map125.size); - for (int _i126 = 0; _i126 < _map125.size; ++_i126) - { - String _key127; // required - Map _val128; // required - _key127 = iprot.readString(); - { - org.apache.thrift7.protocol.TMap _map129 = iprot.readMapBegin(); - _val128 = new HashMap(2*_map129.size); - for (int _i130 = 0; _i130 < _map129.size; ++_i130) - { - String _key131; // required - long _val132; // required - _key131 = iprot.readString(); - _val132 = iprot.readI64(); - _val128.put(_key131, _val132); - } - iprot.readMapEnd(); - } - this.emitted.put(_key127, _val128); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // SEND_TPS - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map133 = iprot.readMapBegin(); - this.send_tps = new HashMap>(2*_map133.size); - for (int _i134 = 0; _i134 < _map133.size; ++_i134) - { - String _key135; // required - Map _val136; // required - _key135 = iprot.readString(); - { - org.apache.thrift7.protocol.TMap _map137 = iprot.readMapBegin(); - _val136 = new HashMap(2*_map137.size); - for (int _i138 = 0; _i138 < _map137.size; ++_i138) - { - String _key139; // required - double _val140; // required - _key139 = iprot.readString(); - _val140 = iprot.readDouble(); - _val136.put(_key139, _val140); - } - iprot.readMapEnd(); - } - this.send_tps.put(_key135, _val136); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 3: // RECV_TPS - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map141 = iprot.readMapBegin(); - this.recv_tps = new HashMap>(2*_map141.size); - for (int _i142 = 0; _i142 < _map141.size; ++_i142) - { - String _key143; // required - Map _val144; // required - _key143 = iprot.readString(); - { - org.apache.thrift7.protocol.TMap _map145 = iprot.readMapBegin(); - _val144 = new HashMap(2*_map145.size); - for (int _i146 = 0; _i146 < _map145.size; ++_i146) - { - GlobalStreamId _key147; // required - double _val148; // required - _key147 = new GlobalStreamId(); - _key147.read(iprot); - _val148 = iprot.readDouble(); - _val144.put(_key147, _val148); - } - iprot.readMapEnd(); - } - this.recv_tps.put(_key143, _val144); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 4: // ACKED - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map149 = iprot.readMapBegin(); - this.acked = new HashMap>(2*_map149.size); - for (int _i150 = 0; _i150 < _map149.size; ++_i150) - { - String _key151; // required - Map _val152; // required - _key151 = iprot.readString(); - { - org.apache.thrift7.protocol.TMap _map153 = iprot.readMapBegin(); - _val152 = new HashMap(2*_map153.size); - for (int _i154 = 0; _i154 < _map153.size; ++_i154) - { - GlobalStreamId _key155; // required - long _val156; // required - _key155 = new GlobalStreamId(); - _key155.read(iprot); - _val156 = iprot.readI64(); - _val152.put(_key155, _val156); - } - iprot.readMapEnd(); - } - this.acked.put(_key151, _val152); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 5: // FAILED - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map157 = iprot.readMapBegin(); - this.failed = new HashMap>(2*_map157.size); - for (int _i158 = 0; _i158 < _map157.size; ++_i158) - { - String _key159; // required - Map _val160; // required - _key159 = iprot.readString(); - { - org.apache.thrift7.protocol.TMap _map161 = iprot.readMapBegin(); - _val160 = new HashMap(2*_map161.size); - for (int _i162 = 0; _i162 < _map161.size; ++_i162) - { - GlobalStreamId _key163; // required - long _val164; // required - _key163 = new GlobalStreamId(); - _key163.read(iprot); - _val164 = iprot.readI64(); - _val160.put(_key163, _val164); - } - iprot.readMapEnd(); - } - this.failed.put(_key159, _val160); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 6: // PROCESS_MS_AVG - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map165 = iprot.readMapBegin(); - this.process_ms_avg = new HashMap>(2*_map165.size); - for (int _i166 = 0; _i166 < _map165.size; ++_i166) - { - String _key167; // required - Map _val168; // required - _key167 = iprot.readString(); - { - org.apache.thrift7.protocol.TMap _map169 = iprot.readMapBegin(); - _val168 = new HashMap(2*_map169.size); - for (int _i170 = 0; _i170 < _map169.size; ++_i170) - { - GlobalStreamId _key171; // required - double _val172; // required - _key171 = new GlobalStreamId(); - _key171.read(iprot); - _val172 = iprot.readDouble(); - _val168.put(_key171, _val172); - } - iprot.readMapEnd(); - } - this.process_ms_avg.put(_key167, _val168); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.emitted != null) { - oprot.writeFieldBegin(EMITTED_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.emitted.size())); - for (Map.Entry> _iter173 : this.emitted.entrySet()) - { - oprot.writeString(_iter173.getKey()); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.I64, _iter173.getValue().size())); - for (Map.Entry _iter174 : _iter173.getValue().entrySet()) - { - oprot.writeString(_iter174.getKey()); - oprot.writeI64(_iter174.getValue()); - } - oprot.writeMapEnd(); - } - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.send_tps != null) { - oprot.writeFieldBegin(SEND_TPS_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.send_tps.size())); - for (Map.Entry> _iter175 : this.send_tps.entrySet()) - { - oprot.writeString(_iter175.getKey()); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.DOUBLE, _iter175.getValue().size())); - for (Map.Entry _iter176 : _iter175.getValue().entrySet()) - { - oprot.writeString(_iter176.getKey()); - oprot.writeDouble(_iter176.getValue()); - } - oprot.writeMapEnd(); - } - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.recv_tps != null) { - oprot.writeFieldBegin(RECV_TPS_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.recv_tps.size())); - for (Map.Entry> _iter177 : this.recv_tps.entrySet()) - { - oprot.writeString(_iter177.getKey()); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRUCT, org.apache.thrift7.protocol.TType.DOUBLE, _iter177.getValue().size())); - for (Map.Entry _iter178 : _iter177.getValue().entrySet()) - { - _iter178.getKey().write(oprot); - oprot.writeDouble(_iter178.getValue()); - } - oprot.writeMapEnd(); - } - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.acked != null) { - oprot.writeFieldBegin(ACKED_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.acked.size())); - for (Map.Entry> _iter179 : this.acked.entrySet()) - { - oprot.writeString(_iter179.getKey()); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRUCT, org.apache.thrift7.protocol.TType.I64, _iter179.getValue().size())); - for (Map.Entry _iter180 : _iter179.getValue().entrySet()) - { - _iter180.getKey().write(oprot); - oprot.writeI64(_iter180.getValue()); - } - oprot.writeMapEnd(); - } - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.failed != null) { - oprot.writeFieldBegin(FAILED_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.failed.size())); - for (Map.Entry> _iter181 : this.failed.entrySet()) - { - oprot.writeString(_iter181.getKey()); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRUCT, org.apache.thrift7.protocol.TType.I64, _iter181.getValue().size())); - for (Map.Entry _iter182 : _iter181.getValue().entrySet()) - { - _iter182.getKey().write(oprot); - oprot.writeI64(_iter182.getValue()); - } - oprot.writeMapEnd(); - } - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.process_ms_avg != null) { - oprot.writeFieldBegin(PROCESS_MS_AVG_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.MAP, this.process_ms_avg.size())); - for (Map.Entry> _iter183 : this.process_ms_avg.entrySet()) - { - oprot.writeString(_iter183.getKey()); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRUCT, org.apache.thrift7.protocol.TType.DOUBLE, _iter183.getValue().size())); - for (Map.Entry _iter184 : _iter183.getValue().entrySet()) - { - _iter184.getKey().write(oprot); - oprot.writeDouble(_iter184.getValue()); - } - oprot.writeMapEnd(); - } - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TaskStats("); - boolean first = true; - - sb.append("emitted:"); - if (this.emitted == null) { - sb.append("null"); - } else { - sb.append(this.emitted); - } - first = false; - if (!first) sb.append(", "); - sb.append("send_tps:"); - if (this.send_tps == null) { - sb.append("null"); - } else { - sb.append(this.send_tps); - } - first = false; - if (!first) sb.append(", "); - sb.append("recv_tps:"); - if (this.recv_tps == null) { - sb.append("null"); - } else { - sb.append(this.recv_tps); - } - first = false; - if (!first) sb.append(", "); - sb.append("acked:"); - if (this.acked == null) { - sb.append("null"); - } else { - sb.append(this.acked); - } - first = false; - if (!first) sb.append(", "); - sb.append("failed:"); - if (this.failed == null) { - sb.append("null"); - } else { - sb.append(this.failed); - } - first = false; - if (!first) sb.append(", "); - sb.append("process_ms_avg:"); - if (this.process_ms_avg == null) { - sb.append("null"); - } else { - sb.append(this.process_ms_avg); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_emitted()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'emitted' is unset! Struct:" + toString()); - } - - if (!is_set_send_tps()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'send_tps' is unset! Struct:" + toString()); - } - - if (!is_set_recv_tps()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'recv_tps' is unset! Struct:" + toString()); - } - - if (!is_set_acked()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'acked' is unset! Struct:" + toString()); - } - - if (!is_set_failed()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'failed' is unset! Struct:" + toString()); - } - - if (!is_set_process_ms_avg()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'process_ms_avg' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/TaskSummary.java b/jstorm-client/src/main/java/backtype/storm/generated/TaskSummary.java deleted file mode 100644 index 348377276..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/TaskSummary.java +++ /dev/null @@ -1,1122 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TaskSummary implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("TaskSummary"); - - private static final org.apache.thrift7.protocol.TField TASK_ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("task_id", org.apache.thrift7.protocol.TType.I32, (short)1); - private static final org.apache.thrift7.protocol.TField COMPONENT_ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("component_id", org.apache.thrift7.protocol.TType.STRING, (short)2); - private static final org.apache.thrift7.protocol.TField HOST_FIELD_DESC = new org.apache.thrift7.protocol.TField("host", org.apache.thrift7.protocol.TType.STRING, (short)3); - private static final org.apache.thrift7.protocol.TField PORT_FIELD_DESC = new org.apache.thrift7.protocol.TField("port", org.apache.thrift7.protocol.TType.I32, (short)4); - private static final org.apache.thrift7.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift7.protocol.TField("uptime_secs", org.apache.thrift7.protocol.TType.I32, (short)5); - private static final org.apache.thrift7.protocol.TField ERRORS_FIELD_DESC = new org.apache.thrift7.protocol.TField("errors", org.apache.thrift7.protocol.TType.LIST, (short)6); - private static final org.apache.thrift7.protocol.TField STATS_FIELD_DESC = new org.apache.thrift7.protocol.TField("stats", org.apache.thrift7.protocol.TType.STRUCT, (short)7); - private static final org.apache.thrift7.protocol.TField COMPONENT_TYPE_FIELD_DESC = new org.apache.thrift7.protocol.TField("component_type", org.apache.thrift7.protocol.TType.STRING, (short)8); - private static final org.apache.thrift7.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift7.protocol.TField("status", org.apache.thrift7.protocol.TType.STRING, (short)9); - - private int task_id; // required - private String component_id; // required - private String host; // required - private int port; // required - private int uptime_secs; // required - private List errors; // required - private TaskStats stats; // required - private String component_type; // required - private String status; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - TASK_ID((short)1, "task_id"), - COMPONENT_ID((short)2, "component_id"), - HOST((short)3, "host"), - PORT((short)4, "port"), - UPTIME_SECS((short)5, "uptime_secs"), - ERRORS((short)6, "errors"), - STATS((short)7, "stats"), - COMPONENT_TYPE((short)8, "component_type"), - STATUS((short)9, "status"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // TASK_ID - return TASK_ID; - case 2: // COMPONENT_ID - return COMPONENT_ID; - case 3: // HOST - return HOST; - case 4: // PORT - return PORT; - case 5: // UPTIME_SECS - return UPTIME_SECS; - case 6: // ERRORS - return ERRORS; - case 7: // STATS - return STATS; - case 8: // COMPONENT_TYPE - return COMPONENT_TYPE; - case 9: // STATUS - return STATUS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __TASK_ID_ISSET_ID = 0; - private static final int __PORT_ISSET_ID = 1; - private static final int __UPTIME_SECS_ISSET_ID = 2; - private BitSet __isset_bit_vector = new BitSet(3); - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.TASK_ID, new org.apache.thrift7.meta_data.FieldMetaData("task_id", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - tmpMap.put(_Fields.COMPONENT_ID, new org.apache.thrift7.meta_data.FieldMetaData("component_id", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.HOST, new org.apache.thrift7.meta_data.FieldMetaData("host", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.PORT, new org.apache.thrift7.meta_data.FieldMetaData("port", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift7.meta_data.FieldMetaData("uptime_secs", org.apache.thrift7.TFieldRequirementType.OPTIONAL, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - tmpMap.put(_Fields.ERRORS, new org.apache.thrift7.meta_data.FieldMetaData("errors", org.apache.thrift7.TFieldRequirementType.OPTIONAL, - new org.apache.thrift7.meta_data.ListMetaData(org.apache.thrift7.protocol.TType.LIST, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, ErrorInfo.class)))); - tmpMap.put(_Fields.STATS, new org.apache.thrift7.meta_data.FieldMetaData("stats", org.apache.thrift7.TFieldRequirementType.OPTIONAL, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, TaskStats.class))); - tmpMap.put(_Fields.COMPONENT_TYPE, new org.apache.thrift7.meta_data.FieldMetaData("component_type", org.apache.thrift7.TFieldRequirementType.OPTIONAL, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.STATUS, new org.apache.thrift7.meta_data.FieldMetaData("status", org.apache.thrift7.TFieldRequirementType.OPTIONAL, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(TaskSummary.class, metaDataMap); - } - - public TaskSummary() { - } - - public TaskSummary( - int task_id, - String component_id, - String host, - int port) - { - this(); - this.task_id = task_id; - set_task_id_isSet(true); - this.component_id = component_id; - this.host = host; - this.port = port; - set_port_isSet(true); - } - - /** - * Performs a deep copy on other. - */ - public TaskSummary(TaskSummary other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); - this.task_id = other.task_id; - if (other.is_set_component_id()) { - this.component_id = other.component_id; - } - if (other.is_set_host()) { - this.host = other.host; - } - this.port = other.port; - this.uptime_secs = other.uptime_secs; - if (other.is_set_errors()) { - List __this__errors = new ArrayList(); - for (ErrorInfo other_element : other.errors) { - __this__errors.add(new ErrorInfo(other_element)); - } - this.errors = __this__errors; - } - if (other.is_set_stats()) { - this.stats = new TaskStats(other.stats); - } - if (other.is_set_component_type()) { - this.component_type = other.component_type; - } - if (other.is_set_status()) { - this.status = other.status; - } - } - - public TaskSummary deepCopy() { - return new TaskSummary(this); - } - - @Override - public void clear() { - set_task_id_isSet(false); - this.task_id = 0; - this.component_id = null; - this.host = null; - set_port_isSet(false); - this.port = 0; - set_uptime_secs_isSet(false); - this.uptime_secs = 0; - this.errors = null; - this.stats = null; - this.component_type = null; - this.status = null; - } - - public int get_task_id() { - return this.task_id; - } - - public void set_task_id(int task_id) { - this.task_id = task_id; - set_task_id_isSet(true); - } - - public void unset_task_id() { - __isset_bit_vector.clear(__TASK_ID_ISSET_ID); - } - - /** Returns true if field task_id is set (has been assigned a value) and false otherwise */ - public boolean is_set_task_id() { - return __isset_bit_vector.get(__TASK_ID_ISSET_ID); - } - - public void set_task_id_isSet(boolean value) { - __isset_bit_vector.set(__TASK_ID_ISSET_ID, value); - } - - public String get_component_id() { - return this.component_id; - } - - public void set_component_id(String component_id) { - this.component_id = component_id; - } - - public void unset_component_id() { - this.component_id = null; - } - - /** Returns true if field component_id is set (has been assigned a value) and false otherwise */ - public boolean is_set_component_id() { - return this.component_id != null; - } - - public void set_component_id_isSet(boolean value) { - if (!value) { - this.component_id = null; - } - } - - public String get_host() { - return this.host; - } - - public void set_host(String host) { - this.host = host; - } - - public void unset_host() { - this.host = null; - } - - /** Returns true if field host is set (has been assigned a value) and false otherwise */ - public boolean is_set_host() { - return this.host != null; - } - - public void set_host_isSet(boolean value) { - if (!value) { - this.host = null; - } - } - - public int get_port() { - return this.port; - } - - public void set_port(int port) { - this.port = port; - set_port_isSet(true); - } - - public void unset_port() { - __isset_bit_vector.clear(__PORT_ISSET_ID); - } - - /** Returns true if field port is set (has been assigned a value) and false otherwise */ - public boolean is_set_port() { - return __isset_bit_vector.get(__PORT_ISSET_ID); - } - - public void set_port_isSet(boolean value) { - __isset_bit_vector.set(__PORT_ISSET_ID, value); - } - - public int get_uptime_secs() { - return this.uptime_secs; - } - - public void set_uptime_secs(int uptime_secs) { - this.uptime_secs = uptime_secs; - set_uptime_secs_isSet(true); - } - - public void unset_uptime_secs() { - __isset_bit_vector.clear(__UPTIME_SECS_ISSET_ID); - } - - /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */ - public boolean is_set_uptime_secs() { - return __isset_bit_vector.get(__UPTIME_SECS_ISSET_ID); - } - - public void set_uptime_secs_isSet(boolean value) { - __isset_bit_vector.set(__UPTIME_SECS_ISSET_ID, value); - } - - public int get_errors_size() { - return (this.errors == null) ? 0 : this.errors.size(); - } - - public java.util.Iterator get_errors_iterator() { - return (this.errors == null) ? null : this.errors.iterator(); - } - - public void add_to_errors(ErrorInfo elem) { - if (this.errors == null) { - this.errors = new ArrayList(); - } - this.errors.add(elem); - } - - public List get_errors() { - return this.errors; - } - - public void set_errors(List errors) { - this.errors = errors; - } - - public void unset_errors() { - this.errors = null; - } - - /** Returns true if field errors is set (has been assigned a value) and false otherwise */ - public boolean is_set_errors() { - return this.errors != null; - } - - public void set_errors_isSet(boolean value) { - if (!value) { - this.errors = null; - } - } - - public TaskStats get_stats() { - return this.stats; - } - - public void set_stats(TaskStats stats) { - this.stats = stats; - } - - public void unset_stats() { - this.stats = null; - } - - /** Returns true if field stats is set (has been assigned a value) and false otherwise */ - public boolean is_set_stats() { - return this.stats != null; - } - - public void set_stats_isSet(boolean value) { - if (!value) { - this.stats = null; - } - } - - public String get_component_type() { - return this.component_type; - } - - public void set_component_type(String component_type) { - this.component_type = component_type; - } - - public void unset_component_type() { - this.component_type = null; - } - - /** Returns true if field component_type is set (has been assigned a value) and false otherwise */ - public boolean is_set_component_type() { - return this.component_type != null; - } - - public void set_component_type_isSet(boolean value) { - if (!value) { - this.component_type = null; - } - } - - public String get_status() { - return this.status; - } - - public void set_status(String status) { - this.status = status; - } - - public void unset_status() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean is_set_status() { - return this.status != null; - } - - public void set_status_isSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case TASK_ID: - if (value == null) { - unset_task_id(); - } else { - set_task_id((Integer)value); - } - break; - - case COMPONENT_ID: - if (value == null) { - unset_component_id(); - } else { - set_component_id((String)value); - } - break; - - case HOST: - if (value == null) { - unset_host(); - } else { - set_host((String)value); - } - break; - - case PORT: - if (value == null) { - unset_port(); - } else { - set_port((Integer)value); - } - break; - - case UPTIME_SECS: - if (value == null) { - unset_uptime_secs(); - } else { - set_uptime_secs((Integer)value); - } - break; - - case ERRORS: - if (value == null) { - unset_errors(); - } else { - set_errors((List)value); - } - break; - - case STATS: - if (value == null) { - unset_stats(); - } else { - set_stats((TaskStats)value); - } - break; - - case COMPONENT_TYPE: - if (value == null) { - unset_component_type(); - } else { - set_component_type((String)value); - } - break; - - case STATUS: - if (value == null) { - unset_status(); - } else { - set_status((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case TASK_ID: - return Integer.valueOf(get_task_id()); - - case COMPONENT_ID: - return get_component_id(); - - case HOST: - return get_host(); - - case PORT: - return Integer.valueOf(get_port()); - - case UPTIME_SECS: - return Integer.valueOf(get_uptime_secs()); - - case ERRORS: - return get_errors(); - - case STATS: - return get_stats(); - - case COMPONENT_TYPE: - return get_component_type(); - - case STATUS: - return get_status(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case TASK_ID: - return is_set_task_id(); - case COMPONENT_ID: - return is_set_component_id(); - case HOST: - return is_set_host(); - case PORT: - return is_set_port(); - case UPTIME_SECS: - return is_set_uptime_secs(); - case ERRORS: - return is_set_errors(); - case STATS: - return is_set_stats(); - case COMPONENT_TYPE: - return is_set_component_type(); - case STATUS: - return is_set_status(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TaskSummary) - return this.equals((TaskSummary)that); - return false; - } - - public boolean equals(TaskSummary that) { - if (that == null) - return false; - - boolean this_present_task_id = true; - boolean that_present_task_id = true; - if (this_present_task_id || that_present_task_id) { - if (!(this_present_task_id && that_present_task_id)) - return false; - if (this.task_id != that.task_id) - return false; - } - - boolean this_present_component_id = true && this.is_set_component_id(); - boolean that_present_component_id = true && that.is_set_component_id(); - if (this_present_component_id || that_present_component_id) { - if (!(this_present_component_id && that_present_component_id)) - return false; - if (!this.component_id.equals(that.component_id)) - return false; - } - - boolean this_present_host = true && this.is_set_host(); - boolean that_present_host = true && that.is_set_host(); - if (this_present_host || that_present_host) { - if (!(this_present_host && that_present_host)) - return false; - if (!this.host.equals(that.host)) - return false; - } - - boolean this_present_port = true; - boolean that_present_port = true; - if (this_present_port || that_present_port) { - if (!(this_present_port && that_present_port)) - return false; - if (this.port != that.port) - return false; - } - - boolean this_present_uptime_secs = true && this.is_set_uptime_secs(); - boolean that_present_uptime_secs = true && that.is_set_uptime_secs(); - if (this_present_uptime_secs || that_present_uptime_secs) { - if (!(this_present_uptime_secs && that_present_uptime_secs)) - return false; - if (this.uptime_secs != that.uptime_secs) - return false; - } - - boolean this_present_errors = true && this.is_set_errors(); - boolean that_present_errors = true && that.is_set_errors(); - if (this_present_errors || that_present_errors) { - if (!(this_present_errors && that_present_errors)) - return false; - if (!this.errors.equals(that.errors)) - return false; - } - - boolean this_present_stats = true && this.is_set_stats(); - boolean that_present_stats = true && that.is_set_stats(); - if (this_present_stats || that_present_stats) { - if (!(this_present_stats && that_present_stats)) - return false; - if (!this.stats.equals(that.stats)) - return false; - } - - boolean this_present_component_type = true && this.is_set_component_type(); - boolean that_present_component_type = true && that.is_set_component_type(); - if (this_present_component_type || that_present_component_type) { - if (!(this_present_component_type && that_present_component_type)) - return false; - if (!this.component_type.equals(that.component_type)) - return false; - } - - boolean this_present_status = true && this.is_set_status(); - boolean that_present_status = true && that.is_set_status(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_task_id = true; - builder.append(present_task_id); - if (present_task_id) - builder.append(task_id); - - boolean present_component_id = true && (is_set_component_id()); - builder.append(present_component_id); - if (present_component_id) - builder.append(component_id); - - boolean present_host = true && (is_set_host()); - builder.append(present_host); - if (present_host) - builder.append(host); - - boolean present_port = true; - builder.append(present_port); - if (present_port) - builder.append(port); - - boolean present_uptime_secs = true && (is_set_uptime_secs()); - builder.append(present_uptime_secs); - if (present_uptime_secs) - builder.append(uptime_secs); - - boolean present_errors = true && (is_set_errors()); - builder.append(present_errors); - if (present_errors) - builder.append(errors); - - boolean present_stats = true && (is_set_stats()); - builder.append(present_stats); - if (present_stats) - builder.append(stats); - - boolean present_component_type = true && (is_set_component_type()); - builder.append(present_component_type); - if (present_component_type) - builder.append(component_type); - - boolean present_status = true && (is_set_status()); - builder.append(present_status); - if (present_status) - builder.append(status); - - return builder.toHashCode(); - } - - public int compareTo(TaskSummary other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TaskSummary typedOther = (TaskSummary)other; - - lastComparison = Boolean.valueOf(is_set_task_id()).compareTo(typedOther.is_set_task_id()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_task_id()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.task_id, typedOther.task_id); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_component_id()).compareTo(typedOther.is_set_component_id()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_component_id()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.component_id, typedOther.component_id); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_host()).compareTo(typedOther.is_set_host()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_host()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.host, typedOther.host); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_port()).compareTo(typedOther.is_set_port()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_port()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.port, typedOther.port); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(typedOther.is_set_uptime_secs()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_uptime_secs()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.uptime_secs, typedOther.uptime_secs); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_errors()).compareTo(typedOther.is_set_errors()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_errors()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.errors, typedOther.errors); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_stats()).compareTo(typedOther.is_set_stats()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_stats()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.stats, typedOther.stats); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_component_type()).compareTo(typedOther.is_set_component_type()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_component_type()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.component_type, typedOther.component_type); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_status()).compareTo(typedOther.is_set_status()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_status()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // TASK_ID - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.task_id = iprot.readI32(); - set_task_id_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // COMPONENT_ID - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.component_id = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 3: // HOST - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.host = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 4: // PORT - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.port = iprot.readI32(); - set_port_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 5: // UPTIME_SECS - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.uptime_secs = iprot.readI32(); - set_uptime_secs_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 6: // ERRORS - if (field.type == org.apache.thrift7.protocol.TType.LIST) { - { - org.apache.thrift7.protocol.TList _list185 = iprot.readListBegin(); - this.errors = new ArrayList(_list185.size); - for (int _i186 = 0; _i186 < _list185.size; ++_i186) - { - ErrorInfo _elem187; // required - _elem187 = new ErrorInfo(); - _elem187.read(iprot); - this.errors.add(_elem187); - } - iprot.readListEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 7: // STATS - if (field.type == org.apache.thrift7.protocol.TType.STRUCT) { - this.stats = new TaskStats(); - this.stats.read(iprot); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 8: // COMPONENT_TYPE - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.component_type = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 9: // STATUS - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.status = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(TASK_ID_FIELD_DESC); - oprot.writeI32(this.task_id); - oprot.writeFieldEnd(); - if (this.component_id != null) { - oprot.writeFieldBegin(COMPONENT_ID_FIELD_DESC); - oprot.writeString(this.component_id); - oprot.writeFieldEnd(); - } - if (this.host != null) { - oprot.writeFieldBegin(HOST_FIELD_DESC); - oprot.writeString(this.host); - oprot.writeFieldEnd(); - } - oprot.writeFieldBegin(PORT_FIELD_DESC); - oprot.writeI32(this.port); - oprot.writeFieldEnd(); - if (is_set_uptime_secs()) { - oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC); - oprot.writeI32(this.uptime_secs); - oprot.writeFieldEnd(); - } - if (this.errors != null) { - if (is_set_errors()) { - oprot.writeFieldBegin(ERRORS_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift7.protocol.TList(org.apache.thrift7.protocol.TType.STRUCT, this.errors.size())); - for (ErrorInfo _iter188 : this.errors) - { - _iter188.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - } - if (this.stats != null) { - if (is_set_stats()) { - oprot.writeFieldBegin(STATS_FIELD_DESC); - this.stats.write(oprot); - oprot.writeFieldEnd(); - } - } - if (this.component_type != null) { - if (is_set_component_type()) { - oprot.writeFieldBegin(COMPONENT_TYPE_FIELD_DESC); - oprot.writeString(this.component_type); - oprot.writeFieldEnd(); - } - } - if (this.status != null) { - if (is_set_status()) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - oprot.writeString(this.status); - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TaskSummary("); - boolean first = true; - - sb.append("task_id:"); - sb.append(this.task_id); - first = false; - if (!first) sb.append(", "); - sb.append("component_id:"); - if (this.component_id == null) { - sb.append("null"); - } else { - sb.append(this.component_id); - } - first = false; - if (!first) sb.append(", "); - sb.append("host:"); - if (this.host == null) { - sb.append("null"); - } else { - sb.append(this.host); - } - first = false; - if (!first) sb.append(", "); - sb.append("port:"); - sb.append(this.port); - first = false; - if (is_set_uptime_secs()) { - if (!first) sb.append(", "); - sb.append("uptime_secs:"); - sb.append(this.uptime_secs); - first = false; - } - if (is_set_errors()) { - if (!first) sb.append(", "); - sb.append("errors:"); - if (this.errors == null) { - sb.append("null"); - } else { - sb.append(this.errors); - } - first = false; - } - if (is_set_stats()) { - if (!first) sb.append(", "); - sb.append("stats:"); - if (this.stats == null) { - sb.append("null"); - } else { - sb.append(this.stats); - } - first = false; - } - if (is_set_component_type()) { - if (!first) sb.append(", "); - sb.append("component_type:"); - if (this.component_type == null) { - sb.append("null"); - } else { - sb.append(this.component_type); - } - first = false; - } - if (is_set_status()) { - if (!first) sb.append(", "); - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_task_id()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'task_id' is unset! Struct:" + toString()); - } - - if (!is_set_component_id()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'component_id' is unset! Struct:" + toString()); - } - - if (!is_set_host()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'host' is unset! Struct:" + toString()); - } - - if (!is_set_port()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'port' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bit_vector = new BitSet(1); - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/ThriftResourceType.java b/jstorm-client/src/main/java/backtype/storm/generated/ThriftResourceType.java deleted file mode 100644 index 34617a68f..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/ThriftResourceType.java +++ /dev/null @@ -1,47 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - - -import java.util.Map; -import java.util.HashMap; -import org.apache.thrift7.TEnum; - -public enum ThriftResourceType implements org.apache.thrift7.TEnum { - UNKNOWN(1), - MEM(2), - NET(3); - - private final int value; - - private ThriftResourceType(int value) { - this.value = value; - } - - /** - * Get the integer value of this enum value, as defined in the Thrift IDL. - */ - public int getValue() { - return value; - } - - /** - * Find a the enum type by its integer value, as defined in the Thrift IDL. - * @return null if the value is not found. - */ - public static ThriftResourceType findByValue(int value) { - switch (value) { - case 1: - return UNKNOWN; - case 2: - return MEM; - case 3: - return NET; - default: - return null; - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/generated/TopologyAssignException.java b/jstorm-client/src/main/java/backtype/storm/generated/TopologyAssignException.java deleted file mode 100644 index 2907dab49..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/TopologyAssignException.java +++ /dev/null @@ -1,328 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TopologyAssignException extends Exception implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("TopologyAssignException"); - - private static final org.apache.thrift7.protocol.TField MSG_FIELD_DESC = new org.apache.thrift7.protocol.TField("msg", org.apache.thrift7.protocol.TType.STRING, (short)1); - - private String msg; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - MSG((short)1, "msg"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // MSG - return MSG; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.MSG, new org.apache.thrift7.meta_data.FieldMetaData("msg", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(TopologyAssignException.class, metaDataMap); - } - - public TopologyAssignException() { - } - - public TopologyAssignException( - String msg) - { - this(); - this.msg = msg; - } - - /** - * Performs a deep copy on other. - */ - public TopologyAssignException(TopologyAssignException other) { - if (other.is_set_msg()) { - this.msg = other.msg; - } - } - - public TopologyAssignException deepCopy() { - return new TopologyAssignException(this); - } - - @Override - public void clear() { - this.msg = null; - } - - public String get_msg() { - return this.msg; - } - - public void set_msg(String msg) { - this.msg = msg; - } - - public void unset_msg() { - this.msg = null; - } - - /** Returns true if field msg is set (has been assigned a value) and false otherwise */ - public boolean is_set_msg() { - return this.msg != null; - } - - public void set_msg_isSet(boolean value) { - if (!value) { - this.msg = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case MSG: - if (value == null) { - unset_msg(); - } else { - set_msg((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case MSG: - return get_msg(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case MSG: - return is_set_msg(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TopologyAssignException) - return this.equals((TopologyAssignException)that); - return false; - } - - public boolean equals(TopologyAssignException that) { - if (that == null) - return false; - - boolean this_present_msg = true && this.is_set_msg(); - boolean that_present_msg = true && that.is_set_msg(); - if (this_present_msg || that_present_msg) { - if (!(this_present_msg && that_present_msg)) - return false; - if (!this.msg.equals(that.msg)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_msg = true && (is_set_msg()); - builder.append(present_msg); - if (present_msg) - builder.append(msg); - - return builder.toHashCode(); - } - - public int compareTo(TopologyAssignException other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TopologyAssignException typedOther = (TopologyAssignException)other; - - lastComparison = Boolean.valueOf(is_set_msg()).compareTo(typedOther.is_set_msg()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_msg()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.msg, typedOther.msg); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // MSG - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.msg = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.msg != null) { - oprot.writeFieldBegin(MSG_FIELD_DESC); - oprot.writeString(this.msg); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TopologyAssignException("); - boolean first = true; - - sb.append("msg:"); - if (this.msg == null) { - sb.append("null"); - } else { - sb.append(this.msg); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_msg()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/TopologyInfo.java b/jstorm-client/src/main/java/backtype/storm/generated/TopologyInfo.java deleted file mode 100644 index a2566dc92..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/TopologyInfo.java +++ /dev/null @@ -1,1022 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TopologyInfo implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("TopologyInfo"); - - private static final org.apache.thrift7.protocol.TField ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("id", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)2); - private static final org.apache.thrift7.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift7.protocol.TField("uptime_secs", org.apache.thrift7.protocol.TType.I32, (short)3); - private static final org.apache.thrift7.protocol.TField WORKERS_FIELD_DESC = new org.apache.thrift7.protocol.TField("workers", org.apache.thrift7.protocol.TType.LIST, (short)4); - private static final org.apache.thrift7.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift7.protocol.TField("status", org.apache.thrift7.protocol.TType.STRING, (short)5); - private static final org.apache.thrift7.protocol.TField TASKS_FIELD_DESC = new org.apache.thrift7.protocol.TField("tasks", org.apache.thrift7.protocol.TType.LIST, (short)6); - private static final org.apache.thrift7.protocol.TField USER_DEF_METRIC_FIELD_DESC = new org.apache.thrift7.protocol.TField("userDefMetric", org.apache.thrift7.protocol.TType.LIST, (short)7); - - private String id; // required - private String name; // required - private int uptime_secs; // required - private List workers; // required - private String status; // required - private List tasks; // required - private List userDefMetric; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - ID((short)1, "id"), - NAME((short)2, "name"), - UPTIME_SECS((short)3, "uptime_secs"), - WORKERS((short)4, "workers"), - STATUS((short)5, "status"), - TASKS((short)6, "tasks"), - USER_DEF_METRIC((short)7, "userDefMetric"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // ID - return ID; - case 2: // NAME - return NAME; - case 3: // UPTIME_SECS - return UPTIME_SECS; - case 4: // WORKERS - return WORKERS; - case 5: // STATUS - return STATUS; - case 6: // TASKS - return TASKS; - case 7: // USER_DEF_METRIC - return USER_DEF_METRIC; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __UPTIME_SECS_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.ID, new org.apache.thrift7.meta_data.FieldMetaData("id", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift7.meta_data.FieldMetaData("uptime_secs", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - tmpMap.put(_Fields.WORKERS, new org.apache.thrift7.meta_data.FieldMetaData("workers", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.ListMetaData(org.apache.thrift7.protocol.TType.LIST, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, WorkerSummary.class)))); - tmpMap.put(_Fields.STATUS, new org.apache.thrift7.meta_data.FieldMetaData("status", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.TASKS, new org.apache.thrift7.meta_data.FieldMetaData("tasks", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.ListMetaData(org.apache.thrift7.protocol.TType.LIST, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, TaskSummary.class)))); - tmpMap.put(_Fields.USER_DEF_METRIC, new org.apache.thrift7.meta_data.FieldMetaData("userDefMetric", org.apache.thrift7.TFieldRequirementType.OPTIONAL, - new org.apache.thrift7.meta_data.ListMetaData(org.apache.thrift7.protocol.TType.LIST, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, UserDefMetric.class)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(TopologyInfo.class, metaDataMap); - } - - public TopologyInfo() { - } - - public TopologyInfo( - String id, - String name, - int uptime_secs, - List workers, - String status, - List tasks) - { - this(); - this.id = id; - this.name = name; - this.uptime_secs = uptime_secs; - set_uptime_secs_isSet(true); - this.workers = workers; - this.status = status; - this.tasks = tasks; - } - - /** - * Performs a deep copy on other. - */ - public TopologyInfo(TopologyInfo other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); - if (other.is_set_id()) { - this.id = other.id; - } - if (other.is_set_name()) { - this.name = other.name; - } - this.uptime_secs = other.uptime_secs; - if (other.is_set_workers()) { - List __this__workers = new ArrayList(); - for (WorkerSummary other_element : other.workers) { - __this__workers.add(new WorkerSummary(other_element)); - } - this.workers = __this__workers; - } - if (other.is_set_status()) { - this.status = other.status; - } - if (other.is_set_tasks()) { - List __this__tasks = new ArrayList(); - for (TaskSummary other_element : other.tasks) { - __this__tasks.add(new TaskSummary(other_element)); - } - this.tasks = __this__tasks; - } - if (other.is_set_userDefMetric()) { - List __this__userDefMetric = new ArrayList(); - for (UserDefMetric other_element : other.userDefMetric) { - __this__userDefMetric.add(new UserDefMetric(other_element)); - } - this.userDefMetric = __this__userDefMetric; - } - } - - public TopologyInfo deepCopy() { - return new TopologyInfo(this); - } - - @Override - public void clear() { - this.id = null; - this.name = null; - set_uptime_secs_isSet(false); - this.uptime_secs = 0; - this.workers = null; - this.status = null; - this.tasks = null; - this.userDefMetric = null; - } - - public String get_id() { - return this.id; - } - - public void set_id(String id) { - this.id = id; - } - - public void unset_id() { - this.id = null; - } - - /** Returns true if field id is set (has been assigned a value) and false otherwise */ - public boolean is_set_id() { - return this.id != null; - } - - public void set_id_isSet(boolean value) { - if (!value) { - this.id = null; - } - } - - public String get_name() { - return this.name; - } - - public void set_name(String name) { - this.name = name; - } - - public void unset_name() { - this.name = null; - } - - /** Returns true if field name is set (has been assigned a value) and false otherwise */ - public boolean is_set_name() { - return this.name != null; - } - - public void set_name_isSet(boolean value) { - if (!value) { - this.name = null; - } - } - - public int get_uptime_secs() { - return this.uptime_secs; - } - - public void set_uptime_secs(int uptime_secs) { - this.uptime_secs = uptime_secs; - set_uptime_secs_isSet(true); - } - - public void unset_uptime_secs() { - __isset_bit_vector.clear(__UPTIME_SECS_ISSET_ID); - } - - /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */ - public boolean is_set_uptime_secs() { - return __isset_bit_vector.get(__UPTIME_SECS_ISSET_ID); - } - - public void set_uptime_secs_isSet(boolean value) { - __isset_bit_vector.set(__UPTIME_SECS_ISSET_ID, value); - } - - public int get_workers_size() { - return (this.workers == null) ? 0 : this.workers.size(); - } - - public java.util.Iterator get_workers_iterator() { - return (this.workers == null) ? null : this.workers.iterator(); - } - - public void add_to_workers(WorkerSummary elem) { - if (this.workers == null) { - this.workers = new ArrayList(); - } - this.workers.add(elem); - } - - public List get_workers() { - return this.workers; - } - - public void set_workers(List workers) { - this.workers = workers; - } - - public void unset_workers() { - this.workers = null; - } - - /** Returns true if field workers is set (has been assigned a value) and false otherwise */ - public boolean is_set_workers() { - return this.workers != null; - } - - public void set_workers_isSet(boolean value) { - if (!value) { - this.workers = null; - } - } - - public String get_status() { - return this.status; - } - - public void set_status(String status) { - this.status = status; - } - - public void unset_status() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean is_set_status() { - return this.status != null; - } - - public void set_status_isSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public int get_tasks_size() { - return (this.tasks == null) ? 0 : this.tasks.size(); - } - - public java.util.Iterator get_tasks_iterator() { - return (this.tasks == null) ? null : this.tasks.iterator(); - } - - public void add_to_tasks(TaskSummary elem) { - if (this.tasks == null) { - this.tasks = new ArrayList(); - } - this.tasks.add(elem); - } - - public List get_tasks() { - return this.tasks; - } - - public void set_tasks(List tasks) { - this.tasks = tasks; - } - - public void unset_tasks() { - this.tasks = null; - } - - /** Returns true if field tasks is set (has been assigned a value) and false otherwise */ - public boolean is_set_tasks() { - return this.tasks != null; - } - - public void set_tasks_isSet(boolean value) { - if (!value) { - this.tasks = null; - } - } - - public int get_userDefMetric_size() { - return (this.userDefMetric == null) ? 0 : this.userDefMetric.size(); - } - - public java.util.Iterator get_userDefMetric_iterator() { - return (this.userDefMetric == null) ? null : this.userDefMetric.iterator(); - } - - public void add_to_userDefMetric(UserDefMetric elem) { - if (this.userDefMetric == null) { - this.userDefMetric = new ArrayList(); - } - this.userDefMetric.add(elem); - } - - public List get_userDefMetric() { - return this.userDefMetric; - } - - public void set_userDefMetric(List userDefMetric) { - this.userDefMetric = userDefMetric; - } - - public void unset_userDefMetric() { - this.userDefMetric = null; - } - - /** Returns true if field userDefMetric is set (has been assigned a value) and false otherwise */ - public boolean is_set_userDefMetric() { - return this.userDefMetric != null; - } - - public void set_userDefMetric_isSet(boolean value) { - if (!value) { - this.userDefMetric = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case ID: - if (value == null) { - unset_id(); - } else { - set_id((String)value); - } - break; - - case NAME: - if (value == null) { - unset_name(); - } else { - set_name((String)value); - } - break; - - case UPTIME_SECS: - if (value == null) { - unset_uptime_secs(); - } else { - set_uptime_secs((Integer)value); - } - break; - - case WORKERS: - if (value == null) { - unset_workers(); - } else { - set_workers((List)value); - } - break; - - case STATUS: - if (value == null) { - unset_status(); - } else { - set_status((String)value); - } - break; - - case TASKS: - if (value == null) { - unset_tasks(); - } else { - set_tasks((List)value); - } - break; - - case USER_DEF_METRIC: - if (value == null) { - unset_userDefMetric(); - } else { - set_userDefMetric((List)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case ID: - return get_id(); - - case NAME: - return get_name(); - - case UPTIME_SECS: - return Integer.valueOf(get_uptime_secs()); - - case WORKERS: - return get_workers(); - - case STATUS: - return get_status(); - - case TASKS: - return get_tasks(); - - case USER_DEF_METRIC: - return get_userDefMetric(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case ID: - return is_set_id(); - case NAME: - return is_set_name(); - case UPTIME_SECS: - return is_set_uptime_secs(); - case WORKERS: - return is_set_workers(); - case STATUS: - return is_set_status(); - case TASKS: - return is_set_tasks(); - case USER_DEF_METRIC: - return is_set_userDefMetric(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TopologyInfo) - return this.equals((TopologyInfo)that); - return false; - } - - public boolean equals(TopologyInfo that) { - if (that == null) - return false; - - boolean this_present_id = true && this.is_set_id(); - boolean that_present_id = true && that.is_set_id(); - if (this_present_id || that_present_id) { - if (!(this_present_id && that_present_id)) - return false; - if (!this.id.equals(that.id)) - return false; - } - - boolean this_present_name = true && this.is_set_name(); - boolean that_present_name = true && that.is_set_name(); - if (this_present_name || that_present_name) { - if (!(this_present_name && that_present_name)) - return false; - if (!this.name.equals(that.name)) - return false; - } - - boolean this_present_uptime_secs = true; - boolean that_present_uptime_secs = true; - if (this_present_uptime_secs || that_present_uptime_secs) { - if (!(this_present_uptime_secs && that_present_uptime_secs)) - return false; - if (this.uptime_secs != that.uptime_secs) - return false; - } - - boolean this_present_workers = true && this.is_set_workers(); - boolean that_present_workers = true && that.is_set_workers(); - if (this_present_workers || that_present_workers) { - if (!(this_present_workers && that_present_workers)) - return false; - if (!this.workers.equals(that.workers)) - return false; - } - - boolean this_present_status = true && this.is_set_status(); - boolean that_present_status = true && that.is_set_status(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_tasks = true && this.is_set_tasks(); - boolean that_present_tasks = true && that.is_set_tasks(); - if (this_present_tasks || that_present_tasks) { - if (!(this_present_tasks && that_present_tasks)) - return false; - if (!this.tasks.equals(that.tasks)) - return false; - } - - boolean this_present_userDefMetric = true && this.is_set_userDefMetric(); - boolean that_present_userDefMetric = true && that.is_set_userDefMetric(); - if (this_present_userDefMetric || that_present_userDefMetric) { - if (!(this_present_userDefMetric && that_present_userDefMetric)) - return false; - if (!this.userDefMetric.equals(that.userDefMetric)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_id = true && (is_set_id()); - builder.append(present_id); - if (present_id) - builder.append(id); - - boolean present_name = true && (is_set_name()); - builder.append(present_name); - if (present_name) - builder.append(name); - - boolean present_uptime_secs = true; - builder.append(present_uptime_secs); - if (present_uptime_secs) - builder.append(uptime_secs); - - boolean present_workers = true && (is_set_workers()); - builder.append(present_workers); - if (present_workers) - builder.append(workers); - - boolean present_status = true && (is_set_status()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_tasks = true && (is_set_tasks()); - builder.append(present_tasks); - if (present_tasks) - builder.append(tasks); - - boolean present_userDefMetric = true && (is_set_userDefMetric()); - builder.append(present_userDefMetric); - if (present_userDefMetric) - builder.append(userDefMetric); - - return builder.toHashCode(); - } - - public int compareTo(TopologyInfo other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TopologyInfo typedOther = (TopologyInfo)other; - - lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_id()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.id, typedOther.id); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_name()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(typedOther.is_set_uptime_secs()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_uptime_secs()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.uptime_secs, typedOther.uptime_secs); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_workers()).compareTo(typedOther.is_set_workers()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_workers()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.workers, typedOther.workers); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_status()).compareTo(typedOther.is_set_status()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_status()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_tasks()).compareTo(typedOther.is_set_tasks()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_tasks()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.tasks, typedOther.tasks); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_userDefMetric()).compareTo(typedOther.is_set_userDefMetric()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_userDefMetric()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.userDefMetric, typedOther.userDefMetric); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // ID - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.id = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // NAME - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.name = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 3: // UPTIME_SECS - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.uptime_secs = iprot.readI32(); - set_uptime_secs_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 4: // WORKERS - if (field.type == org.apache.thrift7.protocol.TType.LIST) { - { - org.apache.thrift7.protocol.TList _list193 = iprot.readListBegin(); - this.workers = new ArrayList(_list193.size); - for (int _i194 = 0; _i194 < _list193.size; ++_i194) - { - WorkerSummary _elem195; // required - _elem195 = new WorkerSummary(); - _elem195.read(iprot); - this.workers.add(_elem195); - } - iprot.readListEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 5: // STATUS - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.status = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 6: // TASKS - if (field.type == org.apache.thrift7.protocol.TType.LIST) { - { - org.apache.thrift7.protocol.TList _list196 = iprot.readListBegin(); - this.tasks = new ArrayList(_list196.size); - for (int _i197 = 0; _i197 < _list196.size; ++_i197) - { - TaskSummary _elem198; // required - _elem198 = new TaskSummary(); - _elem198.read(iprot); - this.tasks.add(_elem198); - } - iprot.readListEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 7: // USER_DEF_METRIC - if (field.type == org.apache.thrift7.protocol.TType.LIST) { - { - org.apache.thrift7.protocol.TList _list199 = iprot.readListBegin(); - this.userDefMetric = new ArrayList(_list199.size); - for (int _i200 = 0; _i200 < _list199.size; ++_i200) - { - UserDefMetric _elem201; // required - _elem201 = new UserDefMetric(); - _elem201.read(iprot); - this.userDefMetric.add(_elem201); - } - iprot.readListEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.id != null) { - oprot.writeFieldBegin(ID_FIELD_DESC); - oprot.writeString(this.id); - oprot.writeFieldEnd(); - } - if (this.name != null) { - oprot.writeFieldBegin(NAME_FIELD_DESC); - oprot.writeString(this.name); - oprot.writeFieldEnd(); - } - oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC); - oprot.writeI32(this.uptime_secs); - oprot.writeFieldEnd(); - if (this.workers != null) { - oprot.writeFieldBegin(WORKERS_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift7.protocol.TList(org.apache.thrift7.protocol.TType.STRUCT, this.workers.size())); - for (WorkerSummary _iter202 : this.workers) - { - _iter202.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (this.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - oprot.writeString(this.status); - oprot.writeFieldEnd(); - } - if (this.tasks != null) { - oprot.writeFieldBegin(TASKS_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift7.protocol.TList(org.apache.thrift7.protocol.TType.STRUCT, this.tasks.size())); - for (TaskSummary _iter203 : this.tasks) - { - _iter203.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - if (this.userDefMetric != null) { - if (is_set_userDefMetric()) { - oprot.writeFieldBegin(USER_DEF_METRIC_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift7.protocol.TList(org.apache.thrift7.protocol.TType.STRUCT, this.userDefMetric.size())); - for (UserDefMetric _iter204 : this.userDefMetric) - { - _iter204.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TopologyInfo("); - boolean first = true; - - sb.append("id:"); - if (this.id == null) { - sb.append("null"); - } else { - sb.append(this.id); - } - first = false; - if (!first) sb.append(", "); - sb.append("name:"); - if (this.name == null) { - sb.append("null"); - } else { - sb.append(this.name); - } - first = false; - if (!first) sb.append(", "); - sb.append("uptime_secs:"); - sb.append(this.uptime_secs); - first = false; - if (!first) sb.append(", "); - sb.append("workers:"); - if (this.workers == null) { - sb.append("null"); - } else { - sb.append(this.workers); - } - first = false; - if (!first) sb.append(", "); - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (!first) sb.append(", "); - sb.append("tasks:"); - if (this.tasks == null) { - sb.append("null"); - } else { - sb.append(this.tasks); - } - first = false; - if (is_set_userDefMetric()) { - if (!first) sb.append(", "); - sb.append("userDefMetric:"); - if (this.userDefMetric == null) { - sb.append("null"); - } else { - sb.append(this.userDefMetric); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_id()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'id' is unset! Struct:" + toString()); - } - - if (!is_set_name()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'name' is unset! Struct:" + toString()); - } - - if (!is_set_uptime_secs()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString()); - } - - if (!is_set_workers()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'workers' is unset! Struct:" + toString()); - } - - if (!is_set_status()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - if (!is_set_tasks()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'tasks' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bit_vector = new BitSet(1); - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/TopologyMetricInfo.java b/jstorm-client/src/main/java/backtype/storm/generated/TopologyMetricInfo.java deleted file mode 100644 index 9d584ec5a..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/TopologyMetricInfo.java +++ /dev/null @@ -1,594 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TopologyMetricInfo implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("TopologyMetricInfo"); - - private static final org.apache.thrift7.protocol.TField TOPOLOGY_ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("topology_id", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField TASK_METRIC_LIST_FIELD_DESC = new org.apache.thrift7.protocol.TField("task_metric_list", org.apache.thrift7.protocol.TType.LIST, (short)2); - private static final org.apache.thrift7.protocol.TField WORKER_METRIC_LIST_FIELD_DESC = new org.apache.thrift7.protocol.TField("worker_metric_list", org.apache.thrift7.protocol.TType.LIST, (short)3); - - private String topology_id; // required - private List task_metric_list; // required - private List worker_metric_list; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - TOPOLOGY_ID((short)1, "topology_id"), - TASK_METRIC_LIST((short)2, "task_metric_list"), - WORKER_METRIC_LIST((short)3, "worker_metric_list"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // TOPOLOGY_ID - return TOPOLOGY_ID; - case 2: // TASK_METRIC_LIST - return TASK_METRIC_LIST; - case 3: // WORKER_METRIC_LIST - return WORKER_METRIC_LIST; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.TOPOLOGY_ID, new org.apache.thrift7.meta_data.FieldMetaData("topology_id", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.TASK_METRIC_LIST, new org.apache.thrift7.meta_data.FieldMetaData("task_metric_list", org.apache.thrift7.TFieldRequirementType.OPTIONAL, - new org.apache.thrift7.meta_data.ListMetaData(org.apache.thrift7.protocol.TType.LIST, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, TaskMetricData.class)))); - tmpMap.put(_Fields.WORKER_METRIC_LIST, new org.apache.thrift7.meta_data.FieldMetaData("worker_metric_list", org.apache.thrift7.TFieldRequirementType.OPTIONAL, - new org.apache.thrift7.meta_data.ListMetaData(org.apache.thrift7.protocol.TType.LIST, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, WorkerMetricData.class)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(TopologyMetricInfo.class, metaDataMap); - } - - public TopologyMetricInfo() { - } - - public TopologyMetricInfo( - String topology_id) - { - this(); - this.topology_id = topology_id; - } - - /** - * Performs a deep copy on other. - */ - public TopologyMetricInfo(TopologyMetricInfo other) { - if (other.is_set_topology_id()) { - this.topology_id = other.topology_id; - } - if (other.is_set_task_metric_list()) { - List __this__task_metric_list = new ArrayList(); - for (TaskMetricData other_element : other.task_metric_list) { - __this__task_metric_list.add(new TaskMetricData(other_element)); - } - this.task_metric_list = __this__task_metric_list; - } - if (other.is_set_worker_metric_list()) { - List __this__worker_metric_list = new ArrayList(); - for (WorkerMetricData other_element : other.worker_metric_list) { - __this__worker_metric_list.add(new WorkerMetricData(other_element)); - } - this.worker_metric_list = __this__worker_metric_list; - } - } - - public TopologyMetricInfo deepCopy() { - return new TopologyMetricInfo(this); - } - - @Override - public void clear() { - this.topology_id = null; - this.task_metric_list = null; - this.worker_metric_list = null; - } - - public String get_topology_id() { - return this.topology_id; - } - - public void set_topology_id(String topology_id) { - this.topology_id = topology_id; - } - - public void unset_topology_id() { - this.topology_id = null; - } - - /** Returns true if field topology_id is set (has been assigned a value) and false otherwise */ - public boolean is_set_topology_id() { - return this.topology_id != null; - } - - public void set_topology_id_isSet(boolean value) { - if (!value) { - this.topology_id = null; - } - } - - public int get_task_metric_list_size() { - return (this.task_metric_list == null) ? 0 : this.task_metric_list.size(); - } - - public java.util.Iterator get_task_metric_list_iterator() { - return (this.task_metric_list == null) ? null : this.task_metric_list.iterator(); - } - - public void add_to_task_metric_list(TaskMetricData elem) { - if (this.task_metric_list == null) { - this.task_metric_list = new ArrayList(); - } - this.task_metric_list.add(elem); - } - - public List get_task_metric_list() { - return this.task_metric_list; - } - - public void set_task_metric_list(List task_metric_list) { - this.task_metric_list = task_metric_list; - } - - public void unset_task_metric_list() { - this.task_metric_list = null; - } - - /** Returns true if field task_metric_list is set (has been assigned a value) and false otherwise */ - public boolean is_set_task_metric_list() { - return this.task_metric_list != null; - } - - public void set_task_metric_list_isSet(boolean value) { - if (!value) { - this.task_metric_list = null; - } - } - - public int get_worker_metric_list_size() { - return (this.worker_metric_list == null) ? 0 : this.worker_metric_list.size(); - } - - public java.util.Iterator get_worker_metric_list_iterator() { - return (this.worker_metric_list == null) ? null : this.worker_metric_list.iterator(); - } - - public void add_to_worker_metric_list(WorkerMetricData elem) { - if (this.worker_metric_list == null) { - this.worker_metric_list = new ArrayList(); - } - this.worker_metric_list.add(elem); - } - - public List get_worker_metric_list() { - return this.worker_metric_list; - } - - public void set_worker_metric_list(List worker_metric_list) { - this.worker_metric_list = worker_metric_list; - } - - public void unset_worker_metric_list() { - this.worker_metric_list = null; - } - - /** Returns true if field worker_metric_list is set (has been assigned a value) and false otherwise */ - public boolean is_set_worker_metric_list() { - return this.worker_metric_list != null; - } - - public void set_worker_metric_list_isSet(boolean value) { - if (!value) { - this.worker_metric_list = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case TOPOLOGY_ID: - if (value == null) { - unset_topology_id(); - } else { - set_topology_id((String)value); - } - break; - - case TASK_METRIC_LIST: - if (value == null) { - unset_task_metric_list(); - } else { - set_task_metric_list((List)value); - } - break; - - case WORKER_METRIC_LIST: - if (value == null) { - unset_worker_metric_list(); - } else { - set_worker_metric_list((List)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case TOPOLOGY_ID: - return get_topology_id(); - - case TASK_METRIC_LIST: - return get_task_metric_list(); - - case WORKER_METRIC_LIST: - return get_worker_metric_list(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case TOPOLOGY_ID: - return is_set_topology_id(); - case TASK_METRIC_LIST: - return is_set_task_metric_list(); - case WORKER_METRIC_LIST: - return is_set_worker_metric_list(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TopologyMetricInfo) - return this.equals((TopologyMetricInfo)that); - return false; - } - - public boolean equals(TopologyMetricInfo that) { - if (that == null) - return false; - - boolean this_present_topology_id = true && this.is_set_topology_id(); - boolean that_present_topology_id = true && that.is_set_topology_id(); - if (this_present_topology_id || that_present_topology_id) { - if (!(this_present_topology_id && that_present_topology_id)) - return false; - if (!this.topology_id.equals(that.topology_id)) - return false; - } - - boolean this_present_task_metric_list = true && this.is_set_task_metric_list(); - boolean that_present_task_metric_list = true && that.is_set_task_metric_list(); - if (this_present_task_metric_list || that_present_task_metric_list) { - if (!(this_present_task_metric_list && that_present_task_metric_list)) - return false; - if (!this.task_metric_list.equals(that.task_metric_list)) - return false; - } - - boolean this_present_worker_metric_list = true && this.is_set_worker_metric_list(); - boolean that_present_worker_metric_list = true && that.is_set_worker_metric_list(); - if (this_present_worker_metric_list || that_present_worker_metric_list) { - if (!(this_present_worker_metric_list && that_present_worker_metric_list)) - return false; - if (!this.worker_metric_list.equals(that.worker_metric_list)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_topology_id = true && (is_set_topology_id()); - builder.append(present_topology_id); - if (present_topology_id) - builder.append(topology_id); - - boolean present_task_metric_list = true && (is_set_task_metric_list()); - builder.append(present_task_metric_list); - if (present_task_metric_list) - builder.append(task_metric_list); - - boolean present_worker_metric_list = true && (is_set_worker_metric_list()); - builder.append(present_worker_metric_list); - if (present_worker_metric_list) - builder.append(worker_metric_list); - - return builder.toHashCode(); - } - - public int compareTo(TopologyMetricInfo other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TopologyMetricInfo typedOther = (TopologyMetricInfo)other; - - lastComparison = Boolean.valueOf(is_set_topology_id()).compareTo(typedOther.is_set_topology_id()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_topology_id()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.topology_id, typedOther.topology_id); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_task_metric_list()).compareTo(typedOther.is_set_task_metric_list()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_task_metric_list()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.task_metric_list, typedOther.task_metric_list); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_worker_metric_list()).compareTo(typedOther.is_set_worker_metric_list()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_worker_metric_list()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.worker_metric_list, typedOther.worker_metric_list); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // TOPOLOGY_ID - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.topology_id = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // TASK_METRIC_LIST - if (field.type == org.apache.thrift7.protocol.TType.LIST) { - { - org.apache.thrift7.protocol.TList _list259 = iprot.readListBegin(); - this.task_metric_list = new ArrayList(_list259.size); - for (int _i260 = 0; _i260 < _list259.size; ++_i260) - { - TaskMetricData _elem261; // required - _elem261 = new TaskMetricData(); - _elem261.read(iprot); - this.task_metric_list.add(_elem261); - } - iprot.readListEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 3: // WORKER_METRIC_LIST - if (field.type == org.apache.thrift7.protocol.TType.LIST) { - { - org.apache.thrift7.protocol.TList _list262 = iprot.readListBegin(); - this.worker_metric_list = new ArrayList(_list262.size); - for (int _i263 = 0; _i263 < _list262.size; ++_i263) - { - WorkerMetricData _elem264; // required - _elem264 = new WorkerMetricData(); - _elem264.read(iprot); - this.worker_metric_list.add(_elem264); - } - iprot.readListEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.topology_id != null) { - oprot.writeFieldBegin(TOPOLOGY_ID_FIELD_DESC); - oprot.writeString(this.topology_id); - oprot.writeFieldEnd(); - } - if (this.task_metric_list != null) { - if (is_set_task_metric_list()) { - oprot.writeFieldBegin(TASK_METRIC_LIST_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift7.protocol.TList(org.apache.thrift7.protocol.TType.STRUCT, this.task_metric_list.size())); - for (TaskMetricData _iter265 : this.task_metric_list) - { - _iter265.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - } - if (this.worker_metric_list != null) { - if (is_set_worker_metric_list()) { - oprot.writeFieldBegin(WORKER_METRIC_LIST_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift7.protocol.TList(org.apache.thrift7.protocol.TType.STRUCT, this.worker_metric_list.size())); - for (WorkerMetricData _iter266 : this.worker_metric_list) - { - _iter266.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TopologyMetricInfo("); - boolean first = true; - - sb.append("topology_id:"); - if (this.topology_id == null) { - sb.append("null"); - } else { - sb.append(this.topology_id); - } - first = false; - if (is_set_task_metric_list()) { - if (!first) sb.append(", "); - sb.append("task_metric_list:"); - if (this.task_metric_list == null) { - sb.append("null"); - } else { - sb.append(this.task_metric_list); - } - first = false; - } - if (is_set_worker_metric_list()) { - if (!first) sb.append(", "); - sb.append("worker_metric_list:"); - if (this.worker_metric_list == null) { - sb.append("null"); - } else { - sb.append(this.worker_metric_list); - } - first = false; - } - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_topology_id()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'topology_id' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/TopologySummary.java b/jstorm-client/src/main/java/backtype/storm/generated/TopologySummary.java deleted file mode 100644 index e2ee70892..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/TopologySummary.java +++ /dev/null @@ -1,900 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TopologySummary implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("TopologySummary"); - - private static final org.apache.thrift7.protocol.TField ID_FIELD_DESC = new org.apache.thrift7.protocol.TField("id", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)2); - private static final org.apache.thrift7.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift7.protocol.TField("status", org.apache.thrift7.protocol.TType.STRING, (short)3); - private static final org.apache.thrift7.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift7.protocol.TField("uptime_secs", org.apache.thrift7.protocol.TType.I32, (short)4); - private static final org.apache.thrift7.protocol.TField NUM_TASKS_FIELD_DESC = new org.apache.thrift7.protocol.TField("num_tasks", org.apache.thrift7.protocol.TType.I32, (short)5); - private static final org.apache.thrift7.protocol.TField NUM_WORKERS_FIELD_DESC = new org.apache.thrift7.protocol.TField("num_workers", org.apache.thrift7.protocol.TType.I32, (short)6); - private static final org.apache.thrift7.protocol.TField ERROR_INFO_FIELD_DESC = new org.apache.thrift7.protocol.TField("error_info", org.apache.thrift7.protocol.TType.STRING, (short)7); - - private String id; // required - private String name; // required - private String status; // required - private int uptime_secs; // required - private int num_tasks; // required - private int num_workers; // required - private String error_info; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - ID((short)1, "id"), - NAME((short)2, "name"), - STATUS((short)3, "status"), - UPTIME_SECS((short)4, "uptime_secs"), - NUM_TASKS((short)5, "num_tasks"), - NUM_WORKERS((short)6, "num_workers"), - ERROR_INFO((short)7, "error_info"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // ID - return ID; - case 2: // NAME - return NAME; - case 3: // STATUS - return STATUS; - case 4: // UPTIME_SECS - return UPTIME_SECS; - case 5: // NUM_TASKS - return NUM_TASKS; - case 6: // NUM_WORKERS - return NUM_WORKERS; - case 7: // ERROR_INFO - return ERROR_INFO; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __UPTIME_SECS_ISSET_ID = 0; - private static final int __NUM_TASKS_ISSET_ID = 1; - private static final int __NUM_WORKERS_ISSET_ID = 2; - private BitSet __isset_bit_vector = new BitSet(3); - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.ID, new org.apache.thrift7.meta_data.FieldMetaData("id", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.STATUS, new org.apache.thrift7.meta_data.FieldMetaData("status", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift7.meta_data.FieldMetaData("uptime_secs", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - tmpMap.put(_Fields.NUM_TASKS, new org.apache.thrift7.meta_data.FieldMetaData("num_tasks", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - tmpMap.put(_Fields.NUM_WORKERS, new org.apache.thrift7.meta_data.FieldMetaData("num_workers", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - tmpMap.put(_Fields.ERROR_INFO, new org.apache.thrift7.meta_data.FieldMetaData("error_info", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(TopologySummary.class, metaDataMap); - } - - public TopologySummary() { - } - - public TopologySummary( - String id, - String name, - String status, - int uptime_secs, - int num_tasks, - int num_workers, - String error_info) - { - this(); - this.id = id; - this.name = name; - this.status = status; - this.uptime_secs = uptime_secs; - set_uptime_secs_isSet(true); - this.num_tasks = num_tasks; - set_num_tasks_isSet(true); - this.num_workers = num_workers; - set_num_workers_isSet(true); - this.error_info = error_info; - } - - /** - * Performs a deep copy on other. - */ - public TopologySummary(TopologySummary other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); - if (other.is_set_id()) { - this.id = other.id; - } - if (other.is_set_name()) { - this.name = other.name; - } - if (other.is_set_status()) { - this.status = other.status; - } - this.uptime_secs = other.uptime_secs; - this.num_tasks = other.num_tasks; - this.num_workers = other.num_workers; - if (other.is_set_error_info()) { - this.error_info = other.error_info; - } - } - - public TopologySummary deepCopy() { - return new TopologySummary(this); - } - - @Override - public void clear() { - this.id = null; - this.name = null; - this.status = null; - set_uptime_secs_isSet(false); - this.uptime_secs = 0; - set_num_tasks_isSet(false); - this.num_tasks = 0; - set_num_workers_isSet(false); - this.num_workers = 0; - this.error_info = null; - } - - public String get_id() { - return this.id; - } - - public void set_id(String id) { - this.id = id; - } - - public void unset_id() { - this.id = null; - } - - /** Returns true if field id is set (has been assigned a value) and false otherwise */ - public boolean is_set_id() { - return this.id != null; - } - - public void set_id_isSet(boolean value) { - if (!value) { - this.id = null; - } - } - - public String get_name() { - return this.name; - } - - public void set_name(String name) { - this.name = name; - } - - public void unset_name() { - this.name = null; - } - - /** Returns true if field name is set (has been assigned a value) and false otherwise */ - public boolean is_set_name() { - return this.name != null; - } - - public void set_name_isSet(boolean value) { - if (!value) { - this.name = null; - } - } - - public String get_status() { - return this.status; - } - - public void set_status(String status) { - this.status = status; - } - - public void unset_status() { - this.status = null; - } - - /** Returns true if field status is set (has been assigned a value) and false otherwise */ - public boolean is_set_status() { - return this.status != null; - } - - public void set_status_isSet(boolean value) { - if (!value) { - this.status = null; - } - } - - public int get_uptime_secs() { - return this.uptime_secs; - } - - public void set_uptime_secs(int uptime_secs) { - this.uptime_secs = uptime_secs; - set_uptime_secs_isSet(true); - } - - public void unset_uptime_secs() { - __isset_bit_vector.clear(__UPTIME_SECS_ISSET_ID); - } - - /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */ - public boolean is_set_uptime_secs() { - return __isset_bit_vector.get(__UPTIME_SECS_ISSET_ID); - } - - public void set_uptime_secs_isSet(boolean value) { - __isset_bit_vector.set(__UPTIME_SECS_ISSET_ID, value); - } - - public int get_num_tasks() { - return this.num_tasks; - } - - public void set_num_tasks(int num_tasks) { - this.num_tasks = num_tasks; - set_num_tasks_isSet(true); - } - - public void unset_num_tasks() { - __isset_bit_vector.clear(__NUM_TASKS_ISSET_ID); - } - - /** Returns true if field num_tasks is set (has been assigned a value) and false otherwise */ - public boolean is_set_num_tasks() { - return __isset_bit_vector.get(__NUM_TASKS_ISSET_ID); - } - - public void set_num_tasks_isSet(boolean value) { - __isset_bit_vector.set(__NUM_TASKS_ISSET_ID, value); - } - - public int get_num_workers() { - return this.num_workers; - } - - public void set_num_workers(int num_workers) { - this.num_workers = num_workers; - set_num_workers_isSet(true); - } - - public void unset_num_workers() { - __isset_bit_vector.clear(__NUM_WORKERS_ISSET_ID); - } - - /** Returns true if field num_workers is set (has been assigned a value) and false otherwise */ - public boolean is_set_num_workers() { - return __isset_bit_vector.get(__NUM_WORKERS_ISSET_ID); - } - - public void set_num_workers_isSet(boolean value) { - __isset_bit_vector.set(__NUM_WORKERS_ISSET_ID, value); - } - - public String get_error_info() { - return this.error_info; - } - - public void set_error_info(String error_info) { - this.error_info = error_info; - } - - public void unset_error_info() { - this.error_info = null; - } - - /** Returns true if field error_info is set (has been assigned a value) and false otherwise */ - public boolean is_set_error_info() { - return this.error_info != null; - } - - public void set_error_info_isSet(boolean value) { - if (!value) { - this.error_info = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case ID: - if (value == null) { - unset_id(); - } else { - set_id((String)value); - } - break; - - case NAME: - if (value == null) { - unset_name(); - } else { - set_name((String)value); - } - break; - - case STATUS: - if (value == null) { - unset_status(); - } else { - set_status((String)value); - } - break; - - case UPTIME_SECS: - if (value == null) { - unset_uptime_secs(); - } else { - set_uptime_secs((Integer)value); - } - break; - - case NUM_TASKS: - if (value == null) { - unset_num_tasks(); - } else { - set_num_tasks((Integer)value); - } - break; - - case NUM_WORKERS: - if (value == null) { - unset_num_workers(); - } else { - set_num_workers((Integer)value); - } - break; - - case ERROR_INFO: - if (value == null) { - unset_error_info(); - } else { - set_error_info((String)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case ID: - return get_id(); - - case NAME: - return get_name(); - - case STATUS: - return get_status(); - - case UPTIME_SECS: - return Integer.valueOf(get_uptime_secs()); - - case NUM_TASKS: - return Integer.valueOf(get_num_tasks()); - - case NUM_WORKERS: - return Integer.valueOf(get_num_workers()); - - case ERROR_INFO: - return get_error_info(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case ID: - return is_set_id(); - case NAME: - return is_set_name(); - case STATUS: - return is_set_status(); - case UPTIME_SECS: - return is_set_uptime_secs(); - case NUM_TASKS: - return is_set_num_tasks(); - case NUM_WORKERS: - return is_set_num_workers(); - case ERROR_INFO: - return is_set_error_info(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof TopologySummary) - return this.equals((TopologySummary)that); - return false; - } - - public boolean equals(TopologySummary that) { - if (that == null) - return false; - - boolean this_present_id = true && this.is_set_id(); - boolean that_present_id = true && that.is_set_id(); - if (this_present_id || that_present_id) { - if (!(this_present_id && that_present_id)) - return false; - if (!this.id.equals(that.id)) - return false; - } - - boolean this_present_name = true && this.is_set_name(); - boolean that_present_name = true && that.is_set_name(); - if (this_present_name || that_present_name) { - if (!(this_present_name && that_present_name)) - return false; - if (!this.name.equals(that.name)) - return false; - } - - boolean this_present_status = true && this.is_set_status(); - boolean that_present_status = true && that.is_set_status(); - if (this_present_status || that_present_status) { - if (!(this_present_status && that_present_status)) - return false; - if (!this.status.equals(that.status)) - return false; - } - - boolean this_present_uptime_secs = true; - boolean that_present_uptime_secs = true; - if (this_present_uptime_secs || that_present_uptime_secs) { - if (!(this_present_uptime_secs && that_present_uptime_secs)) - return false; - if (this.uptime_secs != that.uptime_secs) - return false; - } - - boolean this_present_num_tasks = true; - boolean that_present_num_tasks = true; - if (this_present_num_tasks || that_present_num_tasks) { - if (!(this_present_num_tasks && that_present_num_tasks)) - return false; - if (this.num_tasks != that.num_tasks) - return false; - } - - boolean this_present_num_workers = true; - boolean that_present_num_workers = true; - if (this_present_num_workers || that_present_num_workers) { - if (!(this_present_num_workers && that_present_num_workers)) - return false; - if (this.num_workers != that.num_workers) - return false; - } - - boolean this_present_error_info = true && this.is_set_error_info(); - boolean that_present_error_info = true && that.is_set_error_info(); - if (this_present_error_info || that_present_error_info) { - if (!(this_present_error_info && that_present_error_info)) - return false; - if (!this.error_info.equals(that.error_info)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_id = true && (is_set_id()); - builder.append(present_id); - if (present_id) - builder.append(id); - - boolean present_name = true && (is_set_name()); - builder.append(present_name); - if (present_name) - builder.append(name); - - boolean present_status = true && (is_set_status()); - builder.append(present_status); - if (present_status) - builder.append(status); - - boolean present_uptime_secs = true; - builder.append(present_uptime_secs); - if (present_uptime_secs) - builder.append(uptime_secs); - - boolean present_num_tasks = true; - builder.append(present_num_tasks); - if (present_num_tasks) - builder.append(num_tasks); - - boolean present_num_workers = true; - builder.append(present_num_workers); - if (present_num_workers) - builder.append(num_workers); - - boolean present_error_info = true && (is_set_error_info()); - builder.append(present_error_info); - if (present_error_info) - builder.append(error_info); - - return builder.toHashCode(); - } - - public int compareTo(TopologySummary other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - TopologySummary typedOther = (TopologySummary)other; - - lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_id()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.id, typedOther.id); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_name()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_status()).compareTo(typedOther.is_set_status()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_status()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.status, typedOther.status); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(typedOther.is_set_uptime_secs()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_uptime_secs()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.uptime_secs, typedOther.uptime_secs); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_num_tasks()).compareTo(typedOther.is_set_num_tasks()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_num_tasks()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.num_tasks, typedOther.num_tasks); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_num_workers()).compareTo(typedOther.is_set_num_workers()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_num_workers()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.num_workers, typedOther.num_workers); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_error_info()).compareTo(typedOther.is_set_error_info()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_error_info()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.error_info, typedOther.error_info); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // ID - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.id = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // NAME - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.name = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 3: // STATUS - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.status = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 4: // UPTIME_SECS - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.uptime_secs = iprot.readI32(); - set_uptime_secs_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 5: // NUM_TASKS - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.num_tasks = iprot.readI32(); - set_num_tasks_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 6: // NUM_WORKERS - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.num_workers = iprot.readI32(); - set_num_workers_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 7: // ERROR_INFO - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.error_info = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.id != null) { - oprot.writeFieldBegin(ID_FIELD_DESC); - oprot.writeString(this.id); - oprot.writeFieldEnd(); - } - if (this.name != null) { - oprot.writeFieldBegin(NAME_FIELD_DESC); - oprot.writeString(this.name); - oprot.writeFieldEnd(); - } - if (this.status != null) { - oprot.writeFieldBegin(STATUS_FIELD_DESC); - oprot.writeString(this.status); - oprot.writeFieldEnd(); - } - oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC); - oprot.writeI32(this.uptime_secs); - oprot.writeFieldEnd(); - oprot.writeFieldBegin(NUM_TASKS_FIELD_DESC); - oprot.writeI32(this.num_tasks); - oprot.writeFieldEnd(); - oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC); - oprot.writeI32(this.num_workers); - oprot.writeFieldEnd(); - if (this.error_info != null) { - oprot.writeFieldBegin(ERROR_INFO_FIELD_DESC); - oprot.writeString(this.error_info); - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("TopologySummary("); - boolean first = true; - - sb.append("id:"); - if (this.id == null) { - sb.append("null"); - } else { - sb.append(this.id); - } - first = false; - if (!first) sb.append(", "); - sb.append("name:"); - if (this.name == null) { - sb.append("null"); - } else { - sb.append(this.name); - } - first = false; - if (!first) sb.append(", "); - sb.append("status:"); - if (this.status == null) { - sb.append("null"); - } else { - sb.append(this.status); - } - first = false; - if (!first) sb.append(", "); - sb.append("uptime_secs:"); - sb.append(this.uptime_secs); - first = false; - if (!first) sb.append(", "); - sb.append("num_tasks:"); - sb.append(this.num_tasks); - first = false; - if (!first) sb.append(", "); - sb.append("num_workers:"); - sb.append(this.num_workers); - first = false; - if (!first) sb.append(", "); - sb.append("error_info:"); - if (this.error_info == null) { - sb.append("null"); - } else { - sb.append(this.error_info); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_id()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'id' is unset! Struct:" + toString()); - } - - if (!is_set_name()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'name' is unset! Struct:" + toString()); - } - - if (!is_set_status()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); - } - - if (!is_set_uptime_secs()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString()); - } - - if (!is_set_num_tasks()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'num_tasks' is unset! Struct:" + toString()); - } - - if (!is_set_num_workers()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'num_workers' is unset! Struct:" + toString()); - } - - if (!is_set_error_info()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'error_info' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bit_vector = new BitSet(1); - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/UserDefMetric.java b/jstorm-client/src/main/java/backtype/storm/generated/UserDefMetric.java deleted file mode 100644 index ddf2de32c..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/UserDefMetric.java +++ /dev/null @@ -1,522 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class UserDefMetric implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("UserDefMetric"); - - private static final org.apache.thrift7.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift7.protocol.TField("type", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField NAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("name", org.apache.thrift7.protocol.TType.STRING, (short)2); - private static final org.apache.thrift7.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift7.protocol.TField("value", org.apache.thrift7.protocol.TType.DOUBLE, (short)3); - - private String type; // required - private String name; // required - private double value; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - TYPE((short)1, "type"), - NAME((short)2, "name"), - VALUE((short)3, "value"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // TYPE - return TYPE; - case 2: // NAME - return NAME; - case 3: // VALUE - return VALUE; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __VALUE_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.TYPE, new org.apache.thrift7.meta_data.FieldMetaData("type", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.NAME, new org.apache.thrift7.meta_data.FieldMetaData("name", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.VALUE, new org.apache.thrift7.meta_data.FieldMetaData("value", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(UserDefMetric.class, metaDataMap); - } - - public UserDefMetric() { - } - - public UserDefMetric( - String type, - String name, - double value) - { - this(); - this.type = type; - this.name = name; - this.value = value; - set_value_isSet(true); - } - - /** - * Performs a deep copy on other. - */ - public UserDefMetric(UserDefMetric other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); - if (other.is_set_type()) { - this.type = other.type; - } - if (other.is_set_name()) { - this.name = other.name; - } - this.value = other.value; - } - - public UserDefMetric deepCopy() { - return new UserDefMetric(this); - } - - @Override - public void clear() { - this.type = null; - this.name = null; - set_value_isSet(false); - this.value = 0.0; - } - - public String get_type() { - return this.type; - } - - public void set_type(String type) { - this.type = type; - } - - public void unset_type() { - this.type = null; - } - - /** Returns true if field type is set (has been assigned a value) and false otherwise */ - public boolean is_set_type() { - return this.type != null; - } - - public void set_type_isSet(boolean value) { - if (!value) { - this.type = null; - } - } - - public String get_name() { - return this.name; - } - - public void set_name(String name) { - this.name = name; - } - - public void unset_name() { - this.name = null; - } - - /** Returns true if field name is set (has been assigned a value) and false otherwise */ - public boolean is_set_name() { - return this.name != null; - } - - public void set_name_isSet(boolean value) { - if (!value) { - this.name = null; - } - } - - public double get_value() { - return this.value; - } - - public void set_value(double value) { - this.value = value; - set_value_isSet(true); - } - - public void unset_value() { - __isset_bit_vector.clear(__VALUE_ISSET_ID); - } - - /** Returns true if field value is set (has been assigned a value) and false otherwise */ - public boolean is_set_value() { - return __isset_bit_vector.get(__VALUE_ISSET_ID); - } - - public void set_value_isSet(boolean value) { - __isset_bit_vector.set(__VALUE_ISSET_ID, value); - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case TYPE: - if (value == null) { - unset_type(); - } else { - set_type((String)value); - } - break; - - case NAME: - if (value == null) { - unset_name(); - } else { - set_name((String)value); - } - break; - - case VALUE: - if (value == null) { - unset_value(); - } else { - set_value((Double)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case TYPE: - return get_type(); - - case NAME: - return get_name(); - - case VALUE: - return Double.valueOf(get_value()); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case TYPE: - return is_set_type(); - case NAME: - return is_set_name(); - case VALUE: - return is_set_value(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof UserDefMetric) - return this.equals((UserDefMetric)that); - return false; - } - - public boolean equals(UserDefMetric that) { - if (that == null) - return false; - - boolean this_present_type = true && this.is_set_type(); - boolean that_present_type = true && that.is_set_type(); - if (this_present_type || that_present_type) { - if (!(this_present_type && that_present_type)) - return false; - if (!this.type.equals(that.type)) - return false; - } - - boolean this_present_name = true && this.is_set_name(); - boolean that_present_name = true && that.is_set_name(); - if (this_present_name || that_present_name) { - if (!(this_present_name && that_present_name)) - return false; - if (!this.name.equals(that.name)) - return false; - } - - boolean this_present_value = true; - boolean that_present_value = true; - if (this_present_value || that_present_value) { - if (!(this_present_value && that_present_value)) - return false; - if (this.value != that.value) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_type = true && (is_set_type()); - builder.append(present_type); - if (present_type) - builder.append(type); - - boolean present_name = true && (is_set_name()); - builder.append(present_name); - if (present_name) - builder.append(name); - - boolean present_value = true; - builder.append(present_value); - if (present_value) - builder.append(value); - - return builder.toHashCode(); - } - - public int compareTo(UserDefMetric other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - UserDefMetric typedOther = (UserDefMetric)other; - - lastComparison = Boolean.valueOf(is_set_type()).compareTo(typedOther.is_set_type()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_type()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.type, typedOther.type); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_name()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.name, typedOther.name); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_value()).compareTo(typedOther.is_set_value()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_value()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.value, typedOther.value); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // TYPE - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.type = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // NAME - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.name = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 3: // VALUE - if (field.type == org.apache.thrift7.protocol.TType.DOUBLE) { - this.value = iprot.readDouble(); - set_value_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.type != null) { - oprot.writeFieldBegin(TYPE_FIELD_DESC); - oprot.writeString(this.type); - oprot.writeFieldEnd(); - } - if (this.name != null) { - oprot.writeFieldBegin(NAME_FIELD_DESC); - oprot.writeString(this.name); - oprot.writeFieldEnd(); - } - oprot.writeFieldBegin(VALUE_FIELD_DESC); - oprot.writeDouble(this.value); - oprot.writeFieldEnd(); - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("UserDefMetric("); - boolean first = true; - - sb.append("type:"); - if (this.type == null) { - sb.append("null"); - } else { - sb.append(this.type); - } - first = false; - if (!first) sb.append(", "); - sb.append("name:"); - if (this.name == null) { - sb.append("null"); - } else { - sb.append(this.name); - } - first = false; - if (!first) sb.append(", "); - sb.append("value:"); - sb.append(this.value); - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_type()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'type' is unset! Struct:" + toString()); - } - - if (!is_set_name()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'name' is unset! Struct:" + toString()); - } - - if (!is_set_value()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'value' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bit_vector = new BitSet(1); - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/WorkerMetricData.java b/jstorm-client/src/main/java/backtype/storm/generated/WorkerMetricData.java deleted file mode 100644 index e9bc07333..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/WorkerMetricData.java +++ /dev/null @@ -1,1135 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class WorkerMetricData implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("WorkerMetricData"); - - private static final org.apache.thrift7.protocol.TField HOSTNAME_FIELD_DESC = new org.apache.thrift7.protocol.TField("hostname", org.apache.thrift7.protocol.TType.STRING, (short)1); - private static final org.apache.thrift7.protocol.TField PORT_FIELD_DESC = new org.apache.thrift7.protocol.TField("port", org.apache.thrift7.protocol.TType.I32, (short)2); - private static final org.apache.thrift7.protocol.TField GAUGE_FIELD_DESC = new org.apache.thrift7.protocol.TField("gauge", org.apache.thrift7.protocol.TType.MAP, (short)3); - private static final org.apache.thrift7.protocol.TField COUNTER_FIELD_DESC = new org.apache.thrift7.protocol.TField("counter", org.apache.thrift7.protocol.TType.MAP, (short)4); - private static final org.apache.thrift7.protocol.TField METER_FIELD_DESC = new org.apache.thrift7.protocol.TField("meter", org.apache.thrift7.protocol.TType.MAP, (short)5); - private static final org.apache.thrift7.protocol.TField TIMER_FIELD_DESC = new org.apache.thrift7.protocol.TField("timer", org.apache.thrift7.protocol.TType.MAP, (short)6); - private static final org.apache.thrift7.protocol.TField HISTOGRAM_FIELD_DESC = new org.apache.thrift7.protocol.TField("histogram", org.apache.thrift7.protocol.TType.MAP, (short)7); - - private String hostname; // required - private int port; // required - private Map gauge; // required - private Map counter; // required - private Map meter; // required - private Map timer; // required - private Map histogram; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - HOSTNAME((short)1, "hostname"), - PORT((short)2, "port"), - GAUGE((short)3, "gauge"), - COUNTER((short)4, "counter"), - METER((short)5, "meter"), - TIMER((short)6, "timer"), - HISTOGRAM((short)7, "histogram"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // HOSTNAME - return HOSTNAME; - case 2: // PORT - return PORT; - case 3: // GAUGE - return GAUGE; - case 4: // COUNTER - return COUNTER; - case 5: // METER - return METER; - case 6: // TIMER - return TIMER; - case 7: // HISTOGRAM - return HISTOGRAM; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __PORT_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.HOSTNAME, new org.apache.thrift7.meta_data.FieldMetaData("hostname", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.PORT, new org.apache.thrift7.meta_data.FieldMetaData("port", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - tmpMap.put(_Fields.GAUGE, new org.apache.thrift7.meta_data.FieldMetaData("gauge", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE)))); - tmpMap.put(_Fields.COUNTER, new org.apache.thrift7.meta_data.FieldMetaData("counter", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE)))); - tmpMap.put(_Fields.METER, new org.apache.thrift7.meta_data.FieldMetaData("meter", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE)))); - tmpMap.put(_Fields.TIMER, new org.apache.thrift7.meta_data.FieldMetaData("timer", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE)))); - tmpMap.put(_Fields.HISTOGRAM, new org.apache.thrift7.meta_data.FieldMetaData("histogram", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.MapMetaData(org.apache.thrift7.protocol.TType.MAP, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING), - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.DOUBLE)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(WorkerMetricData.class, metaDataMap); - } - - public WorkerMetricData() { - } - - public WorkerMetricData( - String hostname, - int port, - Map gauge, - Map counter, - Map meter, - Map timer, - Map histogram) - { - this(); - this.hostname = hostname; - this.port = port; - set_port_isSet(true); - this.gauge = gauge; - this.counter = counter; - this.meter = meter; - this.timer = timer; - this.histogram = histogram; - } - - /** - * Performs a deep copy on other. - */ - public WorkerMetricData(WorkerMetricData other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); - if (other.is_set_hostname()) { - this.hostname = other.hostname; - } - this.port = other.port; - if (other.is_set_gauge()) { - Map __this__gauge = new HashMap(); - for (Map.Entry other_element : other.gauge.entrySet()) { - - String other_element_key = other_element.getKey(); - Double other_element_value = other_element.getValue(); - - String __this__gauge_copy_key = other_element_key; - - Double __this__gauge_copy_value = other_element_value; - - __this__gauge.put(__this__gauge_copy_key, __this__gauge_copy_value); - } - this.gauge = __this__gauge; - } - if (other.is_set_counter()) { - Map __this__counter = new HashMap(); - for (Map.Entry other_element : other.counter.entrySet()) { - - String other_element_key = other_element.getKey(); - Double other_element_value = other_element.getValue(); - - String __this__counter_copy_key = other_element_key; - - Double __this__counter_copy_value = other_element_value; - - __this__counter.put(__this__counter_copy_key, __this__counter_copy_value); - } - this.counter = __this__counter; - } - if (other.is_set_meter()) { - Map __this__meter = new HashMap(); - for (Map.Entry other_element : other.meter.entrySet()) { - - String other_element_key = other_element.getKey(); - Double other_element_value = other_element.getValue(); - - String __this__meter_copy_key = other_element_key; - - Double __this__meter_copy_value = other_element_value; - - __this__meter.put(__this__meter_copy_key, __this__meter_copy_value); - } - this.meter = __this__meter; - } - if (other.is_set_timer()) { - Map __this__timer = new HashMap(); - for (Map.Entry other_element : other.timer.entrySet()) { - - String other_element_key = other_element.getKey(); - Double other_element_value = other_element.getValue(); - - String __this__timer_copy_key = other_element_key; - - Double __this__timer_copy_value = other_element_value; - - __this__timer.put(__this__timer_copy_key, __this__timer_copy_value); - } - this.timer = __this__timer; - } - if (other.is_set_histogram()) { - Map __this__histogram = new HashMap(); - for (Map.Entry other_element : other.histogram.entrySet()) { - - String other_element_key = other_element.getKey(); - Double other_element_value = other_element.getValue(); - - String __this__histogram_copy_key = other_element_key; - - Double __this__histogram_copy_value = other_element_value; - - __this__histogram.put(__this__histogram_copy_key, __this__histogram_copy_value); - } - this.histogram = __this__histogram; - } - } - - public WorkerMetricData deepCopy() { - return new WorkerMetricData(this); - } - - @Override - public void clear() { - this.hostname = null; - set_port_isSet(false); - this.port = 0; - this.gauge = null; - this.counter = null; - this.meter = null; - this.timer = null; - this.histogram = null; - } - - public String get_hostname() { - return this.hostname; - } - - public void set_hostname(String hostname) { - this.hostname = hostname; - } - - public void unset_hostname() { - this.hostname = null; - } - - /** Returns true if field hostname is set (has been assigned a value) and false otherwise */ - public boolean is_set_hostname() { - return this.hostname != null; - } - - public void set_hostname_isSet(boolean value) { - if (!value) { - this.hostname = null; - } - } - - public int get_port() { - return this.port; - } - - public void set_port(int port) { - this.port = port; - set_port_isSet(true); - } - - public void unset_port() { - __isset_bit_vector.clear(__PORT_ISSET_ID); - } - - /** Returns true if field port is set (has been assigned a value) and false otherwise */ - public boolean is_set_port() { - return __isset_bit_vector.get(__PORT_ISSET_ID); - } - - public void set_port_isSet(boolean value) { - __isset_bit_vector.set(__PORT_ISSET_ID, value); - } - - public int get_gauge_size() { - return (this.gauge == null) ? 0 : this.gauge.size(); - } - - public void put_to_gauge(String key, double val) { - if (this.gauge == null) { - this.gauge = new HashMap(); - } - this.gauge.put(key, val); - } - - public Map get_gauge() { - return this.gauge; - } - - public void set_gauge(Map gauge) { - this.gauge = gauge; - } - - public void unset_gauge() { - this.gauge = null; - } - - /** Returns true if field gauge is set (has been assigned a value) and false otherwise */ - public boolean is_set_gauge() { - return this.gauge != null; - } - - public void set_gauge_isSet(boolean value) { - if (!value) { - this.gauge = null; - } - } - - public int get_counter_size() { - return (this.counter == null) ? 0 : this.counter.size(); - } - - public void put_to_counter(String key, double val) { - if (this.counter == null) { - this.counter = new HashMap(); - } - this.counter.put(key, val); - } - - public Map get_counter() { - return this.counter; - } - - public void set_counter(Map counter) { - this.counter = counter; - } - - public void unset_counter() { - this.counter = null; - } - - /** Returns true if field counter is set (has been assigned a value) and false otherwise */ - public boolean is_set_counter() { - return this.counter != null; - } - - public void set_counter_isSet(boolean value) { - if (!value) { - this.counter = null; - } - } - - public int get_meter_size() { - return (this.meter == null) ? 0 : this.meter.size(); - } - - public void put_to_meter(String key, double val) { - if (this.meter == null) { - this.meter = new HashMap(); - } - this.meter.put(key, val); - } - - public Map get_meter() { - return this.meter; - } - - public void set_meter(Map meter) { - this.meter = meter; - } - - public void unset_meter() { - this.meter = null; - } - - /** Returns true if field meter is set (has been assigned a value) and false otherwise */ - public boolean is_set_meter() { - return this.meter != null; - } - - public void set_meter_isSet(boolean value) { - if (!value) { - this.meter = null; - } - } - - public int get_timer_size() { - return (this.timer == null) ? 0 : this.timer.size(); - } - - public void put_to_timer(String key, double val) { - if (this.timer == null) { - this.timer = new HashMap(); - } - this.timer.put(key, val); - } - - public Map get_timer() { - return this.timer; - } - - public void set_timer(Map timer) { - this.timer = timer; - } - - public void unset_timer() { - this.timer = null; - } - - /** Returns true if field timer is set (has been assigned a value) and false otherwise */ - public boolean is_set_timer() { - return this.timer != null; - } - - public void set_timer_isSet(boolean value) { - if (!value) { - this.timer = null; - } - } - - public int get_histogram_size() { - return (this.histogram == null) ? 0 : this.histogram.size(); - } - - public void put_to_histogram(String key, double val) { - if (this.histogram == null) { - this.histogram = new HashMap(); - } - this.histogram.put(key, val); - } - - public Map get_histogram() { - return this.histogram; - } - - public void set_histogram(Map histogram) { - this.histogram = histogram; - } - - public void unset_histogram() { - this.histogram = null; - } - - /** Returns true if field histogram is set (has been assigned a value) and false otherwise */ - public boolean is_set_histogram() { - return this.histogram != null; - } - - public void set_histogram_isSet(boolean value) { - if (!value) { - this.histogram = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case HOSTNAME: - if (value == null) { - unset_hostname(); - } else { - set_hostname((String)value); - } - break; - - case PORT: - if (value == null) { - unset_port(); - } else { - set_port((Integer)value); - } - break; - - case GAUGE: - if (value == null) { - unset_gauge(); - } else { - set_gauge((Map)value); - } - break; - - case COUNTER: - if (value == null) { - unset_counter(); - } else { - set_counter((Map)value); - } - break; - - case METER: - if (value == null) { - unset_meter(); - } else { - set_meter((Map)value); - } - break; - - case TIMER: - if (value == null) { - unset_timer(); - } else { - set_timer((Map)value); - } - break; - - case HISTOGRAM: - if (value == null) { - unset_histogram(); - } else { - set_histogram((Map)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case HOSTNAME: - return get_hostname(); - - case PORT: - return Integer.valueOf(get_port()); - - case GAUGE: - return get_gauge(); - - case COUNTER: - return get_counter(); - - case METER: - return get_meter(); - - case TIMER: - return get_timer(); - - case HISTOGRAM: - return get_histogram(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case HOSTNAME: - return is_set_hostname(); - case PORT: - return is_set_port(); - case GAUGE: - return is_set_gauge(); - case COUNTER: - return is_set_counter(); - case METER: - return is_set_meter(); - case TIMER: - return is_set_timer(); - case HISTOGRAM: - return is_set_histogram(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof WorkerMetricData) - return this.equals((WorkerMetricData)that); - return false; - } - - public boolean equals(WorkerMetricData that) { - if (that == null) - return false; - - boolean this_present_hostname = true && this.is_set_hostname(); - boolean that_present_hostname = true && that.is_set_hostname(); - if (this_present_hostname || that_present_hostname) { - if (!(this_present_hostname && that_present_hostname)) - return false; - if (!this.hostname.equals(that.hostname)) - return false; - } - - boolean this_present_port = true; - boolean that_present_port = true; - if (this_present_port || that_present_port) { - if (!(this_present_port && that_present_port)) - return false; - if (this.port != that.port) - return false; - } - - boolean this_present_gauge = true && this.is_set_gauge(); - boolean that_present_gauge = true && that.is_set_gauge(); - if (this_present_gauge || that_present_gauge) { - if (!(this_present_gauge && that_present_gauge)) - return false; - if (!this.gauge.equals(that.gauge)) - return false; - } - - boolean this_present_counter = true && this.is_set_counter(); - boolean that_present_counter = true && that.is_set_counter(); - if (this_present_counter || that_present_counter) { - if (!(this_present_counter && that_present_counter)) - return false; - if (!this.counter.equals(that.counter)) - return false; - } - - boolean this_present_meter = true && this.is_set_meter(); - boolean that_present_meter = true && that.is_set_meter(); - if (this_present_meter || that_present_meter) { - if (!(this_present_meter && that_present_meter)) - return false; - if (!this.meter.equals(that.meter)) - return false; - } - - boolean this_present_timer = true && this.is_set_timer(); - boolean that_present_timer = true && that.is_set_timer(); - if (this_present_timer || that_present_timer) { - if (!(this_present_timer && that_present_timer)) - return false; - if (!this.timer.equals(that.timer)) - return false; - } - - boolean this_present_histogram = true && this.is_set_histogram(); - boolean that_present_histogram = true && that.is_set_histogram(); - if (this_present_histogram || that_present_histogram) { - if (!(this_present_histogram && that_present_histogram)) - return false; - if (!this.histogram.equals(that.histogram)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_hostname = true && (is_set_hostname()); - builder.append(present_hostname); - if (present_hostname) - builder.append(hostname); - - boolean present_port = true; - builder.append(present_port); - if (present_port) - builder.append(port); - - boolean present_gauge = true && (is_set_gauge()); - builder.append(present_gauge); - if (present_gauge) - builder.append(gauge); - - boolean present_counter = true && (is_set_counter()); - builder.append(present_counter); - if (present_counter) - builder.append(counter); - - boolean present_meter = true && (is_set_meter()); - builder.append(present_meter); - if (present_meter) - builder.append(meter); - - boolean present_timer = true && (is_set_timer()); - builder.append(present_timer); - if (present_timer) - builder.append(timer); - - boolean present_histogram = true && (is_set_histogram()); - builder.append(present_histogram); - if (present_histogram) - builder.append(histogram); - - return builder.toHashCode(); - } - - public int compareTo(WorkerMetricData other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - WorkerMetricData typedOther = (WorkerMetricData)other; - - lastComparison = Boolean.valueOf(is_set_hostname()).compareTo(typedOther.is_set_hostname()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_hostname()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.hostname, typedOther.hostname); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_port()).compareTo(typedOther.is_set_port()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_port()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.port, typedOther.port); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_gauge()).compareTo(typedOther.is_set_gauge()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_gauge()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.gauge, typedOther.gauge); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_counter()).compareTo(typedOther.is_set_counter()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_counter()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.counter, typedOther.counter); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_meter()).compareTo(typedOther.is_set_meter()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_meter()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.meter, typedOther.meter); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_timer()).compareTo(typedOther.is_set_timer()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_timer()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.timer, typedOther.timer); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_histogram()).compareTo(typedOther.is_set_histogram()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_histogram()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.histogram, typedOther.histogram); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // HOSTNAME - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.hostname = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // PORT - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.port = iprot.readI32(); - set_port_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 3: // GAUGE - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map234 = iprot.readMapBegin(); - this.gauge = new HashMap(2*_map234.size); - for (int _i235 = 0; _i235 < _map234.size; ++_i235) - { - String _key236; // required - double _val237; // required - _key236 = iprot.readString(); - _val237 = iprot.readDouble(); - this.gauge.put(_key236, _val237); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 4: // COUNTER - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map238 = iprot.readMapBegin(); - this.counter = new HashMap(2*_map238.size); - for (int _i239 = 0; _i239 < _map238.size; ++_i239) - { - String _key240; // required - double _val241; // required - _key240 = iprot.readString(); - _val241 = iprot.readDouble(); - this.counter.put(_key240, _val241); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 5: // METER - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map242 = iprot.readMapBegin(); - this.meter = new HashMap(2*_map242.size); - for (int _i243 = 0; _i243 < _map242.size; ++_i243) - { - String _key244; // required - double _val245; // required - _key244 = iprot.readString(); - _val245 = iprot.readDouble(); - this.meter.put(_key244, _val245); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 6: // TIMER - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map246 = iprot.readMapBegin(); - this.timer = new HashMap(2*_map246.size); - for (int _i247 = 0; _i247 < _map246.size; ++_i247) - { - String _key248; // required - double _val249; // required - _key248 = iprot.readString(); - _val249 = iprot.readDouble(); - this.timer.put(_key248, _val249); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 7: // HISTOGRAM - if (field.type == org.apache.thrift7.protocol.TType.MAP) { - { - org.apache.thrift7.protocol.TMap _map250 = iprot.readMapBegin(); - this.histogram = new HashMap(2*_map250.size); - for (int _i251 = 0; _i251 < _map250.size; ++_i251) - { - String _key252; // required - double _val253; // required - _key252 = iprot.readString(); - _val253 = iprot.readDouble(); - this.histogram.put(_key252, _val253); - } - iprot.readMapEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - if (this.hostname != null) { - oprot.writeFieldBegin(HOSTNAME_FIELD_DESC); - oprot.writeString(this.hostname); - oprot.writeFieldEnd(); - } - oprot.writeFieldBegin(PORT_FIELD_DESC); - oprot.writeI32(this.port); - oprot.writeFieldEnd(); - if (this.gauge != null) { - oprot.writeFieldBegin(GAUGE_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.DOUBLE, this.gauge.size())); - for (Map.Entry _iter254 : this.gauge.entrySet()) - { - oprot.writeString(_iter254.getKey()); - oprot.writeDouble(_iter254.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.counter != null) { - oprot.writeFieldBegin(COUNTER_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.DOUBLE, this.counter.size())); - for (Map.Entry _iter255 : this.counter.entrySet()) - { - oprot.writeString(_iter255.getKey()); - oprot.writeDouble(_iter255.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.meter != null) { - oprot.writeFieldBegin(METER_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.DOUBLE, this.meter.size())); - for (Map.Entry _iter256 : this.meter.entrySet()) - { - oprot.writeString(_iter256.getKey()); - oprot.writeDouble(_iter256.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.timer != null) { - oprot.writeFieldBegin(TIMER_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.DOUBLE, this.timer.size())); - for (Map.Entry _iter257 : this.timer.entrySet()) - { - oprot.writeString(_iter257.getKey()); - oprot.writeDouble(_iter257.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - if (this.histogram != null) { - oprot.writeFieldBegin(HISTOGRAM_FIELD_DESC); - { - oprot.writeMapBegin(new org.apache.thrift7.protocol.TMap(org.apache.thrift7.protocol.TType.STRING, org.apache.thrift7.protocol.TType.DOUBLE, this.histogram.size())); - for (Map.Entry _iter258 : this.histogram.entrySet()) - { - oprot.writeString(_iter258.getKey()); - oprot.writeDouble(_iter258.getValue()); - } - oprot.writeMapEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("WorkerMetricData("); - boolean first = true; - - sb.append("hostname:"); - if (this.hostname == null) { - sb.append("null"); - } else { - sb.append(this.hostname); - } - first = false; - if (!first) sb.append(", "); - sb.append("port:"); - sb.append(this.port); - first = false; - if (!first) sb.append(", "); - sb.append("gauge:"); - if (this.gauge == null) { - sb.append("null"); - } else { - sb.append(this.gauge); - } - first = false; - if (!first) sb.append(", "); - sb.append("counter:"); - if (this.counter == null) { - sb.append("null"); - } else { - sb.append(this.counter); - } - first = false; - if (!first) sb.append(", "); - sb.append("meter:"); - if (this.meter == null) { - sb.append("null"); - } else { - sb.append(this.meter); - } - first = false; - if (!first) sb.append(", "); - sb.append("timer:"); - if (this.timer == null) { - sb.append("null"); - } else { - sb.append(this.timer); - } - first = false; - if (!first) sb.append(", "); - sb.append("histogram:"); - if (this.histogram == null) { - sb.append("null"); - } else { - sb.append(this.histogram); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_hostname()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'hostname' is unset! Struct:" + toString()); - } - - if (!is_set_port()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'port' is unset! Struct:" + toString()); - } - - if (!is_set_gauge()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'gauge' is unset! Struct:" + toString()); - } - - if (!is_set_counter()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'counter' is unset! Struct:" + toString()); - } - - if (!is_set_meter()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'meter' is unset! Struct:" + toString()); - } - - if (!is_set_timer()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'timer' is unset! Struct:" + toString()); - } - - if (!is_set_histogram()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'histogram' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bit_vector = new BitSet(1); - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/generated/WorkerSummary.java b/jstorm-client/src/main/java/backtype/storm/generated/WorkerSummary.java deleted file mode 100644 index 0bf878c0e..000000000 --- a/jstorm-client/src/main/java/backtype/storm/generated/WorkerSummary.java +++ /dev/null @@ -1,560 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.7.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - */ -package backtype.storm.generated; - -import org.apache.commons.lang.builder.HashCodeBuilder; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.EnumMap; -import java.util.Set; -import java.util.HashSet; -import java.util.EnumSet; -import java.util.Collections; -import java.util.BitSet; -import java.nio.ByteBuffer; -import java.util.Arrays; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class WorkerSummary implements org.apache.thrift7.TBase, java.io.Serializable, Cloneable { - private static final org.apache.thrift7.protocol.TStruct STRUCT_DESC = new org.apache.thrift7.protocol.TStruct("WorkerSummary"); - - private static final org.apache.thrift7.protocol.TField PORT_FIELD_DESC = new org.apache.thrift7.protocol.TField("port", org.apache.thrift7.protocol.TType.I32, (short)1); - private static final org.apache.thrift7.protocol.TField TOPOLOGY_FIELD_DESC = new org.apache.thrift7.protocol.TField("topology", org.apache.thrift7.protocol.TType.STRING, (short)2); - private static final org.apache.thrift7.protocol.TField TASKS_FIELD_DESC = new org.apache.thrift7.protocol.TField("tasks", org.apache.thrift7.protocol.TType.LIST, (short)3); - - private int port; // required - private String topology; // required - private List tasks; // required - - /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ - public enum _Fields implements org.apache.thrift7.TFieldIdEnum { - PORT((short)1, "port"), - TOPOLOGY((short)2, "topology"), - TASKS((short)3, "tasks"); - - private static final Map byName = new HashMap(); - - static { - for (_Fields field : EnumSet.allOf(_Fields.class)) { - byName.put(field.getFieldName(), field); - } - } - - /** - * Find the _Fields constant that matches fieldId, or null if its not found. - */ - public static _Fields findByThriftId(int fieldId) { - switch(fieldId) { - case 1: // PORT - return PORT; - case 2: // TOPOLOGY - return TOPOLOGY; - case 3: // TASKS - return TASKS; - default: - return null; - } - } - - /** - * Find the _Fields constant that matches fieldId, throwing an exception - * if it is not found. - */ - public static _Fields findByThriftIdOrThrow(int fieldId) { - _Fields fields = findByThriftId(fieldId); - if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); - return fields; - } - - /** - * Find the _Fields constant that matches name, or null if its not found. - */ - public static _Fields findByName(String name) { - return byName.get(name); - } - - private final short _thriftId; - private final String _fieldName; - - _Fields(short thriftId, String fieldName) { - _thriftId = thriftId; - _fieldName = fieldName; - } - - public short getThriftFieldId() { - return _thriftId; - } - - public String getFieldName() { - return _fieldName; - } - } - - // isset id assignments - private static final int __PORT_ISSET_ID = 0; - private BitSet __isset_bit_vector = new BitSet(1); - - public static final Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> metaDataMap; - static { - Map<_Fields, org.apache.thrift7.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift7.meta_data.FieldMetaData>(_Fields.class); - tmpMap.put(_Fields.PORT, new org.apache.thrift7.meta_data.FieldMetaData("port", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.I32))); - tmpMap.put(_Fields.TOPOLOGY, new org.apache.thrift7.meta_data.FieldMetaData("topology", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.FieldValueMetaData(org.apache.thrift7.protocol.TType.STRING))); - tmpMap.put(_Fields.TASKS, new org.apache.thrift7.meta_data.FieldMetaData("tasks", org.apache.thrift7.TFieldRequirementType.REQUIRED, - new org.apache.thrift7.meta_data.ListMetaData(org.apache.thrift7.protocol.TType.LIST, - new org.apache.thrift7.meta_data.StructMetaData(org.apache.thrift7.protocol.TType.STRUCT, TaskSummary.class)))); - metaDataMap = Collections.unmodifiableMap(tmpMap); - org.apache.thrift7.meta_data.FieldMetaData.addStructMetaDataMap(WorkerSummary.class, metaDataMap); - } - - public WorkerSummary() { - } - - public WorkerSummary( - int port, - String topology, - List tasks) - { - this(); - this.port = port; - set_port_isSet(true); - this.topology = topology; - this.tasks = tasks; - } - - /** - * Performs a deep copy on other. - */ - public WorkerSummary(WorkerSummary other) { - __isset_bit_vector.clear(); - __isset_bit_vector.or(other.__isset_bit_vector); - this.port = other.port; - if (other.is_set_topology()) { - this.topology = other.topology; - } - if (other.is_set_tasks()) { - List __this__tasks = new ArrayList(); - for (TaskSummary other_element : other.tasks) { - __this__tasks.add(new TaskSummary(other_element)); - } - this.tasks = __this__tasks; - } - } - - public WorkerSummary deepCopy() { - return new WorkerSummary(this); - } - - @Override - public void clear() { - set_port_isSet(false); - this.port = 0; - this.topology = null; - this.tasks = null; - } - - public int get_port() { - return this.port; - } - - public void set_port(int port) { - this.port = port; - set_port_isSet(true); - } - - public void unset_port() { - __isset_bit_vector.clear(__PORT_ISSET_ID); - } - - /** Returns true if field port is set (has been assigned a value) and false otherwise */ - public boolean is_set_port() { - return __isset_bit_vector.get(__PORT_ISSET_ID); - } - - public void set_port_isSet(boolean value) { - __isset_bit_vector.set(__PORT_ISSET_ID, value); - } - - public String get_topology() { - return this.topology; - } - - public void set_topology(String topology) { - this.topology = topology; - } - - public void unset_topology() { - this.topology = null; - } - - /** Returns true if field topology is set (has been assigned a value) and false otherwise */ - public boolean is_set_topology() { - return this.topology != null; - } - - public void set_topology_isSet(boolean value) { - if (!value) { - this.topology = null; - } - } - - public int get_tasks_size() { - return (this.tasks == null) ? 0 : this.tasks.size(); - } - - public java.util.Iterator get_tasks_iterator() { - return (this.tasks == null) ? null : this.tasks.iterator(); - } - - public void add_to_tasks(TaskSummary elem) { - if (this.tasks == null) { - this.tasks = new ArrayList(); - } - this.tasks.add(elem); - } - - public List get_tasks() { - return this.tasks; - } - - public void set_tasks(List tasks) { - this.tasks = tasks; - } - - public void unset_tasks() { - this.tasks = null; - } - - /** Returns true if field tasks is set (has been assigned a value) and false otherwise */ - public boolean is_set_tasks() { - return this.tasks != null; - } - - public void set_tasks_isSet(boolean value) { - if (!value) { - this.tasks = null; - } - } - - public void setFieldValue(_Fields field, Object value) { - switch (field) { - case PORT: - if (value == null) { - unset_port(); - } else { - set_port((Integer)value); - } - break; - - case TOPOLOGY: - if (value == null) { - unset_topology(); - } else { - set_topology((String)value); - } - break; - - case TASKS: - if (value == null) { - unset_tasks(); - } else { - set_tasks((List)value); - } - break; - - } - } - - public Object getFieldValue(_Fields field) { - switch (field) { - case PORT: - return Integer.valueOf(get_port()); - - case TOPOLOGY: - return get_topology(); - - case TASKS: - return get_tasks(); - - } - throw new IllegalStateException(); - } - - /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ - public boolean isSet(_Fields field) { - if (field == null) { - throw new IllegalArgumentException(); - } - - switch (field) { - case PORT: - return is_set_port(); - case TOPOLOGY: - return is_set_topology(); - case TASKS: - return is_set_tasks(); - } - throw new IllegalStateException(); - } - - @Override - public boolean equals(Object that) { - if (that == null) - return false; - if (that instanceof WorkerSummary) - return this.equals((WorkerSummary)that); - return false; - } - - public boolean equals(WorkerSummary that) { - if (that == null) - return false; - - boolean this_present_port = true; - boolean that_present_port = true; - if (this_present_port || that_present_port) { - if (!(this_present_port && that_present_port)) - return false; - if (this.port != that.port) - return false; - } - - boolean this_present_topology = true && this.is_set_topology(); - boolean that_present_topology = true && that.is_set_topology(); - if (this_present_topology || that_present_topology) { - if (!(this_present_topology && that_present_topology)) - return false; - if (!this.topology.equals(that.topology)) - return false; - } - - boolean this_present_tasks = true && this.is_set_tasks(); - boolean that_present_tasks = true && that.is_set_tasks(); - if (this_present_tasks || that_present_tasks) { - if (!(this_present_tasks && that_present_tasks)) - return false; - if (!this.tasks.equals(that.tasks)) - return false; - } - - return true; - } - - @Override - public int hashCode() { - HashCodeBuilder builder = new HashCodeBuilder(); - - boolean present_port = true; - builder.append(present_port); - if (present_port) - builder.append(port); - - boolean present_topology = true && (is_set_topology()); - builder.append(present_topology); - if (present_topology) - builder.append(topology); - - boolean present_tasks = true && (is_set_tasks()); - builder.append(present_tasks); - if (present_tasks) - builder.append(tasks); - - return builder.toHashCode(); - } - - public int compareTo(WorkerSummary other) { - if (!getClass().equals(other.getClass())) { - return getClass().getName().compareTo(other.getClass().getName()); - } - - int lastComparison = 0; - WorkerSummary typedOther = (WorkerSummary)other; - - lastComparison = Boolean.valueOf(is_set_port()).compareTo(typedOther.is_set_port()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_port()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.port, typedOther.port); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_topology()).compareTo(typedOther.is_set_topology()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_topology()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.topology, typedOther.topology); - if (lastComparison != 0) { - return lastComparison; - } - } - lastComparison = Boolean.valueOf(is_set_tasks()).compareTo(typedOther.is_set_tasks()); - if (lastComparison != 0) { - return lastComparison; - } - if (is_set_tasks()) { - lastComparison = org.apache.thrift7.TBaseHelper.compareTo(this.tasks, typedOther.tasks); - if (lastComparison != 0) { - return lastComparison; - } - } - return 0; - } - - public _Fields fieldForId(int fieldId) { - return _Fields.findByThriftId(fieldId); - } - - public void read(org.apache.thrift7.protocol.TProtocol iprot) throws org.apache.thrift7.TException { - org.apache.thrift7.protocol.TField field; - iprot.readStructBegin(); - while (true) - { - field = iprot.readFieldBegin(); - if (field.type == org.apache.thrift7.protocol.TType.STOP) { - break; - } - switch (field.id) { - case 1: // PORT - if (field.type == org.apache.thrift7.protocol.TType.I32) { - this.port = iprot.readI32(); - set_port_isSet(true); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 2: // TOPOLOGY - if (field.type == org.apache.thrift7.protocol.TType.STRING) { - this.topology = iprot.readString(); - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - case 3: // TASKS - if (field.type == org.apache.thrift7.protocol.TType.LIST) { - { - org.apache.thrift7.protocol.TList _list189 = iprot.readListBegin(); - this.tasks = new ArrayList(_list189.size); - for (int _i190 = 0; _i190 < _list189.size; ++_i190) - { - TaskSummary _elem191; // required - _elem191 = new TaskSummary(); - _elem191.read(iprot); - this.tasks.add(_elem191); - } - iprot.readListEnd(); - } - } else { - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - break; - default: - org.apache.thrift7.protocol.TProtocolUtil.skip(iprot, field.type); - } - iprot.readFieldEnd(); - } - iprot.readStructEnd(); - validate(); - } - - public void write(org.apache.thrift7.protocol.TProtocol oprot) throws org.apache.thrift7.TException { - validate(); - - oprot.writeStructBegin(STRUCT_DESC); - oprot.writeFieldBegin(PORT_FIELD_DESC); - oprot.writeI32(this.port); - oprot.writeFieldEnd(); - if (this.topology != null) { - oprot.writeFieldBegin(TOPOLOGY_FIELD_DESC); - oprot.writeString(this.topology); - oprot.writeFieldEnd(); - } - if (this.tasks != null) { - oprot.writeFieldBegin(TASKS_FIELD_DESC); - { - oprot.writeListBegin(new org.apache.thrift7.protocol.TList(org.apache.thrift7.protocol.TType.STRUCT, this.tasks.size())); - for (TaskSummary _iter192 : this.tasks) - { - _iter192.write(oprot); - } - oprot.writeListEnd(); - } - oprot.writeFieldEnd(); - } - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder("WorkerSummary("); - boolean first = true; - - sb.append("port:"); - sb.append(this.port); - first = false; - if (!first) sb.append(", "); - sb.append("topology:"); - if (this.topology == null) { - sb.append("null"); - } else { - sb.append(this.topology); - } - first = false; - if (!first) sb.append(", "); - sb.append("tasks:"); - if (this.tasks == null) { - sb.append("null"); - } else { - sb.append(this.tasks); - } - first = false; - sb.append(")"); - return sb.toString(); - } - - public void validate() throws org.apache.thrift7.TException { - // check for required fields - if (!is_set_port()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'port' is unset! Struct:" + toString()); - } - - if (!is_set_topology()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'topology' is unset! Struct:" + toString()); - } - - if (!is_set_tasks()) { - throw new org.apache.thrift7.protocol.TProtocolException("Required field 'tasks' is unset! Struct:" + toString()); - } - - } - - private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { - try { - write(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(out))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - - private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { - try { - // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. - __isset_bit_vector = new BitSet(1); - read(new org.apache.thrift7.protocol.TCompactProtocol(new org.apache.thrift7.transport.TIOStreamTransport(in))); - } catch (org.apache.thrift7.TException te) { - throw new java.io.IOException(te); - } - } - -} - diff --git a/jstorm-client/src/main/java/backtype/storm/grouping/CustomStreamGrouping.java b/jstorm-client/src/main/java/backtype/storm/grouping/CustomStreamGrouping.java deleted file mode 100644 index 15e37a8d0..000000000 --- a/jstorm-client/src/main/java/backtype/storm/grouping/CustomStreamGrouping.java +++ /dev/null @@ -1,29 +0,0 @@ -package backtype.storm.grouping; - -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.task.WorkerTopologyContext; -import java.io.Serializable; -import java.util.List; - -public interface CustomStreamGrouping extends Serializable { - - /** - * Tells the stream grouping at runtime the tasks in the target bolt. This - * information should be used in chooseTasks to determine the target tasks. - * - * It also tells the grouping the metadata on the stream this grouping will - * be used on. - */ - void prepare(WorkerTopologyContext context, GlobalStreamId stream, - List targetTasks); - - /** - * This function implements a custom stream grouping. It takes in as input - * the number of tasks in the target bolt in prepare and returns the tasks - * to send the tuples to. - * - * @param values - * the values to group on - */ - List chooseTasks(int taskId, List values); -} diff --git a/jstorm-client/src/main/java/backtype/storm/hooks/BaseTaskHook.java b/jstorm-client/src/main/java/backtype/storm/hooks/BaseTaskHook.java deleted file mode 100644 index a2aac3386..000000000 --- a/jstorm-client/src/main/java/backtype/storm/hooks/BaseTaskHook.java +++ /dev/null @@ -1,44 +0,0 @@ -package backtype.storm.hooks; - -import backtype.storm.hooks.info.BoltAckInfo; -import backtype.storm.hooks.info.BoltExecuteInfo; -import backtype.storm.hooks.info.BoltFailInfo; -import backtype.storm.hooks.info.EmitInfo; -import backtype.storm.hooks.info.SpoutAckInfo; -import backtype.storm.hooks.info.SpoutFailInfo; -import backtype.storm.task.TopologyContext; -import java.util.Map; - -public class BaseTaskHook implements ITaskHook { - @Override - public void prepare(Map conf, TopologyContext context) { - } - - @Override - public void cleanup() { - } - - @Override - public void emit(EmitInfo info) { - } - - @Override - public void spoutAck(SpoutAckInfo info) { - } - - @Override - public void spoutFail(SpoutFailInfo info) { - } - - @Override - public void boltAck(BoltAckInfo info) { - } - - @Override - public void boltFail(BoltFailInfo info) { - } - - @Override - public void boltExecute(BoltExecuteInfo info) { - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/hooks/ITaskHook.java b/jstorm-client/src/main/java/backtype/storm/hooks/ITaskHook.java deleted file mode 100644 index f705f12b9..000000000 --- a/jstorm-client/src/main/java/backtype/storm/hooks/ITaskHook.java +++ /dev/null @@ -1,28 +0,0 @@ -package backtype.storm.hooks; - -import backtype.storm.hooks.info.BoltAckInfo; -import backtype.storm.hooks.info.BoltExecuteInfo; -import backtype.storm.hooks.info.SpoutFailInfo; -import backtype.storm.hooks.info.SpoutAckInfo; -import backtype.storm.hooks.info.EmitInfo; -import backtype.storm.hooks.info.BoltFailInfo; -import backtype.storm.task.TopologyContext; -import java.util.Map; - -public interface ITaskHook { - void prepare(Map conf, TopologyContext context); - - void cleanup(); - - void emit(EmitInfo info); - - void spoutAck(SpoutAckInfo info); - - void spoutFail(SpoutFailInfo info); - - void boltExecute(BoltExecuteInfo info); - - void boltAck(BoltAckInfo info); - - void boltFail(BoltFailInfo info); -} diff --git a/jstorm-client/src/main/java/backtype/storm/hooks/info/BoltAckInfo.java b/jstorm-client/src/main/java/backtype/storm/hooks/info/BoltAckInfo.java deleted file mode 100644 index b0f0a9b2a..000000000 --- a/jstorm-client/src/main/java/backtype/storm/hooks/info/BoltAckInfo.java +++ /dev/null @@ -1,15 +0,0 @@ -package backtype.storm.hooks.info; - -import backtype.storm.tuple.Tuple; - -public class BoltAckInfo { - public Tuple tuple; - public int ackingTaskId; - public Long processLatencyMs; // null if it wasn't sampled - - public BoltAckInfo(Tuple tuple, int ackingTaskId, Long processLatencyMs) { - this.tuple = tuple; - this.ackingTaskId = ackingTaskId; - this.processLatencyMs = processLatencyMs; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/hooks/info/BoltExecuteInfo.java b/jstorm-client/src/main/java/backtype/storm/hooks/info/BoltExecuteInfo.java deleted file mode 100644 index 31ca3738d..000000000 --- a/jstorm-client/src/main/java/backtype/storm/hooks/info/BoltExecuteInfo.java +++ /dev/null @@ -1,16 +0,0 @@ -package backtype.storm.hooks.info; - -import backtype.storm.tuple.Tuple; - -public class BoltExecuteInfo { - public Tuple tuple; - public int executingTaskId; - public Long executeLatencyMs; // null if it wasn't sampled - - public BoltExecuteInfo(Tuple tuple, int executingTaskId, - Long executeLatencyMs) { - this.tuple = tuple; - this.executingTaskId = executingTaskId; - this.executeLatencyMs = executeLatencyMs; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/hooks/info/BoltFailInfo.java b/jstorm-client/src/main/java/backtype/storm/hooks/info/BoltFailInfo.java deleted file mode 100644 index 3a3dfec27..000000000 --- a/jstorm-client/src/main/java/backtype/storm/hooks/info/BoltFailInfo.java +++ /dev/null @@ -1,15 +0,0 @@ -package backtype.storm.hooks.info; - -import backtype.storm.tuple.Tuple; - -public class BoltFailInfo { - public Tuple tuple; - public int failingTaskId; - public Long failLatencyMs; // null if it wasn't sampled - - public BoltFailInfo(Tuple tuple, int failingTaskId, Long failLatencyMs) { - this.tuple = tuple; - this.failingTaskId = failingTaskId; - this.failLatencyMs = failLatencyMs; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/hooks/info/EmitInfo.java b/jstorm-client/src/main/java/backtype/storm/hooks/info/EmitInfo.java deleted file mode 100644 index 39b9688f5..000000000 --- a/jstorm-client/src/main/java/backtype/storm/hooks/info/EmitInfo.java +++ /dev/null @@ -1,19 +0,0 @@ -package backtype.storm.hooks.info; - -import java.util.Collection; -import java.util.List; - -public class EmitInfo { - public List values; - public String stream; - public int taskId; - public Collection outTasks; - - public EmitInfo(List values, String stream, int taskId, - Collection outTasks) { - this.values = values; - this.stream = stream; - this.taskId = taskId; - this.outTasks = outTasks; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/hooks/info/SpoutAckInfo.java b/jstorm-client/src/main/java/backtype/storm/hooks/info/SpoutAckInfo.java deleted file mode 100644 index f74efaebe..000000000 --- a/jstorm-client/src/main/java/backtype/storm/hooks/info/SpoutAckInfo.java +++ /dev/null @@ -1,14 +0,0 @@ -package backtype.storm.hooks.info; - -public class SpoutAckInfo { - public Object messageId; - public int spoutTaskId; - public Long completeLatencyMs; // null if it wasn't sampled - - public SpoutAckInfo(Object messageId, int spoutTaskId, - Long completeLatencyMs) { - this.messageId = messageId; - this.spoutTaskId = spoutTaskId; - this.completeLatencyMs = completeLatencyMs; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/hooks/info/SpoutFailInfo.java b/jstorm-client/src/main/java/backtype/storm/hooks/info/SpoutFailInfo.java deleted file mode 100644 index 8052b4a3f..000000000 --- a/jstorm-client/src/main/java/backtype/storm/hooks/info/SpoutFailInfo.java +++ /dev/null @@ -1,13 +0,0 @@ -package backtype.storm.hooks.info; - -public class SpoutFailInfo { - public Object messageId; - public int spoutTaskId; - public Long failLatencyMs; // null if it wasn't sampled - - public SpoutFailInfo(Object messageId, int spoutTaskId, Long failLatencyMs) { - this.messageId = messageId; - this.spoutTaskId = spoutTaskId; - this.failLatencyMs = failLatencyMs; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/messaging/IConnection.java b/jstorm-client/src/main/java/backtype/storm/messaging/IConnection.java deleted file mode 100644 index f61e818fc..000000000 --- a/jstorm-client/src/main/java/backtype/storm/messaging/IConnection.java +++ /dev/null @@ -1,36 +0,0 @@ -package backtype.storm.messaging; - -import java.util.List; - -import backtype.storm.utils.DisruptorQueue; - -public interface IConnection { - - /** - * (flags != 1) synchronously - * (flags==1) asynchronously - * - * @param flags - * @return - */ - public TaskMessage recv(int flags); - - /** - * In the new design, receive flow is through registerQueue, - * then push message into queue - * - * @param recvQueu - */ - public void registerQueue(DisruptorQueue recvQueu); - public void enqueue(TaskMessage message); - - public void send(List messages); - public void send(TaskMessage message); - - /** - * close this connection - */ - public void close(); - - public boolean isClosed(); -} diff --git a/jstorm-client/src/main/java/backtype/storm/messaging/IContext.java b/jstorm-client/src/main/java/backtype/storm/messaging/IContext.java deleted file mode 100644 index 760b6e5e8..000000000 --- a/jstorm-client/src/main/java/backtype/storm/messaging/IContext.java +++ /dev/null @@ -1,58 +0,0 @@ -package backtype.storm.messaging; - -import java.util.Map; - -import backtype.storm.utils.DisruptorQueue; - -/** - * This interface needs to be implemented for messaging plugin. - * - * Messaging plugin is specified via Storm config parameter, - * storm.messaging.transport. - * - * A messaging plugin should have a default constructor and implements IContext - * interface. Upon construction, we will invoke IContext::prepare(storm_conf) to - * enable context to be configured according to storm configuration. - */ -public interface IContext { - /** - * This method is invoked at the startup of messaging plugin - * - * @param storm_conf - * storm configuration - */ - public void prepare(Map storm_conf); - - /** - * This method is invoked when a worker is unload a messaging plugin - */ - public void term(); - - /** - * This method establishes a server side connection - * - * @param topology_id - * topology ID - * @param port - * port # - * @param distribute - * true -- receive other worker's data - * @return server side connection - */ - public IConnection bind(String topology_id, int port); - - /** - * This method establish a client side connection to a remote server - * - * @param topology_id - * topology ID - * @param host - * remote host - * @param port - * remote port - * @param distribute - * true -- send other worker data - * @return client side connection - */ - public IConnection connect(String topology_id, String host, int port); -}; diff --git a/jstorm-client/src/main/java/backtype/storm/messaging/TaskMessage.java b/jstorm-client/src/main/java/backtype/storm/messaging/TaskMessage.java deleted file mode 100644 index cab968f6d..000000000 --- a/jstorm-client/src/main/java/backtype/storm/messaging/TaskMessage.java +++ /dev/null @@ -1,51 +0,0 @@ -package backtype.storm.messaging; - -import java.nio.ByteBuffer; - -public class TaskMessage { - private int _task; - private byte[] _message; - - public TaskMessage(int task, byte[] message) { - _task = task; - _message = message; - } - - public int task() { - return _task; - } - - public byte[] message() { - return _message; - } - - public static boolean isEmpty(TaskMessage message) { - if (message == null) { - return true; - }else if (message.message() == null) { - return true; - }else if (message.message().length == 0) { - return true; - } - - return false; - } - - @Deprecated - public ByteBuffer serialize() { - ByteBuffer bb = ByteBuffer.allocate(_message.length + 2); - bb.putShort((short) _task); - bb.put(_message); - return bb; - } - - @Deprecated - public void deserialize(ByteBuffer packet) { - if (packet == null) - return; - _task = packet.getShort(); - _message = new byte[packet.limit() - 2]; - packet.get(_message); - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/messaging/TransportFactory.java b/jstorm-client/src/main/java/backtype/storm/messaging/TransportFactory.java deleted file mode 100644 index 8830496ab..000000000 --- a/jstorm-client/src/main/java/backtype/storm/messaging/TransportFactory.java +++ /dev/null @@ -1,54 +0,0 @@ -package backtype.storm.messaging; - -import java.lang.reflect.Constructor; -import java.lang.reflect.Method; -import java.util.Map; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; - -public class TransportFactory { - public static final Logger LOG = Logger.getLogger(TransportFactory.class); - - public static IContext makeContext(Map storm_conf) { - - // get factory class name - String transport_plugin_klassName = (String) storm_conf - .get(Config.STORM_MESSAGING_TRANSPORT); - LOG.info("JStorm peer transport plugin:" + transport_plugin_klassName); - - IContext transport = null; - try { - // create a factory class - Class klass = Class.forName(transport_plugin_klassName); - // obtain a context object - // Object obj = klass.newInstance(); - Constructor constructor = klass.getDeclaredConstructor(); - constructor.setAccessible(true); - Object obj = constructor.newInstance(); - LOG.info("TransportFactory makeContext: new klass: " + obj); - if (obj instanceof IContext) { - // case 1: plugin is a IContext class - transport = (IContext) obj; - // initialize with storm configuration - transport.prepare(storm_conf); - LOG.info("TransportFactory makeContext: start prepare... " - + storm_conf); - } else { - // case 2: Non-IContext plugin must have a - // makeContext(storm_conf) method that returns IContext object - Method method = klass.getMethod("makeContext", Map.class); - LOG.debug("object:" + obj + " method:" + method); - transport = (IContext) method.invoke(obj, storm_conf); - } - LOG.info("TransportFactory makeContext done..."); - } catch (Exception e) { - throw new RuntimeException( - "Fail to construct messaging plugin from plugin " - + transport_plugin_klassName, e); - } - return transport; - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/metric/LoggingMetricsConsumer.java b/jstorm-client/src/main/java/backtype/storm/metric/LoggingMetricsConsumer.java deleted file mode 100644 index 19c223583..000000000 --- a/jstorm-client/src/main/java/backtype/storm/metric/LoggingMetricsConsumer.java +++ /dev/null @@ -1,59 +0,0 @@ -package backtype.storm.metric; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Collection; -import java.util.Map; - -import backtype.storm.metric.api.IMetricsConsumer; -import backtype.storm.task.IErrorReporter; -import backtype.storm.task.TopologyContext; -import backtype.storm.utils.Utils; - -/* - * Listens for all metrics, dumps them to log - * - * To use, add this to your topology's configuration: - * conf.registerMetricsConsumer(backtype.storm.metrics.LoggingMetricsConsumer.class, 1); - * - * Or edit the storm.yaml config file: - * - * topology.metrics.consumer.register: - * - class: "backtype.storm.metrics.LoggingMetricsConsumer" - * parallelism.hint: 1 - * - */ -public class LoggingMetricsConsumer implements IMetricsConsumer { - public static final Logger LOG = LoggerFactory - .getLogger(LoggingMetricsConsumer.class); - - @Override - public void prepare(Map stormConf, Object registrationArgument, - TopologyContext context, IErrorReporter errorReporter) { - } - - static private String padding = " "; - - @Override - public void handleDataPoints(TaskInfo taskInfo, - Collection dataPoints) { - StringBuilder sb = new StringBuilder(); - String header = String.format("%d\t%15s:%-4d\t%3d:%-11s\t", - taskInfo.timestamp, taskInfo.srcWorkerHost, - taskInfo.srcWorkerPort, taskInfo.srcTaskId, - taskInfo.srcComponentId); - sb.append(header); - for (DataPoint p : dataPoints) { - sb.delete(header.length(), sb.length()); - sb.append(p.name).append(padding) - .delete(header.length() + 23, sb.length()).append("\t") - .append(p.value); - LOG.info(sb.toString()); - } - } - - @Override - public void cleanup() { - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/metric/MetricsConsumerBolt.java b/jstorm-client/src/main/java/backtype/storm/metric/MetricsConsumerBolt.java deleted file mode 100644 index 994cb56fe..000000000 --- a/jstorm-client/src/main/java/backtype/storm/metric/MetricsConsumerBolt.java +++ /dev/null @@ -1,56 +0,0 @@ -package backtype.storm.metric; - -import backtype.storm.Config; -import backtype.storm.metric.api.IMetricsConsumer; -import backtype.storm.task.IBolt; -import backtype.storm.task.IErrorReporter; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Tuple; -import java.util.Collection; -import java.util.Map; - -public class MetricsConsumerBolt implements IBolt { - IMetricsConsumer _metricsConsumer; - String _consumerClassName; - OutputCollector _collector; - Object _registrationArgument; - - public MetricsConsumerBolt(String consumerClassName, - Object registrationArgument) { - _consumerClassName = consumerClassName; - _registrationArgument = registrationArgument; - } - - @Override - public void prepare(Map stormConf, TopologyContext context, - OutputCollector collector) { - try { - _metricsConsumer = (IMetricsConsumer) Class.forName( - _consumerClassName).newInstance(); - } catch (Exception e) { - throw new RuntimeException( - "Could not instantiate a class listed in config under section " - + Config.TOPOLOGY_METRICS_CONSUMER_REGISTER - + " with fully qualified name " - + _consumerClassName, e); - } - _metricsConsumer.prepare(stormConf, _registrationArgument, context, - (IErrorReporter) collector); - _collector = collector; - } - - @Override - public void execute(Tuple input) { - _metricsConsumer.handleDataPoints( - (IMetricsConsumer.TaskInfo) input.getValue(0), - (Collection) input.getValue(1)); - _collector.ack(input); - } - - @Override - public void cleanup() { - _metricsConsumer.cleanup(); - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/metric/SystemBolt.java b/jstorm-client/src/main/java/backtype/storm/metric/SystemBolt.java deleted file mode 100644 index 07bdc28eb..000000000 --- a/jstorm-client/src/main/java/backtype/storm/metric/SystemBolt.java +++ /dev/null @@ -1,157 +0,0 @@ -package backtype.storm.metric; - -import backtype.storm.Config; -import backtype.storm.metric.api.AssignableMetric; -import backtype.storm.metric.api.IMetric; -import backtype.storm.task.IBolt; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Tuple; -import clojure.lang.AFn; -import clojure.lang.IFn; -import clojure.lang.RT; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.lang.management.*; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -// There is one task inside one executor for each worker of the topology. -// TaskID is always -1, therefore you can only send-unanchored tuples to co-located SystemBolt. -// This bolt was conceived to export worker stats via metrics api. -public class SystemBolt implements IBolt { - private static Logger LOG = LoggerFactory.getLogger(SystemBolt.class); - private static boolean _prepareWasCalled = false; - - private static class MemoryUsageMetric implements IMetric { - IFn _getUsage; - - public MemoryUsageMetric(IFn getUsage) { - _getUsage = getUsage; - } - - @Override - public Object getValueAndReset() { - MemoryUsage memUsage; - try { - memUsage = (MemoryUsage) _getUsage.invoke(); - } catch (Exception e) { - LOG.error("Failed to get userage ", e); - throw new RuntimeException(e); - } - HashMap m = new HashMap(); - m.put("maxBytes", memUsage.getMax()); - m.put("committedBytes", memUsage.getCommitted()); - m.put("initBytes", memUsage.getInit()); - m.put("usedBytes", memUsage.getUsed()); - m.put("virtualFreeBytes", memUsage.getMax() - memUsage.getUsed()); - m.put("unusedBytes", memUsage.getCommitted() - memUsage.getUsed()); - return m; - } - } - - // canonically the metrics data exported is time bucketed when doing counts. - // convert the absolute values here into time buckets. - private static class GarbageCollectorMetric implements IMetric { - GarbageCollectorMXBean _gcBean; - Long _collectionCount; - Long _collectionTime; - - public GarbageCollectorMetric(GarbageCollectorMXBean gcBean) { - _gcBean = gcBean; - } - - @Override - public Object getValueAndReset() { - Long collectionCountP = _gcBean.getCollectionCount(); - Long collectionTimeP = _gcBean.getCollectionTime(); - - Map ret = null; - if (_collectionCount != null && _collectionTime != null) { - ret = new HashMap(); - ret.put("count", collectionCountP - _collectionCount); - ret.put("timeMs", collectionTimeP - _collectionTime); - } - - _collectionCount = collectionCountP; - _collectionTime = collectionTimeP; - return ret; - } - } - - @Override - public void prepare(final Map stormConf, TopologyContext context, - OutputCollector collector) { - if (_prepareWasCalled - && !"local".equals(stormConf.get(Config.STORM_CLUSTER_MODE))) { - throw new RuntimeException( - "A single worker should have 1 SystemBolt instance."); - } - _prepareWasCalled = true; - - int bucketSize = RT.intCast(stormConf - .get(Config.TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS)); - - final RuntimeMXBean jvmRT = ManagementFactory.getRuntimeMXBean(); - - context.registerMetric("uptimeSecs", new IMetric() { - @Override - public Object getValueAndReset() { - return jvmRT.getUptime() / 1000.0; - } - }, bucketSize); - - context.registerMetric("startTimeSecs", new IMetric() { - @Override - public Object getValueAndReset() { - return jvmRT.getStartTime() / 1000.0; - } - }, bucketSize); - - context.registerMetric("newWorkerEvent", new IMetric() { - boolean doEvent = true; - - @Override - public Object getValueAndReset() { - if (doEvent) { - doEvent = false; - return 1; - } else - return 0; - } - }, bucketSize); - - final MemoryMXBean jvmMemRT = ManagementFactory.getMemoryMXBean(); - - context.registerMetric("memory/heap", new MemoryUsageMetric(new AFn() { - public Object invoke() { - return jvmMemRT.getHeapMemoryUsage(); - } - }), bucketSize); - context.registerMetric("memory/nonHeap", new MemoryUsageMetric( - new AFn() { - public Object invoke() { - return jvmMemRT.getNonHeapMemoryUsage(); - } - }), bucketSize); - - for (GarbageCollectorMXBean b : ManagementFactory - .getGarbageCollectorMXBeans()) { - context.registerMetric("GC/" + b.getName().replaceAll("\\W", ""), - new GarbageCollectorMetric(b), bucketSize); - } - } - - @Override - public void execute(Tuple input) { - throw new RuntimeException( - "Non-system tuples should never be sent to __system bolt."); - } - - @Override - public void cleanup() { - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/metric/api/AssignableMetric.java b/jstorm-client/src/main/java/backtype/storm/metric/api/AssignableMetric.java deleted file mode 100644 index ed6dc72ec..000000000 --- a/jstorm-client/src/main/java/backtype/storm/metric/api/AssignableMetric.java +++ /dev/null @@ -1,17 +0,0 @@ -package backtype.storm.metric.api; - -public class AssignableMetric implements IMetric { - Object _value; - - public AssignableMetric(Object value) { - _value = value; - } - - public void setValue(Object value) { - _value = value; - } - - public Object getValueAndReset() { - return _value; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/metric/api/CombinedMetric.java b/jstorm-client/src/main/java/backtype/storm/metric/api/CombinedMetric.java deleted file mode 100644 index cf74184a4..000000000 --- a/jstorm-client/src/main/java/backtype/storm/metric/api/CombinedMetric.java +++ /dev/null @@ -1,21 +0,0 @@ -package backtype.storm.metric.api; - -public class CombinedMetric implements IMetric { - private final ICombiner _combiner; - private Object _value; - - public CombinedMetric(ICombiner combiner) { - _combiner = combiner; - _value = _combiner.identity(); - } - - public void update(Object value) { - _value = _combiner.combine(_value, value); - } - - public Object getValueAndReset() { - Object ret = _value; - _value = _combiner.identity(); - return ret; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/metric/api/CountMetric.java b/jstorm-client/src/main/java/backtype/storm/metric/api/CountMetric.java deleted file mode 100644 index 12694cdf8..000000000 --- a/jstorm-client/src/main/java/backtype/storm/metric/api/CountMetric.java +++ /dev/null @@ -1,24 +0,0 @@ -package backtype.storm.metric.api; - -import backtype.storm.metric.api.IMetric; - -public class CountMetric implements IMetric { - long _value = 0; - - public CountMetric() { - } - - public void incr() { - _value++; - } - - public void incrBy(long incrementBy) { - _value += incrementBy; - } - - public Object getValueAndReset() { - long ret = _value; - _value = 0; - return ret; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/metric/api/ICombiner.java b/jstorm-client/src/main/java/backtype/storm/metric/api/ICombiner.java deleted file mode 100644 index cdc93639a..000000000 --- a/jstorm-client/src/main/java/backtype/storm/metric/api/ICombiner.java +++ /dev/null @@ -1,7 +0,0 @@ -package backtype.storm.metric.api; - -public interface ICombiner { - public T identity(); - - public T combine(T a, T b); -} diff --git a/jstorm-client/src/main/java/backtype/storm/metric/api/IMetric.java b/jstorm-client/src/main/java/backtype/storm/metric/api/IMetric.java deleted file mode 100644 index cd507578f..000000000 --- a/jstorm-client/src/main/java/backtype/storm/metric/api/IMetric.java +++ /dev/null @@ -1,5 +0,0 @@ -package backtype.storm.metric.api; - -public interface IMetric { - public Object getValueAndReset(); -} diff --git a/jstorm-client/src/main/java/backtype/storm/metric/api/IMetricsConsumer.java b/jstorm-client/src/main/java/backtype/storm/metric/api/IMetricsConsumer.java deleted file mode 100644 index 51b8d5b6f..000000000 --- a/jstorm-client/src/main/java/backtype/storm/metric/api/IMetricsConsumer.java +++ /dev/null @@ -1,56 +0,0 @@ -package backtype.storm.metric.api; - -import backtype.storm.task.IErrorReporter; -import backtype.storm.task.TopologyContext; -import java.util.Collection; -import java.util.Map; - -public interface IMetricsConsumer { - public static class TaskInfo { - public TaskInfo() { - } - - public TaskInfo(String srcWorkerHost, int srcWorkerPort, - String srcComponentId, int srcTaskId, long timestamp, - int updateIntervalSecs) { - this.srcWorkerHost = srcWorkerHost; - this.srcWorkerPort = srcWorkerPort; - this.srcComponentId = srcComponentId; - this.srcTaskId = srcTaskId; - this.timestamp = timestamp; - this.updateIntervalSecs = updateIntervalSecs; - } - - public String srcWorkerHost; - public int srcWorkerPort; - public String srcComponentId; - public int srcTaskId; - public long timestamp; - public int updateIntervalSecs; - } - - public static class DataPoint { - public DataPoint() { - } - - public DataPoint(String name, Object value) { - this.name = name; - this.value = value; - } - - @Override - public String toString() { - return "[" + name + " = " + value + "]"; - } - - public String name; - public Object value; - } - - void prepare(Map stormConf, Object registrationArgument, - TopologyContext context, IErrorReporter errorReporter); - - void handleDataPoints(TaskInfo taskInfo, Collection dataPoints); - - void cleanup(); -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/metric/api/IReducer.java b/jstorm-client/src/main/java/backtype/storm/metric/api/IReducer.java deleted file mode 100644 index fe221ae75..000000000 --- a/jstorm-client/src/main/java/backtype/storm/metric/api/IReducer.java +++ /dev/null @@ -1,9 +0,0 @@ -package backtype.storm.metric.api; - -public interface IReducer { - T init(); - - T reduce(T accumulator, Object input); - - Object extractResult(T accumulator); -} diff --git a/jstorm-client/src/main/java/backtype/storm/metric/api/IStatefulObject.java b/jstorm-client/src/main/java/backtype/storm/metric/api/IStatefulObject.java deleted file mode 100644 index ab37b2c40..000000000 --- a/jstorm-client/src/main/java/backtype/storm/metric/api/IStatefulObject.java +++ /dev/null @@ -1,5 +0,0 @@ -package backtype.storm.metric.api; - -public interface IStatefulObject { - Object getState(); -} diff --git a/jstorm-client/src/main/java/backtype/storm/metric/api/MeanReducer.java b/jstorm-client/src/main/java/backtype/storm/metric/api/MeanReducer.java deleted file mode 100644 index 86f45935a..000000000 --- a/jstorm-client/src/main/java/backtype/storm/metric/api/MeanReducer.java +++ /dev/null @@ -1,39 +0,0 @@ -package backtype.storm.metric.api; - -import backtype.storm.metric.api.IReducer; - -class MeanReducerState { - public int count = 0; - public double sum = 0.0; -} - -public class MeanReducer implements IReducer { - public MeanReducerState init() { - return new MeanReducerState(); - } - - public MeanReducerState reduce(MeanReducerState acc, Object input) { - acc.count++; - if (input instanceof Double) { - acc.sum += (Double) input; - } else if (input instanceof Long) { - acc.sum += ((Long) input).doubleValue(); - } else if (input instanceof Integer) { - acc.sum += ((Integer) input).doubleValue(); - } else { - throw new RuntimeException( - "MeanReducer::reduce called with unsupported input type `" - + input.getClass() - + "`. Supported types are Double, Long, Integer."); - } - return acc; - } - - public Object extractResult(MeanReducerState acc) { - if (acc.count > 0) { - return new Double(acc.sum / (double) acc.count); - } else { - return null; - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/metric/api/MultiCountMetric.java b/jstorm-client/src/main/java/backtype/storm/metric/api/MultiCountMetric.java deleted file mode 100644 index f550eeb1b..000000000 --- a/jstorm-client/src/main/java/backtype/storm/metric/api/MultiCountMetric.java +++ /dev/null @@ -1,28 +0,0 @@ -package backtype.storm.metric.api; - -import backtype.storm.metric.api.IMetric; -import java.util.HashMap; -import java.util.Map; - -public class MultiCountMetric implements IMetric { - Map _value = new HashMap(); - - public MultiCountMetric() { - } - - public CountMetric scope(String key) { - CountMetric val = _value.get(key); - if (val == null) { - _value.put(key, val = new CountMetric()); - } - return val; - } - - public Object getValueAndReset() { - Map ret = new HashMap(); - for (Map.Entry e : _value.entrySet()) { - ret.put(e.getKey(), e.getValue().getValueAndReset()); - } - return ret; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/metric/api/MultiReducedMetric.java b/jstorm-client/src/main/java/backtype/storm/metric/api/MultiReducedMetric.java deleted file mode 100644 index 5020fd860..000000000 --- a/jstorm-client/src/main/java/backtype/storm/metric/api/MultiReducedMetric.java +++ /dev/null @@ -1,33 +0,0 @@ -package backtype.storm.metric.api; - -import backtype.storm.metric.api.IMetric; -import java.util.HashMap; -import java.util.Map; - -public class MultiReducedMetric implements IMetric { - Map _value = new HashMap(); - IReducer _reducer; - - public MultiReducedMetric(IReducer reducer) { - _reducer = reducer; - } - - public ReducedMetric scope(String key) { - ReducedMetric val = _value.get(key); - if (val == null) { - _value.put(key, val = new ReducedMetric(_reducer)); - } - return val; - } - - public Object getValueAndReset() { - Map ret = new HashMap(); - for (Map.Entry e : _value.entrySet()) { - Object val = e.getValue().getValueAndReset(); - if (val != null) { - ret.put(e.getKey(), val); - } - } - return ret; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/metric/api/ReducedMetric.java b/jstorm-client/src/main/java/backtype/storm/metric/api/ReducedMetric.java deleted file mode 100644 index b2a7bf8c6..000000000 --- a/jstorm-client/src/main/java/backtype/storm/metric/api/ReducedMetric.java +++ /dev/null @@ -1,21 +0,0 @@ -package backtype.storm.metric.api; - -public class ReducedMetric implements IMetric { - private final IReducer _reducer; - private Object _accumulator; - - public ReducedMetric(IReducer reducer) { - _reducer = reducer; - _accumulator = _reducer.init(); - } - - public void update(Object value) { - _accumulator = _reducer.reduce(_accumulator, value); - } - - public Object getValueAndReset() { - Object ret = _reducer.extractResult(_accumulator); - _accumulator = _reducer.init(); - return ret; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/metric/api/StateMetric.java b/jstorm-client/src/main/java/backtype/storm/metric/api/StateMetric.java deleted file mode 100644 index 48170ff0d..000000000 --- a/jstorm-client/src/main/java/backtype/storm/metric/api/StateMetric.java +++ /dev/null @@ -1,14 +0,0 @@ -package backtype.storm.metric.api; - -public class StateMetric implements IMetric { - private IStatefulObject _obj; - - public StateMetric(IStatefulObject obj) { - _obj = obj; - } - - @Override - public Object getValueAndReset() { - return _obj.getState(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/nimbus/DefaultTopologyValidator.java b/jstorm-client/src/main/java/backtype/storm/nimbus/DefaultTopologyValidator.java deleted file mode 100644 index c84be4eaa..000000000 --- a/jstorm-client/src/main/java/backtype/storm/nimbus/DefaultTopologyValidator.java +++ /dev/null @@ -1,16 +0,0 @@ -package backtype.storm.nimbus; - -import backtype.storm.generated.InvalidTopologyException; -import backtype.storm.generated.StormTopology; -import java.util.Map; - -public class DefaultTopologyValidator implements ITopologyValidator { - @Override - public void prepare(Map StormConf) { - } - - @Override - public void validate(String topologyName, Map topologyConf, - StormTopology topology) throws InvalidTopologyException { - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/nimbus/ITopologyValidator.java b/jstorm-client/src/main/java/backtype/storm/nimbus/ITopologyValidator.java deleted file mode 100644 index f73467021..000000000 --- a/jstorm-client/src/main/java/backtype/storm/nimbus/ITopologyValidator.java +++ /dev/null @@ -1,12 +0,0 @@ -package backtype.storm.nimbus; - -import backtype.storm.generated.InvalidTopologyException; -import backtype.storm.generated.StormTopology; -import java.util.Map; - -public interface ITopologyValidator { - void prepare(Map StormConf); - - void validate(String topologyName, Map topologyConf, StormTopology topology) - throws InvalidTopologyException; -} diff --git a/jstorm-client/src/main/java/backtype/storm/planner/CompoundSpout.java b/jstorm-client/src/main/java/backtype/storm/planner/CompoundSpout.java deleted file mode 100644 index 7e5040654..000000000 --- a/jstorm-client/src/main/java/backtype/storm/planner/CompoundSpout.java +++ /dev/null @@ -1,7 +0,0 @@ -package backtype.storm.planner; - -public class CompoundSpout -// implements ISpout -{ - -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/planner/CompoundTask.java b/jstorm-client/src/main/java/backtype/storm/planner/CompoundTask.java deleted file mode 100644 index 71cdb158d..000000000 --- a/jstorm-client/src/main/java/backtype/storm/planner/CompoundTask.java +++ /dev/null @@ -1,7 +0,0 @@ -package backtype.storm.planner; - -public class CompoundTask -// implements IBolt -{ - -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/planner/TaskBundle.java b/jstorm-client/src/main/java/backtype/storm/planner/TaskBundle.java deleted file mode 100644 index c8dadd32a..000000000 --- a/jstorm-client/src/main/java/backtype/storm/planner/TaskBundle.java +++ /dev/null @@ -1,15 +0,0 @@ -package backtype.storm.planner; - -import backtype.storm.task.IBolt; -import java.io.Serializable; - -public class TaskBundle implements Serializable { - public IBolt task; - public int componentId; - - public TaskBundle(IBolt task, int componentId) { - this.task = task; - this.componentId = componentId; - } - -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/scheduler/Cluster.java b/jstorm-client/src/main/java/backtype/storm/scheduler/Cluster.java deleted file mode 100644 index b53f02d1b..000000000 --- a/jstorm-client/src/main/java/backtype/storm/scheduler/Cluster.java +++ /dev/null @@ -1,448 +0,0 @@ -package backtype.storm.scheduler; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -public class Cluster { - - /** - * key: supervisor id, value: supervisor details - */ - private Map supervisors; - /** - * key: topologyId, value: topology's current assignments. - */ - private Map assignments; - - /** - * a map from hostname to supervisor id. - */ - private Map> hostToId; - - private Set blackListedHosts = new HashSet(); - private INimbus inimbus; - - public Cluster(INimbus nimbus, Map supervisors, - Map assignments) { - this.inimbus = nimbus; - this.supervisors = new HashMap( - supervisors.size()); - this.supervisors.putAll(supervisors); - this.assignments = new HashMap( - assignments.size()); - this.assignments.putAll(assignments); - this.hostToId = new HashMap>(); - for (String nodeId : supervisors.keySet()) { - SupervisorDetails supervisor = supervisors.get(nodeId); - String host = supervisor.getHost(); - if (!this.hostToId.containsKey(host)) { - this.hostToId.put(host, new ArrayList()); - } - this.hostToId.get(host).add(nodeId); - } - } - - public void setBlacklistedHosts(Set hosts) { - blackListedHosts = hosts; - } - - public Set getBlacklistedHosts() { - return blackListedHosts; - } - - public void blacklistHost(String host) { - // this is so it plays well with setting blackListedHosts to an - // immutable list - if (blackListedHosts == null) - blackListedHosts = new HashSet(); - if (!(blackListedHosts instanceof HashSet)) - blackListedHosts = new HashSet(blackListedHosts); - blackListedHosts.add(host); - } - - public boolean isBlackListed(String supervisorId) { - return blackListedHosts != null - && blackListedHosts.contains(getHost(supervisorId)); - } - - public boolean isBlacklistedHost(String host) { - return blackListedHosts != null && blackListedHosts.contains(host); - } - - public String getHost(String supervisorId) { - return inimbus.getHostName(supervisors, supervisorId); - } - - /** - * Gets all the topologies which needs scheduling. - * - * @param topologies - * @return - */ - public List needsSchedulingTopologies(Topologies topologies) { - List ret = new ArrayList(); - for (TopologyDetails topology : topologies.getTopologies()) { - if (needsScheduling(topology)) { - ret.add(topology); - } - } - - return ret; - } - - /** - * Does the topology need scheduling? - * - * A topology needs scheduling if one of the following conditions holds: - *
    - *
  • Although the topology is assigned slots, but is squeezed. i.e. the - * topology is assigned less slots than desired.
  • - *
  • There are unassigned executors in this topology
  • - *
- */ - public boolean needsScheduling(TopologyDetails topology) { - int desiredNumWorkers = topology.getNumWorkers(); - int assignedNumWorkers = this.getAssignedNumWorkers(topology); - - if (desiredNumWorkers > assignedNumWorkers) { - return true; - } - - return this.getUnassignedExecutors(topology).size() > 0; - } - - /** - * Gets a executor -> component-id map which needs scheduling in this - * topology. - * - * @param topology - * @return - */ - public Map getNeedsSchedulingExecutorToComponents( - TopologyDetails topology) { - Collection allExecutors = new HashSet( - topology.getExecutors()); - - SchedulerAssignment assignment = this.assignments.get(topology.getId()); - if (assignment != null) { - Collection assignedExecutors = assignment - .getExecutors(); - allExecutors.removeAll(assignedExecutors); - } - - return topology.selectExecutorToComponent(allExecutors); - } - - /** - * Gets a component-id -> executors map which needs scheduling in this - * topology. - * - * @param topology - * @return - */ - public Map> getNeedsSchedulingComponentToExecutors( - TopologyDetails topology) { - Map executorToComponents = this - .getNeedsSchedulingExecutorToComponents(topology); - Map> componentToExecutors = new HashMap>(); - for (ExecutorDetails executor : executorToComponents.keySet()) { - String component = executorToComponents.get(executor); - if (!componentToExecutors.containsKey(component)) { - componentToExecutors.put(component, - new ArrayList()); - } - - componentToExecutors.get(component).add(executor); - } - - return componentToExecutors; - } - - /** - * Get all the used ports of this supervisor. - * - * @param cluster - * @return - */ - public Set getUsedPorts(SupervisorDetails supervisor) { - Map assignments = this.getAssignments(); - Set usedPorts = new HashSet(); - - for (SchedulerAssignment assignment : assignments.values()) { - for (WorkerSlot slot : assignment.getExecutorToSlot().values()) { - if (slot.getNodeId().equals(supervisor.getId())) { - usedPorts.add(slot.getPort()); - } - } - } - - return usedPorts; - } - - /** - * Return the available ports of this supervisor. - * - * @param cluster - * @return - */ - public Set getAvailablePorts(SupervisorDetails supervisor) { - Set usedPorts = this.getUsedPorts(supervisor); - - Set ret = new HashSet(); - ret.addAll(getAssignablePorts(supervisor)); - ret.removeAll(usedPorts); - - return ret; - } - - public Set getAssignablePorts(SupervisorDetails supervisor) { - if (isBlackListed(supervisor.id)) - return new HashSet(); - return supervisor.allPorts; - } - - /** - * Return all the available slots on this supervisor. - * - * @param cluster - * @return - */ - public List getAvailableSlots(SupervisorDetails supervisor) { - Set ports = this.getAvailablePorts(supervisor); - List slots = new ArrayList(ports.size()); - - for (Integer port : ports) { - slots.add(new WorkerSlot(supervisor.getId(), port)); - } - - return slots; - } - - public List getAssignableSlots(SupervisorDetails supervisor) { - Set ports = this.getAssignablePorts(supervisor); - List slots = new ArrayList(ports.size()); - - for (Integer port : ports) { - slots.add(new WorkerSlot(supervisor.getId(), port)); - } - - return slots; - } - - /** - * get the unassigned executors of the topology. - */ - public Collection getUnassignedExecutors( - TopologyDetails topology) { - if (topology == null) { - return new ArrayList(0); - } - - Collection ret = new HashSet(topology.getExecutors()); - - SchedulerAssignment assignment = this.getAssignmentById(topology - .getId()); - if (assignment != null) { - Set assignedExecutors = assignment.getExecutors(); - ret.removeAll(assignedExecutors); - } - - return ret; - } - - /** - * Gets the number of workers assigned to this topology. - * - * @param topology - * @return - */ - public int getAssignedNumWorkers(TopologyDetails topology) { - if (topology == null) { - return 0; - } - SchedulerAssignment assignment = this.getAssignmentById(topology - .getId()); - if (assignment == null) { - return 0; - } - - Set slots = new HashSet(); - slots.addAll(assignment.getExecutorToSlot().values()); - - return slots.size(); - } - - /** - * Assign the slot to the executors for this topology. - * - * @throws RuntimeException - * if the specified slot is already occupied. - */ - public void assign(WorkerSlot slot, String topologyId, - Collection executors) { - if (this.isSlotOccupied(slot)) { - throw new RuntimeException("slot: [" + slot.getNodeId() + ", " - + slot.getPort() + "] is already occupied."); - } - - SchedulerAssignmentImpl assignment = (SchedulerAssignmentImpl) this - .getAssignmentById(topologyId); - if (assignment == null) { - assignment = new SchedulerAssignmentImpl(topologyId, - new HashMap()); - this.assignments.put(topologyId, assignment); - } else { - for (ExecutorDetails executor : executors) { - if (assignment.isExecutorAssigned(executor)) { - throw new RuntimeException( - "the executor is already assigned, you should unassign it before assign it to another slot."); - } - } - } - - assignment.assign(slot, executors); - } - - /** - * Gets all the available slots in the cluster. - * - * @return - */ - public List getAvailableSlots() { - List slots = new ArrayList(); - for (SupervisorDetails supervisor : this.supervisors.values()) { - slots.addAll(this.getAvailableSlots(supervisor)); - } - - return slots; - } - - public List getAssignableSlots() { - List slots = new ArrayList(); - for (SupervisorDetails supervisor : this.supervisors.values()) { - slots.addAll(this.getAssignableSlots(supervisor)); - } - - return slots; - } - - /** - * Free the specified slot. - * - * @param slot - */ - public void freeSlot(WorkerSlot slot) { - // remove the slot from the existing assignments - for (SchedulerAssignmentImpl assignment : this.assignments.values()) { - if (assignment.isSlotOccupied(slot)) { - assignment.unassignBySlot(slot); - } - } - } - - /** - * free the slots. - * - * @param slots - */ - public void freeSlots(Collection slots) { - if (slots != null) { - for (WorkerSlot slot : slots) { - this.freeSlot(slot); - } - } - } - - /** - * Checks the specified slot is occupied. - * - * @param slot - * the slot be to checked. - * @return - */ - public boolean isSlotOccupied(WorkerSlot slot) { - for (SchedulerAssignment assignment : this.assignments.values()) { - if (assignment.isSlotOccupied(slot)) { - return true; - } - } - - return false; - } - - /** - * get the current assignment for the topology. - */ - public SchedulerAssignment getAssignmentById(String topologyId) { - if (this.assignments.containsKey(topologyId)) { - return this.assignments.get(topologyId); - } - - return null; - } - - /** - * Get a specific supervisor with the nodeId - */ - public SupervisorDetails getSupervisorById(String nodeId) { - if (this.supervisors.containsKey(nodeId)) { - return this.supervisors.get(nodeId); - } - - return null; - } - - public Collection getUsedSlots() { - Set ret = new HashSet(); - for (SchedulerAssignmentImpl s : assignments.values()) { - ret.addAll(s.getExecutorToSlot().values()); - } - return ret; - } - - /** - * Get all the supervisors on the specified host. - * - * @param host - * hostname of the supervisor - * @return the SupervisorDetails object. - */ - public List getSupervisorsByHost(String host) { - List nodeIds = this.hostToId.get(host); - List ret = new ArrayList(); - - if (nodeIds != null) { - for (String nodeId : nodeIds) { - ret.add(this.getSupervisorById(nodeId)); - } - } - - return ret; - } - - /** - * Get all the assignments. - */ - public Map getAssignments() { - Map ret = new HashMap( - this.assignments.size()); - - for (String topologyId : this.assignments.keySet()) { - ret.put(topologyId, this.assignments.get(topologyId)); - } - - return ret; - } - - /** - * Get all the supervisors. - */ - public Map getSupervisors() { - return this.supervisors; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/scheduler/ExecutorDetails.java b/jstorm-client/src/main/java/backtype/storm/scheduler/ExecutorDetails.java deleted file mode 100644 index fb04d8451..000000000 --- a/jstorm-client/src/main/java/backtype/storm/scheduler/ExecutorDetails.java +++ /dev/null @@ -1,38 +0,0 @@ -package backtype.storm.scheduler; - -public class ExecutorDetails { - int startTask; - int endTask; - - public ExecutorDetails(int startTask, int endTask) { - this.startTask = startTask; - this.endTask = endTask; - } - - public int getStartTask() { - return startTask; - } - - public int getEndTask() { - return endTask; - } - - public boolean equals(Object other) { - if (other == null || !(other instanceof ExecutorDetails)) { - return false; - } - - ExecutorDetails executor = (ExecutorDetails) other; - return (this.startTask == executor.startTask) - && (this.endTask == executor.endTask); - } - - public int hashCode() { - return this.startTask + 13 * this.endTask; - } - - @Override - public String toString() { - return "[" + this.startTask + ", " + this.endTask + "]"; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/scheduler/INimbus.java b/jstorm-client/src/main/java/backtype/storm/scheduler/INimbus.java deleted file mode 100644 index 2d81b983d..000000000 --- a/jstorm-client/src/main/java/backtype/storm/scheduler/INimbus.java +++ /dev/null @@ -1,28 +0,0 @@ -package backtype.storm.scheduler; - -import java.util.Collection; -import java.util.Map; -import java.util.Set; - -public interface INimbus { - void prepare(Map stormConf, String schedulerLocalDir); - - /** - * Returns all slots that are available for the next round of scheduling. A - * slot is available for scheduling if it is free and can be assigned to, or - * if it is used and can be reassigned. - */ - Collection allSlotsAvailableForScheduling( - Collection existingSupervisors, - Topologies topologies, Set topologiesMissingAssignments); - - // this is called after the assignment is changed in ZK - void assignSlots(Topologies topologies, - Map> newSlotsByTopologyId); - - // map from node id to supervisor details - String getHostName(Map existingSupervisors, - String nodeId); - - IScheduler getForcedScheduler(); -} diff --git a/jstorm-client/src/main/java/backtype/storm/scheduler/IScheduler.java b/jstorm-client/src/main/java/backtype/storm/scheduler/IScheduler.java deleted file mode 100644 index fb6849995..000000000 --- a/jstorm-client/src/main/java/backtype/storm/scheduler/IScheduler.java +++ /dev/null @@ -1,27 +0,0 @@ -package backtype.storm.scheduler; - -import java.util.Map; - -public interface IScheduler { - - void prepare(Map conf); - - /** - * Set assignments for the topologies which needs scheduling. The new - * assignments is available through cluster.getAssignments() - * - * @param topologies - * all the topologies in the cluster, some of them need schedule. - * Topologies object here only contain static information about - * topologies. Information like assignments, slots are all in the - * clusterobject. - * @param cluster - * the cluster these topologies are running in. - * cluster contains everything user need to develop - * a new scheduling logic. e.g. supervisors information, - * available slots, current assignments for all the topologies - * etc. User can set the new assignment for topologies using - * cluster.setAssignmentById - */ - void schedule(Topologies topologies, Cluster cluster); -} diff --git a/jstorm-client/src/main/java/backtype/storm/scheduler/ISupervisor.java b/jstorm-client/src/main/java/backtype/storm/scheduler/ISupervisor.java deleted file mode 100644 index 30548c800..000000000 --- a/jstorm-client/src/main/java/backtype/storm/scheduler/ISupervisor.java +++ /dev/null @@ -1,33 +0,0 @@ -package backtype.storm.scheduler; - -import java.util.Map; -import java.util.Collection; - -public interface ISupervisor { - void prepare(Map stormConf, String schedulerLocalDir); - - // for mesos, this is {hostname}-{topologyid} - /** - * The id used for writing metadata into ZK. - */ - String getSupervisorId(); - - /** - * The id used in assignments. This combined with confirmAssigned decides - * what this supervisor is responsible for. The combination of this and - * getSupervisorId allows Nimbus to assign to a single machine and have - * multiple supervisors on that machine execute the assignment. This is - * important for achieving resource isolation. - */ - String getAssignmentId(); - - Object getMetadata(); - - boolean confirmAssigned(int port); - - // calls this before actually killing the worker locally... - // sends a "task finished" update - void killedWorker(int port); - - void assigned(Collection ports); -} diff --git a/jstorm-client/src/main/java/backtype/storm/scheduler/SchedulerAssignment.java b/jstorm-client/src/main/java/backtype/storm/scheduler/SchedulerAssignment.java deleted file mode 100644 index bf3b2cb31..000000000 --- a/jstorm-client/src/main/java/backtype/storm/scheduler/SchedulerAssignment.java +++ /dev/null @@ -1,45 +0,0 @@ -package backtype.storm.scheduler; - -import java.util.Map; -import java.util.Set; - -public interface SchedulerAssignment { - /** - * Does this slot occupied by this assignment? - * - * @param slot - * @return - */ - public boolean isSlotOccupied(WorkerSlot slot); - - /** - * is the executor assigned? - * - * @param executor - * @return - */ - public boolean isExecutorAssigned(ExecutorDetails executor); - - /** - * get the topology-id this assignment is for. - * - * @return - */ - public String getTopologyId(); - - /** - * get the executor -> slot map. - * - * @return - */ - public Map getExecutorToSlot(); - - /** - * Return the executors covered by this assignments - * - * @return - */ - public Set getExecutors(); - - public Set getSlots(); -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/scheduler/SchedulerAssignmentImpl.java b/jstorm-client/src/main/java/backtype/storm/scheduler/SchedulerAssignmentImpl.java deleted file mode 100644 index 651486311..000000000 --- a/jstorm-client/src/main/java/backtype/storm/scheduler/SchedulerAssignmentImpl.java +++ /dev/null @@ -1,98 +0,0 @@ -package backtype.storm.scheduler; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -//TODO: improve this by maintaining slot -> executors as well for more efficient operations -public class SchedulerAssignmentImpl implements SchedulerAssignment { - /** - * topology-id this assignment is for. - */ - String topologyId; - /** - * assignment detail, a mapping from executor to WorkerSlot - */ - Map executorToSlot; - - public SchedulerAssignmentImpl(String topologyId, - Map executorToSlots) { - this.topologyId = topologyId; - this.executorToSlot = new HashMap(0); - if (executorToSlots != null) { - this.executorToSlot.putAll(executorToSlots); - } - } - - @Override - public Set getSlots() { - return new HashSet(executorToSlot.values()); - } - - /** - * Assign the slot to executors. - * - * @param slot - * @param executors - */ - public void assign(WorkerSlot slot, Collection executors) { - for (ExecutorDetails executor : executors) { - this.executorToSlot.put(executor, slot); - } - } - - /** - * Release the slot occupied by this assignment. - * - * @param slot - */ - public void unassignBySlot(WorkerSlot slot) { - List executors = new ArrayList(); - for (ExecutorDetails executor : this.executorToSlot.keySet()) { - WorkerSlot ws = this.executorToSlot.get(executor); - if (ws.equals(slot)) { - executors.add(executor); - } - } - - // remove - for (ExecutorDetails executor : executors) { - this.executorToSlot.remove(executor); - } - } - - /** - * Does this slot occupied by this assignment? - * - * @param slot - * @return - */ - public boolean isSlotOccupied(WorkerSlot slot) { - return this.executorToSlot.containsValue(slot); - } - - public boolean isExecutorAssigned(ExecutorDetails executor) { - return this.executorToSlot.containsKey(executor); - } - - public String getTopologyId() { - return this.topologyId; - } - - public Map getExecutorToSlot() { - return this.executorToSlot; - } - - /** - * Return the executors covered by this assignments - * - * @return - */ - public Set getExecutors() { - return this.executorToSlot.keySet(); - } -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/scheduler/SupervisorDetails.java b/jstorm-client/src/main/java/backtype/storm/scheduler/SupervisorDetails.java deleted file mode 100644 index adb81cf67..000000000 --- a/jstorm-client/src/main/java/backtype/storm/scheduler/SupervisorDetails.java +++ /dev/null @@ -1,73 +0,0 @@ -package backtype.storm.scheduler; - -import java.util.Collection; -import java.util.HashSet; -import java.util.Set; - -public class SupervisorDetails { - - String id; - /** - * hostname of this supervisor - */ - String host; - Object meta; - /** - * meta data configured for this supervisor - */ - Object schedulerMeta; - /** - * all the ports of the supervisor - */ - Set allPorts; - - public SupervisorDetails(String id, Object meta) { - this.id = id; - this.meta = meta; - allPorts = new HashSet(); - } - - public SupervisorDetails(String id, Object meta, Collection allPorts) { - this.id = id; - this.meta = meta; - setAllPorts(allPorts); - } - - public SupervisorDetails(String id, String host, Object schedulerMeta, - Collection allPorts) { - this.id = id; - this.host = host; - this.schedulerMeta = schedulerMeta; - - setAllPorts(allPorts); - } - - private void setAllPorts(Collection allPorts) { - this.allPorts = new HashSet(); - if (allPorts != null) { - for (Number n : allPorts) { - this.allPorts.add(n.intValue()); - } - } - } - - public String getId() { - return id; - } - - public String getHost() { - return host; - } - - public Object getMeta() { - return meta; - } - - public Set getAllPorts() { - return allPorts; - } - - public Object getSchedulerMeta() { - return this.schedulerMeta; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/scheduler/Topologies.java b/jstorm-client/src/main/java/backtype/storm/scheduler/Topologies.java deleted file mode 100644 index 883213cca..000000000 --- a/jstorm-client/src/main/java/backtype/storm/scheduler/Topologies.java +++ /dev/null @@ -1,42 +0,0 @@ -package backtype.storm.scheduler; - -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; - -public class Topologies { - Map topologies; - Map nameToId; - - public Topologies(Map topologies) { - if (topologies == null) - topologies = new HashMap(); - this.topologies = new HashMap( - topologies.size()); - this.topologies.putAll(topologies); - this.nameToId = new HashMap(topologies.size()); - - for (String topologyId : topologies.keySet()) { - TopologyDetails topology = topologies.get(topologyId); - this.nameToId.put(topology.getName(), topologyId); - } - } - - public TopologyDetails getById(String topologyId) { - return this.topologies.get(topologyId); - } - - public TopologyDetails getByName(String topologyName) { - String topologyId = this.nameToId.get(topologyName); - - if (topologyId == null) { - return null; - } else { - return this.getById(topologyId); - } - } - - public Collection getTopologies() { - return this.topologies.values(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/scheduler/TopologyDetails.java b/jstorm-client/src/main/java/backtype/storm/scheduler/TopologyDetails.java deleted file mode 100644 index fe9138c77..000000000 --- a/jstorm-client/src/main/java/backtype/storm/scheduler/TopologyDetails.java +++ /dev/null @@ -1,76 +0,0 @@ -package backtype.storm.scheduler; - -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; - -import backtype.storm.Config; -import backtype.storm.generated.StormTopology; - -public class TopologyDetails { - String topologyId; - Map topologyConf; - StormTopology topology; - Map executorToComponent; - int numWorkers; - - public TopologyDetails(String topologyId, Map topologyConf, - StormTopology topology, int numWorkers) { - this.topologyId = topologyId; - this.topologyConf = topologyConf; - this.topology = topology; - this.numWorkers = numWorkers; - } - - public TopologyDetails(String topologyId, Map topologyConf, - StormTopology topology, int numWorkers, - Map executorToComponents) { - this(topologyId, topologyConf, topology, numWorkers); - this.executorToComponent = new HashMap(0); - if (executorToComponents != null) { - this.executorToComponent.putAll(executorToComponents); - } - } - - public String getId() { - return topologyId; - } - - public String getName() { - return (String) this.topologyConf.get(Config.TOPOLOGY_NAME); - } - - public Map getConf() { - return topologyConf; - } - - public int getNumWorkers() { - return numWorkers; - } - - public StormTopology getTopology() { - return topology; - } - - public Map getExecutorToComponent() { - return this.executorToComponent; - } - - public Map selectExecutorToComponent( - Collection executors) { - Map ret = new HashMap( - executors.size()); - for (ExecutorDetails executor : executors) { - String compId = this.executorToComponent.get(executor); - if (compId != null) { - ret.put(executor, compId); - } - } - - return ret; - } - - public Collection getExecutors() { - return this.executorToComponent.keySet(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/scheduler/WorkerSlot.java b/jstorm-client/src/main/java/backtype/storm/scheduler/WorkerSlot.java deleted file mode 100644 index a89abab28..000000000 --- a/jstorm-client/src/main/java/backtype/storm/scheduler/WorkerSlot.java +++ /dev/null @@ -1,87 +0,0 @@ -package backtype.storm.scheduler; - -import java.io.Serializable; - -public class WorkerSlot implements Comparable, Serializable { - - private static final long serialVersionUID = -4451854497340313268L; - String nodeId; - int port; - - public WorkerSlot(String nodeId, Number port) { - this.nodeId = nodeId; - this.port = port.intValue(); - } - - public WorkerSlot() { - - } - - public String getNodeId() { - return nodeId; - } - - public int getPort() { - return port; - } - - public void setNodeId(String nodeId) { - this.nodeId = nodeId; - } - - public void setPort(int port) { - this.port = port; - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + ((nodeId == null) ? 0 : nodeId.hashCode()); - result = prime * result + port; - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - WorkerSlot other = (WorkerSlot) obj; - if (nodeId == null) { - if (other.nodeId != null) - return false; - } else if (!nodeId.equals(other.nodeId)) - return false; - if (port != other.port) - return false; - return true; - } - - @Override - public String toString() { - return this.nodeId + ":" + this.port; - } - - @Override - public int compareTo(WorkerSlot o) { - String otherNode = o.getNodeId(); - if (nodeId == null) { - if (otherNode != null) { - return -1; - } else { - return port - o.getPort(); - } - } else { - int ret = nodeId.compareTo(otherNode); - if (ret == 0) { - return port - o.getPort(); - } else { - return ret; - } - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/security/auth/AuthUtils.java b/jstorm-client/src/main/java/backtype/storm/security/auth/AuthUtils.java deleted file mode 100644 index 5306933fb..000000000 --- a/jstorm-client/src/main/java/backtype/storm/security/auth/AuthUtils.java +++ /dev/null @@ -1,94 +0,0 @@ -package backtype.storm.security.auth; - -import backtype.storm.Config; -import javax.security.auth.login.Configuration; -import javax.security.auth.login.AppConfigurationEntry; -import java.security.NoSuchAlgorithmException; -import java.security.URIParameter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.net.URI; -import java.util.Map; - -public class AuthUtils { - private static final Logger LOG = LoggerFactory.getLogger(AuthUtils.class); - public static final String LOGIN_CONTEXT_SERVER = "StormServer"; - public static final String LOGIN_CONTEXT_CLIENT = "StormClient"; - public static final String SERVICE = "storm_thrift_server"; - - /** - * Construct a JAAS configuration object per storm configuration file - * - * @param storm_conf - * Storm configuration - * @return JAAS configuration object - */ - public static Configuration GetConfiguration(Map storm_conf) { - Configuration login_conf = null; - - // find login file configuration from Storm configuration - String loginConfigurationFile = (String) storm_conf - .get("java.security.auth.login.config"); - if ((loginConfigurationFile != null) - && (loginConfigurationFile.length() > 0)) { - try { - URI config_uri = new File(loginConfigurationFile).toURI(); - login_conf = Configuration.getInstance("JavaLoginConfig", - new URIParameter(config_uri)); - } catch (NoSuchAlgorithmException ex1) { - if (ex1.getCause() instanceof FileNotFoundException) - throw new RuntimeException("configuration file " - + loginConfigurationFile + " could not be found"); - else - throw new RuntimeException(ex1); - } catch (Exception ex2) { - throw new RuntimeException(ex2); - } - } - - return login_conf; - } - - /** - * Construct a transport plugin per storm configuration - * - * @param conf - * storm configuration - * @return - */ - public static ITransportPlugin GetTransportPlugin(Map storm_conf, - Configuration login_conf) { - ITransportPlugin transportPlugin = null; - try { - String transport_plugin_klassName = (String) storm_conf - .get(Config.STORM_THRIFT_TRANSPORT_PLUGIN); - Class klass = Class.forName(transport_plugin_klassName); - transportPlugin = (ITransportPlugin) klass.newInstance(); - transportPlugin.prepare(storm_conf, login_conf); - } catch (Exception e) { - throw new RuntimeException(e); - } - return transportPlugin; - } - - public static String get(Configuration configuration, String section, - String key) throws IOException { - AppConfigurationEntry configurationEntries[] = configuration - .getAppConfigurationEntry(section); - if (configurationEntries == null) { - String errorMessage = "Could not find a '" + section - + "' entry in this configuration."; - throw new IOException(errorMessage); - } - - for (AppConfigurationEntry entry : configurationEntries) { - Object val = entry.getOptions().get(key); - if (val != null) - return (String) val; - } - return null; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/security/auth/IAuthorizer.java b/jstorm-client/src/main/java/backtype/storm/security/auth/IAuthorizer.java deleted file mode 100644 index 90b17d0e7..000000000 --- a/jstorm-client/src/main/java/backtype/storm/security/auth/IAuthorizer.java +++ /dev/null @@ -1,38 +0,0 @@ -package backtype.storm.security.auth; - -import java.util.Map; - -/** - * Nimbus could be configured with an authorization plugin. If not specified, - * all requests are authorized. - * - * You could specify the authorization plugin via storm parameter. For example: - * storm -c - * nimbus.authorization.class=backtype.storm.security.auth.NoopAuthorizer ... - * - * You could also specify it via storm.yaml: nimbus.authorization.class: - * backtype.storm.security.auth.NoopAuthorizer - */ -public interface IAuthorizer { - /** - * Invoked once immediately after construction - * - * @param conf - * Storm configuration - */ - void prepare(Map storm_conf); - - /** - * permit() method is invoked for each incoming Thrift request. - * - * @param context - * request context includes info about - * @param operation - * operation name - * @param topology_storm - * configuration of targeted topology - * @return true if the request is authorized, false if reject - */ - public boolean permit(ReqContext context, String operation, - Map topology_conf); -} diff --git a/jstorm-client/src/main/java/backtype/storm/security/auth/ITransportPlugin.java b/jstorm-client/src/main/java/backtype/storm/security/auth/ITransportPlugin.java deleted file mode 100644 index 3e4a35a4e..000000000 --- a/jstorm-client/src/main/java/backtype/storm/security/auth/ITransportPlugin.java +++ /dev/null @@ -1,49 +0,0 @@ -package backtype.storm.security.auth; - -import java.io.IOException; -import java.util.Map; - -import javax.security.auth.login.Configuration; - -import org.apache.thrift7.TProcessor; -import org.apache.thrift7.server.TServer; -import org.apache.thrift7.transport.TTransport; -import org.apache.thrift7.transport.TTransportException; - -/** - * Interface for Thrift Transport plugin - */ -public interface ITransportPlugin { - /** - * Invoked once immediately after construction - * - * @param storm_conf - * Storm configuration - * @param login_conf - * login configuration - */ - void prepare(Map storm_conf, Configuration login_conf); - - /** - * Create a server associated with a given port and service handler - * - * @param port - * listening port - * @param processor - * service handler - * @return server to be binded - */ - public TServer getServer(int port, TProcessor processor) - throws IOException, TTransportException; - - /** - * Connect to the specified server via framed transport - * - * @param transport - * The underlying Thrift transport. - * @param serverHost - * server host - */ - public TTransport connect(TTransport transport, String serverHost) - throws IOException, TTransportException; -} diff --git a/jstorm-client/src/main/java/backtype/storm/security/auth/ReqContext.java b/jstorm-client/src/main/java/backtype/storm/security/auth/ReqContext.java deleted file mode 100644 index e9afde626..000000000 --- a/jstorm-client/src/main/java/backtype/storm/security/auth/ReqContext.java +++ /dev/null @@ -1,91 +0,0 @@ -package backtype.storm.security.auth; - -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; -import java.net.InetAddress; -import com.google.common.annotations.VisibleForTesting; -import java.security.AccessControlContext; -import java.security.AccessController; -import java.security.Principal; -import javax.security.auth.Subject; - -/** - * context request context includes info about (1) remote address, (2) remote - * subject and primary principal (3) request ID - */ -public class ReqContext { - private static final AtomicInteger uniqueId = new AtomicInteger(0); - private Subject _subject; - private InetAddress _remoteAddr; - private Integer _reqID; - private Map _storm_conf; - - /** - * Get a request context associated with current thread - * - * @return - */ - public static ReqContext context() { - return ctxt.get(); - } - - // each thread will have its own request context - private static final ThreadLocal ctxt = new ThreadLocal() { - @Override - protected ReqContext initialValue() { - return new ReqContext(AccessController.getContext()); - } - }; - - // private constructor - @VisibleForTesting - ReqContext(AccessControlContext acl_ctxt) { - _subject = Subject.getSubject(acl_ctxt); - _reqID = uniqueId.incrementAndGet(); - } - - /** - * client address - */ - public void setRemoteAddress(InetAddress addr) { - _remoteAddr = addr; - } - - public InetAddress remoteAddress() { - return _remoteAddr; - } - - /** - * Set remote subject explicitly - */ - public void setSubject(Subject subject) { - _subject = subject; - } - - /** - * Retrieve client subject associated with this request context - */ - public Subject subject() { - return _subject; - } - - /** - * The primary principal associated current subject - */ - public Principal principal() { - if (_subject == null) - return null; - Set princs = _subject.getPrincipals(); - if (princs.size() == 0) - return null; - return (Principal) (princs.toArray()[0]); - } - - /** - * request ID of this request - */ - public Integer requestID() { - return _reqID; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/security/auth/SaslTransportPlugin.java b/jstorm-client/src/main/java/backtype/storm/security/auth/SaslTransportPlugin.java deleted file mode 100644 index 31a4fef35..000000000 --- a/jstorm-client/src/main/java/backtype/storm/security/auth/SaslTransportPlugin.java +++ /dev/null @@ -1,150 +0,0 @@ -package backtype.storm.security.auth; - -import java.io.IOException; -import java.net.Socket; -import java.security.Principal; -import java.util.Map; - -import javax.security.auth.Subject; -import javax.security.auth.login.Configuration; -import javax.security.sasl.SaslServer; - -import org.apache.thrift7.TException; -import org.apache.thrift7.TProcessor; -import org.apache.thrift7.protocol.TBinaryProtocol; -import org.apache.thrift7.protocol.TProtocol; -import org.apache.thrift7.server.TServer; -import org.apache.thrift7.server.TThreadPoolServer; -import org.apache.thrift7.transport.TSaslServerTransport; -import org.apache.thrift7.transport.TServerSocket; -import org.apache.thrift7.transport.TSocket; -import org.apache.thrift7.transport.TTransport; -import org.apache.thrift7.transport.TTransportException; -import org.apache.thrift7.transport.TTransportFactory; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Base class for SASL authentication plugin. - */ -public abstract class SaslTransportPlugin implements ITransportPlugin { - protected Configuration login_conf; - private static final Logger LOG = LoggerFactory - .getLogger(SaslTransportPlugin.class); - - /** - * Invoked once immediately after construction - * - * @param conf - * Storm configuration - * @param login_conf - * login configuration - */ - public void prepare(Map storm_conf, Configuration login_conf) { - this.login_conf = login_conf; - } - - public TServer getServer(int port, TProcessor processor) - throws IOException, TTransportException { - TTransportFactory serverTransportFactory = getServerTransportFactory(); - - // define THsHaServer args - // original: THsHaServer + TNonblockingServerSocket - // option: TThreadPoolServer + TServerSocket - TServerSocket serverTransport = new TServerSocket(port); - TThreadPoolServer.Args server_args = new TThreadPoolServer.Args( - serverTransport).processor(new TUGIWrapProcessor(processor)) - .minWorkerThreads(64).maxWorkerThreads(64) - .protocolFactory(new TBinaryProtocol.Factory()); - if (serverTransportFactory != null) - server_args.transportFactory(serverTransportFactory); - - // construct THsHaServer - return new TThreadPoolServer(server_args); - } - - /** - * All subclass must implement this method - * - * @return - * @throws IOException - */ - protected abstract TTransportFactory getServerTransportFactory() - throws IOException; - - /** - * Processor that pulls the SaslServer object out of the transport, and - * assumes the remote user's UGI before calling through to the original - * processor. - * - * This is used on the server side to set the UGI for each specific call. - */ - private class TUGIWrapProcessor implements TProcessor { - final TProcessor wrapped; - - TUGIWrapProcessor(TProcessor wrapped) { - this.wrapped = wrapped; - } - - public boolean process(final TProtocol inProt, final TProtocol outProt) - throws TException { - // populating request context - ReqContext req_context = ReqContext.context(); - - TTransport trans = inProt.getTransport(); - // Sasl transport - TSaslServerTransport saslTrans = (TSaslServerTransport) trans; - - // remote address - TSocket tsocket = (TSocket) saslTrans.getUnderlyingTransport(); - Socket socket = tsocket.getSocket(); - req_context.setRemoteAddress(socket.getInetAddress()); - - // remote subject - SaslServer saslServer = saslTrans.getSaslServer(); - String authId = saslServer.getAuthorizationID(); - Subject remoteUser = new Subject(); - remoteUser.getPrincipals().add(new User(authId)); - req_context.setSubject(remoteUser); - - // invoke service handler - return wrapped.process(inProt, outProt); - } - } - - public static class User implements Principal { - private final String name; - - public User(String name) { - this.name = name; - } - - /** - * Get the full name of the user. - */ - public String getName() { - return name; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } else if (o == null || getClass() != o.getClass()) { - return false; - } else { - return (name.equals(((User) o).name)); - } - } - - @Override - public int hashCode() { - return name.hashCode(); - } - - @Override - public String toString() { - return name; - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/security/auth/SimpleTransportPlugin.java b/jstorm-client/src/main/java/backtype/storm/security/auth/SimpleTransportPlugin.java deleted file mode 100644 index e118f7d5c..000000000 --- a/jstorm-client/src/main/java/backtype/storm/security/auth/SimpleTransportPlugin.java +++ /dev/null @@ -1,119 +0,0 @@ -package backtype.storm.security.auth; - -import java.io.IOException; -import java.net.InetAddress; -import java.net.Socket; -import java.net.UnknownHostException; -import java.util.Map; - -import javax.security.auth.login.Configuration; - -import org.apache.thrift7.TException; -import org.apache.thrift7.TProcessor; -import org.apache.thrift7.protocol.TBinaryProtocol; -import org.apache.thrift7.protocol.TProtocol; -import org.apache.thrift7.server.THsHaServer; -import org.apache.thrift7.server.TServer; -import org.apache.thrift7.transport.TFramedTransport; -import org.apache.thrift7.transport.TMemoryInputTransport; -import org.apache.thrift7.transport.TNonblockingServerSocket; -import org.apache.thrift7.transport.TSocket; -import org.apache.thrift7.transport.TTransport; -import org.apache.thrift7.transport.TTransportException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Simple transport for Thrift plugin. - * - * This plugin is designed to be backward compatible with existing Storm code. - */ -public class SimpleTransportPlugin implements ITransportPlugin { - protected Configuration login_conf; - private static final Logger LOG = LoggerFactory - .getLogger(SimpleTransportPlugin.class); - - /** - * Invoked once immediately after construction - * - * @param conf - * Storm configuration - * @param login_conf - * login configuration - */ - public void prepare(Map storm_conf, Configuration login_conf) { - this.login_conf = login_conf; - } - - /** - * We will let Thrift to apply default transport factory - */ - public TServer getServer(int port, TProcessor processor) - throws IOException, TTransportException { - TNonblockingServerSocket serverTransport = new TNonblockingServerSocket( - port); - THsHaServer.Args server_args = new THsHaServer.Args(serverTransport) - .processor(new SimpleWrapProcessor(processor)) - .workerThreads(64) - .protocolFactory(new TBinaryProtocol.Factory()); - - // construct THsHaServer - return new THsHaServer(server_args); - } - - /** - * Connect to the specified server via framed transport - * - * @param transport - * The underlying Thrift transport. - */ - public TTransport connect(TTransport transport, String serverHost) - throws TTransportException { - // create a framed transport - TTransport conn = new TFramedTransport(transport); - - // connect - conn.open(); - LOG.debug("Simple client transport has been established"); - - return conn; - } - - /** - * Processor that populate simple transport info into ReqContext, and then - * invoke a service handler - */ - private class SimpleWrapProcessor implements TProcessor { - final TProcessor wrapped; - - SimpleWrapProcessor(TProcessor wrapped) { - this.wrapped = wrapped; - } - - public boolean process(final TProtocol inProt, final TProtocol outProt) - throws TException { - // populating request context - ReqContext req_context = ReqContext.context(); - - TTransport trans = inProt.getTransport(); - if (trans instanceof TMemoryInputTransport) { - try { - req_context.setRemoteAddress(InetAddress.getLocalHost()); - } catch (UnknownHostException e) { - throw new RuntimeException(e); - } - } else if (trans instanceof TSocket) { - TSocket tsocket = (TSocket) trans; - // remote address - Socket socket = tsocket.getSocket(); - req_context.setRemoteAddress(socket.getInetAddress()); - } - - // anonymous user - req_context.setSubject(null); - - // invoke service handler - return wrapped.process(inProt, outProt); - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/security/auth/ThriftClient.java b/jstorm-client/src/main/java/backtype/storm/security/auth/ThriftClient.java deleted file mode 100644 index 5a78a709a..000000000 --- a/jstorm-client/src/main/java/backtype/storm/security/auth/ThriftClient.java +++ /dev/null @@ -1,138 +0,0 @@ -package backtype.storm.security.auth; - -import java.io.IOException; -import java.security.InvalidParameterException; -import java.util.List; -import java.util.Map; - -import javax.security.auth.login.Configuration; - -import org.apache.curator.framework.CuratorFramework; -import org.apache.thrift7.protocol.TBinaryProtocol; -import org.apache.thrift7.protocol.TProtocol; -import org.apache.thrift7.transport.TSocket; -import org.apache.thrift7.transport.TTransport; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import backtype.storm.Config; -import backtype.storm.utils.Utils; - -public class ThriftClient { - private static final Logger LOG = LoggerFactory - .getLogger(ThriftClient.class); - private static final String MASTER_PATH = "/nimbus_master"; - private TTransport _transport; - protected TProtocol _protocol; - private String masterHost; - - private Map conf; - - public ThriftClient(Map storm_conf) throws Exception { - this(storm_conf, null); - } - - @SuppressWarnings("unchecked") - public ThriftClient(Map storm_conf, Integer timeout) throws Exception { - conf = storm_conf; - masterHost = getMaster(storm_conf, timeout); - connect(storm_conf, timeout); - } - - public static String getMaster(Map storm_conf, Integer timeout) throws Exception { - CuratorFramework zkobj = null; - String masterHost = null; - - try { - String root = String.valueOf(storm_conf - .get(Config.STORM_ZOOKEEPER_ROOT)); - String zkMasterDir = root + MASTER_PATH; - - - zkobj = Utils.newCurator(storm_conf, - (List) storm_conf.get(Config.STORM_ZOOKEEPER_SERVERS), - storm_conf.get(Config.STORM_ZOOKEEPER_PORT), zkMasterDir); - zkobj.start(); - if (zkobj.checkExists().forPath("/") == null) { - throw new RuntimeException("No alive nimbus "); - } - - masterHost = new String(zkobj.getData().forPath("/")); - - LOG.info("masterHost:" + masterHost); - return masterHost; - }finally { - if (zkobj != null) { - zkobj.close(); - zkobj = null; - } - } - } - - public TTransport transport() { - return _transport; - } - - protected void connect(Map storm_conf, Integer timeout) - throws Exception { - try { - String[] host_port = masterHost.split(":"); - if (host_port.length != 2) { - throw new InvalidParameterException("Host format error: " - + masterHost); - } - String host = host_port[0]; - int port = Integer.parseInt(host_port[1]); - LOG.info("Begin to connect " + host + ":" + port); - - // locate login configuration - Configuration login_conf = AuthUtils.GetConfiguration(storm_conf); - - // construct a transport plugin - ITransportPlugin transportPlugin = AuthUtils.GetTransportPlugin( - storm_conf, login_conf); - - // create a socket with server - if (host == null) { - throw new IllegalArgumentException("host is not set"); - } - if (port <= 0) { - throw new IllegalArgumentException("invalid port: " + port); - } - - TSocket socket = new TSocket(host, port); - if (timeout != null) { - socket.setTimeout(timeout); - } - final TTransport underlyingTransport = socket; - - // establish client-server transport via plugin - _transport = transportPlugin.connect(underlyingTransport, host); - } catch (IOException ex) { - throw new RuntimeException("Create transport error"); - } - _protocol = null; - if (_transport != null) - _protocol = new TBinaryProtocol(_transport); - } - - - - - public void close() { - if (_transport != null) - _transport.close(); - } - - public String getMasterHost() { - return masterHost; - } - - public Map getConf() { - return conf; - } - - protected void flush() { - - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/security/auth/ThriftServer.java b/jstorm-client/src/main/java/backtype/storm/security/auth/ThriftServer.java deleted file mode 100644 index d1b57590b..000000000 --- a/jstorm-client/src/main/java/backtype/storm/security/auth/ThriftServer.java +++ /dev/null @@ -1,70 +0,0 @@ -package backtype.storm.security.auth; - -import java.util.Map; - -import javax.security.auth.login.Configuration; - -import org.apache.thrift7.TProcessor; -import org.apache.thrift7.server.TServer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class ThriftServer { - private static final Logger LOG = LoggerFactory - .getLogger(ThriftServer.class); - private Map _storm_conf; // storm configuration - protected TProcessor _processor = null; - private int _port = 0; - private TServer _server = null; - private Configuration _login_conf; - - public ThriftServer(Map storm_conf, TProcessor processor, int port) { - try { - _storm_conf = storm_conf; - _processor = processor; - _port = port; - - // retrieve authentication configuration - _login_conf = AuthUtils.GetConfiguration(_storm_conf); - } catch (Exception x) { - LOG.error(x.getMessage(), x); - } - } - - public void stop() { - if (_server != null) - _server.stop(); - } - - /** - * Is ThriftServer listening to requests? - * - * @return - */ - public boolean isServing() { - if (_server == null) - return false; - return _server.isServing(); - } - - public void serve() { - try { - // locate our thrift transport plugin - ITransportPlugin transportPlugin = AuthUtils.GetTransportPlugin( - _storm_conf, _login_conf); - - // server - _server = transportPlugin.getServer(_port, _processor); - - // start accepting requests - _server.serve(); - } catch (Exception ex) { - LOG.error("ThriftServer is being stopped due to: " + ex, ex); - if (_server != null) - _server.stop(); - Runtime.getRuntime().halt(1); // shutdown server process since we - // could not handle Thrift requests - // any more - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/security/auth/authorizer/DenyAuthorizer.java b/jstorm-client/src/main/java/backtype/storm/security/auth/authorizer/DenyAuthorizer.java deleted file mode 100644 index 9414eaf10..000000000 --- a/jstorm-client/src/main/java/backtype/storm/security/auth/authorizer/DenyAuthorizer.java +++ /dev/null @@ -1,52 +0,0 @@ -package backtype.storm.security.auth.authorizer; - -import java.util.Map; - -import backtype.storm.Config; -import backtype.storm.security.auth.IAuthorizer; -import backtype.storm.security.auth.ReqContext; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * An authorization implementation that denies everything, for testing purposes - */ -public class DenyAuthorizer implements IAuthorizer { - private static final Logger LOG = LoggerFactory - .getLogger(DenyAuthorizer.class); - - /** - * Invoked once immediately after construction - * - * @param conf - * Storm configuration - */ - public void prepare(Map conf) { - } - - /** - * permit() method is invoked for each incoming Thrift request - * - * @param contrext - * request context - * @param operation - * operation name - * @param topology_storm - * configuration of targeted topology - * @return true if the request is authorized, false if reject - */ - public boolean permit(ReqContext context, String operation, - Map topology_conf) { - LOG.info("[req " - + context.requestID() - + "] Access " - + " from: " - + (context.remoteAddress() == null ? "null" : context - .remoteAddress().toString()) + " principal:" - + (context.principal() == null ? "null" : context.principal()) - + " op:" + operation + " topoology:" - + topology_conf.get(Config.TOPOLOGY_NAME)); - return false; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/security/auth/authorizer/NoopAuthorizer.java b/jstorm-client/src/main/java/backtype/storm/security/auth/authorizer/NoopAuthorizer.java deleted file mode 100644 index b61483542..000000000 --- a/jstorm-client/src/main/java/backtype/storm/security/auth/authorizer/NoopAuthorizer.java +++ /dev/null @@ -1,53 +0,0 @@ -package backtype.storm.security.auth.authorizer; - -import java.util.Map; - -import backtype.storm.Config; -import backtype.storm.security.auth.IAuthorizer; -import backtype.storm.security.auth.ReqContext; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * A no-op authorization implementation that illustrate info available for - * authorization decisions. - */ -public class NoopAuthorizer implements IAuthorizer { - private static final Logger LOG = LoggerFactory - .getLogger(NoopAuthorizer.class); - - /** - * Invoked once immediately after construction - * - * @param conf - * Storm configuration - */ - public void prepare(Map conf) { - } - - /** - * permit() method is invoked for each incoming Thrift request - * - * @param context - * request context includes info about - * @param operation - * operation name - * @param topology_storm - * configuration of targeted topology - * @return true if the request is authorized, false if reject - */ - public boolean permit(ReqContext context, String operation, - Map topology_conf) { - LOG.info("[req " - + context.requestID() - + "] Access " - + " from: " - + (context.remoteAddress() == null ? "null" : context - .remoteAddress().toString()) + " principal:" - + (context.principal() == null ? "null" : context.principal()) - + " op:" + operation + " topoology:" - + topology_conf.get(Config.TOPOLOGY_NAME)); - return true; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/security/auth/digest/ClientCallbackHandler.java b/jstorm-client/src/main/java/backtype/storm/security/auth/digest/ClientCallbackHandler.java deleted file mode 100644 index c15ff7d0a..000000000 --- a/jstorm-client/src/main/java/backtype/storm/security/auth/digest/ClientCallbackHandler.java +++ /dev/null @@ -1,101 +0,0 @@ -package backtype.storm.security.auth.digest; - -import java.io.IOException; -import javax.security.auth.callback.Callback; -import javax.security.auth.callback.CallbackHandler; -import javax.security.auth.callback.NameCallback; -import javax.security.auth.callback.PasswordCallback; -import javax.security.auth.callback.UnsupportedCallbackException; -import javax.security.sasl.AuthorizeCallback; -import javax.security.sasl.RealmCallback; -import javax.security.auth.login.AppConfigurationEntry; -import javax.security.auth.login.Configuration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import backtype.storm.security.auth.AuthUtils; - -/** - * client side callback handler. - */ -public class ClientCallbackHandler implements CallbackHandler { - private static final String USERNAME = "username"; - private static final String PASSWORD = "password"; - private static final Logger LOG = LoggerFactory - .getLogger(ClientCallbackHandler.class); - private String _username = null; - private String _password = null; - - /** - * Constructor based on a JAAS configuration - * - * For digest, you should have a pair of user name and password defined. - * - * @param configuration - * @throws IOException - */ - public ClientCallbackHandler(Configuration configuration) - throws IOException { - if (configuration == null) - return; - AppConfigurationEntry configurationEntries[] = configuration - .getAppConfigurationEntry(AuthUtils.LOGIN_CONTEXT_CLIENT); - if (configurationEntries == null) { - String errorMessage = "Could not find a '" - + AuthUtils.LOGIN_CONTEXT_CLIENT - + "' entry in this configuration: Client cannot start."; - throw new IOException(errorMessage); - } - - _password = ""; - for (AppConfigurationEntry entry : configurationEntries) { - if (entry.getOptions().get(USERNAME) != null) { - _username = (String) entry.getOptions().get(USERNAME); - } - if (entry.getOptions().get(PASSWORD) != null) { - _password = (String) entry.getOptions().get(PASSWORD); - } - } - } - - /** - * This method is invoked by SASL for authentication challenges - * - * @param callbacks - * a collection of challenge callbacks - */ - public void handle(Callback[] callbacks) throws IOException, - UnsupportedCallbackException { - for (Callback c : callbacks) { - if (c instanceof NameCallback) { - LOG.debug("name callback"); - NameCallback nc = (NameCallback) c; - nc.setName(_username); - } else if (c instanceof PasswordCallback) { - LOG.debug("password callback"); - PasswordCallback pc = (PasswordCallback) c; - if (_password != null) { - pc.setPassword(_password.toCharArray()); - } - } else if (c instanceof AuthorizeCallback) { - LOG.debug("authorization callback"); - AuthorizeCallback ac = (AuthorizeCallback) c; - String authid = ac.getAuthenticationID(); - String authzid = ac.getAuthorizationID(); - if (authid.equals(authzid)) { - ac.setAuthorized(true); - } else { - ac.setAuthorized(false); - } - if (ac.isAuthorized()) { - ac.setAuthorizedID(authzid); - } - } else if (c instanceof RealmCallback) { - RealmCallback rc = (RealmCallback) c; - ((RealmCallback) c).setText(rc.getDefaultText()); - } else { - throw new UnsupportedCallbackException(c); - } - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java b/jstorm-client/src/main/java/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java deleted file mode 100644 index 05f19c5b2..000000000 --- a/jstorm-client/src/main/java/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java +++ /dev/null @@ -1,52 +0,0 @@ -package backtype.storm.security.auth.digest; - -import java.io.IOException; - -import javax.security.auth.callback.CallbackHandler; - -import org.apache.thrift7.transport.TSaslClientTransport; -import org.apache.thrift7.transport.TSaslServerTransport; -import org.apache.thrift7.transport.TTransport; -import org.apache.thrift7.transport.TTransportException; -import org.apache.thrift7.transport.TTransportFactory; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import backtype.storm.security.auth.AuthUtils; -import backtype.storm.security.auth.SaslTransportPlugin; - -public class DigestSaslTransportPlugin extends SaslTransportPlugin { - public static final String DIGEST = "DIGEST-MD5"; - private static final Logger LOG = LoggerFactory - .getLogger(DigestSaslTransportPlugin.class); - - protected TTransportFactory getServerTransportFactory() throws IOException { - // create an authentication callback handler - CallbackHandler serer_callback_handler = new ServerCallbackHandler( - login_conf); - - // create a transport factory that will invoke our auth callback for - // digest - TSaslServerTransport.Factory factory = new TSaslServerTransport.Factory(); - factory.addServerDefinition(DIGEST, AuthUtils.SERVICE, "localhost", - null, serer_callback_handler); - - LOG.info("SASL DIGEST-MD5 transport factory will be used"); - return factory; - } - - public TTransport connect(TTransport transport, String serverHost) - throws TTransportException, IOException { - ClientCallbackHandler client_callback_handler = new ClientCallbackHandler( - login_conf); - TSaslClientTransport wrapper_transport = new TSaslClientTransport( - DIGEST, null, AuthUtils.SERVICE, serverHost, null, - client_callback_handler, transport); - - wrapper_transport.open(); - LOG.debug("SASL DIGEST-MD5 client transport has been established"); - - return wrapper_transport; - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/security/auth/digest/ServerCallbackHandler.java b/jstorm-client/src/main/java/backtype/storm/security/auth/digest/ServerCallbackHandler.java deleted file mode 100644 index 09662e557..000000000 --- a/jstorm-client/src/main/java/backtype/storm/security/auth/digest/ServerCallbackHandler.java +++ /dev/null @@ -1,109 +0,0 @@ -package backtype.storm.security.auth.digest; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import javax.security.auth.callback.Callback; -import javax.security.auth.callback.CallbackHandler; -import javax.security.auth.callback.NameCallback; -import javax.security.auth.callback.PasswordCallback; -import javax.security.auth.callback.UnsupportedCallbackException; -import javax.security.auth.login.AppConfigurationEntry; -import javax.security.auth.login.Configuration; -import javax.security.sasl.AuthorizeCallback; -import javax.security.sasl.RealmCallback; - -import backtype.storm.security.auth.AuthUtils; - -/** - * SASL server side collback handler - */ -public class ServerCallbackHandler implements CallbackHandler { - private static final String USER_PREFIX = "user_"; - private static final Logger LOG = LoggerFactory - .getLogger(ServerCallbackHandler.class); - private static final String SYSPROP_SUPER_PASSWORD = "storm.SASLAuthenticationProvider.superPassword"; - - private String userName; - private final Map credentials = new HashMap(); - - public ServerCallbackHandler(Configuration configuration) - throws IOException { - if (configuration == null) - return; - - AppConfigurationEntry configurationEntries[] = configuration - .getAppConfigurationEntry(AuthUtils.LOGIN_CONTEXT_SERVER); - if (configurationEntries == null) { - String errorMessage = "Could not find a '" - + AuthUtils.LOGIN_CONTEXT_SERVER - + "' entry in this configuration: Server cannot start."; - throw new IOException(errorMessage); - } - credentials.clear(); - for (AppConfigurationEntry entry : configurationEntries) { - Map options = entry.getOptions(); - // Populate DIGEST-MD5 user -> password map with JAAS configuration - // entries from the "Server" section. - // Usernames are distinguished from other options by prefixing the - // username with a "user_" prefix. - for (Map.Entry pair : options.entrySet()) { - String key = pair.getKey(); - if (key.startsWith(USER_PREFIX)) { - String userName = key.substring(USER_PREFIX.length()); - credentials.put(userName, (String) pair.getValue()); - } - } - } - } - - public void handle(Callback[] callbacks) - throws UnsupportedCallbackException { - for (Callback callback : callbacks) { - if (callback instanceof NameCallback) { - handleNameCallback((NameCallback) callback); - } else if (callback instanceof PasswordCallback) { - handlePasswordCallback((PasswordCallback) callback); - } else if (callback instanceof RealmCallback) { - handleRealmCallback((RealmCallback) callback); - } else if (callback instanceof AuthorizeCallback) { - handleAuthorizeCallback((AuthorizeCallback) callback); - } - } - } - - private void handleNameCallback(NameCallback nc) { - LOG.debug("handleNameCallback"); - userName = nc.getDefaultName(); - nc.setName(nc.getDefaultName()); - } - - private void handlePasswordCallback(PasswordCallback pc) { - LOG.debug("handlePasswordCallback"); - if ("super".equals(this.userName) - && System.getProperty(SYSPROP_SUPER_PASSWORD) != null) { - // superuser: use Java system property for password, if available. - pc.setPassword(System.getProperty(SYSPROP_SUPER_PASSWORD) - .toCharArray()); - } else if (credentials.containsKey(userName)) { - pc.setPassword(credentials.get(userName).toCharArray()); - } else { - LOG.warn("No password found for user: " + userName); - } - } - - private void handleRealmCallback(RealmCallback rc) { - LOG.debug("handleRealmCallback: " + rc.getDefaultText()); - rc.setText(rc.getDefaultText()); - } - - private void handleAuthorizeCallback(AuthorizeCallback ac) { - String authenticationID = ac.getAuthenticationID(); - LOG.debug("Successfully authenticated client: authenticationID=" - + authenticationID); - ac.setAuthorizedID(authenticationID); - ac.setAuthorized(true); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/security/serialization/BlowfishTupleSerializer.java b/jstorm-client/src/main/java/backtype/storm/security/serialization/BlowfishTupleSerializer.java deleted file mode 100644 index 6ded0d91c..000000000 --- a/jstorm-client/src/main/java/backtype/storm/security/serialization/BlowfishTupleSerializer.java +++ /dev/null @@ -1,80 +0,0 @@ -package backtype.storm.security.serialization; - -import java.util.Map; -import org.apache.commons.codec.binary.Hex; -import org.apache.log4j.Logger; - -import javax.crypto.KeyGenerator; -import javax.crypto.SecretKey; - -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.io.Input; -import com.esotericsoftware.kryo.io.Output; -import com.esotericsoftware.kryo.Serializer; -import com.esotericsoftware.kryo.serializers.BlowfishSerializer; - -import backtype.storm.serialization.types.ListDelegateSerializer; -import backtype.storm.utils.ListDelegate; -import backtype.storm.Config; - -/** - * Apply Blowfish encrption for tuple communication to bolts - */ -public class BlowfishTupleSerializer extends Serializer { - /** - * The secret key (if any) for data encryption by blowfish payload - * serialization factory (BlowfishSerializationFactory). You should use in - * via - * "storm -c topology.tuple.serializer.blowfish.key=YOURKEY -c topology.tuple.serializer=backtype.storm.security.serialization.BlowfishTupleSerializer jar ..." - * . - */ - public static String SECRET_KEY = "topology.tuple.serializer.blowfish.key"; - private static final Logger LOG = Logger - .getLogger(BlowfishTupleSerializer.class); - private BlowfishSerializer _serializer; - - public BlowfishTupleSerializer(Kryo kryo, Map storm_conf) { - String encryption_key = null; - try { - encryption_key = (String) storm_conf.get(SECRET_KEY); - LOG.debug("Blowfish serializer being constructed ..."); - if (encryption_key == null) { - throw new RuntimeException( - "Blowfish encryption key not specified"); - } - byte[] bytes = Hex.decodeHex(encryption_key.toCharArray()); - _serializer = new BlowfishSerializer(new ListDelegateSerializer(), - bytes); - } catch (org.apache.commons.codec.DecoderException ex) { - throw new RuntimeException("Blowfish encryption key invalid", ex); - } - } - - @Override - public void write(Kryo kryo, Output output, ListDelegate object) { - _serializer.write(kryo, output, object); - } - - @Override - public ListDelegate read(Kryo kryo, Input input, Class type) { - return (ListDelegate) _serializer.read(kryo, input, type); - } - - /** - * Produce a blowfish key to be used in "Storm jar" command - */ - public static void main(String[] args) { - try { - KeyGenerator kgen = KeyGenerator.getInstance("Blowfish"); - SecretKey skey = kgen.generateKey(); - byte[] raw = skey.getEncoded(); - String keyString = new String(Hex.encodeHex(raw)); - System.out.println("storm -c " + SECRET_KEY + "=" + keyString - + " -c " + Config.TOPOLOGY_TUPLE_SERIALIZER + "=" - + BlowfishTupleSerializer.class.getName() + " ..."); - } catch (Exception ex) { - LOG.error(ex.getMessage()); - ex.printStackTrace(); - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/serialization/DefaultKryoFactory.java b/jstorm-client/src/main/java/backtype/storm/serialization/DefaultKryoFactory.java deleted file mode 100644 index c9afec43a..000000000 --- a/jstorm-client/src/main/java/backtype/storm/serialization/DefaultKryoFactory.java +++ /dev/null @@ -1,47 +0,0 @@ -package backtype.storm.serialization; - -import backtype.storm.Config; -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.Serializer; -import java.util.Map; - -public class DefaultKryoFactory implements IKryoFactory { - - public static class KryoSerializableDefault extends Kryo { - boolean _override = false; - - public void overrideDefault(boolean value) { - _override = value; - } - - @Override - public Serializer getDefaultSerializer(Class type) { - if (_override) { - return new SerializableSerializer(); - } else { - return super.getDefaultSerializer(type); - } - } - } - - @Override - public Kryo getKryo(Map conf) { - KryoSerializableDefault k = new KryoSerializableDefault(); - k.setRegistrationRequired(!((Boolean) conf - .get(Config.TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION))); - k.setReferences(false); - return k; - } - - @Override - public void preRegister(Kryo k, Map conf) { - } - - public void postRegister(Kryo k, Map conf) { - ((KryoSerializableDefault) k).overrideDefault(true); - } - - @Override - public void postDecorate(Kryo k, Map conf) { - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/serialization/IKryoDecorator.java b/jstorm-client/src/main/java/backtype/storm/serialization/IKryoDecorator.java deleted file mode 100644 index f2f07ad0c..000000000 --- a/jstorm-client/src/main/java/backtype/storm/serialization/IKryoDecorator.java +++ /dev/null @@ -1,7 +0,0 @@ -package backtype.storm.serialization; - -import com.esotericsoftware.kryo.Kryo; - -public interface IKryoDecorator { - void decorate(Kryo k); -} diff --git a/jstorm-client/src/main/java/backtype/storm/serialization/IKryoFactory.java b/jstorm-client/src/main/java/backtype/storm/serialization/IKryoFactory.java deleted file mode 100644 index 5cb9ab322..000000000 --- a/jstorm-client/src/main/java/backtype/storm/serialization/IKryoFactory.java +++ /dev/null @@ -1,25 +0,0 @@ -package backtype.storm.serialization; - -import com.esotericsoftware.kryo.Kryo; -import java.util.Map; - -/** - * An interface that controls the Kryo instance used by Storm for serialization. - * The lifecycle is: - * - * 1. The Kryo instance is constructed using getKryo 2. Storm registers the - * default classes (e.g. arrays, lists, maps, etc.) 3. Storm calls preRegister - * hook 4. Storm registers all user-defined registrations through - * topology.kryo.register 5. Storm calls postRegister hook 6. Storm calls all - * user-defined decorators through topology.kryo.decorators 7. Storm calls - * postDecorate hook - */ -public interface IKryoFactory { - Kryo getKryo(Map conf); - - void preRegister(Kryo k, Map conf); - - void postRegister(Kryo k, Map conf); - - void postDecorate(Kryo k, Map conf); -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/serialization/ITupleDeserializer.java b/jstorm-client/src/main/java/backtype/storm/serialization/ITupleDeserializer.java deleted file mode 100644 index 1ae2c9f66..000000000 --- a/jstorm-client/src/main/java/backtype/storm/serialization/ITupleDeserializer.java +++ /dev/null @@ -1,9 +0,0 @@ -package backtype.storm.serialization; - -import backtype.storm.tuple.Tuple; - -import java.io.IOException; - -public interface ITupleDeserializer { - Tuple deserialize(byte[] ser); -} diff --git a/jstorm-client/src/main/java/backtype/storm/serialization/ITupleSerializer.java b/jstorm-client/src/main/java/backtype/storm/serialization/ITupleSerializer.java deleted file mode 100644 index 9ad7cae2b..000000000 --- a/jstorm-client/src/main/java/backtype/storm/serialization/ITupleSerializer.java +++ /dev/null @@ -1,15 +0,0 @@ -package backtype.storm.serialization; - -import backtype.storm.tuple.Tuple; - -public interface ITupleSerializer { - /** - * serialize targetTask before the tuple, it should be stored in 4 bytes - * - * @param targetTask - * @param tuple - * @return - */ - byte[] serialize(Tuple tuple); - // long crc32(Tuple tuple); -} diff --git a/jstorm-client/src/main/java/backtype/storm/serialization/KryoTupleDeserializer.java b/jstorm-client/src/main/java/backtype/storm/serialization/KryoTupleDeserializer.java deleted file mode 100644 index 12d919323..000000000 --- a/jstorm-client/src/main/java/backtype/storm/serialization/KryoTupleDeserializer.java +++ /dev/null @@ -1,90 +0,0 @@ -package backtype.storm.serialization; - -import backtype.storm.task.GeneralTopologyContext; -import backtype.storm.tuple.MessageId; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.TupleImplExt; - -import com.esotericsoftware.kryo.io.Input; - -import java.io.IOException; -import java.net.URLClassLoader; -import java.util.List; -import java.util.Map; - -import org.apache.log4j.Logger; - -public class KryoTupleDeserializer implements ITupleDeserializer { - private static final Logger LOG = Logger.getLogger(KryoTupleDeserializer.class); - - public static final boolean USE_RAW_PACKET = true; - - GeneralTopologyContext _context; - KryoValuesDeserializer _kryo; - SerializationFactory.IdDictionary _ids; - Input _kryoInput; - - public KryoTupleDeserializer(final Map conf, - final GeneralTopologyContext context) { - _kryo = new KryoValuesDeserializer(conf); - _context = context; - _ids = new SerializationFactory.IdDictionary(context.getRawTopology()); - _kryoInput = new Input(1); - } - - public Tuple deserialize(byte[] ser) { - - int targetTaskId = 0; - int taskId = 0; - int streamId = 0; - String componentName = null; - String streamName = null; - MessageId id = null; - - try { - - _kryoInput.setBuffer(ser); - - targetTaskId = _kryoInput.readInt(); - taskId = _kryoInput.readInt(true); - streamId = _kryoInput.readInt(true); - componentName = _context.getComponentId(taskId); - streamName = _ids.getStreamName(componentName, streamId); - id = MessageId.deserialize(_kryoInput); - List values = _kryo.deserializeFrom(_kryoInput); - TupleImplExt tuple = new TupleImplExt(_context, values, taskId, - streamName, id); - tuple.setTargetTaskId(targetTaskId); - return tuple; - } catch (Throwable e) { - StringBuilder sb = new StringBuilder(); - - sb.append("Deserialize error:"); - sb.append("targetTaskId:").append(targetTaskId); - sb.append(",taskId:").append(taskId); - sb.append(",streamId:").append(streamId); - sb.append(",componentName:").append(componentName); - sb.append(",streamName:").append(streamName); - sb.append(",MessageId").append(id); - - LOG.info(sb.toString(), e ); - throw new RuntimeException(e); - } - } - - /** - * just get target taskId - * - * @param ser - * @return - */ - public static int deserializeTaskId(byte[] ser) { - Input _kryoInput = new Input(1); - - _kryoInput.setBuffer(ser); - - int targetTaskId = _kryoInput.readInt(); - - return targetTaskId; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/serialization/KryoTupleSerializer.java b/jstorm-client/src/main/java/backtype/storm/serialization/KryoTupleSerializer.java deleted file mode 100644 index e04b145ba..000000000 --- a/jstorm-client/src/main/java/backtype/storm/serialization/KryoTupleSerializer.java +++ /dev/null @@ -1,67 +0,0 @@ -package backtype.storm.serialization; - -import backtype.storm.task.GeneralTopologyContext; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.TupleExt; - -import com.esotericsoftware.kryo.io.Output; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Map; - -public class KryoTupleSerializer implements ITupleSerializer { - KryoValuesSerializer _kryo; - SerializationFactory.IdDictionary _ids; - Output _kryoOut; - - public KryoTupleSerializer(final Map conf, - final GeneralTopologyContext context) { - _kryo = new KryoValuesSerializer(conf); - _kryoOut = new Output(2000, 2000000000); - _ids = new SerializationFactory.IdDictionary(context.getRawTopology()); - } - - /** - * @@@ in the furture, it will skill serialize 'targetTask' through check - * some flag - * @see backtype.storm.serialization.ITupleSerializer#serialize(int, - * backtype.storm.tuple.Tuple) - */ - public byte[] serialize(Tuple tuple) { - try { - - _kryoOut.clear(); - if (tuple instanceof TupleExt) { - _kryoOut.writeInt(((TupleExt) tuple).getTargetTaskId()); - } - - _kryoOut.writeInt(tuple.getSourceTask(), true); - _kryoOut.writeInt( - _ids.getStreamId(tuple.getSourceComponent(), - tuple.getSourceStreamId()), true); - tuple.getMessageId().serialize(_kryoOut); - _kryo.serializeInto(tuple.getValues(), _kryoOut); - return _kryoOut.toBytes(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - public static byte[] serialize(int targetTask) { - ByteBuffer buff = ByteBuffer.allocate((Integer.SIZE / 8)); - buff.putInt(targetTask); - byte[] rtn = buff.array(); - return rtn; - } - - // public long crc32(Tuple tuple) { - // try { - // CRC32OutputStream hasher = new CRC32OutputStream(); - // _kryo.serializeInto(tuple.getValues(), hasher); - // return hasher.getValue(); - // } catch (IOException e) { - // throw new RuntimeException(e); - // } - // } -} diff --git a/jstorm-client/src/main/java/backtype/storm/serialization/KryoValuesDeserializer.java b/jstorm-client/src/main/java/backtype/storm/serialization/KryoValuesDeserializer.java deleted file mode 100644 index c1f3a8064..000000000 --- a/jstorm-client/src/main/java/backtype/storm/serialization/KryoValuesDeserializer.java +++ /dev/null @@ -1,39 +0,0 @@ -package backtype.storm.serialization; - -import java.io.IOException; -import java.net.URLClassLoader; -import java.util.List; -import java.util.Map; - -import backtype.storm.utils.ListDelegate; - -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.io.Input; - -public class KryoValuesDeserializer { - - Kryo _kryo; - Input _kryoInput; - - public KryoValuesDeserializer(Map conf) { - this._kryo = SerializationFactory.getKryo(conf); - this._kryoInput = new Input(1); - } - - public List deserializeFrom(Input input) { - ListDelegate delegate = (ListDelegate) _kryo.readObject(input, - ListDelegate.class); - return delegate.getDelegate(); - } - - public List deserialize(byte[] ser) throws IOException { - _kryoInput.setBuffer(ser); - return deserializeFrom(_kryoInput); - } - - public Object deserializeObject(byte[] ser) throws IOException { - _kryoInput.setBuffer(ser); - return _kryo.readClassAndObject(_kryoInput); - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/serialization/KryoValuesSerializer.java b/jstorm-client/src/main/java/backtype/storm/serialization/KryoValuesSerializer.java deleted file mode 100644 index 6072282fd..000000000 --- a/jstorm-client/src/main/java/backtype/storm/serialization/KryoValuesSerializer.java +++ /dev/null @@ -1,45 +0,0 @@ -package backtype.storm.serialization; - -import backtype.storm.utils.ListDelegate; -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.io.Output; -import java.io.IOException; -import java.util.List; -import java.util.Map; - -public class KryoValuesSerializer { - Kryo _kryo; - ListDelegate _delegate; - Output _kryoOut; - - public KryoValuesSerializer(Map conf) { - _kryo = SerializationFactory.getKryo(conf); - _delegate = new ListDelegate(); - _kryoOut = new Output(2000, 2000000000); - } - - public void serializeInto(List values, Output out) - throws IOException { - // this ensures that list of values is always written the same way, - // regardless - // of whether it's a java collection or one of clojure's persistent - // collections - // (which have different serializers) - // Doing this lets us deserialize as ArrayList and avoid writing the - // class here - _delegate.setDelegate(values); - _kryo.writeObject(out, _delegate); - } - - public byte[] serialize(List values) throws IOException { - _kryoOut.clear(); - serializeInto(values, _kryoOut); - return _kryoOut.toBytes(); - } - - public byte[] serializeObject(Object obj) { - _kryoOut.clear(); - _kryo.writeClassAndObject(_kryoOut, obj); - return _kryoOut.toBytes(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/serialization/SerializableSerializer.java b/jstorm-client/src/main/java/backtype/storm/serialization/SerializableSerializer.java deleted file mode 100644 index 4fcaf02ba..000000000 --- a/jstorm-client/src/main/java/backtype/storm/serialization/SerializableSerializer.java +++ /dev/null @@ -1,46 +0,0 @@ -package backtype.storm.serialization; - -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.Serializer; -import com.esotericsoftware.kryo.io.Input; -import com.esotericsoftware.kryo.io.Output; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; - -import org.apache.commons.io.input.ClassLoaderObjectInputStream; - -public class SerializableSerializer extends Serializer { - - @Override - public void write(Kryo kryo, Output output, Object object) { - ByteArrayOutputStream bos = new ByteArrayOutputStream(); - try { - ObjectOutputStream oos = new ObjectOutputStream(bos); - oos.writeObject(object); - oos.flush(); - } catch (IOException e) { - throw new RuntimeException(e); - } - byte[] ser = bos.toByteArray(); - output.writeInt(ser.length); - output.writeBytes(ser); - } - - @Override - public Object read(Kryo kryo, Input input, Class c) { - int len = input.readInt(); - byte[] ser = new byte[len]; - input.readBytes(ser); - ByteArrayInputStream bis = new ByteArrayInputStream(ser); - try { - ClassLoaderObjectInputStream ois = new ClassLoaderObjectInputStream( - kryo.getClassLoader(), bis); - return ois.readObject(); - } catch (Exception e) { - throw new RuntimeException(e); - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/serialization/SerializationFactory.java b/jstorm-client/src/main/java/backtype/storm/serialization/SerializationFactory.java deleted file mode 100644 index 88f7803a8..000000000 --- a/jstorm-client/src/main/java/backtype/storm/serialization/SerializationFactory.java +++ /dev/null @@ -1,242 +0,0 @@ -package backtype.storm.serialization; - -import backtype.storm.Config; -import backtype.storm.generated.ComponentCommon; -import backtype.storm.generated.StormTopology; -import backtype.storm.serialization.types.ArrayListSerializer; -import backtype.storm.serialization.types.ListDelegateSerializer; -import backtype.storm.serialization.types.HashMapSerializer; -import backtype.storm.serialization.types.HashSetSerializer; -import backtype.storm.transactional.TransactionAttempt; -import backtype.storm.tuple.Values; -import backtype.storm.utils.ListDelegate; -import backtype.storm.utils.Utils; -import backtype.storm.utils.WorkerClassLoader; -import carbonite.JavaBridge; -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.Serializer; -import com.esotericsoftware.kryo.serializers.DefaultSerializers.BigIntegerSerializer; -import java.math.BigInteger; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class SerializationFactory { - public static final Logger LOG = LoggerFactory - .getLogger(SerializationFactory.class); - - public static Kryo getKryo(Map conf) { - IKryoFactory kryoFactory = (IKryoFactory) Utils - .newInstance((String) conf.get(Config.TOPOLOGY_KRYO_FACTORY)); - Kryo k = kryoFactory.getKryo(conf); - if (WorkerClassLoader.getInstance() != null) - k.setClassLoader(WorkerClassLoader.getInstance()); - k.register(byte[].class); - - /* tuple payload serializer is specified via configuration */ - String payloadSerializerName = (String) conf - .get(Config.TOPOLOGY_TUPLE_SERIALIZER); - try { - Class serializerClass = Class.forName( - payloadSerializerName, true, k.getClassLoader()); - Serializer serializer = resolveSerializerInstance(k, - ListDelegate.class, serializerClass, conf); - k.register(ListDelegate.class, serializer); - } catch (ClassNotFoundException ex) { - throw new RuntimeException(ex); - } - - k.register(ArrayList.class, new ArrayListSerializer()); - k.register(HashMap.class, new HashMapSerializer()); - k.register(HashSet.class, new HashSetSerializer()); - k.register(BigInteger.class, new BigIntegerSerializer()); - k.register(TransactionAttempt.class); - k.register(Values.class); - k.register(backtype.storm.metric.api.IMetricsConsumer.DataPoint.class); - k.register(backtype.storm.metric.api.IMetricsConsumer.TaskInfo.class); - try { - JavaBridge.registerPrimitives(k); - JavaBridge.registerCollections(k); - } catch (Exception e) { - throw new RuntimeException(e); - } - - Map registrations = normalizeKryoRegister(conf); - - kryoFactory.preRegister(k, conf); - - boolean skipMissing = (Boolean) conf - .get(Config.TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS); - for (String klassName : registrations.keySet()) { - String serializerClassName = registrations.get(klassName); - try { - Class klass = Class.forName( - klassName, true, k.getClassLoader()); - - Class serializerClass = null; - if (serializerClassName != null) - serializerClass = Class.forName( - serializerClassName, true, k.getClassLoader()); - if (serializerClass == null) { - k.register(klass); - } else { - k.register( - klass, - resolveSerializerInstance(k, klass, - serializerClass, conf)); - } - } catch (ClassNotFoundException e) { - if (skipMissing) { - LOG.info("Could not find serialization or class for " - + serializerClassName - + ". Skipping registration..."); - } else { - throw new RuntimeException(e); - } - } - } - - kryoFactory.postRegister(k, conf); - - if (conf.get(Config.TOPOLOGY_KRYO_DECORATORS) != null) { - for (String klassName : (List) conf - .get(Config.TOPOLOGY_KRYO_DECORATORS)) { - try { - Class klass = Class.forName( - klassName, true, k.getClassLoader()); - IKryoDecorator decorator = (IKryoDecorator) klass - .newInstance(); - decorator.decorate(k); - } catch (ClassNotFoundException e) { - if (skipMissing) { - LOG.info("Could not find kryo decorator named " - + klassName + ". Skipping registration..."); - } else { - throw new RuntimeException(e); - } - } catch (InstantiationException e) { - throw new RuntimeException(e); - } catch (IllegalAccessException e) { - throw new RuntimeException(e); - } - } - } - - kryoFactory.postDecorate(k, conf); - - return k; - } - - public static class IdDictionary { - Map> streamNametoId = new HashMap>(); - Map> streamIdToName = new HashMap>(); - - public IdDictionary(StormTopology topology) { - List componentNames = new ArrayList(topology - .get_spouts().keySet()); - componentNames.addAll(topology.get_bolts().keySet()); - componentNames.addAll(topology.get_state_spouts().keySet()); - - for (String name : componentNames) { - ComponentCommon common = Utils.getComponentCommon(topology, - name); - List streams = new ArrayList(common - .get_streams().keySet()); - streamNametoId.put(name, idify(streams)); - streamIdToName.put(name, - Utils.reverseMap(streamNametoId.get(name))); - } - } - - public int getStreamId(String component, String stream) { - return streamNametoId.get(component).get(stream); - } - - public String getStreamName(String component, int stream) { - return streamIdToName.get(component).get(stream); - } - - private static Map idify(List names) { - Collections.sort(names); - Map ret = new HashMap(); - int i = 1; - for (String name : names) { - ret.put(name, i); - i++; - } - return ret; - } - } - - private static Serializer resolveSerializerInstance(Kryo k, - Class superClass, Class serializerClass, - Map conf) { - try { - try { - return serializerClass.getConstructor(Kryo.class, Class.class, - Map.class).newInstance(k, superClass, conf); - } catch (Exception ex1) { - try { - return serializerClass.getConstructor(Kryo.class, - Class.class).newInstance(k, superClass); - } catch (Exception ex2) { - try { - return serializerClass.getConstructor(Kryo.class, - Map.class).newInstance(k, conf); - } catch (Exception ex3) { - try { - return serializerClass.getConstructor(Kryo.class) - .newInstance(k); - } catch (Exception ex4) { - try { - return serializerClass.getConstructor( - Class.class, Map.class).newInstance( - superClass, conf); - } catch (Exception ex5) { - try { - return serializerClass.getConstructor( - Class.class) - .newInstance(superClass); - } catch (Exception ex6) { - return serializerClass.newInstance(); - } - } - } - } - } - } - } catch (Exception ex) { - throw new IllegalArgumentException("Unable to create serializer \"" - + serializerClass.getName() + "\" for class: " - + superClass.getName(), ex); - } - } - - private static Map normalizeKryoRegister(Map conf) { - // TODO: de-duplicate this logic with the code in nimbus - Object res = conf.get(Config.TOPOLOGY_KRYO_REGISTER); - if (res == null) - return new TreeMap(); - Map ret = new HashMap(); - if (res instanceof Map) { - ret = (Map) res; - } else { - for (Object o : (List) res) { - if (o instanceof Map) { - ret.putAll((Map) o); - } else { - ret.put((String) o, null); - } - } - } - - // ensure always same order for registrations with TreeMap - return new TreeMap(ret); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/serialization/types/ArrayListSerializer.java b/jstorm-client/src/main/java/backtype/storm/serialization/types/ArrayListSerializer.java deleted file mode 100644 index e403a9521..000000000 --- a/jstorm-client/src/main/java/backtype/storm/serialization/types/ArrayListSerializer.java +++ /dev/null @@ -1,14 +0,0 @@ -package backtype.storm.serialization.types; - -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.io.Input; -import com.esotericsoftware.kryo.serializers.CollectionSerializer; -import java.util.ArrayList; -import java.util.Collection; - -public class ArrayListSerializer extends CollectionSerializer { - @Override - public Collection create(Kryo kryo, Input input, Class type) { - return new ArrayList(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/serialization/types/HashMapSerializer.java b/jstorm-client/src/main/java/backtype/storm/serialization/types/HashMapSerializer.java deleted file mode 100644 index c1f74562c..000000000 --- a/jstorm-client/src/main/java/backtype/storm/serialization/types/HashMapSerializer.java +++ /dev/null @@ -1,14 +0,0 @@ -package backtype.storm.serialization.types; - -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.io.Input; -import com.esotericsoftware.kryo.serializers.MapSerializer; -import java.util.HashMap; -import java.util.Map; - -public class HashMapSerializer extends MapSerializer { - @Override - public Map create(Kryo kryo, Input input, Class type) { - return new HashMap(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/serialization/types/HashSetSerializer.java b/jstorm-client/src/main/java/backtype/storm/serialization/types/HashSetSerializer.java deleted file mode 100644 index b28bbd6a2..000000000 --- a/jstorm-client/src/main/java/backtype/storm/serialization/types/HashSetSerializer.java +++ /dev/null @@ -1,14 +0,0 @@ -package backtype.storm.serialization.types; - -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.io.Input; -import com.esotericsoftware.kryo.serializers.CollectionSerializer; -import java.util.Collection; -import java.util.HashSet; - -public class HashSetSerializer extends CollectionSerializer { - @Override - public Collection create(Kryo kryo, Input input, Class type) { - return new HashSet(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/serialization/types/ListDelegateSerializer.java b/jstorm-client/src/main/java/backtype/storm/serialization/types/ListDelegateSerializer.java deleted file mode 100644 index 67242a2b8..000000000 --- a/jstorm-client/src/main/java/backtype/storm/serialization/types/ListDelegateSerializer.java +++ /dev/null @@ -1,14 +0,0 @@ -package backtype.storm.serialization.types; - -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.io.Input; -import com.esotericsoftware.kryo.serializers.CollectionSerializer; -import backtype.storm.utils.ListDelegate; -import java.util.Collection; - -public class ListDelegateSerializer extends CollectionSerializer { - @Override - public Collection create(Kryo kryo, Input input, Class type) { - return new ListDelegate(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/spout/IMultiSchemableSpout.java b/jstorm-client/src/main/java/backtype/storm/spout/IMultiSchemableSpout.java deleted file mode 100644 index ba313247c..000000000 --- a/jstorm-client/src/main/java/backtype/storm/spout/IMultiSchemableSpout.java +++ /dev/null @@ -1,7 +0,0 @@ -package backtype.storm.spout; - -public interface IMultiSchemableSpout { - MultiScheme getScheme(); - - void setScheme(MultiScheme scheme); -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/spout/ISchemableSpout.java b/jstorm-client/src/main/java/backtype/storm/spout/ISchemableSpout.java deleted file mode 100644 index 5bbc86988..000000000 --- a/jstorm-client/src/main/java/backtype/storm/spout/ISchemableSpout.java +++ /dev/null @@ -1,7 +0,0 @@ -package backtype.storm.spout; - -public interface ISchemableSpout { - Scheme getScheme(); - - void setScheme(Scheme scheme); -} diff --git a/jstorm-client/src/main/java/backtype/storm/spout/ISpout.java b/jstorm-client/src/main/java/backtype/storm/spout/ISpout.java deleted file mode 100644 index 67f94f38a..000000000 --- a/jstorm-client/src/main/java/backtype/storm/spout/ISpout.java +++ /dev/null @@ -1,116 +0,0 @@ -package backtype.storm.spout; - -import backtype.storm.task.TopologyContext; -import java.util.Map; -import java.io.Serializable; - -/** - * ISpout is the core interface for implementing spouts. A Spout is responsible - * for feeding messages into the topology for processing. For every tuple - * emitted by a spout, Storm will track the (potentially very large) DAG of - * tuples generated based on a tuple emitted by the spout. When Storm detects - * that every tuple in that DAG has been successfully processed, it will send an - * ack message to the Spout. - * - *

- * If a tuple fails to be fully process within the configured timeout for the - * topology (see {@link backtype.storm.Config}), Storm will send a fail message - * to the spout for the message. - *

- * - *

- * When a Spout emits a tuple, it can tag the tuple with a message id. The - * message id can be any type. When Storm acks or fails a message, it will pass - * back to the spout the same message id to identify which tuple it's referring - * to. If the spout leaves out the message id, or sets it to null, then Storm - * will not track the message and the spout will not receive any ack or fail - * callbacks for the message. - *

- * - *

- * Storm executes ack, fail, and nextTuple all on the same thread. This means - * that an implementor of an ISpout does not need to worry about concurrency - * issues between those methods. However, it also means that an implementor must - * ensure that nextTuple is non-blocking: otherwise the method could block acks - * and fails that are pending to be processed. - *

- */ -public interface ISpout extends Serializable { - /** - * Called when a task for this component is initialized within a worker on - * the cluster. It provides the spout with the environment in which the - * spout executes. - * - *

- * This includes the: - *

- * - * @param conf - * The Storm configuration for this spout. This is the - * configuration provided to the topology merged in with cluster - * configuration on this machine. - * @param context - * This object can be used to get information about this task's - * place within the topology, including the task id and component - * id of this task, input and output information, etc. - * @param collector - * The collector is used to emit tuples from this spout. Tuples - * can be emitted at any time, including the open and close - * methods. The collector is thread-safe and should be saved as - * an instance variable of this spout object. - */ - void open(Map conf, TopologyContext context, SpoutOutputCollector collector); - - /** - * Called when an ISpout is going to be shutdown. There is no guarentee that - * close will be called, because the supervisor kill -9's worker processes - * on the cluster. - * - *

- * The one context where close is guaranteed to be called is a topology is - * killed when running Storm in local mode. - *

- */ - void close(); - - /** - * Called when a spout has been activated out of a deactivated mode. - * nextTuple will be called on this spout soon. A spout can become activated - * after having been deactivated when the topology is manipulated using the - * `storm` client. - */ - void activate(); - - /** - * Called when a spout has been deactivated. nextTuple will not be called - * while a spout is deactivated. The spout may or may not be reactivated in - * the future. - */ - void deactivate(); - - /** - * When this method is called, Storm is requesting that the Spout emit - * tuples to the output collector. This method should be non-blocking, so if - * the Spout has no tuples to emit, this method should return. nextTuple, - * ack, and fail are all called in a tight loop in a single thread in the - * spout task. When there are no tuples to emit, it is courteous to have - * nextTuple sleep for a short amount of time (like a single millisecond) so - * as not to waste too much CPU. - */ - void nextTuple(); - - /** - * Storm has determined that the tuple emitted by this spout with the msgId - * identifier has been fully processed. Typically, an implementation of this - * method will take that message off the queue and prevent it from being - * replayed. - */ - void ack(Object msgId); - - /** - * The tuple emitted by this spout with the msgId identifier has failed to - * be fully processed. Typically, an implementation of this method will put - * that message back on the queue to be replayed at a later time. - */ - void fail(Object msgId); -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/spout/ISpoutOutputCollector.java b/jstorm-client/src/main/java/backtype/storm/spout/ISpoutOutputCollector.java deleted file mode 100644 index 6b66b00af..000000000 --- a/jstorm-client/src/main/java/backtype/storm/spout/ISpoutOutputCollector.java +++ /dev/null @@ -1,15 +0,0 @@ -package backtype.storm.spout; - -import java.util.List; - -public interface ISpoutOutputCollector { - /** - * Returns the task ids that received the tuples. - */ - List emit(String streamId, List tuple, Object messageId); - - void emitDirect(int taskId, String streamId, List tuple, - Object messageId); - - void reportError(Throwable error); -} diff --git a/jstorm-client/src/main/java/backtype/storm/spout/ISpoutWaitStrategy.java b/jstorm-client/src/main/java/backtype/storm/spout/ISpoutWaitStrategy.java deleted file mode 100644 index f5a25b3ec..000000000 --- a/jstorm-client/src/main/java/backtype/storm/spout/ISpoutWaitStrategy.java +++ /dev/null @@ -1,18 +0,0 @@ -package backtype.storm.spout; - -import java.util.Map; - -/** - * The strategy a spout needs to use when its waiting. Waiting is triggered in - * one of two conditions: - * - * 1. nextTuple emits no tuples 2. The spout has hit maxSpoutPending and can't - * emit any more tuples - * - * The default strategy sleeps for one millisecond. - */ -public interface ISpoutWaitStrategy { - void prepare(Map conf); - - void emptyEmit(long streak); -} diff --git a/jstorm-client/src/main/java/backtype/storm/spout/MultiScheme.java b/jstorm-client/src/main/java/backtype/storm/spout/MultiScheme.java deleted file mode 100644 index e67d03679..000000000 --- a/jstorm-client/src/main/java/backtype/storm/spout/MultiScheme.java +++ /dev/null @@ -1,12 +0,0 @@ -package backtype.storm.spout; - -import java.util.List; -import java.io.Serializable; - -import backtype.storm.tuple.Fields; - -public interface MultiScheme extends Serializable { - public Iterable> deserialize(byte[] ser); - - public Fields getOutputFields(); -} diff --git a/jstorm-client/src/main/java/backtype/storm/spout/NothingEmptyEmitStrategy.java b/jstorm-client/src/main/java/backtype/storm/spout/NothingEmptyEmitStrategy.java deleted file mode 100644 index c084b10d5..000000000 --- a/jstorm-client/src/main/java/backtype/storm/spout/NothingEmptyEmitStrategy.java +++ /dev/null @@ -1,14 +0,0 @@ -package backtype.storm.spout; - -import java.util.Map; - -public class NothingEmptyEmitStrategy implements ISpoutWaitStrategy { - @Override - public void emptyEmit(long streak) { - } - - @Override - public void prepare(Map conf) { - throw new UnsupportedOperationException("Not supported yet."); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/spout/RawMultiScheme.java b/jstorm-client/src/main/java/backtype/storm/spout/RawMultiScheme.java deleted file mode 100644 index 2446b45b5..000000000 --- a/jstorm-client/src/main/java/backtype/storm/spout/RawMultiScheme.java +++ /dev/null @@ -1,20 +0,0 @@ -package backtype.storm.spout; - -import java.util.List; - -import backtype.storm.tuple.Fields; - -import static backtype.storm.utils.Utils.tuple; -import static java.util.Arrays.asList; - -public class RawMultiScheme implements MultiScheme { - @Override - public Iterable> deserialize(byte[] ser) { - return asList(tuple(ser)); - } - - @Override - public Fields getOutputFields() { - return new Fields("bytes"); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/spout/RawScheme.java b/jstorm-client/src/main/java/backtype/storm/spout/RawScheme.java deleted file mode 100644 index 46e9d1c81..000000000 --- a/jstorm-client/src/main/java/backtype/storm/spout/RawScheme.java +++ /dev/null @@ -1,15 +0,0 @@ -package backtype.storm.spout; - -import backtype.storm.tuple.Fields; -import java.util.List; -import static backtype.storm.utils.Utils.tuple; - -public class RawScheme implements Scheme { - public List deserialize(byte[] ser) { - return tuple(ser); - } - - public Fields getOutputFields() { - return new Fields("bytes"); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/spout/Scheme.java b/jstorm-client/src/main/java/backtype/storm/spout/Scheme.java deleted file mode 100644 index 26bf3aef0..000000000 --- a/jstorm-client/src/main/java/backtype/storm/spout/Scheme.java +++ /dev/null @@ -1,11 +0,0 @@ -package backtype.storm.spout; - -import backtype.storm.tuple.Fields; -import java.io.Serializable; -import java.util.List; - -public interface Scheme extends Serializable { - public List deserialize(byte[] ser); - - public Fields getOutputFields(); -} diff --git a/jstorm-client/src/main/java/backtype/storm/spout/SchemeAsMultiScheme.java b/jstorm-client/src/main/java/backtype/storm/spout/SchemeAsMultiScheme.java deleted file mode 100644 index cc80ef932..000000000 --- a/jstorm-client/src/main/java/backtype/storm/spout/SchemeAsMultiScheme.java +++ /dev/null @@ -1,28 +0,0 @@ -package backtype.storm.spout; - -import java.util.Arrays; -import java.util.List; - -import backtype.storm.tuple.Fields; - -public class SchemeAsMultiScheme implements MultiScheme { - public final Scheme scheme; - - public SchemeAsMultiScheme(Scheme scheme) { - this.scheme = scheme; - } - - @Override - public Iterable> deserialize(final byte[] ser) { - List o = scheme.deserialize(ser); - if (o == null) - return null; - else - return Arrays.asList(o); - } - - @Override - public Fields getOutputFields() { - return scheme.getOutputFields(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/spout/SleepSpoutWaitStrategy.java b/jstorm-client/src/main/java/backtype/storm/spout/SleepSpoutWaitStrategy.java deleted file mode 100644 index 0aa7f64cc..000000000 --- a/jstorm-client/src/main/java/backtype/storm/spout/SleepSpoutWaitStrategy.java +++ /dev/null @@ -1,25 +0,0 @@ -package backtype.storm.spout; - -import backtype.storm.Config; -import java.util.Map; - -public class SleepSpoutWaitStrategy implements ISpoutWaitStrategy { - - long sleepMillis; - - @Override - public void prepare(Map conf) { - sleepMillis = ((Number) conf - .get(Config.TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS)) - .longValue(); - } - - @Override - public void emptyEmit(long streak) { - try { - Thread.sleep(sleepMillis); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/spout/SpoutOutputCollector.java b/jstorm-client/src/main/java/backtype/storm/spout/SpoutOutputCollector.java deleted file mode 100644 index 069fb992d..000000000 --- a/jstorm-client/src/main/java/backtype/storm/spout/SpoutOutputCollector.java +++ /dev/null @@ -1,125 +0,0 @@ -package backtype.storm.spout; - -import backtype.storm.task.OutputCollector; -import backtype.storm.utils.Utils; -import java.util.List; - -/** - * This output collector exposes the API for emitting tuples from an - * {@link backtype.storm.topology.IRichSpout}. The main difference between this - * output collector and {@link OutputCollector} for - * {@link backtype.storm.topology.IRichBolt} is that spouts can tag messages - * with ids so that they can be acked or failed later on. This is the Spout - * portion of Storm's API to guarantee that each message is fully processed at - * least once. - */ -public class SpoutOutputCollector implements ISpoutOutputCollector { - ISpoutOutputCollector _delegate; - - public SpoutOutputCollector(ISpoutOutputCollector delegate) { - _delegate = delegate; - } - - /** - * Emits a new tuple to the specified output stream with the given message - * ID. When Storm detects that this tuple has been fully processed, or has - * failed to be fully processed, the spout will receive an ack or fail - * callback respectively with the messageId as long as the messageId was not - * null. If the messageId was null, Storm will not track the tuple and no - * callback will be received. The emitted values must be immutable. - * - * @return the list of task ids that this tuple was sent to - */ - public List emit(String streamId, List tuple, - Object messageId) { - return _delegate.emit(streamId, tuple, messageId); - } - - /** - * Emits a new tuple to the default output stream with the given message ID. - * When Storm detects that this tuple has been fully processed, or has - * failed to be fully processed, the spout will receive an ack or fail - * callback respectively with the messageId as long as the messageId was not - * null. If the messageId was null, Storm will not track the tuple and no - * callback will be received. The emitted values must be immutable. - * - * @return the list of task ids that this tuple was sent to - */ - public List emit(List tuple, Object messageId) { - return emit(Utils.DEFAULT_STREAM_ID, tuple, messageId); - } - - /** - * Emits a tuple to the default output stream with a null message id. Storm - * will not track this message so ack and fail will never be called for this - * tuple. The emitted values must be immutable. - */ - public List emit(List tuple) { - return emit(tuple, null); - } - - /** - * Emits a tuple to the specified output stream with a null message id. - * Storm will not track this message so ack and fail will never be called - * for this tuple. The emitted values must be immutable. - */ - public List emit(String streamId, List tuple) { - return emit(streamId, tuple, null); - } - - /** - * Emits a tuple to the specified task on the specified output stream. This - * output stream must have been declared as a direct stream, and the - * specified task must use a direct grouping on this stream to receive the - * message. The emitted values must be immutable. - */ - public void emitDirect(int taskId, String streamId, List tuple, - Object messageId) { - _delegate.emitDirect(taskId, streamId, tuple, messageId); - } - - /** - * Emits a tuple to the specified task on the default output stream. This - * output stream must have been declared as a direct stream, and the - * specified task must use a direct grouping on this stream to receive the - * message. The emitted values must be immutable. - */ - public void emitDirect(int taskId, List tuple, Object messageId) { - emitDirect(taskId, Utils.DEFAULT_STREAM_ID, tuple, messageId); - } - - /** - * Emits a tuple to the specified task on the specified output stream. This - * output stream must have been declared as a direct stream, and the - * specified task must use a direct grouping on this stream to receive the - * message. The emitted values must be immutable. - * - *

- * Because no message id is specified, Storm will not track this message so - * ack and fail will never be called for this tuple. - *

- */ - public void emitDirect(int taskId, String streamId, List tuple) { - emitDirect(taskId, streamId, tuple, null); - } - - /** - * Emits a tuple to the specified task on the default output stream. This - * output stream must have been declared as a direct stream, and the - * specified task must use a direct grouping on this stream to receive the - * message. The emitted values must be immutable. - * - *

- * Because no message id is specified, Storm will not track this message so - * ack and fail will never be called for this tuple. - *

- */ - public void emitDirect(int taskId, List tuple) { - emitDirect(taskId, tuple, null); - } - - @Override - public void reportError(Throwable error) { - _delegate.reportError(error); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/state/IStateSpout.java b/jstorm-client/src/main/java/backtype/storm/state/IStateSpout.java deleted file mode 100644 index 2c8b30053..000000000 --- a/jstorm-client/src/main/java/backtype/storm/state/IStateSpout.java +++ /dev/null @@ -1,15 +0,0 @@ -package backtype.storm.state; - -import backtype.storm.task.TopologyContext; -import java.io.Serializable; -import java.util.Map; - -public interface IStateSpout extends Serializable { - void open(Map conf, TopologyContext context); - - void close(); - - void nextTuple(StateSpoutOutputCollector collector); - - void synchronize(SynchronizeOutputCollector collector); -} diff --git a/jstorm-client/src/main/java/backtype/storm/state/IStateSpoutOutputCollector.java b/jstorm-client/src/main/java/backtype/storm/state/IStateSpoutOutputCollector.java deleted file mode 100644 index d26ed6b8e..000000000 --- a/jstorm-client/src/main/java/backtype/storm/state/IStateSpoutOutputCollector.java +++ /dev/null @@ -1,5 +0,0 @@ -package backtype.storm.state; - -public interface IStateSpoutOutputCollector extends ISynchronizeOutputCollector { - void remove(int streamId, Object id); -} diff --git a/jstorm-client/src/main/java/backtype/storm/state/ISubscribedState.java b/jstorm-client/src/main/java/backtype/storm/state/ISubscribedState.java deleted file mode 100644 index 4256a0ac8..000000000 --- a/jstorm-client/src/main/java/backtype/storm/state/ISubscribedState.java +++ /dev/null @@ -1,9 +0,0 @@ -package backtype.storm.state; - -import backtype.storm.tuple.Tuple; - -public interface ISubscribedState { - void set(Object id, Tuple tuple); - - void remove(Object id); -} diff --git a/jstorm-client/src/main/java/backtype/storm/state/ISynchronizeOutputCollector.java b/jstorm-client/src/main/java/backtype/storm/state/ISynchronizeOutputCollector.java deleted file mode 100644 index 97a8a8ec4..000000000 --- a/jstorm-client/src/main/java/backtype/storm/state/ISynchronizeOutputCollector.java +++ /dev/null @@ -1,7 +0,0 @@ -package backtype.storm.state; - -import java.util.List; - -public interface ISynchronizeOutputCollector { - void add(int streamId, Object id, List tuple); -} diff --git a/jstorm-client/src/main/java/backtype/storm/state/StateSpoutOutputCollector.java b/jstorm-client/src/main/java/backtype/storm/state/StateSpoutOutputCollector.java deleted file mode 100644 index 3156e957c..000000000 --- a/jstorm-client/src/main/java/backtype/storm/state/StateSpoutOutputCollector.java +++ /dev/null @@ -1,11 +0,0 @@ -package backtype.storm.state; - -public class StateSpoutOutputCollector extends SynchronizeOutputCollector - implements IStateSpoutOutputCollector { - - @Override - public void remove(int streamId, Object id) { - throw new UnsupportedOperationException("Not supported yet."); - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/state/SynchronizeOutputCollector.java b/jstorm-client/src/main/java/backtype/storm/state/SynchronizeOutputCollector.java deleted file mode 100644 index 9474fa2af..000000000 --- a/jstorm-client/src/main/java/backtype/storm/state/SynchronizeOutputCollector.java +++ /dev/null @@ -1,12 +0,0 @@ -package backtype.storm.state; - -import java.util.List; - -public class SynchronizeOutputCollector implements ISynchronizeOutputCollector { - - @Override - public void add(int streamId, Object id, List tuple) { - throw new UnsupportedOperationException("Not supported yet."); - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/task/GeneralTopologyContext.java b/jstorm-client/src/main/java/backtype/storm/task/GeneralTopologyContext.java deleted file mode 100644 index 4817fb4f4..000000000 --- a/jstorm-client/src/main/java/backtype/storm/task/GeneralTopologyContext.java +++ /dev/null @@ -1,206 +0,0 @@ -package backtype.storm.task; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.json.simple.JSONAware; - -import backtype.storm.Config; -import backtype.storm.Constants; -import backtype.storm.generated.ComponentCommon; -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.generated.Grouping; -import backtype.storm.generated.StormTopology; -import backtype.storm.tuple.Fields; -import backtype.storm.utils.ThriftTopologyUtils; -import backtype.storm.utils.Utils; - -public class GeneralTopologyContext implements JSONAware { - private StormTopology _topology; - private Map _taskToComponent; - private Map> _componentToTasks; - private Map> _componentToStreamToFields; - private String _topologyId; - protected Map _stormConf; - - // pass in componentToSortedTasks for the case of running tons of tasks in - // single executor - public GeneralTopologyContext(StormTopology topology, Map stormConf, - Map taskToComponent, - Map> componentToSortedTasks, - Map> componentToStreamToFields, - String topologyId) { - _topology = topology; - _stormConf = stormConf; - _taskToComponent = taskToComponent; - _topologyId = topologyId; - _componentToTasks = componentToSortedTasks; - _componentToStreamToFields = componentToStreamToFields; - } - - /** - * Gets the unique id assigned to this topology. The id is the storm name - * with a unique nonce appended to it. - * - * @return the topology id - */ - public String getTopologyId() { - return _topologyId; - } - - /** - * Please use the getTopologId() instead. - * - * @return the topology id - */ - @Deprecated - public String getStormId() { - return _topologyId; - } - - /** - * Gets the Thrift object representing the topology. - * - * @return the Thrift definition representing the topology - */ - public StormTopology getRawTopology() { - return _topology; - } - - /** - * Gets the component id for the specified task id. The component id maps to - * a component id specified for a Spout or Bolt in the topology definition. - * - * @param taskId - * the task id - * @return the component id for the input task id - */ - public String getComponentId(int taskId) { - if (taskId == Constants.SYSTEM_TASK_ID) { - return Constants.SYSTEM_COMPONENT_ID; - } else { - return _taskToComponent.get(taskId); - } - } - - /** - * Gets the set of streams declared for the specified component. - */ - public Set getComponentStreams(String componentId) { - return getComponentCommon(componentId).get_streams().keySet(); - } - - /** - * Gets the task ids allocated for the given component id. The task ids are - * always returned in ascending order. - */ - public List getComponentTasks(String componentId) { - List ret = _componentToTasks.get(componentId); - if (ret == null) - return new ArrayList(); - else - return new ArrayList(ret); - } - - /** - * Gets the declared output fields for the specified component/stream. - */ - public Fields getComponentOutputFields(String componentId, String streamId) { - Fields ret = _componentToStreamToFields.get(componentId).get(streamId); - if (ret == null) { - throw new IllegalArgumentException( - "No output fields defined for component:stream " - + componentId + ":" + streamId); - } - return ret; - } - - /** - * Gets the declared output fields for the specified global stream id. - */ - public Fields getComponentOutputFields(GlobalStreamId id) { - return getComponentOutputFields(id.get_componentId(), id.get_streamId()); - } - - /** - * Gets the declared inputs to the specified component. - * - * @return A map from subscribed component/stream to the grouping subscribed - * with. - */ - public Map getSources(String componentId) { - return getComponentCommon(componentId).get_inputs(); - } - - /** - * Gets information about who is consuming the outputs of the specified - * component, and how. - * - * @return Map from stream id to component id to the Grouping used. - */ - public Map> getTargets(String componentId) { - Map> ret = new HashMap>(); - for (String otherComponentId : getComponentIds()) { - Map inputs = getComponentCommon( - otherComponentId).get_inputs(); - for (GlobalStreamId id : inputs.keySet()) { - if (id.get_componentId().equals(componentId)) { - Map curr = ret.get(id.get_streamId()); - if (curr == null) - curr = new HashMap(); - curr.put(otherComponentId, inputs.get(id)); - ret.put(id.get_streamId(), curr); - } - } - } - return ret; - } - - @Override - public String toJSONString() { - Map obj = new HashMap(); - obj.put("task->component", _taskToComponent); - // TODO: jsonify StormTopology - // at the minimum should send source info - return Utils.to_json(obj); - } - - /** - * Gets a map from task id to component id. - */ - public Map getTaskToComponent() { - return _taskToComponent; - } - - /** - * Gets a list of all component ids in this topology - */ - public Set getComponentIds() { - return ThriftTopologyUtils.getComponentIds(getRawTopology()); - } - - public ComponentCommon getComponentCommon(String componentId) { - return ThriftTopologyUtils.getComponentCommon(getRawTopology(), - componentId); - } - - public int maxTopologyMessageTimeout() { - Integer max = Utils.getInt(_stormConf - .get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS)); - for (String spout : getRawTopology().get_spouts().keySet()) { - ComponentCommon common = getComponentCommon(spout); - String jsonConf = common.get_json_conf(); - if (jsonConf != null) { - Map conf = (Map) Utils.from_json(jsonConf); - Object comp = conf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS); - if (comp != null) { - max = Math.max(Utils.getInt(comp), max); - } - } - } - return max; - } -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/task/IBolt.java b/jstorm-client/src/main/java/backtype/storm/task/IBolt.java deleted file mode 100644 index bfffa141d..000000000 --- a/jstorm-client/src/main/java/backtype/storm/task/IBolt.java +++ /dev/null @@ -1,98 +0,0 @@ -package backtype.storm.task; - -import backtype.storm.tuple.Tuple; -import java.util.Map; -import java.io.Serializable; - -/** - * An IBolt represents a component that takes tuples as input and produces - * tuples as output. An IBolt can do everything from filtering to joining to - * functions to aggregations. It does not have to process a tuple immediately - * and may hold onto tuples to process later. - * - *

- * A bolt's lifecycle is as follows: - *

- * - *

- * IBolt object created on client machine. The IBolt is serialized into the - * topology (using Java serialization) and submitted to the master machine of - * the cluster (Nimbus). Nimbus then launches workers which deserialize the - * object, call prepare on it, and then start processing tuples. - *

- * - *

- * If you want to parameterize an IBolt, you should set the parameter's through - * its constructor and save the parameterization state as instance variables - * (which will then get serialized and shipped to every task executing this bolt - * across the cluster). - *

- * - *

- * When defining bolts in Java, you should use the IRichBolt interface which - * adds necessary methods for using the Java TopologyBuilder API. - *

- */ -public interface IBolt extends Serializable { - /** - * Called when a task for this component is initialized within a worker on - * the cluster. It provides the bolt with the environment in which the bolt - * executes. - * - *

- * This includes the: - *

- * - * @param stormConf - * The Storm configuration for this bolt. This is the - * configuration provided to the topology merged in with cluster - * configuration on this machine. - * @param context - * This object can be used to get information about this task's - * place within the topology, including the task id and component - * id of this task, input and output information, etc. - * @param collector - * The collector is used to emit tuples from this bolt. Tuples - * can be emitted at any time, including the prepare and cleanup - * methods. The collector is thread-safe and should be saved as - * an instance variable of this bolt object. - */ - void prepare(Map stormConf, TopologyContext context, - OutputCollector collector); - - /** - * Process a single tuple of input. The Tuple object contains metadata on it - * about which component/stream/task it came from. The values of the Tuple - * can be accessed using Tuple#getValue. The IBolt does not have to process - * the Tuple immediately. It is perfectly fine to hang onto a tuple and - * process it later (for instance, to do an aggregation or join). - * - *

- * Tuples should be emitted using the OutputCollector provided through the - * prepare method. It is required that all input tuples are acked or failed - * at some point using the OutputCollector. Otherwise, Storm will be unable - * to determine when tuples coming off the spouts have been completed. - *

- * - *

- * For the common case of acking an input tuple at the end of the execute - * method, see IBasicBolt which automates this. - *

- * - * @param input - * The input tuple to be processed. - */ - void execute(Tuple input); - - /** - * Called when an IBolt is going to be shutdown. There is no guarentee that - * cleanup will be called, because the supervisor kill -9's worker processes - * on the cluster. - * - *

- * The one context where cleanup is guaranteed to be called is when a - * topology is killed when running Storm in local mode. - *

- */ - void cleanup(); -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/task/IErrorReporter.java b/jstorm-client/src/main/java/backtype/storm/task/IErrorReporter.java deleted file mode 100644 index ae0471071..000000000 --- a/jstorm-client/src/main/java/backtype/storm/task/IErrorReporter.java +++ /dev/null @@ -1,5 +0,0 @@ -package backtype.storm.task; - -public interface IErrorReporter { - void reportError(Throwable error); -} diff --git a/jstorm-client/src/main/java/backtype/storm/task/IMetricsContext.java b/jstorm-client/src/main/java/backtype/storm/task/IMetricsContext.java deleted file mode 100644 index d4ace69f0..000000000 --- a/jstorm-client/src/main/java/backtype/storm/task/IMetricsContext.java +++ /dev/null @@ -1,18 +0,0 @@ -package backtype.storm.task; - -import backtype.storm.metric.api.CombinedMetric; -import backtype.storm.metric.api.ICombiner; -import backtype.storm.metric.api.IMetric; -import backtype.storm.metric.api.IReducer; -import backtype.storm.metric.api.ReducedMetric; - -public interface IMetricsContext { - T registerMetric(String name, T metric, - int timeBucketSizeInSecs); - - ReducedMetric registerMetric(String name, IReducer reducer, - int timeBucketSizeInSecs); - - CombinedMetric registerMetric(String name, ICombiner combiner, - int timeBucketSizeInSecs); -} diff --git a/jstorm-client/src/main/java/backtype/storm/task/IOutputCollector.java b/jstorm-client/src/main/java/backtype/storm/task/IOutputCollector.java deleted file mode 100644 index dcf22172a..000000000 --- a/jstorm-client/src/main/java/backtype/storm/task/IOutputCollector.java +++ /dev/null @@ -1,20 +0,0 @@ -package backtype.storm.task; - -import backtype.storm.tuple.Tuple; -import java.util.Collection; -import java.util.List; - -public interface IOutputCollector extends IErrorReporter { - /** - * Returns the task ids that received the tuples. - */ - List emit(String streamId, Collection anchors, - List tuple); - - void emitDirect(int taskId, String streamId, Collection anchors, - List tuple); - - void ack(Tuple input); - - void fail(Tuple input); -} diff --git a/jstorm-client/src/main/java/backtype/storm/task/OutputCollector.java b/jstorm-client/src/main/java/backtype/storm/task/OutputCollector.java deleted file mode 100644 index 9cee3a86c..000000000 --- a/jstorm-client/src/main/java/backtype/storm/task/OutputCollector.java +++ /dev/null @@ -1,245 +0,0 @@ -package backtype.storm.task; - -import backtype.storm.tuple.Tuple; -import backtype.storm.utils.Utils; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; - -/** - * This output collector exposes the API for emitting tuples from an IRichBolt. - * This is the core API for emitting tuples. For a simpler API, and a more - * restricted form of stream processing, see IBasicBolt and - * BasicOutputCollector. - */ -public class OutputCollector implements IOutputCollector { - private IOutputCollector _delegate; - - public OutputCollector(IOutputCollector delegate) { - _delegate = delegate; - } - - /** - * Emits a new tuple to a specific stream with a single anchor. The emitted - * values must be immutable. - * - * @param streamId - * the stream to emit to - * @param anchor - * the tuple to anchor to - * @param tuple - * the new output tuple from this bolt - * @return the list of task ids that this new tuple was sent to - */ - public List emit(String streamId, Tuple anchor, List tuple) { - return emit(streamId, Arrays.asList(anchor), tuple); - } - - /** - * Emits a new unanchored tuple to the specified stream. Because it's - * unanchored, if a failure happens downstream, this new tuple won't affect - * whether any spout tuples are considered failed or not. The emitted values - * must be immutable. - * - * @param streamId - * the stream to emit to - * @param tuple - * the new output tuple from this bolt - * @return the list of task ids that this new tuple was sent to - */ - public List emit(String streamId, List tuple) { - return emit(streamId, (List) null, tuple); - } - - /** - * Emits a new tuple to the default stream anchored on a group of input - * tuples. The emitted values must be immutable. - * - * @param anchors - * the tuples to anchor to - * @param tuple - * the new output tuple from this bolt - * @return the list of task ids that this new tuple was sent to - */ - public List emit(Collection anchors, List tuple) { - return emit(Utils.DEFAULT_STREAM_ID, anchors, tuple); - } - - /** - * Emits a new tuple to the default stream anchored on a single tuple. The - * emitted values must be immutable. - * - * @param anchor - * the tuple to anchor to - * @param tuple - * the new output tuple from this bolt - * @return the list of task ids that this new tuple was sent to - */ - public List emit(Tuple anchor, List tuple) { - return emit(Utils.DEFAULT_STREAM_ID, anchor, tuple); - } - - /** - * Emits a new unanchored tuple to the default stream. Beacuse it's - * unanchored, if a failure happens downstream, this new tuple won't affect - * whether any spout tuples are considered failed or not. The emitted values - * must be immutable. - * - * @param tuple - * the new output tuple from this bolt - * @return the list of task ids that this new tuple was sent to - */ - public List emit(List tuple) { - return emit(Utils.DEFAULT_STREAM_ID, tuple); - } - - /** - * Emits a tuple directly to the specified task id on the specified stream. - * If the target bolt does not subscribe to this bolt using a direct - * grouping, the tuple will not be sent. If the specified output stream is - * not declared as direct, or the target bolt subscribes with a non-direct - * grouping, an error will occur at runtime. The emitted values must be - * immutable. - * - * @param taskId - * the taskId to send the new tuple to - * @param streamId - * the stream to send the tuple on. It must be declared as a - * direct stream in the topology definition. - * @param anchor - * the tuple to anchor to - * @param tuple - * the new output tuple from this bolt - */ - public void emitDirect(int taskId, String streamId, Tuple anchor, - List tuple) { - emitDirect(taskId, streamId, Arrays.asList(anchor), tuple); - } - - /** - * Emits a tuple directly to the specified task id on the specified stream. - * If the target bolt does not subscribe to this bolt using a direct - * grouping, the tuple will not be sent. If the specified output stream is - * not declared as direct, or the target bolt subscribes with a non-direct - * grouping, an error will occur at runtime. Note that this method does not - * use anchors, so downstream failures won't affect the failure status of - * any spout tuples. The emitted values must be immutable. - * - * @param taskId - * the taskId to send the new tuple to - * @param streamId - * the stream to send the tuple on. It must be declared as a - * direct stream in the topology definition. - * @param tuple - * the new output tuple from this bolt - */ - public void emitDirect(int taskId, String streamId, List tuple) { - emitDirect(taskId, streamId, (List) null, tuple); - } - - /** - * Emits a tuple directly to the specified task id on the default stream. If - * the target bolt does not subscribe to this bolt using a direct grouping, - * the tuple will not be sent. If the specified output stream is not - * declared as direct, or the target bolt subscribes with a non-direct - * grouping, an error will occur at runtime. The emitted values must be - * immutable. - * - *

- * The default stream must be declared as direct in the topology definition. - * See OutputDeclarer#declare for how this is done when defining topologies - * in Java. - *

- * - * @param taskId - * the taskId to send the new tuple to - * @param anchosr - * the tuples to anchor to - * @param tuple - * the new output tuple from this bolt - */ - public void emitDirect(int taskId, Collection anchors, - List tuple) { - emitDirect(taskId, Utils.DEFAULT_STREAM_ID, anchors, tuple); - } - - /** - * Emits a tuple directly to the specified task id on the default stream. If - * the target bolt does not subscribe to this bolt using a direct grouping, - * the tuple will not be sent. If the specified output stream is not - * declared as direct, or the target bolt subscribes with a non-direct - * grouping, an error will occur at runtime. The emitted values must be - * immutable. - * - *

- * The default stream must be declared as direct in the topology definition. - * See OutputDeclarer#declare for how this is done when defining topologies - * in Java. - *

- * - * @param taskId - * the taskId to send the new tuple to - * @param anchor - * the tuple to anchor to - * @param tuple - * the new output tuple from this bolt - */ - public void emitDirect(int taskId, Tuple anchor, List tuple) { - emitDirect(taskId, Utils.DEFAULT_STREAM_ID, anchor, tuple); - } - - /** - * Emits a tuple directly to the specified task id on the default stream. If - * the target bolt does not subscribe to this bolt using a direct grouping, - * the tuple will not be sent. If the specified output stream is not - * declared as direct, or the target bolt subscribes with a non-direct - * grouping, an error will occur at runtime. The emitted values must be - * immutable. - * - *

- * The default stream must be declared as direct in the topology definition. - * See OutputDeclarer#declare for how this is done when defining topologies - * in Java. - *

- * - *

- * Note that this method does not use anchors, so downstream failures won't - * affect the failure status of any spout tuples. - *

- * - * @param taskId - * the taskId to send the new tuple to - * @param tuple - * the new output tuple from this bolt - */ - public void emitDirect(int taskId, List tuple) { - emitDirect(taskId, Utils.DEFAULT_STREAM_ID, tuple); - } - - @Override - public List emit(String streamId, Collection anchors, - List tuple) { - return _delegate.emit(streamId, anchors, tuple); - } - - @Override - public void emitDirect(int taskId, String streamId, - Collection anchors, List tuple) { - _delegate.emitDirect(taskId, streamId, anchors, tuple); - } - - @Override - public void ack(Tuple input) { - _delegate.ack(input); - } - - @Override - public void fail(Tuple input) { - _delegate.fail(input); - } - - @Override - public void reportError(Throwable error) { - _delegate.reportError(error); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/task/TopologyContext.java b/jstorm-client/src/main/java/backtype/storm/task/TopologyContext.java deleted file mode 100644 index 34ef4fa9e..000000000 --- a/jstorm-client/src/main/java/backtype/storm/task/TopologyContext.java +++ /dev/null @@ -1,317 +0,0 @@ -package backtype.storm.task; - -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.generated.Grouping; -import backtype.storm.generated.StormTopology; -import backtype.storm.hooks.ITaskHook; -import backtype.storm.metric.api.IMetric; -import backtype.storm.metric.api.IReducer; -import backtype.storm.metric.api.ICombiner; -import backtype.storm.metric.api.ReducedMetric; -import backtype.storm.metric.api.CombinedMetric; -import backtype.storm.state.ISubscribedState; -import backtype.storm.tuple.Fields; -import backtype.storm.utils.Utils; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import org.apache.commons.lang.NotImplementedException; - -/** - * A TopologyContext is given to bolts and spouts in their "prepare" and "open" - * methods, respectively. This object provides information about the component's - * place within the topology, such as task ids, inputs and outputs, etc. - * - *

- * The TopologyContext is also used to declare ISubscribedState objects to - * synchronize state with StateSpouts this object is subscribed to. - *

- */ -public class TopologyContext extends WorkerTopologyContext implements - IMetricsContext { - private Integer _taskId; - private Map _taskData = new HashMap(); - private List _hooks = new ArrayList(); - private Map _executorData; - private Map>> _registeredMetrics; - private clojure.lang.Atom _openOrPrepareWasCalled; - - public TopologyContext(StormTopology topology, Map stormConf, - Map taskToComponent, - Map> componentToSortedTasks, - Map> componentToStreamToFields, - String topologyId, String codeDir, String pidDir, Integer taskId, - Integer workerPort, List workerTasks, - Map defaultResources, - Map userResources, - Map executorData, Map registeredMetrics, - clojure.lang.Atom openOrPrepareWasCalled) { - super(topology, stormConf, taskToComponent, componentToSortedTasks, - componentToStreamToFields, topologyId, codeDir, pidDir, - workerPort, workerTasks, defaultResources, userResources); - _taskId = taskId; - _executorData = executorData; - _registeredMetrics = registeredMetrics; - _openOrPrepareWasCalled = openOrPrepareWasCalled; - } - - /** - * All state from all subscribed state spouts streams will be synced with - * the provided object. - * - *

- * It is recommended that your ISubscribedState object is kept as an - * instance variable of this object. The recommended usage of this method is - * as follows: - *

- * - *

- * _myState = context.setAllSubscribedState(new MyState()); - *

- * - * @param obj - * Provided ISubscribedState implementation - * @return Returns the ISubscribedState object provided - */ - public T setAllSubscribedState(T obj) { - // check that only subscribed to one component/stream for statespout - // setsubscribedstate appropriately - throw new NotImplementedException(); - } - - /** - * Synchronizes the default stream from the specified state spout component - * id with the provided ISubscribedState object. - * - *

- * The recommended usage of this method is as follows: - *

- *

- * _myState = context.setSubscribedState(componentId, new MyState()); - *

- * - * @param componentId - * the id of the StateSpout component to subscribe to - * @param obj - * Provided ISubscribedState implementation - * @return Returns the ISubscribedState object provided - */ - public T setSubscribedState( - String componentId, T obj) { - return setSubscribedState(componentId, Utils.DEFAULT_STREAM_ID, obj); - } - - /** - * Synchronizes the specified stream from the specified state spout - * component id with the provided ISubscribedState object. - * - *

- * The recommended usage of this method is as follows: - *

- *

- * _myState = context.setSubscribedState(componentId, streamId, new - * MyState()); - *

- * - * @param componentId - * the id of the StateSpout component to subscribe to - * @param streamId - * the stream to subscribe to - * @param obj - * Provided ISubscribedState implementation - * @return Returns the ISubscribedState object provided - */ - public T setSubscribedState( - String componentId, String streamId, T obj) { - throw new NotImplementedException(); - } - - /** - * Gets the task id of this task. - * - * @return the task id - */ - public int getThisTaskId() { - return _taskId; - } - - /** - * Gets the component id for this task. The component id maps to a component - * id specified for a Spout or Bolt in the topology definition. - * - * @return - */ - public String getThisComponentId() { - return getComponentId(_taskId); - } - - /** - * Gets the declared output fields for the specified stream id for the - * component this task is a part of. - */ - public Fields getThisOutputFields(String streamId) { - return getComponentOutputFields(getThisComponentId(), streamId); - } - - /** - * Gets the set of streams declared for the component of this task. - */ - public Set getThisStreams() { - return getComponentStreams(getThisComponentId()); - } - - /** - * Gets the index of this task id in - * getComponentTasks(getThisComponentId()). An example use case for this - * method is determining which task accesses which resource in a distributed - * resource to ensure an even distribution. - */ - public int getThisTaskIndex() { - List tasks = new ArrayList( - getComponentTasks(getThisComponentId())); - Collections.sort(tasks); - for (int i = 0; i < tasks.size(); i++) { - if (tasks.get(i) == getThisTaskId()) { - return i; - } - } - throw new RuntimeException( - "Fatal: could not find this task id in this component"); - } - - /** - * Gets the declared inputs to this component. - * - * @return A map from subscribed component/stream to the grouping subscribed - * with. - */ - public Map getThisSources() { - return getSources(getThisComponentId()); - } - - /** - * Gets information about who is consuming the outputs of this component, - * and how. - * - * @return Map from stream id to component id to the Grouping used. - */ - public Map> getThisTargets() { - return getTargets(getThisComponentId()); - } - - public void setTaskData(String name, Object data) { - _taskData.put(name, data); - } - - public Object getTaskData(String name) { - return _taskData.get(name); - } - - public void setExecutorData(String name, Object data) { - _executorData.put(name, data); - } - - public Object getExecutorData(String name) { - return _executorData.get(name); - } - - public void addTaskHook(ITaskHook hook) { - hook.prepare(_stormConf, this); - _hooks.add(hook); - } - - public Collection getHooks() { - return _hooks; - } - - /* - * Register a IMetric instance. Storm will then call getValueAndReset on the - * metric every timeBucketSizeInSecs and the returned value is sent to all - * metrics consumers. You must call this during IBolt::prepare or - * ISpout::open. - * - * @return The IMetric argument unchanged. - */ - public T registerMetric(String name, T metric, - int timeBucketSizeInSecs) { - if ((Boolean) _openOrPrepareWasCalled.deref() == true) { - throw new RuntimeException( - "TopologyContext.registerMetric can only be called from within overridden " - + "IBolt::prepare() or ISpout::open() method."); - } - - if (metric == null) { - throw new IllegalArgumentException("Cannot register a null metric"); - } - - if (timeBucketSizeInSecs <= 0) { - throw new IllegalArgumentException("TopologyContext.registerMetric can only be called with timeBucketSizeInSecs " + - "greater than or equal to 1 second."); - } - - if (getRegisteredMetricByName(name) != null) { - throw new RuntimeException("The same metric name `" + name + "` was registered twice." ); - } - - Map m1 = _registeredMetrics; - if (!m1.containsKey(timeBucketSizeInSecs)) { - m1.put(timeBucketSizeInSecs, new HashMap()); - } - - Map m2 = (Map) m1.get(timeBucketSizeInSecs); - if (!m2.containsKey(_taskId)) { - m2.put(_taskId, new HashMap()); - } - - Map m3 = (Map) m2.get(_taskId); - if (m3.containsKey(name)) { - throw new RuntimeException("The same metric name `" + name - + "` was registered twice."); - } else { - m3.put(name, metric); - } - - return metric; - } - - /** - * Get component's metric from registered metrics by name. - * Notice: Normally, one component can only register one metric name once. - * But now registerMetric has a bug(https://issues.apache.org/jira/browse/STORM-254) - * cause the same metric name can register twice. - * So we just return the first metric we meet. - */ - public IMetric getRegisteredMetricByName(String name) { - IMetric metric = null; - - for (Map> taskIdToNameToMetric: _registeredMetrics.values()) { - Map nameToMetric = taskIdToNameToMetric.get(_taskId); - if (nameToMetric != null) { - metric = nameToMetric.get(name); - if (metric != null) { - //we just return the first metric we meet - break; - } - } - } - - return metric; - } - - /* - * Convinience method for registering ReducedMetric. - */ - public ReducedMetric registerMetric(String name, IReducer reducer, int timeBucketSizeInSecs) { - return registerMetric(name, new ReducedMetric(reducer), timeBucketSizeInSecs); - } - /* - * Convinience method for registering CombinedMetric. - */ - public CombinedMetric registerMetric(String name, ICombiner combiner, int timeBucketSizeInSecs) { - return registerMetric(name, new CombinedMetric(combiner), timeBucketSizeInSecs); - } -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/task/WorkerTopologyContext.java b/jstorm-client/src/main/java/backtype/storm/task/WorkerTopologyContext.java deleted file mode 100644 index 42e88dcaf..000000000 --- a/jstorm-client/src/main/java/backtype/storm/task/WorkerTopologyContext.java +++ /dev/null @@ -1,84 +0,0 @@ -package backtype.storm.task; - -import backtype.storm.generated.StormTopology; -import backtype.storm.tuple.Fields; -import java.io.File; -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ExecutorService; - -public class WorkerTopologyContext extends GeneralTopologyContext { - public static final String SHARED_EXECUTOR = "executor"; - - private Integer _workerPort; - private List _workerTasks; - private String _codeDir; - private String _pidDir; - Map _userResources; - Map _defaultResources; - - public WorkerTopologyContext(StormTopology topology, Map stormConf, - Map taskToComponent, - Map> componentToSortedTasks, - Map> componentToStreamToFields, - String topologyId, String codeDir, String pidDir, Integer workerPort, - List workerTasks, Map defaultResources, - Map userResources) { - super(topology, stormConf, taskToComponent, componentToSortedTasks, - componentToStreamToFields, topologyId); - _codeDir = codeDir; - _defaultResources = defaultResources; - _userResources = userResources; - try { - if (pidDir != null) { - _pidDir = new File(pidDir).getCanonicalPath(); - } else { - _pidDir = null; - } - } catch (IOException e) { - throw new RuntimeException("Could not get canonical path for " - + _pidDir, e); - } - _workerPort = workerPort; - _workerTasks = workerTasks; - } - - /** - * Gets all the task ids that are running in this worker process (including - * the task for this task). - */ - public List getThisWorkerTasks() { - return _workerTasks; - } - - public Integer getThisWorkerPort() { - return _workerPort; - } - - /** - * Gets the location of the external resources for this worker on the local - * filesystem. These external resources typically include bolts implemented - * in other languages, such as Ruby or Python. - */ - public String getCodeDir() { - return _codeDir; - } - - /** - * If this task spawns any subprocesses, those subprocesses must immediately - * write their PID to this directory on the local filesystem to ensure that - * Storm properly destroys that process when the worker is shutdown. - */ - public String getPIDDir() { - return _pidDir; - } - - public Object getResource(String name) { - return _userResources.get(name); - } - - public ExecutorService getSharedExecutor() { - return (ExecutorService) _defaultResources.get(SHARED_EXECUTOR); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/AckFailDelegate.java b/jstorm-client/src/main/java/backtype/storm/testing/AckFailDelegate.java deleted file mode 100644 index 131dee756..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/AckFailDelegate.java +++ /dev/null @@ -1,9 +0,0 @@ -package backtype.storm.testing; - -import java.io.Serializable; - -public interface AckFailDelegate extends Serializable { - public void ack(Object id); - - public void fail(Object id); -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/AckFailMapTracker.java b/jstorm-client/src/main/java/backtype/storm/testing/AckFailMapTracker.java deleted file mode 100644 index 68d334d76..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/AckFailMapTracker.java +++ /dev/null @@ -1,35 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.utils.RegisteredGlobalState; -import java.util.HashSet; -import java.util.Set; - -public class AckFailMapTracker implements AckFailDelegate { - - String _acked; - String _failed; - - public AckFailMapTracker() { - _acked = RegisteredGlobalState.registerState(new HashSet()); - _failed = RegisteredGlobalState.registerState(new HashSet()); - } - - public boolean isAcked(Object id) { - return ((Set) RegisteredGlobalState.getState(_acked)).contains(id); - } - - public boolean isFailed(Object id) { - return ((Set) RegisteredGlobalState.getState(_failed)).contains(id); - } - - @Override - public void ack(Object id) { - ((Set) RegisteredGlobalState.getState(_acked)).add(id); - } - - @Override - public void fail(Object id) { - ((Set) RegisteredGlobalState.getState(_failed)).add(id); - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/AckTracker.java b/jstorm-client/src/main/java/backtype/storm/testing/AckTracker.java deleted file mode 100644 index 134f8f061..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/AckTracker.java +++ /dev/null @@ -1,35 +0,0 @@ -package backtype.storm.testing; - -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicInteger; - -public class AckTracker implements AckFailDelegate { - private static Map acks = new ConcurrentHashMap(); - - private String _id; - - public AckTracker() { - _id = UUID.randomUUID().toString(); - acks.put(_id, new AtomicInteger(0)); - } - - @Override - public void ack(Object id) { - acks.get(_id).incrementAndGet(); - } - - @Override - public void fail(Object id) { - } - - public int getNumAcks() { - return acks.get(_id).intValue(); - } - - public void resetNumAcks() { - acks.get(_id).set(0); - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/BatchNumberList.java b/jstorm-client/src/main/java/backtype/storm/testing/BatchNumberList.java deleted file mode 100644 index dd6530e4f..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/BatchNumberList.java +++ /dev/null @@ -1,55 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.coordination.BatchOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseBatchBolt; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -public class BatchNumberList extends BaseBatchBolt { - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(new Fields("word", "list")); - } - - String _wordComponent; - - public BatchNumberList(String wordComponent) { - _wordComponent = wordComponent; - } - - String word = null; - List intSet = new ArrayList(); - BatchOutputCollector _collector; - - @Override - public void prepare(Map conf, TopologyContext context, - BatchOutputCollector collector, Object id) { - _collector = collector; - } - - @Override - public void execute(Tuple tuple) { - if (tuple.getSourceComponent().equals(_wordComponent)) { - this.word = tuple.getString(1); - } else { - intSet.add(tuple.getInteger(1)); - } - } - - @Override - public void finishBatch() { - if (word != null) { - Collections.sort(intSet); - _collector.emit(new Values(word, intSet)); - } - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/BatchProcessWord.java b/jstorm-client/src/main/java/backtype/storm/testing/BatchProcessWord.java deleted file mode 100644 index 21f316f5b..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/BatchProcessWord.java +++ /dev/null @@ -1,23 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.topology.BasicOutputCollector; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseBasicBolt; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; - -public class BatchProcessWord extends BaseBasicBolt { - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(new Fields("id", "size")); - } - - @Override - public void execute(Tuple input, BasicOutputCollector collector) { - collector.emit(new Values(input.getValue(0), input.getString(1) - .length())); - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/BatchRepeatA.java b/jstorm-client/src/main/java/backtype/storm/testing/BatchRepeatA.java deleted file mode 100644 index 9cb06c358..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/BatchRepeatA.java +++ /dev/null @@ -1,30 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.topology.BasicOutputCollector; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseBasicBolt; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; - -public class BatchRepeatA extends BaseBasicBolt { - - @Override - public void execute(Tuple input, BasicOutputCollector collector) { - Object id = input.getValue(0); - String word = input.getString(1); - for (int i = 0; i < word.length(); i++) { - if (word.charAt(i) == 'a') { - collector.emit("multi", new Values(id, word.substring(0, i))); - } - } - collector.emit("single", new Values(id, word)); - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declareStream("multi", new Fields("id", "word")); - declarer.declareStream("single", new Fields("id", "word")); - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/BoltTracker.java b/jstorm-client/src/main/java/backtype/storm/testing/BoltTracker.java deleted file mode 100644 index fb928c8bb..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/BoltTracker.java +++ /dev/null @@ -1,24 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import java.util.HashMap; -import java.util.Map; - -public class BoltTracker extends NonRichBoltTracker implements IRichBolt { - IRichBolt _richDelegate; - - public BoltTracker(IRichBolt delegate, String id) { - super(delegate, id); - _richDelegate = delegate; - } - - public void declareOutputFields(OutputFieldsDeclarer declarer) { - _richDelegate.declareOutputFields(declarer); - } - - @Override - public Map getComponentConfiguration() { - return new HashMap(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/CompleteTopologyParam.java b/jstorm-client/src/main/java/backtype/storm/testing/CompleteTopologyParam.java deleted file mode 100644 index 4017ddbf1..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/CompleteTopologyParam.java +++ /dev/null @@ -1,57 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.Config; - -/** - * The param class for the Testing.completeTopology. - */ -public class CompleteTopologyParam { - /** - * The mocked spout sources - */ - private MockedSources mockedSources; - /** - * the config for the topology when it was submitted to the cluster - */ - private Config stormConf; - /** - * whether cleanup the state? - */ - private Boolean cleanupState; - /** - * the topology name you want to submit to the cluster - */ - private String topologyName; - - public MockedSources getMockedSources() { - return mockedSources; - } - - public void setMockedSources(MockedSources mockedSources) { - this.mockedSources = mockedSources; - } - - public Config getStormConf() { - return stormConf; - } - - public void setStormConf(Config stormConf) { - this.stormConf = stormConf; - } - - public Boolean getCleanupState() { - return cleanupState; - } - - public void setCleanupState(Boolean cleanupState) { - this.cleanupState = cleanupState; - } - - public String getTopologyName() { - return topologyName; - } - - public void setTopologyName(String topologyName) { - this.topologyName = topologyName; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/CountingBatchBolt.java b/jstorm-client/src/main/java/backtype/storm/testing/CountingBatchBolt.java deleted file mode 100644 index 75dca1adb..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/CountingBatchBolt.java +++ /dev/null @@ -1,39 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseBatchBolt; -import backtype.storm.coordination.BatchOutputCollector; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; -import java.util.Map; - -public class CountingBatchBolt extends BaseBatchBolt { - BatchOutputCollector _collector; - Object _id; - int _count = 0; - - @Override - public void prepare(Map conf, TopologyContext context, - BatchOutputCollector collector, Object id) { - _collector = collector; - _id = id; - } - - @Override - public void execute(Tuple tuple) { - _count++; - } - - @Override - public void finishBatch() { - _collector.emit(new Values(_id, _count)); - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(new Fields("tx", "count")); - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/CountingCommitBolt.java b/jstorm-client/src/main/java/backtype/storm/testing/CountingCommitBolt.java deleted file mode 100644 index e3533a6b5..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/CountingCommitBolt.java +++ /dev/null @@ -1,42 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.coordination.BatchOutputCollector; -import backtype.storm.topology.base.BaseTransactionalBolt; -import backtype.storm.transactional.ICommitter; -import backtype.storm.transactional.TransactionAttempt; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; -import java.util.Map; - -public class CountingCommitBolt extends BaseTransactionalBolt implements - ICommitter { - BatchOutputCollector _collector; - TransactionAttempt _id; - int _count = 0; - - @Override - public void prepare(Map conf, TopologyContext context, - BatchOutputCollector collector, TransactionAttempt id) { - _id = id; - _collector = collector; - } - - @Override - public void execute(Tuple tuple) { - _count++; - } - - @Override - public void finishBatch() { - _collector.emit(new Values(_id, _count)); - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(new Fields("tx", "count")); - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/FeederSpout.java b/jstorm-client/src/main/java/backtype/storm/testing/FeederSpout.java deleted file mode 100644 index 871e57374..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/FeederSpout.java +++ /dev/null @@ -1,83 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.topology.OutputFieldsDeclarer; -import java.util.Map; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.base.BaseRichSpout; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Values; -import backtype.storm.utils.InprocMessaging; -import java.util.HashMap; -import java.util.List; -import java.util.UUID; - -public class FeederSpout extends BaseRichSpout { - private int _id; - private Fields _outFields; - private SpoutOutputCollector _collector; - private AckFailDelegate _ackFailDelegate; - - public FeederSpout(Fields outFields) { - _id = InprocMessaging.acquireNewPort(); - _outFields = outFields; - } - - public void setAckFailDelegate(AckFailDelegate d) { - _ackFailDelegate = d; - } - - public void feed(List tuple) { - feed(tuple, UUID.randomUUID().toString()); - } - - public void feed(List tuple, Object msgId) { - InprocMessaging.sendMessage(_id, new Values(tuple, msgId)); - } - - public void open(Map conf, TopologyContext context, - SpoutOutputCollector collector) { - _collector = collector; - } - - public void close() { - - } - - public void nextTuple() { - List toEmit = (List) InprocMessaging.pollMessage(_id); - if (toEmit != null) { - List tuple = (List) toEmit.get(0); - Object msgId = toEmit.get(1); - - _collector.emit(tuple, msgId); - } else { - try { - Thread.sleep(1); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - } - - public void ack(Object msgId) { - if (_ackFailDelegate != null) { - _ackFailDelegate.ack(msgId); - } - } - - public void fail(Object msgId) { - if (_ackFailDelegate != null) { - _ackFailDelegate.fail(msgId); - } - } - - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(_outFields); - } - - @Override - public Map getComponentConfiguration() { - return new HashMap(); - } -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/testing/FixedTuple.java b/jstorm-client/src/main/java/backtype/storm/testing/FixedTuple.java deleted file mode 100644 index e4cc08989..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/FixedTuple.java +++ /dev/null @@ -1,25 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.utils.Utils; -import java.io.Serializable; -import java.util.List; - -public class FixedTuple implements Serializable { - public String stream; - public List values; - - public FixedTuple(List values) { - this.stream = Utils.DEFAULT_STREAM_ID; - this.values = values; - } - - public FixedTuple(String stream, List values) { - this.stream = stream; - this.values = values; - } - - @Override - public String toString() { - return stream + ":" + "<" + values.toString() + ">"; - } -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/testing/FixedTupleSpout.java b/jstorm-client/src/main/java/backtype/storm/testing/FixedTupleSpout.java deleted file mode 100644 index e463df0cb..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/FixedTupleSpout.java +++ /dev/null @@ -1,164 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichSpout; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import backtype.storm.utils.Utils; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import static backtype.storm.utils.Utils.get; - -public class FixedTupleSpout implements IRichSpout { - private static final Map acked = new HashMap(); - private static final Map failed = new HashMap(); - - public static int getNumAcked(String stormId) { - synchronized (acked) { - return get(acked, stormId, 0); - } - } - - public static int getNumFailed(String stormId) { - synchronized (failed) { - return get(failed, stormId, 0); - } - } - - public static void clear(String stormId) { - acked.remove(stormId); - failed.remove(stormId); - } - - private List _tuples; - private SpoutOutputCollector _collector; - - private TopologyContext _context; - private List _serveTuples; - private Map _pending; - - private String _id; - private String _fieldName; - - public FixedTupleSpout(List tuples) { - this(tuples, null); - } - - public FixedTupleSpout(List tuples, String fieldName) { - _id = UUID.randomUUID().toString(); - synchronized (acked) { - acked.put(_id, 0); - } - synchronized (failed) { - failed.put(_id, 0); - } - _tuples = new ArrayList(); - for (Object o : tuples) { - FixedTuple ft; - if (o instanceof FixedTuple) { - ft = (FixedTuple) o; - } else { - ft = new FixedTuple((List) o); - } - _tuples.add(ft); - } - _fieldName = fieldName; - } - - public List getSourceTuples() { - return _tuples; - } - - public int getCompleted() { - int ackedAmt; - int failedAmt; - - synchronized (acked) { - ackedAmt = acked.get(_id); - } - synchronized (failed) { - failedAmt = failed.get(_id); - } - return ackedAmt + failedAmt; - } - - public void cleanup() { - synchronized (acked) { - acked.remove(_id); - } - synchronized (failed) { - failed.remove(_id); - } - } - - public void open(Map conf, TopologyContext context, - SpoutOutputCollector collector) { - _context = context; - List tasks = context.getComponentTasks(context - .getThisComponentId()); - int startIndex; - for (startIndex = 0; startIndex < tasks.size(); startIndex++) { - if (tasks.get(startIndex) == context.getThisTaskId()) { - break; - } - } - _collector = collector; - _pending = new HashMap(); - _serveTuples = new ArrayList(); - for (int i = startIndex; i < _tuples.size(); i += tasks.size()) { - _serveTuples.add(_tuples.get(i)); - } - } - - public void close() { - } - - public void nextTuple() { - if (_serveTuples.size() > 0) { - FixedTuple ft = _serveTuples.remove(0); - String id = UUID.randomUUID().toString(); - _pending.put(id, ft); - _collector.emit(ft.stream, ft.values, id); - } else { - Utils.sleep(100); - } - } - - public void ack(Object msgId) { - synchronized (acked) { - int curr = get(acked, _id, 0); - acked.put(_id, curr + 1); - } - } - - public void fail(Object msgId) { - synchronized (failed) { - int curr = get(failed, _id, 0); - failed.put(_id, curr + 1); - } - } - - @Override - public void activate() { - } - - @Override - public void deactivate() { - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - if (_fieldName != null) { - declarer.declare(new Fields(_fieldName)); - } - } - - @Override - public Map getComponentConfiguration() { - return null; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/IdentityBolt.java b/jstorm-client/src/main/java/backtype/storm/testing/IdentityBolt.java deleted file mode 100644 index b3f8d8784..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/IdentityBolt.java +++ /dev/null @@ -1,25 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.topology.BasicOutputCollector; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseBasicBolt; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; - -public class IdentityBolt extends BaseBasicBolt { - Fields _fields; - - public IdentityBolt(Fields fields) { - _fields = fields; - } - - @Override - public void execute(Tuple input, BasicOutputCollector collector) { - collector.emit(input.getValues()); - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(_fields); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/KeyedCountingBatchBolt.java b/jstorm-client/src/main/java/backtype/storm/testing/KeyedCountingBatchBolt.java deleted file mode 100644 index 58ae38019..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/KeyedCountingBatchBolt.java +++ /dev/null @@ -1,45 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseBatchBolt; -import backtype.storm.coordination.BatchOutputCollector; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; -import backtype.storm.utils.Utils; -import java.util.HashMap; -import java.util.Map; - -public class KeyedCountingBatchBolt extends BaseBatchBolt { - BatchOutputCollector _collector; - Object _id; - Map _counts = new HashMap(); - - @Override - public void prepare(Map conf, TopologyContext context, - BatchOutputCollector collector, Object id) { - _collector = collector; - _id = id; - } - - @Override - public void execute(Tuple tuple) { - Object key = tuple.getValue(1); - int curr = Utils.get(_counts, key, 0); - _counts.put(key, curr + 1); - } - - @Override - public void finishBatch() { - for (Object key : _counts.keySet()) { - _collector.emit(new Values(_id, key, _counts.get(key))); - } - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(new Fields("tx", "key", "count")); - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/KeyedCountingCommitterBolt.java b/jstorm-client/src/main/java/backtype/storm/testing/KeyedCountingCommitterBolt.java deleted file mode 100644 index a170130f2..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/KeyedCountingCommitterBolt.java +++ /dev/null @@ -1,8 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.transactional.ICommitter; - -public class KeyedCountingCommitterBolt extends KeyedCountingBatchBolt - implements ICommitter { - -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/KeyedSummingBatchBolt.java b/jstorm-client/src/main/java/backtype/storm/testing/KeyedSummingBatchBolt.java deleted file mode 100644 index c12a31905..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/KeyedSummingBatchBolt.java +++ /dev/null @@ -1,45 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.coordination.BatchOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseBatchBolt; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; -import backtype.storm.utils.Utils; -import clojure.lang.Numbers; -import java.util.HashMap; -import java.util.Map; - -public class KeyedSummingBatchBolt extends BaseBatchBolt { - BatchOutputCollector _collector; - Object _id; - Map _sums = new HashMap(); - - @Override - public void prepare(Map conf, TopologyContext context, - BatchOutputCollector collector, Object id) { - _collector = collector; - _id = id; - } - - @Override - public void execute(Tuple tuple) { - Object key = tuple.getValue(1); - Number curr = Utils.get(_sums, key, 0); - _sums.put(key, Numbers.add(curr, tuple.getValue(2))); - } - - @Override - public void finishBatch() { - for (Object key : _sums.keySet()) { - _collector.emit(new Values(_id, key, _sums.get(key))); - } - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(new Fields("tx", "key", "sum")); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/MemoryTransactionalSpout.java b/jstorm-client/src/main/java/backtype/storm/testing/MemoryTransactionalSpout.java deleted file mode 100644 index bf6286fd4..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/MemoryTransactionalSpout.java +++ /dev/null @@ -1,187 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.Config; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.transactional.TransactionAttempt; -import backtype.storm.coordination.BatchOutputCollector; -import backtype.storm.transactional.partitioned.IPartitionedTransactionalSpout; -import backtype.storm.transactional.partitioned.IPartitionedTransactionalSpout.Emitter; -import backtype.storm.tuple.Fields; -import backtype.storm.utils.RegisteredGlobalState; -import backtype.storm.utils.Utils; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class MemoryTransactionalSpout implements - IPartitionedTransactionalSpout { - public static String TX_FIELD = MemoryTransactionalSpout.class.getName() - + "/id"; - - private String _id; - private String _finishedPartitionsId; - private int _takeAmt; - private Fields _outFields; - private Map>> _initialPartitions; - - public MemoryTransactionalSpout( - Map>> partitions, Fields outFields, - int takeAmt) { - _id = RegisteredGlobalState.registerState(partitions); - Map finished = Collections - .synchronizedMap(new HashMap()); - _finishedPartitionsId = RegisteredGlobalState.registerState(finished); - _takeAmt = takeAmt; - _outFields = outFields; - _initialPartitions = partitions; - } - - public boolean isExhaustedTuples() { - Map statuses = getFinishedStatuses(); - for (Integer partition : getQueues().keySet()) { - if (!statuses.containsKey(partition) - || !getFinishedStatuses().get(partition)) { - return false; - } - } - return true; - } - - class Coordinator implements IPartitionedTransactionalSpout.Coordinator { - - @Override - public int numPartitions() { - return getQueues().size(); - } - - @Override - public boolean isReady() { - return true; - } - - @Override - public void close() { - } - } - - class Emitter - implements - IPartitionedTransactionalSpout.Emitter { - - Integer _maxSpoutPending; - Map _emptyPartitions = new HashMap(); - - public Emitter(Map conf) { - Object c = conf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING); - if (c == null) - _maxSpoutPending = 1; - else - _maxSpoutPending = Utils.getInt(c); - } - - @Override - public MemoryTransactionalSpoutMeta emitPartitionBatchNew( - TransactionAttempt tx, BatchOutputCollector collector, - int partition, MemoryTransactionalSpoutMeta lastPartitionMeta) { - int index; - if (lastPartitionMeta == null) { - index = 0; - } else { - index = lastPartitionMeta.index + lastPartitionMeta.amt; - } - List> queue = getQueues().get(partition); - int total = queue.size(); - int left = total - index; - int toTake = Math.min(left, _takeAmt); - - MemoryTransactionalSpoutMeta ret = new MemoryTransactionalSpoutMeta( - index, toTake); - emitPartitionBatch(tx, collector, partition, ret); - if (toTake == 0) { - // this is a pretty hacky way to determine when all the - // partitions have been committed - // wait until we've emitted max-spout-pending empty partitions - // for the partition - int curr = Utils.get(_emptyPartitions, partition, 0) + 1; - _emptyPartitions.put(partition, curr); - if (curr > _maxSpoutPending) { - Map finishedStatuses = getFinishedStatuses(); - // will be null in remote mode - if (finishedStatuses != null) { - finishedStatuses.put(partition, true); - } - } - } - return ret; - } - - @Override - public void emitPartitionBatch(TransactionAttempt tx, - BatchOutputCollector collector, int partition, - MemoryTransactionalSpoutMeta partitionMeta) { - List> queue = getQueues().get(partition); - for (int i = partitionMeta.index; i < partitionMeta.index - + partitionMeta.amt; i++) { - List toEmit = new ArrayList(queue.get(i)); - toEmit.add(0, tx); - collector.emit(toEmit); - } - } - - @Override - public void close() { - } - } - - @Override - public IPartitionedTransactionalSpout.Coordinator getCoordinator(Map conf, - TopologyContext context) { - return new Coordinator(); - } - - @Override - public IPartitionedTransactionalSpout.Emitter getEmitter( - Map conf, TopologyContext context) { - return new Emitter(conf); - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - List toDeclare = new ArrayList(_outFields.toList()); - toDeclare.add(0, TX_FIELD); - declarer.declare(new Fields(toDeclare)); - } - - @Override - public Map getComponentConfiguration() { - Config conf = new Config(); - conf.registerSerialization(MemoryTransactionalSpoutMeta.class); - return conf; - } - - public void startup() { - getFinishedStatuses().clear(); - } - - public void cleanup() { - RegisteredGlobalState.clearState(_id); - RegisteredGlobalState.clearState(_finishedPartitionsId); - } - - private Map>> getQueues() { - Map>> ret = (Map>>) RegisteredGlobalState - .getState(_id); - if (ret != null) - return ret; - else - return _initialPartitions; - } - - private Map getFinishedStatuses() { - return (Map) RegisteredGlobalState - .getState(_finishedPartitionsId); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/MemoryTransactionalSpoutMeta.java b/jstorm-client/src/main/java/backtype/storm/testing/MemoryTransactionalSpoutMeta.java deleted file mode 100644 index 4d87a66c0..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/MemoryTransactionalSpoutMeta.java +++ /dev/null @@ -1,21 +0,0 @@ -package backtype.storm.testing; - -public class MemoryTransactionalSpoutMeta { - int index; - int amt; - - // for kryo compatibility - public MemoryTransactionalSpoutMeta() { - - } - - public MemoryTransactionalSpoutMeta(int index, int amt) { - this.index = index; - this.amt = amt; - } - - @Override - public String toString() { - return "index: " + index + "; amt: " + amt; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/MkClusterParam.java b/jstorm-client/src/main/java/backtype/storm/testing/MkClusterParam.java deleted file mode 100644 index 8ec710222..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/MkClusterParam.java +++ /dev/null @@ -1,46 +0,0 @@ -package backtype.storm.testing; - -import java.util.Map; - -/** - * The param arg for Testing.withSimulatedTimeCluster and - * Testing.withTrackedCluster - */ -public class MkClusterParam { - /** - * count of supervisors for the cluster. - */ - private Integer supervisors; - /** - * count of port for each supervisor - */ - private Integer portsPerSupervisor; - /** - * cluster config - */ - private Map daemonConf; - - public Integer getSupervisors() { - return supervisors; - } - - public void setSupervisors(Integer supervisors) { - this.supervisors = supervisors; - } - - public Integer getPortsPerSupervisor() { - return portsPerSupervisor; - } - - public void setPortsPerSupervisor(Integer portsPerSupervisor) { - this.portsPerSupervisor = portsPerSupervisor; - } - - public Map getDaemonConf() { - return daemonConf; - } - - public void setDaemonConf(Map daemonConf) { - this.daemonConf = daemonConf; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/MkTupleParam.java b/jstorm-client/src/main/java/backtype/storm/testing/MkTupleParam.java deleted file mode 100644 index a98704d5a..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/MkTupleParam.java +++ /dev/null @@ -1,37 +0,0 @@ -package backtype.storm.testing; - -import java.util.ArrayList; -import java.util.List; - -public class MkTupleParam { - private String stream; - private String component; - private List fields; - - public String getStream() { - return stream; - } - - public void setStream(String stream) { - this.stream = stream; - } - - public String getComponent() { - return component; - } - - public void setComponent(String component) { - this.component = component; - } - - public List getFields() { - return fields; - } - - public void setFields(String... fields) { - this.fields = new ArrayList(); - for (int i = 0; i < fields.length; i++) { - this.fields.add(fields[i]); - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/MockedSources.java b/jstorm-client/src/main/java/backtype/storm/testing/MockedSources.java deleted file mode 100644 index 6c61edbc4..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/MockedSources.java +++ /dev/null @@ -1,46 +0,0 @@ -package backtype.storm.testing; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import backtype.storm.tuple.Values; -import backtype.storm.utils.Utils; - -public class MockedSources { - /** - * mocked spout sources for the [spout, stream] pair. - */ - private Map> data = new HashMap>(); - - /** - * add mock data for the spout. - * - * @param spoutId - * the spout to be mocked - * @param streamId - * the stream of the spout to be mocked - * @param objects - * the mocked data - */ - public void addMockData(String spoutId, String streamId, Values... valueses) { - if (!data.containsKey(spoutId)) { - data.put(spoutId, new ArrayList()); - } - - List tuples = data.get(spoutId); - for (int i = 0; i < valueses.length; i++) { - FixedTuple tuple = new FixedTuple(streamId, valueses[i]); - tuples.add(tuple); - } - } - - public void addMockData(String spoutId, Values... valueses) { - this.addMockData(spoutId, Utils.DEFAULT_STREAM_ID, valueses); - } - - public Map> getData() { - return this.data; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/NGrouping.java b/jstorm-client/src/main/java/backtype/storm/testing/NGrouping.java deleted file mode 100644 index 9954b06d4..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/NGrouping.java +++ /dev/null @@ -1,34 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.grouping.CustomStreamGrouping; -import backtype.storm.task.WorkerTopologyContext; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -public class NGrouping implements CustomStreamGrouping { - int _n; - List _outTasks; - - public NGrouping(int n) { - _n = n; - } - - @Override - public void prepare(WorkerTopologyContext context, GlobalStreamId stream, - List targetTasks) { - targetTasks = new ArrayList(targetTasks); - Collections.sort(targetTasks); - _outTasks = new ArrayList(); - for (int i = 0; i < _n; i++) { - _outTasks.add(targetTasks.get(i)); - } - } - - @Override - public List chooseTasks(int taskId, List values) { - return _outTasks; - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/NonRichBoltTracker.java b/jstorm-client/src/main/java/backtype/storm/testing/NonRichBoltTracker.java deleted file mode 100644 index c5918ade1..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/NonRichBoltTracker.java +++ /dev/null @@ -1,34 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.task.IBolt; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Tuple; -import backtype.storm.utils.RegisteredGlobalState; -import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; - -public class NonRichBoltTracker implements IBolt { - IBolt _delegate; - String _trackId; - - public NonRichBoltTracker(IBolt delegate, String id) { - _delegate = delegate; - _trackId = id; - } - - public void prepare(Map stormConf, TopologyContext context, - OutputCollector collector) { - _delegate.prepare(stormConf, context, collector); - } - - public void execute(Tuple input) { - _delegate.execute(input); - Map stats = (Map) RegisteredGlobalState.getState(_trackId); - ((AtomicInteger) stats.get("processed")).incrementAndGet(); - } - - public void cleanup() { - _delegate.cleanup(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/OpaqueMemoryTransactionalSpout.java b/jstorm-client/src/main/java/backtype/storm/testing/OpaqueMemoryTransactionalSpout.java deleted file mode 100644 index 371c6223d..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/OpaqueMemoryTransactionalSpout.java +++ /dev/null @@ -1,190 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.Config; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.transactional.TransactionAttempt; -import backtype.storm.coordination.BatchOutputCollector; -import backtype.storm.transactional.partitioned.IOpaquePartitionedTransactionalSpout; -import backtype.storm.tuple.Fields; -import backtype.storm.utils.RegisteredGlobalState; -import backtype.storm.utils.Utils; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * This spout only works in local mode. - */ -public class OpaqueMemoryTransactionalSpout implements - IOpaquePartitionedTransactionalSpout { - public static String TX_FIELD = MemoryTransactionalSpout.class.getName() - + "/id"; - - private String _id; - private String _finishedPartitionsId; - private String _disabledId; - private int _takeAmt; - private Fields _outFields; - - public OpaqueMemoryTransactionalSpout( - Map>> partitions, Fields outFields, - int takeAmt) { - _id = RegisteredGlobalState.registerState(partitions); - - Map finished = Collections - .synchronizedMap(new HashMap()); - _finishedPartitionsId = RegisteredGlobalState.registerState(finished); - - Map disabled = Collections - .synchronizedMap(new HashMap()); - _disabledId = RegisteredGlobalState.registerState(disabled); - - _takeAmt = takeAmt; - _outFields = outFields; - } - - public void setDisabled(Integer partition, boolean disabled) { - getDisabledStatuses().put(partition, disabled); - } - - public boolean isExhaustedTuples() { - Map statuses = getFinishedStatuses(); - for (Integer partition : getQueues().keySet()) { - if (!statuses.containsKey(partition) - || !getFinishedStatuses().get(partition)) { - return false; - } - } - return true; - } - - @Override - public IOpaquePartitionedTransactionalSpout.Emitter getEmitter( - Map conf, TopologyContext context) { - return new Emitter(conf); - } - - @Override - public IOpaquePartitionedTransactionalSpout.Coordinator getCoordinator( - Map conf, TopologyContext context) { - return new Coordinator(); - } - - class Coordinator implements - IOpaquePartitionedTransactionalSpout.Coordinator { - @Override - public boolean isReady() { - return true; - } - - @Override - public void close() { - } - } - - class Emitter - implements - IOpaquePartitionedTransactionalSpout.Emitter { - - Integer _maxSpoutPending; - Map _emptyPartitions = new HashMap(); - - public Emitter(Map conf) { - Object c = conf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING); - if (c == null) - _maxSpoutPending = 1; - else - _maxSpoutPending = Utils.getInt(c); - } - - @Override - public MemoryTransactionalSpoutMeta emitPartitionBatch( - TransactionAttempt tx, BatchOutputCollector collector, - int partition, MemoryTransactionalSpoutMeta lastPartitionMeta) { - if (!Boolean.FALSE.equals(getDisabledStatuses().get(partition))) { - int index; - if (lastPartitionMeta == null) { - index = 0; - } else { - index = lastPartitionMeta.index + lastPartitionMeta.amt; - } - List> queue = getQueues().get(partition); - int total = queue.size(); - int left = total - index; - int toTake = Math.min(left, _takeAmt); - - MemoryTransactionalSpoutMeta ret = new MemoryTransactionalSpoutMeta( - index, toTake); - for (int i = ret.index; i < ret.index + ret.amt; i++) { - List toEmit = new ArrayList(queue.get(i)); - toEmit.add(0, tx); - collector.emit(toEmit); - } - if (toTake == 0) { - // this is a pretty hacky way to determine when all the - // partitions have been committed - // wait until we've emitted max-spout-pending empty - // partitions for the partition - int curr = Utils.get(_emptyPartitions, partition, 0) + 1; - _emptyPartitions.put(partition, curr); - if (curr > _maxSpoutPending) { - getFinishedStatuses().put(partition, true); - } - } - return ret; - } else { - return null; - } - } - - @Override - public void close() { - } - - @Override - public int numPartitions() { - return getQueues().size(); - } - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - List toDeclare = new ArrayList(_outFields.toList()); - toDeclare.add(0, TX_FIELD); - declarer.declare(new Fields(toDeclare)); - } - - @Override - public Map getComponentConfiguration() { - Config conf = new Config(); - conf.registerSerialization(MemoryTransactionalSpoutMeta.class); - return conf; - } - - public void startup() { - getFinishedStatuses().clear(); - } - - public void cleanup() { - RegisteredGlobalState.clearState(_id); - RegisteredGlobalState.clearState(_finishedPartitionsId); - } - - private Map>> getQueues() { - return (Map>>) RegisteredGlobalState - .getState(_id); - } - - private Map getFinishedStatuses() { - return (Map) RegisteredGlobalState - .getState(_finishedPartitionsId); - } - - private Map getDisabledStatuses() { - return (Map) RegisteredGlobalState - .getState(_disabledId); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/PrepareBatchBolt.java b/jstorm-client/src/main/java/backtype/storm/testing/PrepareBatchBolt.java deleted file mode 100644 index 207455b7c..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/PrepareBatchBolt.java +++ /dev/null @@ -1,33 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.topology.BasicOutputCollector; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseBasicBolt; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.utils.Utils; -import java.util.ArrayList; -import java.util.List; - -public class PrepareBatchBolt extends BaseBasicBolt { - Fields _outFields; - - public PrepareBatchBolt(Fields outFields) { - _outFields = outFields; - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(_outFields); - } - - @Override - public void execute(Tuple input, BasicOutputCollector collector) { - long id = Utils.secureRandomLong(); - List toEmit = new ArrayList(); - toEmit.add(id); - toEmit.addAll(input.getValues()); - collector.emit(toEmit); - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/SpoutTracker.java b/jstorm-client/src/main/java/backtype/storm/testing/SpoutTracker.java deleted file mode 100644 index a712ee809..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/SpoutTracker.java +++ /dev/null @@ -1,88 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.spout.ISpoutOutputCollector; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichSpout; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseRichSpout; -import backtype.storm.utils.RegisteredGlobalState; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; - -public class SpoutTracker extends BaseRichSpout { - IRichSpout _delegate; - SpoutTrackOutputCollector _tracker; - String _trackId; - - private class SpoutTrackOutputCollector implements ISpoutOutputCollector { - public int transferred = 0; - public int emitted = 0; - public SpoutOutputCollector _collector; - - public SpoutTrackOutputCollector(SpoutOutputCollector collector) { - _collector = collector; - } - - private void recordSpoutEmit() { - Map stats = (Map) RegisteredGlobalState.getState(_trackId); - ((AtomicInteger) stats.get("spout-emitted")).incrementAndGet(); - - } - - public List emit(String streamId, List tuple, - Object messageId) { - List ret = _collector.emit(streamId, tuple, messageId); - recordSpoutEmit(); - return ret; - } - - public void emitDirect(int taskId, String streamId, List tuple, - Object messageId) { - _collector.emitDirect(taskId, streamId, tuple, messageId); - recordSpoutEmit(); - } - - @Override - public void reportError(Throwable error) { - _collector.reportError(error); - } - } - - public SpoutTracker(IRichSpout delegate, String trackId) { - _delegate = delegate; - _trackId = trackId; - } - - public void open(Map conf, TopologyContext context, - SpoutOutputCollector collector) { - _tracker = new SpoutTrackOutputCollector(collector); - _delegate.open(conf, context, new SpoutOutputCollector(_tracker)); - } - - public void close() { - _delegate.close(); - } - - public void nextTuple() { - _delegate.nextTuple(); - } - - public void ack(Object msgId) { - _delegate.ack(msgId); - Map stats = (Map) RegisteredGlobalState.getState(_trackId); - ((AtomicInteger) stats.get("processed")).incrementAndGet(); - } - - public void fail(Object msgId) { - _delegate.fail(msgId); - Map stats = (Map) RegisteredGlobalState.getState(_trackId); - ((AtomicInteger) stats.get("processed")).incrementAndGet(); - } - - public void declareOutputFields(OutputFieldsDeclarer declarer) { - _delegate.declareOutputFields(declarer); - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/TestAggregatesCounter.java b/jstorm-client/src/main/java/backtype/storm/testing/TestAggregatesCounter.java deleted file mode 100644 index 359fa0000..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/TestAggregatesCounter.java +++ /dev/null @@ -1,46 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.topology.base.BaseRichBolt; -import backtype.storm.task.OutputCollector; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Fields; -import java.util.Map; -import backtype.storm.task.TopologyContext; -import java.util.HashMap; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import static backtype.storm.utils.Utils.tuple; - -public class TestAggregatesCounter extends BaseRichBolt { - public static Logger LOG = LoggerFactory.getLogger(TestWordCounter.class); - - Map _counts; - OutputCollector _collector; - - public void prepare(Map stormConf, TopologyContext context, - OutputCollector collector) { - _collector = collector; - _counts = new HashMap(); - } - - public void execute(Tuple input) { - String word = (String) input.getValues().get(0); - int count = (Integer) input.getValues().get(1); - _counts.put(word, count); - int globalCount = 0; - for (String w : _counts.keySet()) { - globalCount += _counts.get(w); - } - _collector.emit(tuple(globalCount)); - _collector.ack(input); - } - - public void cleanup() { - - } - - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(new Fields("agg-global")); - } -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/testing/TestConfBolt.java b/jstorm-client/src/main/java/backtype/storm/testing/TestConfBolt.java deleted file mode 100644 index c6fe3d64e..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/TestConfBolt.java +++ /dev/null @@ -1,44 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.BasicOutputCollector; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseBasicBolt; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; -import java.util.Map; - -public class TestConfBolt extends BaseBasicBolt { - Map _componentConf; - Map _conf; - - public TestConfBolt() { - this(null); - } - - public TestConfBolt(Map componentConf) { - _componentConf = componentConf; - } - - @Override - public void prepare(Map conf, TopologyContext context) { - _conf = conf; - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(new Fields("conf", "value")); - } - - @Override - public void execute(Tuple input, BasicOutputCollector collector) { - String name = input.getString(0); - collector.emit(new Values(name, _conf.get(name))); - } - - @Override - public Map getComponentConfiguration() { - return _componentConf; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/TestGlobalCount.java b/jstorm-client/src/main/java/backtype/storm/testing/TestGlobalCount.java deleted file mode 100644 index 9f9e42198..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/TestGlobalCount.java +++ /dev/null @@ -1,43 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.topology.base.BaseRichBolt; -import backtype.storm.task.OutputCollector; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Fields; -import java.util.Map; -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Values; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TestGlobalCount extends BaseRichBolt { - public static Logger LOG = LoggerFactory.getLogger(TestWordCounter.class); - - private int _count; - OutputCollector _collector; - - public void prepare(Map stormConf, TopologyContext context, - OutputCollector collector) { - _collector = collector; - _count = 0; - } - - public void execute(Tuple input) { - _count++; - _collector.emit(input, new Values(_count)); - _collector.ack(input); - } - - public void cleanup() { - - } - - public Fields getOutputFields() { - return new Fields("global-count"); - } - - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(new Fields("global-count")); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/TestJob.java b/jstorm-client/src/main/java/backtype/storm/testing/TestJob.java deleted file mode 100644 index 35a2592ce..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/TestJob.java +++ /dev/null @@ -1,24 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.ILocalCluster; - -/** - * This is the core interface for the storm java testing, usually we put our - * java unit testing logic in the run method. A sample code will be: - * Testing.withSimulatedTimeLocalCluster(new TestJob() { - * public void run(Cluster cluster) { - * // your testing logic here. - * } - * }); - */ -public interface TestJob { - /** - * run the testing logic with the cluster. - * - * @param cluster - * the cluster which created by - * Testing.withSimulatedTimeLocalCluster and - * Testing.withTrackedCluster. - */ - public void run(ILocalCluster cluster) throws Exception; -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/TestKryoDecorator.java b/jstorm-client/src/main/java/backtype/storm/testing/TestKryoDecorator.java deleted file mode 100644 index fd9053c21..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/TestKryoDecorator.java +++ /dev/null @@ -1,14 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.serialization.IKryoDecorator; -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.Serializer; -import com.esotericsoftware.kryo.io.Input; -import com.esotericsoftware.kryo.io.Output; - -public class TestKryoDecorator implements IKryoDecorator { - - public void decorate(Kryo k) { - k.register(TestSerObject.class); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/TestPlannerBolt.java b/jstorm-client/src/main/java/backtype/storm/testing/TestPlannerBolt.java deleted file mode 100644 index 69e175e7b..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/TestPlannerBolt.java +++ /dev/null @@ -1,28 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.task.OutputCollector; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Fields; -import java.util.Map; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.base.BaseRichBolt; - -public class TestPlannerBolt extends BaseRichBolt { - public void prepare(Map stormConf, TopologyContext context, - OutputCollector collector) { - - } - - public void execute(Tuple input) { - - } - - public Fields getOutputFields() { - return new Fields("field1", "field2"); - } - - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(getOutputFields()); - } -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/testing/TestPlannerSpout.java b/jstorm-client/src/main/java/backtype/storm/testing/TestPlannerSpout.java deleted file mode 100644 index c00dc45f4..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/TestPlannerSpout.java +++ /dev/null @@ -1,67 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.Config; -import backtype.storm.topology.OutputFieldsDeclarer; -import java.util.Map; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.base.BaseRichSpout; -import backtype.storm.tuple.Fields; -import backtype.storm.utils.Utils; -import java.util.HashMap; - -public class TestPlannerSpout extends BaseRichSpout { - boolean _isDistributed; - Fields _outFields; - - public TestPlannerSpout(Fields outFields, boolean isDistributed) { - _isDistributed = isDistributed; - _outFields = outFields; - } - - public TestPlannerSpout(boolean isDistributed) { - this(new Fields("field1", "field2"), isDistributed); - } - - public TestPlannerSpout(Fields outFields) { - this(outFields, true); - } - - public Fields getOutputFields() { - return _outFields; - } - - public void open(Map conf, TopologyContext context, - SpoutOutputCollector collector) { - - } - - public void close() { - - } - - public void nextTuple() { - Utils.sleep(100); - } - - public void ack(Object msgId) { - - } - - public void fail(Object msgId) { - - } - - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(getOutputFields()); - } - - @Override - public Map getComponentConfiguration() { - Map ret = new HashMap(); - if (!_isDistributed) { - ret.put(Config.TOPOLOGY_MAX_TASK_PARALLELISM, 1); - } - return ret; - } -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/testing/TestSerObject.java b/jstorm-client/src/main/java/backtype/storm/testing/TestSerObject.java deleted file mode 100644 index ab356a334..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/TestSerObject.java +++ /dev/null @@ -1,40 +0,0 @@ -package backtype.storm.testing; - -import java.io.Serializable; - -public class TestSerObject implements Serializable { - public int f1; - public int f2; - - public TestSerObject(int f1, int f2) { - this.f1 = f1; - this.f2 = f2; - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + f1; - result = prime * result + f2; - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - TestSerObject other = (TestSerObject) obj; - if (f1 != other.f1) - return false; - if (f2 != other.f2) - return false; - return true; - } - - -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/TestWordCounter.java b/jstorm-client/src/main/java/backtype/storm/testing/TestWordCounter.java deleted file mode 100644 index 3572b9de4..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/TestWordCounter.java +++ /dev/null @@ -1,43 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.topology.base.BaseBasicBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Fields; -import java.util.Map; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.BasicOutputCollector; -import java.util.HashMap; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import static backtype.storm.utils.Utils.tuple; - -public class TestWordCounter extends BaseBasicBolt { - public static Logger LOG = LoggerFactory.getLogger(TestWordCounter.class); - - Map _counts; - - public void prepare(Map stormConf, TopologyContext context) { - _counts = new HashMap(); - } - - public void execute(Tuple input, BasicOutputCollector collector) { - String word = (String) input.getValues().get(0); - int count = 0; - if (_counts.containsKey(word)) { - count = _counts.get(word); - } - count++; - _counts.put(word, count); - collector.emit(tuple(word, count)); - } - - public void cleanup() { - - } - - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(new Fields("word", "count")); - } - -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/testing/TestWordSpout.java b/jstorm-client/src/main/java/backtype/storm/testing/TestWordSpout.java deleted file mode 100644 index 432e5de9d..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/TestWordSpout.java +++ /dev/null @@ -1,70 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.Config; -import backtype.storm.topology.OutputFieldsDeclarer; -import java.util.Map; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.base.BaseRichSpout; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Values; -import backtype.storm.utils.Utils; -import java.util.HashMap; -import java.util.Random; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TestWordSpout extends BaseRichSpout { - public static Logger LOG = LoggerFactory.getLogger(TestWordSpout.class); - boolean _isDistributed; - SpoutOutputCollector _collector; - - public TestWordSpout() { - this(true); - } - - public TestWordSpout(boolean isDistributed) { - _isDistributed = isDistributed; - } - - public void open(Map conf, TopologyContext context, - SpoutOutputCollector collector) { - _collector = collector; - } - - public void close() { - - } - - public void nextTuple() { - Utils.sleep(100); - final String[] words = new String[] { "nathan", "mike", "jackson", - "golda", "bertels" }; - final Random rand = new Random(); - final String word = words[rand.nextInt(words.length)]; - _collector.emit(new Values(word)); - } - - public void ack(Object msgId) { - - } - - public void fail(Object msgId) { - - } - - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(new Fields("word")); - } - - @Override - public Map getComponentConfiguration() { - if (!_isDistributed) { - Map ret = new HashMap(); - ret.put(Config.TOPOLOGY_MAX_TASK_PARALLELISM, 1); - return ret; - } else { - return null; - } - } -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/testing/TrackedTopology.java b/jstorm-client/src/main/java/backtype/storm/testing/TrackedTopology.java deleted file mode 100644 index dc98ba5df..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/TrackedTopology.java +++ /dev/null @@ -1,17 +0,0 @@ -package backtype.storm.testing; - -import java.util.HashMap; -import java.util.Map; - -import backtype.storm.generated.StormTopology; -import clojure.lang.Keyword; - -public class TrackedTopology extends HashMap { - public TrackedTopology(Map map) { - super(map); - } - - public StormTopology getTopology() { - return (StormTopology) get(Keyword.intern("topology")); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/testing/TupleCaptureBolt.java b/jstorm-client/src/main/java/backtype/storm/testing/TupleCaptureBolt.java deleted file mode 100644 index 33c07b70d..000000000 --- a/jstorm-client/src/main/java/backtype/storm/testing/TupleCaptureBolt.java +++ /dev/null @@ -1,68 +0,0 @@ -package backtype.storm.testing; - -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Tuple; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; - -public class TupleCaptureBolt implements IRichBolt { - public static transient Map>> emitted_tuples = new HashMap>>(); - - private String _name; - private transient OutputCollector _collector; - - public TupleCaptureBolt() { - _name = UUID.randomUUID().toString(); - emitted_tuples.put(_name, new HashMap>()); - } - - public void prepare(Map stormConf, TopologyContext context, - OutputCollector collector) { - _collector = collector; - } - - public void execute(Tuple input) { - String component = input.getSourceComponent(); - Map> captured = emitted_tuples.get(_name); - if (!captured.containsKey(component)) { - captured.put(component, new ArrayList()); - } - captured.get(component).add( - new FixedTuple(input.getSourceStreamId(), input.getValues())); - _collector.ack(input); - } - - public Map> getResults() { - return emitted_tuples.get(_name); - } - - public void cleanup() { - } - - public Map> getAndRemoveResults() { - return emitted_tuples.remove(_name); - } - - public Map> getAndClearResults() { - Map> ret = new HashMap>( - emitted_tuples.get(_name)); - emitted_tuples.get(_name).clear(); - return ret; - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - } - - @Override - public Map getComponentConfiguration() { - return null; - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/BaseConfigurationDeclarer.java b/jstorm-client/src/main/java/backtype/storm/topology/BaseConfigurationDeclarer.java deleted file mode 100644 index fcf90b55c..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/BaseConfigurationDeclarer.java +++ /dev/null @@ -1,41 +0,0 @@ -package backtype.storm.topology; - -import backtype.storm.Config; -import java.util.HashMap; -import java.util.Map; - -public abstract class BaseConfigurationDeclarer - implements ComponentConfigurationDeclarer { - @Override - public T addConfiguration(String config, Object value) { - Map configMap = new HashMap(); - configMap.put(config, value); - return addConfigurations(configMap); - } - - @Override - public T setDebug(boolean debug) { - return addConfiguration(Config.TOPOLOGY_DEBUG, debug); - } - - @Override - public T setMaxTaskParallelism(Number val) { - if (val != null) - val = val.intValue(); - return addConfiguration(Config.TOPOLOGY_MAX_TASK_PARALLELISM, val); - } - - @Override - public T setMaxSpoutPending(Number val) { - if (val != null) - val = val.intValue(); - return addConfiguration(Config.TOPOLOGY_MAX_SPOUT_PENDING, val); - } - - @Override - public T setNumTasks(Number val) { - if (val != null) - val = val.intValue(); - return addConfiguration(Config.TOPOLOGY_TASKS, val); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/BasicBoltExecutor.java b/jstorm-client/src/main/java/backtype/storm/topology/BasicBoltExecutor.java deleted file mode 100644 index 586f17ce5..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/BasicBoltExecutor.java +++ /dev/null @@ -1,50 +0,0 @@ -package backtype.storm.topology; - -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Tuple; -import java.util.Map; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class BasicBoltExecutor implements IRichBolt { - public static Logger LOG = LoggerFactory.getLogger(BasicBoltExecutor.class); - - private IBasicBolt _bolt; - private transient BasicOutputCollector _collector; - - public BasicBoltExecutor(IBasicBolt bolt) { - _bolt = bolt; - } - - public void declareOutputFields(OutputFieldsDeclarer declarer) { - _bolt.declareOutputFields(declarer); - } - - public void prepare(Map stormConf, TopologyContext context, - OutputCollector collector) { - _bolt.prepare(stormConf, context); - _collector = new BasicOutputCollector(collector); - } - - public void execute(Tuple input) { - _collector.setContext(input); - try { - _bolt.execute(input, _collector); - _collector.getOutputter().ack(input); - } catch (FailedException e) { - if (e instanceof ReportedFailedException) { - _collector.reportError(e); - } - _collector.getOutputter().fail(input); - } - } - - public void cleanup() { - _bolt.cleanup(); - } - - public Map getComponentConfiguration() { - return _bolt.getComponentConfiguration(); - } -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/topology/BasicOutputCollector.java b/jstorm-client/src/main/java/backtype/storm/topology/BasicOutputCollector.java deleted file mode 100644 index a0a892f11..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/BasicOutputCollector.java +++ /dev/null @@ -1,44 +0,0 @@ -package backtype.storm.topology; - -import backtype.storm.task.IOutputCollector; -import backtype.storm.task.OutputCollector; -import backtype.storm.tuple.Tuple; -import backtype.storm.utils.Utils; -import java.util.List; - -public class BasicOutputCollector implements IBasicOutputCollector { - private OutputCollector out; - private Tuple inputTuple; - - public BasicOutputCollector(OutputCollector out) { - this.out = out; - } - - public List emit(String streamId, List tuple) { - return out.emit(streamId, inputTuple, tuple); - } - - public List emit(List tuple) { - return emit(Utils.DEFAULT_STREAM_ID, tuple); - } - - public void setContext(Tuple inputTuple) { - this.inputTuple = inputTuple; - } - - public void emitDirect(int taskId, String streamId, List tuple) { - out.emitDirect(taskId, streamId, inputTuple, tuple); - } - - public void emitDirect(int taskId, List tuple) { - emitDirect(taskId, Utils.DEFAULT_STREAM_ID, tuple); - } - - protected IOutputCollector getOutputter() { - return out; - } - - public void reportError(Throwable t) { - out.reportError(t); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/BoltDeclarer.java b/jstorm-client/src/main/java/backtype/storm/topology/BoltDeclarer.java deleted file mode 100644 index 0e9cd58cf..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/BoltDeclarer.java +++ /dev/null @@ -1,6 +0,0 @@ -package backtype.storm.topology; - -public interface BoltDeclarer extends InputDeclarer, - ComponentConfigurationDeclarer { - -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/ComponentConfigurationDeclarer.java b/jstorm-client/src/main/java/backtype/storm/topology/ComponentConfigurationDeclarer.java deleted file mode 100644 index 3c599802d..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/ComponentConfigurationDeclarer.java +++ /dev/null @@ -1,18 +0,0 @@ -package backtype.storm.topology; - -import java.util.Map; - -public interface ComponentConfigurationDeclarer { - T addConfigurations(Map conf); - - T addConfiguration(String config, Object value); - - T setDebug(boolean debug); - - T setMaxTaskParallelism(Number val); - - T setMaxSpoutPending(Number val); - - @Deprecated - T setNumTasks(Number val); -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/FailedException.java b/jstorm-client/src/main/java/backtype/storm/topology/FailedException.java deleted file mode 100644 index 793f53ec3..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/FailedException.java +++ /dev/null @@ -1,19 +0,0 @@ -package backtype.storm.topology; - -public class FailedException extends RuntimeException { - public FailedException() { - super(); - } - - public FailedException(String msg) { - super(msg); - } - - public FailedException(String msg, Throwable cause) { - super(msg, cause); - } - - public FailedException(Throwable cause) { - super(cause); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/IBasicBolt.java b/jstorm-client/src/main/java/backtype/storm/topology/IBasicBolt.java deleted file mode 100644 index e5f303e98..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/IBasicBolt.java +++ /dev/null @@ -1,20 +0,0 @@ -package backtype.storm.topology; - -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Tuple; -import java.util.Map; - -public interface IBasicBolt extends IComponent { - void prepare(Map stormConf, TopologyContext context); - - /** - * Process the input tuple and optionally emit new tuples based on the input - * tuple. - * - * All acking is managed for you. Throw a FailedException if you want to - * fail the tuple. - */ - void execute(Tuple input, BasicOutputCollector collector); - - void cleanup(); -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/topology/IBasicOutputCollector.java b/jstorm-client/src/main/java/backtype/storm/topology/IBasicOutputCollector.java deleted file mode 100644 index 44c0fe164..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/IBasicOutputCollector.java +++ /dev/null @@ -1,11 +0,0 @@ -package backtype.storm.topology; - -import java.util.List; - -public interface IBasicOutputCollector { - List emit(String streamId, List tuple); - - void emitDirect(int taskId, String streamId, List tuple); - - void reportError(Throwable t); -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/IComponent.java b/jstorm-client/src/main/java/backtype/storm/topology/IComponent.java deleted file mode 100644 index 88f1450d8..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/IComponent.java +++ /dev/null @@ -1,30 +0,0 @@ -package backtype.storm.topology; - -import java.io.Serializable; -import java.util.Map; - -/** - * Common methods for all possible components in a topology. This interface is - * used when defining topologies using the Java API. - */ -public interface IComponent extends Serializable { - - /** - * Declare the output schema for all the streams of this topology. - * - * @param declarer - * this is used to declare output stream ids, output fields, and - * whether or not each output stream is a direct stream - */ - void declareOutputFields(OutputFieldsDeclarer declarer); - - /** - * Declare configuration specific to this component. Only a subset of the - * "topology.*" configs can be overridden. The component configuration can - * be further overridden when constructing the topology using - * {@link TopologyBuilder} - * - */ - Map getComponentConfiguration(); - -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/IRichBolt.java b/jstorm-client/src/main/java/backtype/storm/topology/IRichBolt.java deleted file mode 100644 index b79126ced..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/IRichBolt.java +++ /dev/null @@ -1,12 +0,0 @@ -package backtype.storm.topology; - -import backtype.storm.task.IBolt; - -/** - * When writing topologies using Java, {@link IRichBolt} and {@link IRichSpout} - * are the main interfaces to use to implement components of the topology. - * - */ -public interface IRichBolt extends IBolt, IComponent { - -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/IRichSpout.java b/jstorm-client/src/main/java/backtype/storm/topology/IRichSpout.java deleted file mode 100644 index 6953f6618..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/IRichSpout.java +++ /dev/null @@ -1,12 +0,0 @@ -package backtype.storm.topology; - -import backtype.storm.spout.ISpout; - -/** - * When writing topologies using Java, {@link IRichBolt} and {@link IRichSpout} - * are the main interfaces to use to implement components of the topology. - * - */ -public interface IRichSpout extends ISpout, IComponent { - -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/IRichStateSpout.java b/jstorm-client/src/main/java/backtype/storm/topology/IRichStateSpout.java deleted file mode 100644 index ffc2ec289..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/IRichStateSpout.java +++ /dev/null @@ -1,7 +0,0 @@ -package backtype.storm.topology; - -import backtype.storm.state.IStateSpout; - -public interface IRichStateSpout extends IStateSpout, IComponent { - -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/InputDeclarer.java b/jstorm-client/src/main/java/backtype/storm/topology/InputDeclarer.java deleted file mode 100644 index b97dacacd..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/InputDeclarer.java +++ /dev/null @@ -1,48 +0,0 @@ -package backtype.storm.topology; - -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.generated.Grouping; -import backtype.storm.grouping.CustomStreamGrouping; -import backtype.storm.tuple.Fields; - -public interface InputDeclarer { - public T fieldsGrouping(String componentId, Fields fields); - - public T fieldsGrouping(String componentId, String streamId, Fields fields); - - public T globalGrouping(String componentId); - - public T globalGrouping(String componentId, String streamId); - - public T shuffleGrouping(String componentId); - - public T shuffleGrouping(String componentId, String streamId); - - public T localOrShuffleGrouping(String componentId); - - public T localOrShuffleGrouping(String componentId, String streamId); - - public T localFirstGrouping(String componentId); - - public T localFirstGrouping(String componentId, String streamId); - - public T noneGrouping(String componentId); - - public T noneGrouping(String componentId, String streamId); - - public T allGrouping(String componentId); - - public T allGrouping(String componentId, String streamId); - - public T directGrouping(String componentId); - - public T directGrouping(String componentId, String streamId); - - public T customGrouping(String componentId, CustomStreamGrouping grouping); - - public T customGrouping(String componentId, String streamId, - CustomStreamGrouping grouping); - - public T grouping(GlobalStreamId id, Grouping grouping); - -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/OutputFieldsDeclarer.java b/jstorm-client/src/main/java/backtype/storm/topology/OutputFieldsDeclarer.java deleted file mode 100644 index a981cc625..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/OutputFieldsDeclarer.java +++ /dev/null @@ -1,16 +0,0 @@ -package backtype.storm.topology; - -import backtype.storm.tuple.Fields; - -public interface OutputFieldsDeclarer { - /** - * Uses default stream id. - */ - public void declare(Fields fields); - - public void declare(boolean direct, Fields fields); - - public void declareStream(String streamId, Fields fields); - - public void declareStream(String streamId, boolean direct, Fields fields); -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/OutputFieldsGetter.java b/jstorm-client/src/main/java/backtype/storm/topology/OutputFieldsGetter.java deleted file mode 100644 index 1a6c0c2cb..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/OutputFieldsGetter.java +++ /dev/null @@ -1,36 +0,0 @@ -package backtype.storm.topology; - -import backtype.storm.generated.StreamInfo; -import backtype.storm.tuple.Fields; -import backtype.storm.utils.Utils; -import java.util.HashMap; -import java.util.Map; - -public class OutputFieldsGetter implements OutputFieldsDeclarer { - private Map _fields = new HashMap(); - - public void declare(Fields fields) { - declare(false, fields); - } - - public void declare(boolean direct, Fields fields) { - declareStream(Utils.DEFAULT_STREAM_ID, direct, fields); - } - - public void declareStream(String streamId, Fields fields) { - declareStream(streamId, false, fields); - } - - public void declareStream(String streamId, boolean direct, Fields fields) { - if (_fields.containsKey(streamId)) { - throw new IllegalArgumentException("Fields for " + streamId - + " already set"); - } - _fields.put(streamId, new StreamInfo(fields.toList(), direct)); - } - - public Map getFieldsDeclaration() { - return _fields; - } - -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/topology/ReportedFailedException.java b/jstorm-client/src/main/java/backtype/storm/topology/ReportedFailedException.java deleted file mode 100644 index 74bd0de4d..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/ReportedFailedException.java +++ /dev/null @@ -1,19 +0,0 @@ -package backtype.storm.topology; - -public class ReportedFailedException extends FailedException { - public ReportedFailedException() { - super(); - } - - public ReportedFailedException(String msg) { - super(msg); - } - - public ReportedFailedException(String msg, Throwable cause) { - super(msg, cause); - } - - public ReportedFailedException(Throwable cause) { - super(cause); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/SpoutDeclarer.java b/jstorm-client/src/main/java/backtype/storm/topology/SpoutDeclarer.java deleted file mode 100644 index b14d496b1..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/SpoutDeclarer.java +++ /dev/null @@ -1,6 +0,0 @@ -package backtype.storm.topology; - -public interface SpoutDeclarer extends - ComponentConfigurationDeclarer { - -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/TopologyBuilder.java b/jstorm-client/src/main/java/backtype/storm/topology/TopologyBuilder.java deleted file mode 100644 index 2b2f55ba5..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/TopologyBuilder.java +++ /dev/null @@ -1,424 +0,0 @@ -package backtype.storm.topology; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Map; - -import backtype.storm.Config; -import backtype.storm.generated.Bolt; -import backtype.storm.generated.ComponentCommon; -import backtype.storm.generated.ComponentObject; -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.generated.Grouping; -import backtype.storm.generated.NullStruct; -import backtype.storm.generated.SpoutSpec; -import backtype.storm.generated.StateSpoutSpec; -import backtype.storm.generated.StormTopology; -import backtype.storm.grouping.CustomStreamGrouping; -import backtype.storm.tuple.Fields; -import backtype.storm.utils.Utils; - -/** - * TopologyBuilder exposes the Java API for specifying a topology for Storm to - * execute. Topologies are Thrift structures in the end, but since the Thrift - * API is so verbose, TopologyBuilder greatly eases the process of creating - * topologies. The template for creating and submitting a topology looks - * something like: - * - *
- * TopologyBuilder builder = new TopologyBuilder();
- * 
- * builder.setSpout("1", new TestWordSpout(true), 5);
- * builder.setSpout("2", new TestWordSpout(true), 3);
- * builder.setBolt("3", new TestWordCounter(), 3)
- * 		.fieldsGrouping("1", new Fields("word"))
- * 		.fieldsGrouping("2", new Fields("word"));
- * builder.setBolt("4", new TestGlobalCount()).globalGrouping("1");
- * 
- * Map conf = new HashMap();
- * conf.put(Config.TOPOLOGY_WORKERS, 4);
- * 
- * StormSubmitter.submitTopology("mytopology", conf, builder.createTopology());
- * 
- * - * Running the exact same topology in local mode (in process), and configuring - * it to log all tuples emitted, looks like the following. Note that it lets the - * topology run for 10 seconds before shutting down the local cluster. - * - *
- * TopologyBuilder builder = new TopologyBuilder();
- * 
- * builder.setSpout("1", new TestWordSpout(true), 5);
- * builder.setSpout("2", new TestWordSpout(true), 3);
- * builder.setBolt("3", new TestWordCounter(), 3)
- * 		.fieldsGrouping("1", new Fields("word"))
- * 		.fieldsGrouping("2", new Fields("word"));
- * builder.setBolt("4", new TestGlobalCount()).globalGrouping("1");
- * 
- * Map conf = new HashMap();
- * conf.put(Config.TOPOLOGY_WORKERS, 4);
- * conf.put(Config.TOPOLOGY_DEBUG, true);
- * 
- * LocalCluster cluster = new LocalCluster();
- * cluster.submitTopology("mytopology", conf, builder.createTopology());
- * Utils.sleep(10000);
- * cluster.shutdown();
- * 
- * - *

- * The pattern for TopologyBuilder is to map component ids to components using - * the setSpout and setBolt methods. Those methods return objects that are then - * used to declare the inputs for that component. - *

- */ -public class TopologyBuilder { - private Map _bolts = new HashMap(); - private Map _spouts = new HashMap(); - private Map _commons = new HashMap(); - - // private Map> _inputs = new - // HashMap>(); - - private Map _stateSpouts = new HashMap(); - - public StormTopology createTopology() { - Map boltSpecs = new HashMap(); - Map spoutSpecs = new HashMap(); - for (String boltId : _bolts.keySet()) { - IRichBolt bolt = _bolts.get(boltId); - ComponentCommon common = getComponentCommon(boltId, bolt); - boltSpecs.put( - boltId, - new Bolt(ComponentObject.serialized_java(Utils - .serialize(bolt)), common)); - } - for (String spoutId : _spouts.keySet()) { - IRichSpout spout = _spouts.get(spoutId); - ComponentCommon common = getComponentCommon(spoutId, spout); - spoutSpecs.put( - spoutId, - new SpoutSpec(ComponentObject.serialized_java(Utils - .serialize(spout)), common)); - - } - return new StormTopology(spoutSpecs, boltSpecs, - new HashMap()); - } - - /** - * Define a new bolt in this topology with parallelism of just one thread. - * - * @param id - * the id of this component. This id is referenced by other - * components that want to consume this bolt's outputs. - * @param bolt - * the bolt - * @return use the returned object to declare the inputs to this component - */ - public BoltDeclarer setBolt(String id, IRichBolt bolt) { - return setBolt(id, bolt, null); - } - - /** - * Define a new bolt in this topology with the specified amount of - * parallelism. - * - * @param id - * the id of this component. This id is referenced by other - * components that want to consume this bolt's outputs. - * @param bolt - * the bolt - * @param parallelism_hint - * the number of tasks that should be assigned to execute this - * bolt. Each task will run on a thread in a process somewhere - * around the cluster. - * @return use the returned object to declare the inputs to this component - */ - public BoltDeclarer setBolt(String id, IRichBolt bolt, - Number parallelism_hint) { - validateUnusedId(id); - initCommon(id, bolt, parallelism_hint); - _bolts.put(id, bolt); - return new BoltGetter(id); - } - - /** - * Define a new bolt in this topology. This defines a basic bolt, which is a - * simpler to use but more restricted kind of bolt. Basic bolts are intended - * for non-aggregation processing and automate the anchoring/acking process - * to achieve proper reliability in the topology. - * - * @param id - * the id of this component. This id is referenced by other - * components that want to consume this bolt's outputs. - * @param bolt - * the basic bolt - * @return use the returned object to declare the inputs to this component - */ - public BoltDeclarer setBolt(String id, IBasicBolt bolt) { - return setBolt(id, bolt, null); - } - - /** - * Define a new bolt in this topology. This defines a basic bolt, which is a - * simpler to use but more restricted kind of bolt. Basic bolts are intended - * for non-aggregation processing and automate the anchoring/acking process - * to achieve proper reliability in the topology. - * - * @param id - * the id of this component. This id is referenced by other - * components that want to consume this bolt's outputs. - * @param bolt - * the basic bolt - * @param parallelism_hint - * the number of tasks that should be assigned to execute this - * bolt. Each task will run on a thread in a process somwehere - * around the cluster. - * @return use the returned object to declare the inputs to this component - */ - public BoltDeclarer setBolt(String id, IBasicBolt bolt, - Number parallelism_hint) { - return setBolt(id, new BasicBoltExecutor(bolt), parallelism_hint); - } - - /** - * Define a new spout in this topology. - * - * @param id - * the id of this component. This id is referenced by other - * components that want to consume this spout's outputs. - * @param spout - * the spout - */ - public SpoutDeclarer setSpout(String id, IRichSpout spout) { - return setSpout(id, spout, null); - } - - /** - * Define a new spout in this topology with the specified parallelism. If - * the spout declares itself as non-distributed, the parallelism_hint will - * be ignored and only one task will be allocated to this component. - * - * @param id - * the id of this component. This id is referenced by other - * components that want to consume this spout's outputs. - * @param parallelism_hint - * the number of tasks that should be assigned to execute this - * spout. Each task will run on a thread in a process somwehere - * around the cluster. - * @param spout - * the spout - */ - public SpoutDeclarer setSpout(String id, IRichSpout spout, - Number parallelism_hint) { - validateUnusedId(id); - initCommon(id, spout, parallelism_hint); - _spouts.put(id, spout); - return new SpoutGetter(id); - } - - public void setStateSpout(String id, IRichStateSpout stateSpout) { - setStateSpout(id, stateSpout, null); - } - - public void setStateSpout(String id, IRichStateSpout stateSpout, - Number parallelism_hint) { - validateUnusedId(id); - // TODO: finish - } - - private void validateUnusedId(String id) { - if (_bolts.containsKey(id)) { - throw new IllegalArgumentException( - "Bolt has already been declared for id " + id); - } - if (_spouts.containsKey(id)) { - throw new IllegalArgumentException( - "Spout has already been declared for id " + id); - } - if (_stateSpouts.containsKey(id)) { - throw new IllegalArgumentException( - "State spout has already been declared for id " + id); - } - } - - private ComponentCommon getComponentCommon(String id, IComponent component) { - ComponentCommon ret = new ComponentCommon(_commons.get(id)); - - OutputFieldsGetter getter = new OutputFieldsGetter(); - component.declareOutputFields(getter); - ret.set_streams(getter.getFieldsDeclaration()); - return ret; - } - - private void initCommon(String id, IComponent component, Number parallelism) { - ComponentCommon common = new ComponentCommon(); - common.set_inputs(new HashMap()); - if (parallelism != null) - common.set_parallelism_hint(parallelism.intValue()); - else { - common.set_parallelism_hint(Integer.valueOf(1)); - } - Map conf = component.getComponentConfiguration(); - if (conf != null) - common.set_json_conf(Utils.to_json(conf)); - _commons.put(id, common); - } - - protected class ConfigGetter - extends BaseConfigurationDeclarer { - String _id; - - public ConfigGetter(String id) { - _id = id; - } - - @Override - public T addConfigurations(Map conf) { - if (conf != null && conf.containsKey(Config.TOPOLOGY_KRYO_REGISTER)) { - throw new IllegalArgumentException( - "Cannot set serializations for a component using fluent API"); - } - String currConf = _commons.get(_id).get_json_conf(); - _commons.get(_id).set_json_conf( - mergeIntoJson(parseJson(currConf), conf)); - return (T) this; - } - } - - protected class SpoutGetter extends ConfigGetter implements - SpoutDeclarer { - public SpoutGetter(String id) { - super(id); - } - } - - protected class BoltGetter extends ConfigGetter implements - BoltDeclarer { - private String _boltId; - - public BoltGetter(String boltId) { - super(boltId); - _boltId = boltId; - } - - public BoltDeclarer fieldsGrouping(String componentId, Fields fields) { - return fieldsGrouping(componentId, Utils.DEFAULT_STREAM_ID, fields); - } - - public BoltDeclarer fieldsGrouping(String componentId, String streamId, - Fields fields) { - return grouping(componentId, streamId, - Grouping.fields(fields.toList())); - } - - public BoltDeclarer globalGrouping(String componentId) { - return globalGrouping(componentId, Utils.DEFAULT_STREAM_ID); - } - - public BoltDeclarer globalGrouping(String componentId, String streamId) { - return grouping(componentId, streamId, - Grouping.fields(new ArrayList())); - } - - public BoltDeclarer shuffleGrouping(String componentId) { - return shuffleGrouping(componentId, Utils.DEFAULT_STREAM_ID); - } - - public BoltDeclarer shuffleGrouping(String componentId, String streamId) { - return grouping(componentId, streamId, - Grouping.shuffle(new NullStruct())); - } - - public BoltDeclarer localOrShuffleGrouping(String componentId) { - return localOrShuffleGrouping(componentId, Utils.DEFAULT_STREAM_ID); - } - - public BoltDeclarer localOrShuffleGrouping(String componentId, - String streamId) { - return grouping(componentId, streamId, - Grouping.local_or_shuffle(new NullStruct())); - } - - @Override - public BoltDeclarer localFirstGrouping(String componentId) { - return localFirstGrouping(componentId, Utils.DEFAULT_STREAM_ID); - } - - @Override - public BoltDeclarer localFirstGrouping(String componentId, - String streamId) { - return grouping(componentId, streamId, - Grouping.localFirst(new NullStruct())); - } - - public BoltDeclarer noneGrouping(String componentId) { - return noneGrouping(componentId, Utils.DEFAULT_STREAM_ID); - } - - public BoltDeclarer noneGrouping(String componentId, String streamId) { - return grouping(componentId, streamId, - Grouping.none(new NullStruct())); - } - - public BoltDeclarer allGrouping(String componentId) { - return allGrouping(componentId, Utils.DEFAULT_STREAM_ID); - } - - public BoltDeclarer allGrouping(String componentId, String streamId) { - return grouping(componentId, streamId, - Grouping.all(new NullStruct())); - } - - public BoltDeclarer directGrouping(String componentId) { - return directGrouping(componentId, Utils.DEFAULT_STREAM_ID); - } - - public BoltDeclarer directGrouping(String componentId, String streamId) { - return grouping(componentId, streamId, - Grouping.direct(new NullStruct())); - } - - private BoltDeclarer grouping(String componentId, String streamId, - Grouping grouping) { - _commons.get(_boltId).put_to_inputs( - new GlobalStreamId(componentId, streamId), grouping); - return this; - } - - @Override - public BoltDeclarer customGrouping(String componentId, - CustomStreamGrouping grouping) { - return customGrouping(componentId, Utils.DEFAULT_STREAM_ID, - grouping); - } - - @Override - public BoltDeclarer customGrouping(String componentId, String streamId, - CustomStreamGrouping grouping) { - return grouping(componentId, streamId, - Grouping.custom_serialized(Utils.serialize(grouping))); - } - - @Override - public BoltDeclarer grouping(GlobalStreamId id, Grouping grouping) { - return grouping(id.get_componentId(), id.get_streamId(), grouping); - } - - - } - - private static Map parseJson(String json) { - if (json == null) - return new HashMap(); - else - return (Map) Utils.from_json(json); - } - - private static String mergeIntoJson(Map into, Map newMap) { - Map res = new HashMap(into); - if (newMap != null) - res.putAll(newMap); - return Utils.to_json(res); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/base/BaseBasicBolt.java b/jstorm-client/src/main/java/backtype/storm/topology/base/BaseBasicBolt.java deleted file mode 100644 index 2e1820751..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/base/BaseBasicBolt.java +++ /dev/null @@ -1,16 +0,0 @@ -package backtype.storm.topology.base; - -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IBasicBolt; -import java.util.Map; - -public abstract class BaseBasicBolt extends BaseComponent implements IBasicBolt { - - @Override - public void prepare(Map stormConf, TopologyContext context) { - } - - @Override - public void cleanup() { - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/base/BaseBatchBolt.java b/jstorm-client/src/main/java/backtype/storm/topology/base/BaseBatchBolt.java deleted file mode 100644 index 9171392be..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/base/BaseBatchBolt.java +++ /dev/null @@ -1,9 +0,0 @@ -package backtype.storm.topology.base; - -import backtype.storm.coordination.IBatchBolt; -import java.util.Map; - -public abstract class BaseBatchBolt extends BaseComponent implements - IBatchBolt { - -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/base/BaseComponent.java b/jstorm-client/src/main/java/backtype/storm/topology/base/BaseComponent.java deleted file mode 100644 index 1b1449aa9..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/base/BaseComponent.java +++ /dev/null @@ -1,11 +0,0 @@ -package backtype.storm.topology.base; - -import backtype.storm.topology.IComponent; -import java.util.Map; - -public abstract class BaseComponent implements IComponent { - @Override - public Map getComponentConfiguration() { - return null; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/base/BaseOpaquePartitionedTransactionalSpout.java b/jstorm-client/src/main/java/backtype/storm/topology/base/BaseOpaquePartitionedTransactionalSpout.java deleted file mode 100644 index baf0c65dd..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/base/BaseOpaquePartitionedTransactionalSpout.java +++ /dev/null @@ -1,8 +0,0 @@ -package backtype.storm.topology.base; - -import backtype.storm.transactional.partitioned.IOpaquePartitionedTransactionalSpout; - -public abstract class BaseOpaquePartitionedTransactionalSpout extends - BaseComponent implements IOpaquePartitionedTransactionalSpout { - -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/base/BasePartitionedTransactionalSpout.java b/jstorm-client/src/main/java/backtype/storm/topology/base/BasePartitionedTransactionalSpout.java deleted file mode 100644 index 2a9f298a5..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/base/BasePartitionedTransactionalSpout.java +++ /dev/null @@ -1,9 +0,0 @@ -package backtype.storm.topology.base; - -import backtype.storm.transactional.partitioned.IPartitionedTransactionalSpout; -import java.util.Map; - -public abstract class BasePartitionedTransactionalSpout extends - BaseComponent implements IPartitionedTransactionalSpout { - -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/base/BaseRichBolt.java b/jstorm-client/src/main/java/backtype/storm/topology/base/BaseRichBolt.java deleted file mode 100644 index e6a759269..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/base/BaseRichBolt.java +++ /dev/null @@ -1,9 +0,0 @@ -package backtype.storm.topology.base; - -import backtype.storm.topology.IRichBolt; - -public abstract class BaseRichBolt extends BaseComponent implements IRichBolt { - @Override - public void cleanup() { - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/base/BaseRichSpout.java b/jstorm-client/src/main/java/backtype/storm/topology/base/BaseRichSpout.java deleted file mode 100644 index 9c0d73379..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/base/BaseRichSpout.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * To change this template, choose Tools | Templates - * and open the template in the editor. - */ -package backtype.storm.topology.base; - -import backtype.storm.topology.IRichSpout; - -/** - * - * @author nathan - */ -public abstract class BaseRichSpout extends BaseComponent implements IRichSpout { - @Override - public void close() { - } - - @Override - public void activate() { - } - - @Override - public void deactivate() { - } - - @Override - public void ack(Object msgId) { - } - - @Override - public void fail(Object msgId) { - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/base/BaseTransactionalBolt.java b/jstorm-client/src/main/java/backtype/storm/topology/base/BaseTransactionalBolt.java deleted file mode 100644 index 77647c019..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/base/BaseTransactionalBolt.java +++ /dev/null @@ -1,8 +0,0 @@ -package backtype.storm.topology.base; - -import backtype.storm.transactional.TransactionAttempt; - -public abstract class BaseTransactionalBolt extends - BaseBatchBolt { - -} diff --git a/jstorm-client/src/main/java/backtype/storm/topology/base/BaseTransactionalSpout.java b/jstorm-client/src/main/java/backtype/storm/topology/base/BaseTransactionalSpout.java deleted file mode 100644 index 01945eaff..000000000 --- a/jstorm-client/src/main/java/backtype/storm/topology/base/BaseTransactionalSpout.java +++ /dev/null @@ -1,9 +0,0 @@ -package backtype.storm.topology.base; - -import backtype.storm.transactional.ITransactionalSpout; -import java.util.Map; - -public abstract class BaseTransactionalSpout extends BaseComponent implements - ITransactionalSpout { - -} diff --git a/jstorm-client/src/main/java/backtype/storm/transactional/ICommitter.java b/jstorm-client/src/main/java/backtype/storm/transactional/ICommitter.java deleted file mode 100644 index e764fb798..000000000 --- a/jstorm-client/src/main/java/backtype/storm/transactional/ICommitter.java +++ /dev/null @@ -1,9 +0,0 @@ -package backtype.storm.transactional; - -/** - * This marks an IBatchBolt within a transactional topology as a committer. This - * causes the finishBatch method to be called in order of the transactions. - */ -public interface ICommitter { - -} diff --git a/jstorm-client/src/main/java/backtype/storm/transactional/ICommitterTransactionalSpout.java b/jstorm-client/src/main/java/backtype/storm/transactional/ICommitterTransactionalSpout.java deleted file mode 100644 index 58b8e1984..000000000 --- a/jstorm-client/src/main/java/backtype/storm/transactional/ICommitterTransactionalSpout.java +++ /dev/null @@ -1,13 +0,0 @@ -package backtype.storm.transactional; - -import backtype.storm.task.TopologyContext; -import java.util.Map; - -public interface ICommitterTransactionalSpout extends ITransactionalSpout { - public interface Emitter extends ITransactionalSpout.Emitter { - void commit(TransactionAttempt attempt); - } - - @Override - public Emitter getEmitter(Map conf, TopologyContext context); -} diff --git a/jstorm-client/src/main/java/backtype/storm/transactional/ITransactionalSpout.java b/jstorm-client/src/main/java/backtype/storm/transactional/ITransactionalSpout.java deleted file mode 100644 index 13399e133..000000000 --- a/jstorm-client/src/main/java/backtype/storm/transactional/ITransactionalSpout.java +++ /dev/null @@ -1,88 +0,0 @@ -package backtype.storm.transactional; - -import backtype.storm.coordination.BatchOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IComponent; -import java.math.BigInteger; -import java.util.Map; - -public interface ITransactionalSpout extends IComponent { - public interface Coordinator { - /** - * Create metadata for this particular transaction id which has never - * been emitted before. The metadata should contain whatever is - * necessary to be able to replay the exact batch for the transaction at - * a later point. - * - * The metadata is stored in Zookeeper. - * - * Storm uses the Kryo serializations configured in the component - * configuration for this spout to serialize and deserialize the - * metadata. - * - * @param txid - * The id of the transaction. - * @param prevMetadata - * The metadata of the previous transaction - * @return the metadata for this new transaction - */ - X initializeTransaction(BigInteger txid, X prevMetadata); - - /** - * Returns true if its ok to emit start a new transaction, false - * otherwise (will skip this transaction). - * - * You should sleep here if you want a delay between asking for the next - * transaction (this will be called repeatedly in a loop). - */ - boolean isReady(); - - /** - * Release any resources from this coordinator. - */ - void close(); - } - - public interface Emitter { - /** - * Emit a batch for the specified transaction attempt and metadata for - * the transaction. The metadata was created by the Coordinator in the - * initializeTranaction method. This method must always emit the same - * batch of tuples across all tasks for the same transaction id. - * - * The first field of all emitted tuples must contain the provided - * TransactionAttempt. - * - */ - void emitBatch(TransactionAttempt tx, X coordinatorMeta, - BatchOutputCollector collector); - - /** - * Any state for transactions prior to the provided transaction id can - * be safely cleaned up, so this method should clean up that state. - */ - void cleanupBefore(BigInteger txid); - - /** - * Release any resources held by this emitter. - */ - void close(); - } - - /** - * The coordinator for a TransactionalSpout runs in a single thread and - * indicates when batches of tuples should be emitted and when transactions - * should commit. The Coordinator that you provide in a TransactionalSpout - * provides metadata for each transaction so that the transactions can be - * replayed. - */ - Coordinator getCoordinator(Map conf, TopologyContext context); - - /** - * The emitter for a TransactionalSpout runs as many tasks across the - * cluster. Emitters are responsible for emitting batches of tuples for a - * transaction and must ensure that the same batch of tuples is always - * emitted for the same transaction id. - */ - Emitter getEmitter(Map conf, TopologyContext context); -} diff --git a/jstorm-client/src/main/java/backtype/storm/transactional/TransactionAttempt.java b/jstorm-client/src/main/java/backtype/storm/transactional/TransactionAttempt.java deleted file mode 100644 index 2d02de16a..000000000 --- a/jstorm-client/src/main/java/backtype/storm/transactional/TransactionAttempt.java +++ /dev/null @@ -1,44 +0,0 @@ -package backtype.storm.transactional; - -import java.math.BigInteger; - -public class TransactionAttempt { - BigInteger _txid; - long _attemptId; - - // for kryo compatibility - public TransactionAttempt() { - - } - - public TransactionAttempt(BigInteger txid, long attemptId) { - _txid = txid; - _attemptId = attemptId; - } - - public BigInteger getTransactionId() { - return _txid; - } - - public long getAttemptId() { - return _attemptId; - } - - @Override - public int hashCode() { - return _txid.hashCode(); - } - - @Override - public boolean equals(Object o) { - if (!(o instanceof TransactionAttempt)) - return false; - TransactionAttempt other = (TransactionAttempt) o; - return _txid.equals(other._txid) && _attemptId == other._attemptId; - } - - @Override - public String toString() { - return "" + _txid + ":" + _attemptId; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/transactional/TransactionalSpoutBatchExecutor.java b/jstorm-client/src/main/java/backtype/storm/transactional/TransactionalSpoutBatchExecutor.java deleted file mode 100644 index daea107ae..000000000 --- a/jstorm-client/src/main/java/backtype/storm/transactional/TransactionalSpoutBatchExecutor.java +++ /dev/null @@ -1,84 +0,0 @@ -package backtype.storm.transactional; - -import backtype.storm.coordination.BatchOutputCollectorImpl; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.FailedException; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Tuple; -import java.math.BigInteger; -import java.util.Map; -import java.util.TreeMap; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TransactionalSpoutBatchExecutor implements IRichBolt { - public static Logger LOG = LoggerFactory - .getLogger(TransactionalSpoutBatchExecutor.class); - - BatchOutputCollectorImpl _collector; - ITransactionalSpout _spout; - ITransactionalSpout.Emitter _emitter; - - TreeMap _activeTransactions = new TreeMap(); - - public TransactionalSpoutBatchExecutor(ITransactionalSpout spout) { - _spout = spout; - } - - @Override - public void prepare(Map conf, TopologyContext context, - OutputCollector collector) { - _collector = new BatchOutputCollectorImpl(collector); - _emitter = _spout.getEmitter(conf, context); - } - - @Override - public void execute(Tuple input) { - TransactionAttempt attempt = (TransactionAttempt) input.getValue(0); - try { - if (input.getSourceStreamId().equals( - TransactionalSpoutCoordinator.TRANSACTION_COMMIT_STREAM_ID)) { - if (attempt.equals(_activeTransactions.get(attempt - .getTransactionId()))) { - ((ICommitterTransactionalSpout.Emitter) _emitter) - .commit(attempt); - _activeTransactions.remove(attempt.getTransactionId()); - _collector.ack(input); - } else { - _collector.fail(input); - } - } else { - _emitter.emitBatch(attempt, input.getValue(1), _collector); - _activeTransactions.put(attempt.getTransactionId(), attempt); - _collector.ack(input); - BigInteger committed = (BigInteger) input.getValue(2); - if (committed != null) { - // valid to delete before what's been committed since - // those batches will never be accessed again - _activeTransactions.headMap(committed).clear(); - _emitter.cleanupBefore(committed); - } - } - } catch (FailedException e) { - LOG.warn("Failed to emit batch for transaction", e); - _collector.fail(input); - } - } - - @Override - public void cleanup() { - _emitter.close(); - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - _spout.declareOutputFields(declarer); - } - - @Override - public Map getComponentConfiguration() { - return _spout.getComponentConfiguration(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/transactional/TransactionalSpoutCoordinator.java b/jstorm-client/src/main/java/backtype/storm/transactional/TransactionalSpoutCoordinator.java deleted file mode 100644 index 4810903f4..000000000 --- a/jstorm-client/src/main/java/backtype/storm/transactional/TransactionalSpoutCoordinator.java +++ /dev/null @@ -1,220 +0,0 @@ -package backtype.storm.transactional; - -import backtype.storm.Config; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.FailedException; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseRichSpout; -import backtype.storm.transactional.state.RotatingTransactionalState; -import backtype.storm.transactional.state.TransactionalState; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Values; -import backtype.storm.utils.Utils; -import java.math.BigInteger; -import java.util.Map; -import java.util.TreeMap; -import java.util.Random; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class TransactionalSpoutCoordinator extends BaseRichSpout { - public static final Logger LOG = LoggerFactory - .getLogger(TransactionalSpoutCoordinator.class); - - public static final BigInteger INIT_TXID = BigInteger.ONE; - - public static final String TRANSACTION_BATCH_STREAM_ID = TransactionalSpoutCoordinator.class - .getName() + "/batch"; - public static final String TRANSACTION_COMMIT_STREAM_ID = TransactionalSpoutCoordinator.class - .getName() + "/commit"; - - private static final String CURRENT_TX = "currtx"; - private static final String META_DIR = "meta"; - - private ITransactionalSpout _spout; - private ITransactionalSpout.Coordinator _coordinator; - private TransactionalState _state; - private RotatingTransactionalState _coordinatorState; - - TreeMap _activeTx = new TreeMap(); - - private SpoutOutputCollector _collector; - private Random _rand; - BigInteger _currTransaction; - int _maxTransactionActive; - StateInitializer _initializer; - - public TransactionalSpoutCoordinator(ITransactionalSpout spout) { - _spout = spout; - } - - public ITransactionalSpout getSpout() { - return _spout; - } - - @Override - public void open(Map conf, TopologyContext context, - SpoutOutputCollector collector) { - _rand = new Random(Utils.secureRandomLong()); - _state = TransactionalState.newCoordinatorState(conf, - (String) conf.get(Config.TOPOLOGY_TRANSACTIONAL_ID), - _spout.getComponentConfiguration()); - _coordinatorState = new RotatingTransactionalState(_state, META_DIR, - true); - _collector = collector; - _coordinator = _spout.getCoordinator(conf, context); - _currTransaction = getStoredCurrTransaction(_state); - Object active = conf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING); - if (active == null) { - _maxTransactionActive = 1; - } else { - _maxTransactionActive = Utils.getInt(active); - } - _initializer = new StateInitializer(); - } - - @Override - public void close() { - _state.close(); - } - - @Override - public void nextTuple() { - sync(); - } - - @Override - public void ack(Object msgId) { - TransactionAttempt tx = (TransactionAttempt) msgId; - TransactionStatus status = _activeTx.get(tx.getTransactionId()); - if (status != null && tx.equals(status.attempt)) { - if (status.status == AttemptStatus.PROCESSING) { - status.status = AttemptStatus.PROCESSED; - } else if (status.status == AttemptStatus.COMMITTING) { - _activeTx.remove(tx.getTransactionId()); - _coordinatorState.cleanupBefore(tx.getTransactionId()); - _currTransaction = nextTransactionId(tx.getTransactionId()); - _state.setData(CURRENT_TX, _currTransaction); - } - sync(); - } - } - - @Override - public void fail(Object msgId) { - TransactionAttempt tx = (TransactionAttempt) msgId; - TransactionStatus stored = _activeTx.remove(tx.getTransactionId()); - if (stored != null && tx.equals(stored.attempt)) { - _activeTx.tailMap(tx.getTransactionId()).clear(); - sync(); - } - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - // in partitioned example, in case an emitter task receives a later - // transaction than it's emitted so far, - // when it sees the earlier txid it should know to emit nothing - declarer.declareStream(TRANSACTION_BATCH_STREAM_ID, new Fields("tx", - "tx-meta", "committed-txid")); - declarer.declareStream(TRANSACTION_COMMIT_STREAM_ID, new Fields("tx")); - } - - private void sync() { - // note that sometimes the tuples active may be less than - // max_spout_pending, e.g. - // max_spout_pending = 3 - // tx 1, 2, 3 active, tx 2 is acked. there won't be a commit for tx 2 - // (because tx 1 isn't committed yet), - // and there won't be a batch for tx 4 because there's max_spout_pending - // tx active - TransactionStatus maybeCommit = _activeTx.get(_currTransaction); - if (maybeCommit != null - && maybeCommit.status == AttemptStatus.PROCESSED) { - maybeCommit.status = AttemptStatus.COMMITTING; - _collector.emit(TRANSACTION_COMMIT_STREAM_ID, new Values( - maybeCommit.attempt), maybeCommit.attempt); - } - - try { - if (_activeTx.size() < _maxTransactionActive) { - BigInteger curr = _currTransaction; - for (int i = 0; i < _maxTransactionActive; i++) { - if ((_coordinatorState.hasCache(curr) || _coordinator - .isReady()) && !_activeTx.containsKey(curr)) { - TransactionAttempt attempt = new TransactionAttempt( - curr, _rand.nextLong()); - Object state = _coordinatorState.getState(curr, - _initializer); - _activeTx.put(curr, new TransactionStatus(attempt)); - _collector - .emit(TRANSACTION_BATCH_STREAM_ID, - new Values( - attempt, - state, - previousTransactionId(_currTransaction)), - attempt); - } - curr = nextTransactionId(curr); - } - } - } catch (FailedException e) { - LOG.warn("Failed to get metadata for a transaction", e); - } - } - - @Override - public Map getComponentConfiguration() { - Config ret = new Config(); - ret.setMaxTaskParallelism(1); - return ret; - } - - private static enum AttemptStatus { - PROCESSING, PROCESSED, COMMITTING - } - - private static class TransactionStatus { - TransactionAttempt attempt; - AttemptStatus status; - - public TransactionStatus(TransactionAttempt attempt) { - this.attempt = attempt; - this.status = AttemptStatus.PROCESSING; - } - - @Override - public String toString() { - return attempt.toString() + " <" + status.toString() + ">"; - } - } - - private BigInteger nextTransactionId(BigInteger id) { - return id.add(BigInteger.ONE); - } - - private BigInteger previousTransactionId(BigInteger id) { - if (id.equals(INIT_TXID)) { - return null; - } else { - return id.subtract(BigInteger.ONE); - } - } - - private BigInteger getStoredCurrTransaction(TransactionalState state) { - BigInteger ret = (BigInteger) state.getData(CURRENT_TX); - if (ret == null) - return INIT_TXID; - else - return ret; - } - - private class StateInitializer implements - RotatingTransactionalState.StateInitializer { - @Override - public Object init(BigInteger txid, Object lastState) { - return _coordinator.initializeTransaction(txid, lastState); - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/transactional/TransactionalTopologyBuilder.java b/jstorm-client/src/main/java/backtype/storm/transactional/TransactionalTopologyBuilder.java deleted file mode 100644 index b4e437b14..000000000 --- a/jstorm-client/src/main/java/backtype/storm/transactional/TransactionalTopologyBuilder.java +++ /dev/null @@ -1,566 +0,0 @@ -package backtype.storm.transactional; - -import backtype.storm.coordination.IBatchBolt; -import backtype.storm.coordination.BatchBoltExecutor; -import backtype.storm.Config; -import backtype.storm.Constants; -import backtype.storm.coordination.CoordinatedBolt; -import backtype.storm.coordination.CoordinatedBolt.IdStreamSpec; -import backtype.storm.coordination.CoordinatedBolt.SourceArgs; -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.generated.Grouping; -import backtype.storm.generated.StormTopology; -import backtype.storm.grouping.CustomStreamGrouping; -import backtype.storm.topology.BaseConfigurationDeclarer; -import backtype.storm.topology.BasicBoltExecutor; -import backtype.storm.topology.BoltDeclarer; -import backtype.storm.topology.IBasicBolt; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.InputDeclarer; -import backtype.storm.topology.SpoutDeclarer; -import backtype.storm.topology.TopologyBuilder; -import backtype.storm.transactional.partitioned.IOpaquePartitionedTransactionalSpout; -import backtype.storm.transactional.partitioned.IPartitionedTransactionalSpout; -import backtype.storm.transactional.partitioned.OpaquePartitionedTransactionalSpoutExecutor; -import backtype.storm.transactional.partitioned.PartitionedTransactionalSpoutExecutor; -import backtype.storm.tuple.Fields; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -/** - * Trident subsumes the functionality provided by transactional topologies, so - * this class is deprecated. - * - */ -@Deprecated -public class TransactionalTopologyBuilder { - String _id; - String _spoutId; - ITransactionalSpout _spout; - Map _bolts = new HashMap(); - Integer _spoutParallelism; - List _spoutConfs = new ArrayList(); - - // id is used to store the state of this transactionalspout in zookeeper - // it would be very dangerous to have 2 topologies active with the same id - // in the same cluster - public TransactionalTopologyBuilder(String id, String spoutId, - ITransactionalSpout spout, Number spoutParallelism) { - _id = id; - _spoutId = spoutId; - _spout = spout; - _spoutParallelism = (spoutParallelism == null) ? null - : spoutParallelism.intValue(); - } - - public TransactionalTopologyBuilder(String id, String spoutId, - ITransactionalSpout spout) { - this(id, spoutId, spout, null); - } - - public TransactionalTopologyBuilder(String id, String spoutId, - IPartitionedTransactionalSpout spout, Number spoutParallelism) { - this(id, spoutId, new PartitionedTransactionalSpoutExecutor(spout), - spoutParallelism); - } - - public TransactionalTopologyBuilder(String id, String spoutId, - IPartitionedTransactionalSpout spout) { - this(id, spoutId, spout, null); - } - - public TransactionalTopologyBuilder(String id, String spoutId, - IOpaquePartitionedTransactionalSpout spout, Number spoutParallelism) { - this(id, spoutId, - new OpaquePartitionedTransactionalSpoutExecutor(spout), - spoutParallelism); - } - - public TransactionalTopologyBuilder(String id, String spoutId, - IOpaquePartitionedTransactionalSpout spout) { - this(id, spoutId, spout, null); - } - - public SpoutDeclarer getSpoutDeclarer() { - return new SpoutDeclarerImpl(); - } - - public BoltDeclarer setBolt(String id, IBatchBolt bolt) { - return setBolt(id, bolt, null); - } - - public BoltDeclarer setBolt(String id, IBatchBolt bolt, Number parallelism) { - return setBolt(id, new BatchBoltExecutor(bolt), parallelism, - bolt instanceof ICommitter); - } - - public BoltDeclarer setCommitterBolt(String id, IBatchBolt bolt) { - return setCommitterBolt(id, bolt, null); - } - - public BoltDeclarer setCommitterBolt(String id, IBatchBolt bolt, - Number parallelism) { - return setBolt(id, new BatchBoltExecutor(bolt), parallelism, true); - } - - public BoltDeclarer setBolt(String id, IBasicBolt bolt) { - return setBolt(id, bolt, null); - } - - public BoltDeclarer setBolt(String id, IBasicBolt bolt, Number parallelism) { - return setBolt(id, new BasicBoltExecutor(bolt), parallelism, false); - } - - private BoltDeclarer setBolt(String id, IRichBolt bolt, Number parallelism, - boolean committer) { - Integer p = null; - if (parallelism != null) - p = parallelism.intValue(); - Component component = new Component(bolt, p, committer); - _bolts.put(id, component); - return new BoltDeclarerImpl(component); - } - - public TopologyBuilder buildTopologyBuilder() { - String coordinator = _spoutId + "/coordinator"; - TopologyBuilder builder = new TopologyBuilder(); - SpoutDeclarer declarer = builder.setSpout(coordinator, - new TransactionalSpoutCoordinator(_spout)); - for (Map conf : _spoutConfs) { - declarer.addConfigurations(conf); - } - declarer.addConfiguration(Config.TOPOLOGY_TRANSACTIONAL_ID, _id); - - BoltDeclarer emitterDeclarer = builder - .setBolt( - _spoutId, - new CoordinatedBolt( - new TransactionalSpoutBatchExecutor(_spout), - null, null), _spoutParallelism) - .allGrouping( - coordinator, - TransactionalSpoutCoordinator.TRANSACTION_BATCH_STREAM_ID) - .addConfiguration(Config.TOPOLOGY_TRANSACTIONAL_ID, _id); - if (_spout instanceof ICommitterTransactionalSpout) { - emitterDeclarer.allGrouping(coordinator, - TransactionalSpoutCoordinator.TRANSACTION_COMMIT_STREAM_ID); - } - for (String id : _bolts.keySet()) { - Component component = _bolts.get(id); - Map coordinatedArgs = new HashMap(); - // get all source component - for (String c : componentBoltSubscriptions(component)) { - coordinatedArgs.put(c, SourceArgs.all()); - } - - IdStreamSpec idSpec = null; - if (component.committer) { - idSpec = IdStreamSpec - .makeDetectSpec( - coordinator, - TransactionalSpoutCoordinator.TRANSACTION_COMMIT_STREAM_ID); - } - BoltDeclarer input = builder.setBolt(id, new CoordinatedBolt( - component.bolt, coordinatedArgs, idSpec), - component.parallelism); - for (Map conf : component.componentConfs) { - input.addConfigurations(conf); - } - for (String c : componentBoltSubscriptions(component)) { - input.directGrouping(c, Constants.COORDINATED_STREAM_ID); - } - for (InputDeclaration d : component.declarations) { - d.declare(input); - } - if (component.committer) { - input.allGrouping( - coordinator, - TransactionalSpoutCoordinator.TRANSACTION_COMMIT_STREAM_ID); - } - } - return builder; - } - - public StormTopology buildTopology() { - return buildTopologyBuilder().createTopology(); - } - - private Set componentBoltSubscriptions(Component component) { - Set ret = new HashSet(); - for (InputDeclaration d : component.declarations) { - ret.add(d.getComponent()); - } - return ret; - } - - private static class Component { - public IRichBolt bolt; - public Integer parallelism; - public List declarations = new ArrayList(); - public List componentConfs = new ArrayList(); - public boolean committer; - - public Component(IRichBolt bolt, Integer parallelism, boolean committer) { - this.bolt = bolt; - this.parallelism = parallelism; - this.committer = committer; - } - } - - private static interface InputDeclaration { - void declare(InputDeclarer declarer); - - String getComponent(); - } - - private class SpoutDeclarerImpl extends - BaseConfigurationDeclarer implements SpoutDeclarer { - @Override - public SpoutDeclarer addConfigurations(Map conf) { - _spoutConfs.add(conf); - return this; - } - } - - private class BoltDeclarerImpl extends - BaseConfigurationDeclarer implements BoltDeclarer { - Component _component; - - public BoltDeclarerImpl(Component component) { - _component = component; - } - - @Override - public BoltDeclarer fieldsGrouping(final String component, - final Fields fields) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.fieldsGrouping(component, fields); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer fieldsGrouping(final String component, - final String streamId, final Fields fields) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.fieldsGrouping(component, streamId, fields); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer globalGrouping(final String component) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.globalGrouping(component); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer globalGrouping(final String component, - final String streamId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.globalGrouping(component, streamId); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer shuffleGrouping(final String component) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.shuffleGrouping(component); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer shuffleGrouping(final String component, - final String streamId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.shuffleGrouping(component, streamId); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer localOrShuffleGrouping(final String component) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.localOrShuffleGrouping(component); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer localOrShuffleGrouping(final String component, - final String streamId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.localOrShuffleGrouping(component, streamId); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer localFirstGrouping(final String component) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.localFirstGrouping(component); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer localFirstGrouping(final String component, - final String streamId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.localFirstGrouping(component, streamId); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer noneGrouping(final String component) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.noneGrouping(component); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer noneGrouping(final String component, - final String streamId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.noneGrouping(component, streamId); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer allGrouping(final String component) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.allGrouping(component); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer allGrouping(final String component, - final String streamId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.allGrouping(component, streamId); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer directGrouping(final String component) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.directGrouping(component); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer directGrouping(final String component, - final String streamId) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.directGrouping(component, streamId); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer customGrouping(final String component, - final CustomStreamGrouping grouping) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.customGrouping(component, grouping); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer customGrouping(final String component, - final String streamId, final CustomStreamGrouping grouping) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.customGrouping(component, streamId, grouping); - } - - @Override - public String getComponent() { - return component; - } - }); - return this; - } - - @Override - public BoltDeclarer grouping(final GlobalStreamId stream, - final Grouping grouping) { - addDeclaration(new InputDeclaration() { - @Override - public void declare(InputDeclarer declarer) { - declarer.grouping(stream, grouping); - } - - @Override - public String getComponent() { - return stream.get_componentId(); - } - }); - return this; - } - - private void addDeclaration(InputDeclaration declaration) { - _component.declarations.add(declaration); - } - - @Override - public BoltDeclarer addConfigurations(Map conf) { - _component.componentConfs.add(conf); - return this; - } - - - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/transactional/partitioned/IOpaquePartitionedTransactionalSpout.java b/jstorm-client/src/main/java/backtype/storm/transactional/partitioned/IOpaquePartitionedTransactionalSpout.java deleted file mode 100644 index 65c077299..000000000 --- a/jstorm-client/src/main/java/backtype/storm/transactional/partitioned/IOpaquePartitionedTransactionalSpout.java +++ /dev/null @@ -1,46 +0,0 @@ -package backtype.storm.transactional.partitioned; - -import backtype.storm.coordination.BatchOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IComponent; -import backtype.storm.transactional.TransactionAttempt; -import java.util.Map; - -/** - * This defines a transactional spout which does *not* necessarily replay the - * same batch every time it emits a batch for a transaction id. - */ -public interface IOpaquePartitionedTransactionalSpout extends IComponent { - public interface Coordinator { - /** - * Returns true if its ok to emit start a new transaction, false - * otherwise (will skip this transaction). - * - * You should sleep here if you want a delay between asking for the next - * transaction (this will be called repeatedly in a loop). - */ - boolean isReady(); - - void close(); - } - - public interface Emitter { - /** - * Emit a batch of tuples for a partition/transaction. - * - * Return the metadata describing this batch that will be used as - * lastPartitionMeta for defining the parameters of the next batch. - */ - X emitPartitionBatch(TransactionAttempt tx, - BatchOutputCollector collector, int partition, - X lastPartitionMeta); - - int numPartitions(); - - void close(); - } - - Emitter getEmitter(Map conf, TopologyContext context); - - Coordinator getCoordinator(Map conf, TopologyContext context); -} diff --git a/jstorm-client/src/main/java/backtype/storm/transactional/partitioned/IPartitionedTransactionalSpout.java b/jstorm-client/src/main/java/backtype/storm/transactional/partitioned/IPartitionedTransactionalSpout.java deleted file mode 100644 index 31e4c417e..000000000 --- a/jstorm-client/src/main/java/backtype/storm/transactional/partitioned/IPartitionedTransactionalSpout.java +++ /dev/null @@ -1,60 +0,0 @@ -package backtype.storm.transactional.partitioned; - -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IComponent; -import backtype.storm.transactional.TransactionAttempt; -import backtype.storm.coordination.BatchOutputCollector; -import java.util.Map; - -/** - * This interface defines a transactional spout that reads its tuples from a - * partitioned set of brokers. It automates the storing of metadata for each - * partition to ensure that the same batch is always emitted for the same - * transaction id. The partition metadata is stored in Zookeeper. - */ -public interface IPartitionedTransactionalSpout extends IComponent { - public interface Coordinator { - /** - * Return the number of partitions currently in the source of data. The - * idea is is that if a new partition is added and a prior transaction - * is replayed, it doesn't emit tuples for the new partition because it - * knows how many partitions were in that transaction. - */ - int numPartitions(); - - /** - * Returns true if its ok to emit start a new transaction, false - * otherwise (will skip this transaction). - * - * You should sleep here if you want a delay between asking for the next - * transaction (this will be called repeatedly in a loop). - */ - boolean isReady(); - - void close(); - } - - public interface Emitter { - /** - * Emit a batch of tuples for a partition/transaction that's never been - * emitted before. Return the metadata that can be used to reconstruct - * this partition/batch in the future. - */ - X emitPartitionBatchNew(TransactionAttempt tx, - BatchOutputCollector collector, int partition, - X lastPartitionMeta); - - /** - * Emit a batch of tuples for a partition/transaction that has been - * emitted before, using the metadata created when it was first emitted. - */ - void emitPartitionBatch(TransactionAttempt tx, - BatchOutputCollector collector, int partition, X partitionMeta); - - void close(); - } - - Coordinator getCoordinator(Map conf, TopologyContext context); - - Emitter getEmitter(Map conf, TopologyContext context); -} diff --git a/jstorm-client/src/main/java/backtype/storm/transactional/partitioned/OpaquePartitionedTransactionalSpoutExecutor.java b/jstorm-client/src/main/java/backtype/storm/transactional/partitioned/OpaquePartitionedTransactionalSpoutExecutor.java deleted file mode 100644 index 4bc877fb7..000000000 --- a/jstorm-client/src/main/java/backtype/storm/transactional/partitioned/OpaquePartitionedTransactionalSpoutExecutor.java +++ /dev/null @@ -1,153 +0,0 @@ -package backtype.storm.transactional.partitioned; - -import backtype.storm.Config; -import backtype.storm.coordination.BatchOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.transactional.ICommitterTransactionalSpout; -import backtype.storm.transactional.ITransactionalSpout; -import backtype.storm.transactional.TransactionAttempt; -import backtype.storm.transactional.state.RotatingTransactionalState; -import backtype.storm.transactional.state.TransactionalState; -import java.math.BigInteger; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.TreeMap; - -public class OpaquePartitionedTransactionalSpoutExecutor implements - ICommitterTransactionalSpout { - IOpaquePartitionedTransactionalSpout _spout; - - public class Coordinator implements ITransactionalSpout.Coordinator { - IOpaquePartitionedTransactionalSpout.Coordinator _coordinator; - - public Coordinator(Map conf, TopologyContext context) { - _coordinator = _spout.getCoordinator(conf, context); - } - - @Override - public Object initializeTransaction(BigInteger txid, Object prevMetadata) { - return null; - } - - @Override - public boolean isReady() { - return _coordinator.isReady(); - } - - @Override - public void close() { - _coordinator.close(); - } - } - - public class Emitter implements ICommitterTransactionalSpout.Emitter { - IOpaquePartitionedTransactionalSpout.Emitter _emitter; - TransactionalState _state; - TreeMap> _cachedMetas = new TreeMap>(); - Map _partitionStates = new HashMap(); - int _index; - int _numTasks; - - public Emitter(Map conf, TopologyContext context) { - _emitter = _spout.getEmitter(conf, context); - _index = context.getThisTaskIndex(); - _numTasks = context.getComponentTasks(context.getThisComponentId()) - .size(); - _state = TransactionalState.newUserState(conf, - (String) conf.get(Config.TOPOLOGY_TRANSACTIONAL_ID), - getComponentConfiguration()); - List existingPartitions = _state.list(""); - for (String p : existingPartitions) { - int partition = Integer.parseInt(p); - if ((partition - _index) % _numTasks == 0) { - _partitionStates.put(partition, - new RotatingTransactionalState(_state, p)); - } - } - } - - @Override - public void emitBatch(TransactionAttempt tx, Object coordinatorMeta, - BatchOutputCollector collector) { - Map metas = new HashMap(); - _cachedMetas.put(tx.getTransactionId(), metas); - int partitions = _emitter.numPartitions(); - Entry> entry = _cachedMetas - .lowerEntry(tx.getTransactionId()); - Map prevCached; - if (entry != null) { - prevCached = entry.getValue(); - } else { - prevCached = new HashMap(); - } - - for (int i = _index; i < partitions; i += _numTasks) { - RotatingTransactionalState state = _partitionStates.get(i); - if (state == null) { - state = new RotatingTransactionalState(_state, "" + i); - _partitionStates.put(i, state); - } - state.removeState(tx.getTransactionId()); - Object lastMeta = prevCached.get(i); - if (lastMeta == null) - lastMeta = state.getLastState(); - Object meta = _emitter.emitPartitionBatch(tx, collector, i, - lastMeta); - metas.put(i, meta); - } - } - - @Override - public void cleanupBefore(BigInteger txid) { - for (RotatingTransactionalState state : _partitionStates.values()) { - state.cleanupBefore(txid); - } - } - - @Override - public void commit(TransactionAttempt attempt) { - BigInteger txid = attempt.getTransactionId(); - Map metas = _cachedMetas.remove(txid); - for (Integer partition : metas.keySet()) { - Object meta = metas.get(partition); - _partitionStates.get(partition).overrideState(txid, meta); - } - } - - @Override - public void close() { - _emitter.close(); - } - } - - public OpaquePartitionedTransactionalSpoutExecutor( - IOpaquePartitionedTransactionalSpout spout) { - _spout = spout; - } - - @Override - public ITransactionalSpout.Coordinator getCoordinator(Map conf, - TopologyContext context) { - return new Coordinator(conf, context); - } - - @Override - public ICommitterTransactionalSpout.Emitter getEmitter(Map conf, - TopologyContext context) { - return new Emitter(conf, context); - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - _spout.declareOutputFields(declarer); - } - - @Override - public Map getComponentConfiguration() { - return _spout.getComponentConfiguration(); - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/transactional/partitioned/PartitionedTransactionalSpoutExecutor.java b/jstorm-client/src/main/java/backtype/storm/transactional/partitioned/PartitionedTransactionalSpoutExecutor.java deleted file mode 100644 index 51bb34efb..000000000 --- a/jstorm-client/src/main/java/backtype/storm/transactional/partitioned/PartitionedTransactionalSpoutExecutor.java +++ /dev/null @@ -1,136 +0,0 @@ -package backtype.storm.transactional.partitioned; - -import backtype.storm.Config; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.transactional.ITransactionalSpout; -import backtype.storm.transactional.TransactionAttempt; -import backtype.storm.coordination.BatchOutputCollector; -import backtype.storm.transactional.state.RotatingTransactionalState; -import backtype.storm.transactional.state.TransactionalState; -import java.math.BigInteger; -import java.util.HashMap; -import java.util.Map; - -public class PartitionedTransactionalSpoutExecutor implements - ITransactionalSpout { - IPartitionedTransactionalSpout _spout; - - public PartitionedTransactionalSpoutExecutor( - IPartitionedTransactionalSpout spout) { - _spout = spout; - } - - public IPartitionedTransactionalSpout getPartitionedSpout() { - return _spout; - } - - class Coordinator implements ITransactionalSpout.Coordinator { - private IPartitionedTransactionalSpout.Coordinator _coordinator; - - public Coordinator(Map conf, TopologyContext context) { - _coordinator = _spout.getCoordinator(conf, context); - } - - @Override - public Integer initializeTransaction(BigInteger txid, - Integer prevMetadata) { - return _coordinator.numPartitions(); - } - - @Override - public boolean isReady() { - return _coordinator.isReady(); - } - - @Override - public void close() { - _coordinator.close(); - } - } - - class Emitter implements ITransactionalSpout.Emitter { - private IPartitionedTransactionalSpout.Emitter _emitter; - private TransactionalState _state; - private Map _partitionStates = new HashMap(); - private int _index; - private int _numTasks; - - public Emitter(Map conf, TopologyContext context) { - _emitter = _spout.getEmitter(conf, context); - _state = TransactionalState.newUserState(conf, - (String) conf.get(Config.TOPOLOGY_TRANSACTIONAL_ID), - getComponentConfiguration()); - _index = context.getThisTaskIndex(); - _numTasks = context.getComponentTasks(context.getThisComponentId()) - .size(); - } - - @Override - public void emitBatch(final TransactionAttempt tx, - final Integer partitions, final BatchOutputCollector collector) { - for (int i = _index; i < partitions; i += _numTasks) { - if (!_partitionStates.containsKey(i)) { - _partitionStates.put(i, new RotatingTransactionalState( - _state, "" + i)); - } - RotatingTransactionalState state = _partitionStates.get(i); - final int partition = i; - Object meta = state.getStateOrCreate(tx.getTransactionId(), - new RotatingTransactionalState.StateInitializer() { - @Override - public Object init(BigInteger txid, Object lastState) { - return _emitter.emitPartitionBatchNew(tx, - collector, partition, lastState); - } - }); - // it's null if one of: - // a) a later transaction batch was emitted before this, so we - // should skip this batch - // b) if didn't exist and was created (in which case the - // StateInitializer was invoked and - // it was emitted - if (meta != null) { - _emitter.emitPartitionBatch(tx, collector, partition, meta); - } - } - - } - - @Override - public void cleanupBefore(BigInteger txid) { - for (RotatingTransactionalState state : _partitionStates.values()) { - state.cleanupBefore(txid); - } - } - - @Override - public void close() { - _state.close(); - _emitter.close(); - } - } - - @Override - public ITransactionalSpout.Coordinator getCoordinator(Map conf, - TopologyContext context) { - return new Coordinator(conf, context); - } - - @Override - public ITransactionalSpout.Emitter getEmitter(Map conf, - TopologyContext context) { - return new Emitter(conf, context); - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - _spout.declareOutputFields(declarer); - } - - @Override - public Map getComponentConfiguration() { - return _spout.getComponentConfiguration(); - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/transactional/state/RotatingTransactionalState.java b/jstorm-client/src/main/java/backtype/storm/transactional/state/RotatingTransactionalState.java deleted file mode 100644 index 2ee9f857e..000000000 --- a/jstorm-client/src/main/java/backtype/storm/transactional/state/RotatingTransactionalState.java +++ /dev/null @@ -1,143 +0,0 @@ -package backtype.storm.transactional.state; - -import backtype.storm.transactional.TransactionalSpoutCoordinator; - -import java.math.BigInteger; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.SortedMap; -import java.util.TreeMap; - -/** - * A map from txid to a value. Automatically deletes txids that have been - * committed. - */ -public class RotatingTransactionalState { - public static interface StateInitializer { - Object init(BigInteger txid, Object lastState); - } - - private TransactionalState _state; - private String _subdir; - private boolean _strictOrder; - - private TreeMap _curr = new TreeMap(); - - public RotatingTransactionalState(TransactionalState state, String subdir, - boolean strictOrder) { - _state = state; - _subdir = subdir; - _strictOrder = strictOrder; - state.mkdir(subdir); - sync(); - } - - public RotatingTransactionalState(TransactionalState state, String subdir) { - this(state, subdir, false); - } - - public Object getLastState() { - if (_curr.isEmpty()) - return null; - else - return _curr.lastEntry().getValue(); - } - - public void overrideState(BigInteger txid, Object state) { - _state.setData(txPath(txid), state); - _curr.put(txid, state); - } - - public void removeState(BigInteger txid) { - if (_curr.containsKey(txid)) { - _curr.remove(txid); - _state.delete(txPath(txid)); - } - } - - public Object getState(BigInteger txid, StateInitializer init) { - if (!_curr.containsKey(txid)) { - SortedMap prevMap = _curr.headMap(txid); - SortedMap afterMap = _curr.tailMap(txid); - - BigInteger prev = null; - if (!prevMap.isEmpty()) - prev = prevMap.lastKey(); - - if (_strictOrder) { - if (prev == null - && !txid.equals(TransactionalSpoutCoordinator.INIT_TXID)) { - throw new IllegalStateException( - "Trying to initialize transaction for which there should be a previous state"); - } - if (prev != null && !prev.equals(txid.subtract(BigInteger.ONE))) { - throw new IllegalStateException( - "Expecting previous txid state to be the previous transaction"); - } - if (!afterMap.isEmpty()) { - throw new IllegalStateException( - "Expecting tx state to be initialized in strict order but there are txids after that have state"); - } - } - - Object data; - if (afterMap.isEmpty()) { - Object prevData; - if (prev != null) { - prevData = _curr.get(prev); - } else { - prevData = null; - } - data = init.init(txid, prevData); - } else { - data = null; - } - _curr.put(txid, data); - _state.setData(txPath(txid), data); - } - return _curr.get(txid); - } - - public boolean hasCache(BigInteger txid) { - return _curr.containsKey(txid); - } - - /** - * Returns null if it was created, the value otherwise. - */ - public Object getStateOrCreate(BigInteger txid, StateInitializer init) { - if (_curr.containsKey(txid)) { - return _curr.get(txid); - } else { - getState(txid, init); - return null; - } - } - - public void cleanupBefore(BigInteger txid) { - Set toDelete = new HashSet(); - toDelete.addAll(_curr.headMap(txid).keySet()); - for (BigInteger tx : toDelete) { - _curr.remove(tx); - _state.delete(txPath(tx)); - } - } - - private void sync() { - List txids = _state.list(_subdir); - for (String txid_s : txids) { - Object data = _state.getData(txPath(txid_s)); - _curr.put(new BigInteger(txid_s), data); - } - } - - private String txPath(BigInteger tx) { - return txPath(tx.toString()); - } - - private String txPath(String tx) { - return _subdir + "/" + tx; - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/transactional/state/TransactionalState.java b/jstorm-client/src/main/java/backtype/storm/transactional/state/TransactionalState.java deleted file mode 100644 index 11b835941..000000000 --- a/jstorm-client/src/main/java/backtype/storm/transactional/state/TransactionalState.java +++ /dev/null @@ -1,132 +0,0 @@ -package backtype.storm.transactional.state; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.curator.framework.CuratorFramework; -import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.KeeperException; - -import backtype.storm.Config; -import backtype.storm.serialization.KryoValuesDeserializer; -import backtype.storm.serialization.KryoValuesSerializer; -import backtype.storm.utils.Utils; - -public class TransactionalState { - CuratorFramework _curator; - KryoValuesSerializer _ser; - KryoValuesDeserializer _des; - - public static TransactionalState newUserState(Map conf, String id, - Map componentConf) { - return new TransactionalState(conf, id, componentConf, "user"); - } - - public static TransactionalState newCoordinatorState(Map conf, String id, - Map componentConf) { - return new TransactionalState(conf, id, componentConf, "coordinator"); - } - - protected TransactionalState(Map conf, String id, Map componentConf, - String subroot) { - try { - conf = new HashMap(conf); - // ensure that the serialization registrations are consistent with - // the declarations in this spout - if (componentConf != null) { - conf.put(Config.TOPOLOGY_KRYO_REGISTER, - componentConf.get(Config.TOPOLOGY_KRYO_REGISTER)); - } - String rootDir = conf.get(Config.TRANSACTIONAL_ZOOKEEPER_ROOT) - + "/" + id + "/" + subroot; - List servers = (List) getWithBackup(conf, - Config.TRANSACTIONAL_ZOOKEEPER_SERVERS, - Config.STORM_ZOOKEEPER_SERVERS); - Object port = getWithBackup(conf, - Config.TRANSACTIONAL_ZOOKEEPER_PORT, - Config.STORM_ZOOKEEPER_PORT); - CuratorFramework initter = Utils.newCuratorStarted(conf, servers, - port); - try { - initter.create().creatingParentsIfNeeded().forPath(rootDir); - } catch (KeeperException.NodeExistsException e) { - - } - - initter.close(); - - _curator = Utils.newCuratorStarted(conf, servers, port, rootDir); - _ser = new KryoValuesSerializer(conf); - _des = new KryoValuesDeserializer(conf); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - public void setData(String path, Object obj) { - path = "/" + path; - byte[] ser = _ser.serializeObject(obj); - try { - if (_curator.checkExists().forPath(path) != null) { - _curator.setData().forPath(path, ser); - } else { - _curator.create().creatingParentsIfNeeded() - .withMode(CreateMode.PERSISTENT).forPath(path, ser); - } - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - public void delete(String path) { - path = "/" + path; - try { - _curator.delete().forPath(path); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - public List list(String path) { - path = "/" + path; - try { - if (_curator.checkExists().forPath(path) == null) { - return new ArrayList(); - } else { - return _curator.getChildren().forPath(path); - } - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - public void mkdir(String path) { - setData(path, 7); - } - - public Object getData(String path) { - path = "/" + path; - try { - if (_curator.checkExists().forPath(path) != null) { - return _des.deserializeObject(_curator.getData().forPath(path)); - } else { - return null; - } - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - public void close() { - _curator.close(); - } - - private Object getWithBackup(Map amap, Object primary, Object backup) { - Object ret = amap.get(primary); - if (ret == null) - return amap.get(backup); - return ret; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/tuple/Fields.java b/jstorm-client/src/main/java/backtype/storm/tuple/Fields.java deleted file mode 100644 index dc9b8bfff..000000000 --- a/jstorm-client/src/main/java/backtype/storm/tuple/Fields.java +++ /dev/null @@ -1,82 +0,0 @@ -package backtype.storm.tuple; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.io.Serializable; - -public class Fields implements Iterable, Serializable { - private List _fields; - private Map _index = new HashMap(); - - public Fields(String... fields) { - this(Arrays.asList(fields)); - } - - public Fields(List fields) { - _fields = new ArrayList(fields.size()); - for (String field : fields) { - if (_fields.contains(field)) - throw new IllegalArgumentException(String.format( - "duplicate field '%s'", field)); - _fields.add(field); - } - index(); - } - - public List select(Fields selector, List tuple) { - List ret = new ArrayList(selector.size()); - for (String s : selector) { - ret.add(tuple.get(_index.get(s))); - } - return ret; - } - - public List toList() { - return new ArrayList(_fields); - } - - public int size() { - return _fields.size(); - } - - public String get(int index) { - return _fields.get(index); - } - - public Iterator iterator() { - return _fields.iterator(); - } - - /** - * Returns the position of the specified field. - */ - public int fieldIndex(String field) { - Integer ret = _index.get(field); - if (ret == null) { - throw new IllegalArgumentException(field + " does not exist"); - } - return ret; - } - - /** - * Returns true if this contains the specified name of the field. - */ - public boolean contains(String field) { - return _index.containsKey(field); - } - - private void index() { - for (int i = 0; i < _fields.size(); i++) { - _index.put(_fields.get(i), i); - } - } - - @Override - public String toString() { - return _fields.toString(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/tuple/MessageId.java b/jstorm-client/src/main/java/backtype/storm/tuple/MessageId.java deleted file mode 100644 index b1bd68a0a..000000000 --- a/jstorm-client/src/main/java/backtype/storm/tuple/MessageId.java +++ /dev/null @@ -1,86 +0,0 @@ -package backtype.storm.tuple; - -import backtype.storm.utils.Utils; -import com.esotericsoftware.kryo.io.Input; -import com.esotericsoftware.kryo.io.Output; -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Random; -import java.util.Set; - -public class MessageId { - private Map _anchorsToIds; - - @Deprecated - public static long generateId() { - return Utils.secureRandomLong(); - } - - public static long generateId(Random rand) { - return rand.nextLong(); - } - - public static MessageId makeUnanchored() { - return makeId(new HashMap()); - } - - public static MessageId makeId(Map anchorsToIds) { - return new MessageId(anchorsToIds); - } - - public static MessageId makeRootId(long id, long val) { - Map anchorsToIds = new HashMap(); - anchorsToIds.put(id, val); - return new MessageId(anchorsToIds); - } - - protected MessageId(Map anchorsToIds) { - _anchorsToIds = anchorsToIds; - } - - public Map getAnchorsToIds() { - return _anchorsToIds; - } - - public Set getAnchors() { - return _anchorsToIds.keySet(); - } - - @Override - public int hashCode() { - return _anchorsToIds.hashCode(); - } - - @Override - public boolean equals(Object other) { - if (other instanceof MessageId) { - return _anchorsToIds.equals(((MessageId) other)._anchorsToIds); - } else { - return false; - } - } - - @Override - public String toString() { - return _anchorsToIds.toString(); - } - - public void serialize(Output out) throws IOException { - out.writeInt(_anchorsToIds.size(), true); - for (Entry anchorToId : _anchorsToIds.entrySet()) { - out.writeLong(anchorToId.getKey()); - out.writeLong(anchorToId.getValue()); - } - } - - public static MessageId deserialize(Input in) throws IOException { - int numAnchors = in.readInt(true); - Map anchorsToIds = new HashMap(); - for (int i = 0; i < numAnchors; i++) { - anchorsToIds.put(in.readLong(), in.readLong()); - } - return new MessageId(anchorsToIds); - } -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/tuple/Tuple.java b/jstorm-client/src/main/java/backtype/storm/tuple/Tuple.java deleted file mode 100644 index f170cd283..000000000 --- a/jstorm-client/src/main/java/backtype/storm/tuple/Tuple.java +++ /dev/null @@ -1,43 +0,0 @@ -package backtype.storm.tuple; - -import backtype.storm.generated.GlobalStreamId; -import java.util.List; - -/** - * The tuple is the main data structure in Storm. A tuple is a named list of values, - * where each value can be any type. Tuples are dynamically typed -- the types of the fields - * do not need to be declared. Tuples have helper methods like getInteger and getString - * to get field values without having to cast the result. - * - * Storm needs to know how to serialize all the values in a tuple. By default, Storm - * knows how to serialize the primitive types, strings, and byte arrays. If you want to - * use another type, you'll need to implement and register a serializer for that type. - * See {@link http://github.com/nathanmarz/storm/wiki/Serialization} for more info. - */ -public interface Tuple extends ITuple{ - - /** - * Returns the global stream id (component + stream) of this tuple. - */ - public GlobalStreamId getSourceGlobalStreamid(); - - /** - * Gets the id of the component that created this tuple. - */ - public String getSourceComponent(); - - /** - * Gets the id of the task that created this tuple. - */ - public int getSourceTask(); - - /** - * Gets the id of the stream that this tuple was emitted to. - */ - public String getSourceStreamId(); - - /** - * Gets the message id that associated with this tuple. - */ - public MessageId getMessageId(); -} diff --git a/jstorm-client/src/main/java/backtype/storm/tuple/TupleExt.java b/jstorm-client/src/main/java/backtype/storm/tuple/TupleExt.java deleted file mode 100644 index 730734215..000000000 --- a/jstorm-client/src/main/java/backtype/storm/tuple/TupleExt.java +++ /dev/null @@ -1,12 +0,0 @@ -package backtype.storm.tuple; - -public interface TupleExt extends Tuple { - /** - * Get Target TaskId - * - * @return - */ - int getTargetTaskId(); - - void setTargetTaskId(int targetTaskId); -} diff --git a/jstorm-client/src/main/java/backtype/storm/tuple/TupleImpl.java b/jstorm-client/src/main/java/backtype/storm/tuple/TupleImpl.java deleted file mode 100644 index 2f47f6eb7..000000000 --- a/jstorm-client/src/main/java/backtype/storm/tuple/TupleImpl.java +++ /dev/null @@ -1,342 +0,0 @@ -package backtype.storm.tuple; - -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.task.GeneralTopologyContext; -import backtype.storm.utils.IndifferentAccessMap; -import clojure.lang.ASeq; -import clojure.lang.Counted; -import clojure.lang.IMeta; -import clojure.lang.IPersistentMap; -import clojure.lang.ISeq; -import clojure.lang.Indexed; -import clojure.lang.Keyword; -import clojure.lang.MapEntry; -import clojure.lang.Obj; -import clojure.lang.PersistentArrayMap; -import clojure.lang.Seqable; -import clojure.lang.Symbol; -import java.util.List; - -public class TupleImpl extends IndifferentAccessMap implements Seqable, - Indexed, IMeta, Tuple { - private List values; - private int taskId; - private String streamId; - private GeneralTopologyContext context; - private MessageId id; - private IPersistentMap _meta = null; - - public TupleImpl(GeneralTopologyContext context, List values, - int taskId, String streamId, MessageId id) { - this.values = values; - this.taskId = taskId; - this.streamId = streamId; - this.id = id; - this.context = context; - - String componentId = context.getComponentId(taskId); - Fields schema = context.getComponentOutputFields(componentId, streamId); - if (values.size() != schema.size()) { - throw new IllegalArgumentException( - "Tuple created with wrong number of fields. " + "Expected " - + schema.size() + " fields but got " - + values.size() + " fields"); - } - } - - public TupleImpl(GeneralTopologyContext context, List values, - int taskId, String streamId) { - this(context, values, taskId, streamId, MessageId.makeUnanchored()); - } - - Long _processSampleStartTime = null; - Long _executeSampleStartTime = null; - - public void setProcessSampleStartTime(long ms) { - _processSampleStartTime = ms; - } - - public Long getProcessSampleStartTime() { - return _processSampleStartTime; - } - - public void setExecuteSampleStartTime(long ms) { - _executeSampleStartTime = ms; - } - - public Long getExecuteSampleStartTime() { - return _executeSampleStartTime; - } - - long _outAckVal = 0; - - public void updateAckVal(long val) { - _outAckVal = _outAckVal ^ val; - } - - public long getAckVal() { - return _outAckVal; - } - - public int size() { - return values.size(); - } - - public int fieldIndex(String field) { - return getFields().fieldIndex(field); - } - - public boolean contains(String field) { - return getFields().contains(field); - } - - public Object getValue(int i) { - return values.get(i); - } - - public String getString(int i) { - return (String) values.get(i); - } - - public Integer getInteger(int i) { - return (Integer) values.get(i); - } - - public Long getLong(int i) { - return (Long) values.get(i); - } - - public Boolean getBoolean(int i) { - return (Boolean) values.get(i); - } - - public Short getShort(int i) { - return (Short) values.get(i); - } - - public Byte getByte(int i) { - return (Byte) values.get(i); - } - - public Double getDouble(int i) { - return (Double) values.get(i); - } - - public Float getFloat(int i) { - return (Float) values.get(i); - } - - public byte[] getBinary(int i) { - return (byte[]) values.get(i); - } - - public Object getValueByField(String field) { - return values.get(fieldIndex(field)); - } - - public String getStringByField(String field) { - return (String) values.get(fieldIndex(field)); - } - - public Integer getIntegerByField(String field) { - return (Integer) values.get(fieldIndex(field)); - } - - public Long getLongByField(String field) { - return (Long) values.get(fieldIndex(field)); - } - - public Boolean getBooleanByField(String field) { - return (Boolean) values.get(fieldIndex(field)); - } - - public Short getShortByField(String field) { - return (Short) values.get(fieldIndex(field)); - } - - public Byte getByteByField(String field) { - return (Byte) values.get(fieldIndex(field)); - } - - public Double getDoubleByField(String field) { - return (Double) values.get(fieldIndex(field)); - } - - public Float getFloatByField(String field) { - return (Float) values.get(fieldIndex(field)); - } - - public byte[] getBinaryByField(String field) { - return (byte[]) values.get(fieldIndex(field)); - } - - public List getValues() { - return values; - } - - public Fields getFields() { - return context.getComponentOutputFields(getSourceComponent(), - getSourceStreamId()); - } - - public List select(Fields selector) { - return getFields().select(selector, values); - } - - public GlobalStreamId getSourceGlobalStreamid() { - return new GlobalStreamId(getSourceComponent(), streamId); - } - - public String getSourceComponent() { - return context.getComponentId(taskId); - } - - public int getSourceTask() { - return taskId; - } - - public String getSourceStreamId() { - return streamId; - } - - public MessageId getMessageId() { - return id; - } - - @Override - public String toString() { - return "source: " + getSourceComponent() + ":" + taskId + ", stream: " - + streamId + ", id: " + id.toString() + ", " - + values.toString(); - } - - @Override - public boolean equals(Object other) { - return this == other; - } - - @Override - public int hashCode() { - return System.identityHashCode(this); - } - - private final Keyword makeKeyword(String name) { - return Keyword.intern(Symbol.create(name)); - } - - /* ILookup */ - @Override - public Object valAt(Object o) { - try { - if (o instanceof Keyword) { - return getValueByField(((Keyword) o).getName()); - } else if (o instanceof String) { - return getValueByField((String) o); - } - } catch (IllegalArgumentException e) { - } - return null; - } - - /* Seqable */ - public ISeq seq() { - if (values.size() > 0) { - return new Seq(getFields().toList(), values, 0); - } - return null; - } - - static class Seq extends ASeq implements Counted { - final List fields; - final List values; - final int i; - - Seq(List fields, List values, int i) { - this.fields = fields; - this.values = values; - assert i >= 0; - this.i = i; - } - - public Seq(IPersistentMap meta, List fields, - List values, int i) { - super(meta); - this.fields = fields; - this.values = values; - assert i >= 0; - this.i = i; - } - - public Object first() { - return new MapEntry(fields.get(i), values.get(i)); - } - - public ISeq next() { - if (i + 1 < fields.size()) { - return new Seq(fields, values, i + 1); - } - return null; - } - - public int count() { - assert fields.size() - i >= 0 : "index out of bounds"; - // i being the position in the fields of this seq, the remainder of - // the seq is the size - return fields.size() - i; - } - - public Obj withMeta(IPersistentMap meta) { - return new Seq(meta, fields, values, i); - } - } - - /* Indexed */ - public Object nth(int i) { - if (i < values.size()) { - return values.get(i); - } else { - return null; - } - } - - public Object nth(int i, Object notfound) { - Object ret = nth(i); - if (ret == null) - ret = notfound; - return ret; - } - - /* Counted */ - public int count() { - return values.size(); - } - - /* IMeta */ - public IPersistentMap meta() { - if (_meta == null) { - _meta = new PersistentArrayMap(new Object[] { - makeKeyword("stream"), getSourceStreamId(), - makeKeyword("component"), getSourceComponent(), - makeKeyword("task"), getSourceTask() }); - } - return _meta; - } - - private PersistentArrayMap toMap() { - Object array[] = new Object[values.size() * 2]; - List fields = getFields().toList(); - for (int i = 0; i < values.size(); i++) { - array[i * 2] = fields.get(i); - array[(i * 2) + 1] = values.get(i); - } - return new PersistentArrayMap(array); - } - - public IPersistentMap getMap() { - if (_map == null) { - setMap(toMap()); - } - return _map; - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/tuple/TupleImplExt.java b/jstorm-client/src/main/java/backtype/storm/tuple/TupleImplExt.java deleted file mode 100644 index 5d4b487cd..000000000 --- a/jstorm-client/src/main/java/backtype/storm/tuple/TupleImplExt.java +++ /dev/null @@ -1,31 +0,0 @@ -package backtype.storm.tuple; - -import java.util.List; - -import backtype.storm.task.GeneralTopologyContext; - -public class TupleImplExt extends TupleImpl implements TupleExt { - - protected int targetTaskId; - - public TupleImplExt(GeneralTopologyContext context, List values, - int taskId, String streamId) { - super(context, values, taskId, streamId); - } - - public TupleImplExt(GeneralTopologyContext context, List values, - int taskId, String streamId, MessageId id) { - super(context, values, taskId, streamId, id); - } - - @Override - public int getTargetTaskId() { - return targetTaskId; - } - - @Override - public void setTargetTaskId(int targetTaskId) { - this.targetTaskId = targetTaskId; - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/tuple/Values.java b/jstorm-client/src/main/java/backtype/storm/tuple/Values.java deleted file mode 100644 index d374f67f3..000000000 --- a/jstorm-client/src/main/java/backtype/storm/tuple/Values.java +++ /dev/null @@ -1,20 +0,0 @@ -package backtype.storm.tuple; - -import java.util.ArrayList; - -/** - * A convenience class for making tuple values using new Values("field1", 2, 3) - * syntax. - */ -public class Values extends ArrayList { - public Values() { - - } - - public Values(Object... vals) { - super(vals.length); - for (Object o : vals) { - add(o); - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/BufferFileInputStream.java b/jstorm-client/src/main/java/backtype/storm/utils/BufferFileInputStream.java deleted file mode 100644 index c3e1a209f..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/BufferFileInputStream.java +++ /dev/null @@ -1,37 +0,0 @@ -package backtype.storm.utils; - -import java.io.FileInputStream; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.util.Arrays; - -public class BufferFileInputStream { - byte[] buffer; - FileInputStream stream; - - public BufferFileInputStream(String file, int bufferSize) - throws FileNotFoundException { - stream = new FileInputStream(file); - buffer = new byte[bufferSize]; - } - - public BufferFileInputStream(String file) throws FileNotFoundException { - this(file, 15 * 1024); - } - - public byte[] read() throws IOException { - int length = stream.read(buffer); - if (length == -1) { - close(); - return new byte[0]; - } else if (length == buffer.length) { - return buffer; - } else { - return Arrays.copyOf(buffer, length); - } - } - - public void close() throws IOException { - stream.close(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/CRC32OutputStream.java b/jstorm-client/src/main/java/backtype/storm/utils/CRC32OutputStream.java deleted file mode 100644 index 46265b034..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/CRC32OutputStream.java +++ /dev/null @@ -1,27 +0,0 @@ -package backtype.storm.utils; - -import java.io.IOException; -import java.io.OutputStream; -import java.util.zip.CRC32; - -public class CRC32OutputStream extends OutputStream { - private CRC32 hasher; - - public CRC32OutputStream() { - hasher = new CRC32(); - } - - public long getValue() { - return hasher.getValue(); - } - - @Override - public void write(int i) throws IOException { - hasher.update(i); - } - - @Override - public void write(byte[] bytes, int start, int end) throws IOException { - hasher.update(bytes, start, end); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/ClojureTimerTask.java b/jstorm-client/src/main/java/backtype/storm/utils/ClojureTimerTask.java deleted file mode 100644 index b9094e2d4..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/ClojureTimerTask.java +++ /dev/null @@ -1,18 +0,0 @@ -package backtype.storm.utils; - -import clojure.lang.IFn; -import java.util.TimerTask; - -public class ClojureTimerTask extends TimerTask { - IFn _afn; - - public ClojureTimerTask(IFn afn) { - super(); - _afn = afn; - } - - @Override - public void run() { - _afn.run(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/Container.java b/jstorm-client/src/main/java/backtype/storm/utils/Container.java deleted file mode 100644 index b8a6f1222..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/Container.java +++ /dev/null @@ -1,7 +0,0 @@ -package backtype.storm.utils; - -import java.io.Serializable; - -public class Container implements Serializable { - public Object object; -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/utils/DRPCClient.java b/jstorm-client/src/main/java/backtype/storm/utils/DRPCClient.java deleted file mode 100644 index 975d7d8c6..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/DRPCClient.java +++ /dev/null @@ -1,70 +0,0 @@ -package backtype.storm.utils; - -import org.apache.thrift7.TException; -import org.apache.thrift7.protocol.TBinaryProtocol; -import org.apache.thrift7.transport.TFramedTransport; -import org.apache.thrift7.transport.TSocket; -import org.apache.thrift7.transport.TTransport; - -import backtype.storm.generated.DRPCExecutionException; -import backtype.storm.generated.DistributedRPC; - -public class DRPCClient implements DistributedRPC.Iface { - private TTransport conn; - private DistributedRPC.Client client; - private String host; - private int port; - private Integer timeout; - - public DRPCClient(String host, int port, Integer timeout) { - try { - this.host = host; - this.port = port; - this.timeout = timeout; - connect(); - } catch (TException e) { - throw new RuntimeException(e); - } - } - - public DRPCClient(String host, int port) { - this(host, port, null); - } - - private void connect() throws TException { - TSocket socket = new TSocket(host, port); - if (timeout != null) { - socket.setTimeout(timeout); - } - conn = new TFramedTransport(socket); - client = new DistributedRPC.Client(new TBinaryProtocol(conn)); - conn.open(); - } - - public String getHost() { - return host; - } - - public int getPort() { - return port; - } - - public String execute(String func, String args) throws TException, - DRPCExecutionException { - try { - if (client == null) - connect(); - return client.execute(func, args); - } catch (TException e) { - client = null; - throw e; - } catch (DRPCExecutionException e) { - client = null; - throw e; - } - } - - public void close() { - conn.close(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/DisruptorQueue.java b/jstorm-client/src/main/java/backtype/storm/utils/DisruptorQueue.java deleted file mode 100644 index 8d9e86110..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/DisruptorQueue.java +++ /dev/null @@ -1,71 +0,0 @@ -package backtype.storm.utils; - -import backtype.storm.metric.api.IStatefulObject; - -import com.lmax.disruptor.EventHandler; -import com.lmax.disruptor.InsufficientCapacityException; -import com.lmax.disruptor.WaitStrategy; -import com.lmax.disruptor.dsl.ProducerType; - -/** - * - * A single consumer queue that uses the LMAX Disruptor. They key to the - * performance is the ability to catch up to the producer by processing tuples - * in batches. - */ -public abstract class DisruptorQueue implements IStatefulObject { - public static void setUseSleep(boolean useSleep) { - DisruptorQueueImpl.setUseSleep(useSleep); - } - - private static boolean CAPACITY_LIMITED = false; - - public static void setLimited(boolean limited) { - CAPACITY_LIMITED = limited; - } - - public static DisruptorQueue mkInstance(String queueName, - ProducerType producerType, int bufferSize, WaitStrategy wait) { - if (CAPACITY_LIMITED == true) { - return new DisruptorQueueImpl(queueName, producerType, bufferSize, - wait); - } else { - return new DisruptorWrapBlockingQueue(queueName, producerType, - bufferSize, wait); - } - } - - public abstract String getName(); - - - - public abstract void haltWithInterrupt(); - - public abstract Object poll(); - - public abstract Object take(); - - public abstract void consumeBatch(EventHandler handler); - - public abstract void consumeBatchWhenAvailable(EventHandler handler); - - public abstract void publish(Object obj); - - public abstract void publish(Object obj, boolean block) - throws InsufficientCapacityException; - - public abstract void consumerStarted(); - - public abstract void clear(); - - public abstract long population(); - - public abstract long capacity(); - - public abstract long writePos(); - - public abstract long readPos(); - - public abstract float pctFull(); - -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/DisruptorQueueImpl.java b/jstorm-client/src/main/java/backtype/storm/utils/DisruptorQueueImpl.java deleted file mode 100644 index 0c334b54c..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/DisruptorQueueImpl.java +++ /dev/null @@ -1,298 +0,0 @@ -package backtype.storm.utils; - -import java.util.HashMap; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantReadWriteLock; - -import org.apache.log4j.Logger; - -import backtype.storm.metric.api.IStatefulObject; -import backtype.storm.utils.disruptor.AbstractSequencerExt; -import backtype.storm.utils.disruptor.RingBuffer; - -import com.lmax.disruptor.AlertException; -import com.lmax.disruptor.EventFactory; -import com.lmax.disruptor.EventHandler; -import com.lmax.disruptor.InsufficientCapacityException; -import com.lmax.disruptor.Sequence; -import com.lmax.disruptor.SequenceBarrier; -import com.lmax.disruptor.TimeoutException; -import com.lmax.disruptor.WaitStrategy; -import com.lmax.disruptor.dsl.ProducerType; - -/** - * - * A single consumer queue that uses the LMAX Disruptor. They key to the - * performance is the ability to catch up to the producer by processing tuples - * in batches. - */ -public class DisruptorQueueImpl extends DisruptorQueue { - private static final Logger LOG = Logger.getLogger(DisruptorQueueImpl.class); - static boolean useSleep = true; - public static void setUseSleep(boolean useSleep) { - AbstractSequencerExt.setWaitSleep(useSleep); - } - - private static final Object FLUSH_CACHE = new Object(); - private static final Object INTERRUPT = new Object(); - private static final String PREFIX = "disruptor-"; - - private final String _queueName; - private final RingBuffer _buffer; - private final Sequence _consumer; - private final SequenceBarrier _barrier; - - // TODO: consider having a threadlocal cache of this variable to speed up - // reads? - volatile boolean consumerStartedFlag = false; - - private final HashMap state = new HashMap(4); - private final ConcurrentLinkedQueue _cache = new ConcurrentLinkedQueue(); - private final ReentrantReadWriteLock cacheLock = new ReentrantReadWriteLock(); - private final Lock readLock = cacheLock.readLock(); - private final Lock writeLock = cacheLock.writeLock(); - - public DisruptorQueueImpl(String queueName, ProducerType producerType, - int bufferSize, WaitStrategy wait) { - this._queueName = PREFIX + queueName; - _buffer = RingBuffer.create(producerType, new ObjectEventFactory(), - bufferSize, wait); - _consumer = new Sequence(); - _barrier = _buffer.newBarrier(); - _buffer.addGatingSequences(_consumer); - if (producerType == ProducerType.SINGLE) { - consumerStartedFlag = true; - } else { - // make sure we flush the pending messages in cache first - if (bufferSize < 2) { - throw new RuntimeException("QueueSize must >= 2"); - } - try { - publishDirect(FLUSH_CACHE, true); - } catch (InsufficientCapacityException e) { - throw new RuntimeException("This code should be unreachable!", - e); - } - } - } - - public String getName() { - return _queueName; - } - - public void consumeBatch(EventHandler handler) { - consumeBatchToCursor(_barrier.getCursor(), handler); - } - - public void haltWithInterrupt() { - publish(INTERRUPT); - } - - public Object poll() { - // @@@ - // should use _cache.isEmpty, but it is slow - // I will change the logic later - if (consumerStartedFlag == false) { - return _cache.poll(); - } - - final long nextSequence = _consumer.get() + 1; - if (nextSequence <= _barrier.getCursor()) { - MutableObject mo = _buffer.get(nextSequence); - _consumer.set(nextSequence); - Object ret = mo.o; - mo.setObject(null); - return ret; - } - return null; - } - - public Object take() { - // @@@ - // should use _cache.isEmpty, but it is slow - // I will change the logic later - if (consumerStartedFlag == false) { - return _cache.poll(); - } - - final long nextSequence = _consumer.get() + 1; - // final long availableSequence; - try { - _barrier.waitFor(nextSequence); - } catch (AlertException e) { - LOG.error(e.getCause(), e); - throw new RuntimeException(e); - } catch (InterruptedException e) { - LOG.error("InterruptedException " + e.getCause()); - // throw new RuntimeException(e); - return null; - } catch (TimeoutException e) { - LOG.error(e.getCause(), e); - return null; - } - MutableObject mo = _buffer.get(nextSequence); - _consumer.set(nextSequence); - Object ret = mo.o; - mo.setObject(null); - return ret; - } - - public void consumeBatchWhenAvailable(EventHandler handler) { - try { - final long nextSequence = _consumer.get() + 1; - final long availableSequence = _barrier.waitFor(nextSequence); - if (availableSequence >= nextSequence) { - consumeBatchToCursor(availableSequence, handler); - } - } catch (AlertException e) { - LOG.error(e.getCause(), e); - throw new RuntimeException(e); - } catch (InterruptedException e) { - LOG.error("InterruptedException " + e.getCause()); - return; - }catch (TimeoutException e) { - LOG.error(e.getCause(), e); - return ; - } - } - - public void consumeBatchToCursor(long cursor, EventHandler handler){ - for (long curr = _consumer.get() + 1; curr <= cursor; curr++) { - try { - MutableObject mo = _buffer.get(curr); - Object o = mo.o; - mo.setObject(null); - if (o == FLUSH_CACHE) { - Object c = null; - while (true) { - c = _cache.poll(); - if (c == null) - break; - else - handler.onEvent(c, curr, true); - } - } else if (o == INTERRUPT) { - throw new InterruptedException( - "Disruptor processing interrupted"); - } else { - handler.onEvent(o, curr, curr == cursor); - } - } catch (InterruptedException e) { - // throw new RuntimeException(e); - LOG.error(e.getCause()); - return; - } catch (Exception e) { - LOG.error(e.getCause(), e); - throw new RuntimeException(e); - } - } - // TODO: only set this if the consumer cursor has changed? - _consumer.set(cursor); - } - - /* - * Caches until consumerStarted is called, upon which the cache is flushed - * to the consumer - */ - public void publish(Object obj) { - try { - publish(obj, true); - } catch (InsufficientCapacityException ex) { - throw new RuntimeException("This code should be unreachable!"); - } - } - - public void tryPublish(Object obj) throws InsufficientCapacityException { - publish(obj, false); - } - - public void publish(Object obj, boolean block) - throws InsufficientCapacityException { - - boolean publishNow = consumerStartedFlag; - - if (!publishNow) { - readLock.lock(); - try { - publishNow = consumerStartedFlag; - if (!publishNow) { - _cache.add(obj); - } - } finally { - readLock.unlock(); - } - } - - if (publishNow) { - publishDirect(obj, block); - } - } - - protected void publishDirect(Object obj, boolean block) - throws InsufficientCapacityException { - final long id; - if (block) { - id = _buffer.next(); - } else { - id = _buffer.tryNext(1); - } - final MutableObject m = _buffer.get(id); - m.setObject(obj); - _buffer.publish(id); - } - - public void consumerStarted() { - - writeLock.lock(); - consumerStartedFlag = true; - - writeLock.unlock(); - } - - public void clear() { - while (population() != 0L) { - poll(); - } - } - - public long population() { - return (writePos() - readPos()); - } - - public long capacity() { - return _buffer.getBufferSize(); - } - - public long writePos() { - return _buffer.getCursor(); - } - - public long readPos() { - return _consumer.get(); - } - - public float pctFull() { - return (1.0F * population() / capacity()); - } - - @Override - public Object getState() { - // get readPos then writePos so it's never an under-estimate - long rp = readPos(); - long wp = writePos(); - state.put("capacity", capacity()); - state.put("population", wp - rp); - state.put("write_pos", wp); - state.put("read_pos", rp); - return state; - } - - public static class ObjectEventFactory implements - EventFactory { - @Override - public MutableObject newInstance() { - return new MutableObject(); - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/DisruptorWrapBlockingQueue.java b/jstorm-client/src/main/java/backtype/storm/utils/DisruptorWrapBlockingQueue.java deleted file mode 100644 index a701f39e8..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/DisruptorWrapBlockingQueue.java +++ /dev/null @@ -1,192 +0,0 @@ -package backtype.storm.utils; - -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.LinkedBlockingDeque; -import java.util.concurrent.TimeUnit; - -import org.apache.log4j.Logger; - -import backtype.storm.metric.api.IStatefulObject; - -import com.lmax.disruptor.EventFactory; -import com.lmax.disruptor.EventHandler; -import com.lmax.disruptor.InsufficientCapacityException; -import com.lmax.disruptor.WaitStrategy; -import com.lmax.disruptor.dsl.ProducerType; - -/** - * - * A single consumer queue that uses the LMAX Disruptor. They key to the - * performance is the ability to catch up to the producer by processing tuples - * in batches. - */ -public class DisruptorWrapBlockingQueue extends DisruptorQueue { - private static final Logger LOG = Logger - .getLogger(DisruptorWrapBlockingQueue.class); - - private static final long QUEUE_CAPACITY = 512; - private LinkedBlockingDeque queue; - - private String queueName; - - public DisruptorWrapBlockingQueue(String queueName, - ProducerType producerType, int bufferSize, WaitStrategy wait) { - this.queueName = queueName; - queue = new LinkedBlockingDeque(); - } - - public String getName() { - return queueName; - } - - // poll method - public void consumeBatch(EventHandler handler) { - consumeBatchToCursor(0, handler); - } - - public void haltWithInterrupt() { - } - - public Object poll() { - return queue.poll(); - } - - public Object take() { - try { - return queue.take(); - } catch (InterruptedException e) { - return null; - } - } - - public void drainQueue(Object object, EventHandler handler) { - while (object != null) { - try { - handler.onEvent(object, 0, false); - object = queue.poll(); - } catch (InterruptedException e) { - LOG.warn("Occur interrupt error, " + object); - break; - } catch (Exception e) { - throw new RuntimeException(e); - } - } - } - - public void consumeBatchWhenAvailable(EventHandler handler) { - Object object = queue.poll(); - if (object == null) { - try { - object = queue.take(); - } catch (InterruptedException e) { - LOG.warn("Occur interrupt error, " + object); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - drainQueue(object, handler); - - } - - public void consumeBatchToCursor(long cursor, EventHandler handler) { - Object object = queue.poll(); - drainQueue(object, handler); - } - - /* - * Caches until consumerStarted is called, upon which the cache is flushed - * to the consumer - */ - public void publish(Object obj) { - boolean isSuccess = queue.offer(obj); - while (isSuccess == false) { - try { - Thread.sleep(1); - } catch (InterruptedException e) { - } - isSuccess = queue.offer(obj); - } - - } - - public void tryPublish(Object obj) throws InsufficientCapacityException { - boolean isSuccess = queue.offer(obj); - if (isSuccess == false) { - throw InsufficientCapacityException.INSTANCE; - } - - } - - public void publish(Object obj, boolean block) - throws InsufficientCapacityException { - if (block == true) { - publish(obj); - } else { - tryPublish(obj); - } - } - - public void consumerStarted() { - } - - private void flushCache() { - } - - public void clear() { - queue.clear(); - } - - public long population() { - return queue.size(); - } - - public long capacity() { - long used = queue.size(); - if (used < QUEUE_CAPACITY) { - return QUEUE_CAPACITY; - } else { - return used; - } - } - - public long writePos() { - return 0; - } - - public long readPos() { - return queue.size(); - } - - public float pctFull() { - long used = queue.size(); - if (used < QUEUE_CAPACITY) { - return (1.0F * used / QUEUE_CAPACITY); - } else { - return 1.0f; - } - } - - @Override - public Object getState() { - Map state = new HashMap(); - // get readPos then writePos so it's never an under-estimate - long rp = readPos(); - long wp = writePos(); - state.put("capacity", capacity()); - state.put("population", wp - rp); - state.put("write_pos", wp); - state.put("read_pos", rp); - return state; - } - - public static class ObjectEventFactory implements - EventFactory { - @Override - public MutableObject newInstance() { - return new MutableObject(); - } - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/IndifferentAccessMap.java b/jstorm-client/src/main/java/backtype/storm/utils/IndifferentAccessMap.java deleted file mode 100644 index 74c4a63f8..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/IndifferentAccessMap.java +++ /dev/null @@ -1,169 +0,0 @@ -package backtype.storm.utils; - -import clojure.lang.ILookup; -import clojure.lang.ISeq; -import clojure.lang.AFn; -import clojure.lang.IPersistentMap; -import clojure.lang.PersistentArrayMap; -import clojure.lang.IMapEntry; -import clojure.lang.IPersistentCollection; -import clojure.lang.Keyword; -import java.util.Iterator; -import java.util.Map; -import java.util.Collection; -import java.util.Set; - -public class IndifferentAccessMap extends AFn implements ILookup, - IPersistentMap, Map { - - protected IPersistentMap _map; - - protected IndifferentAccessMap() { - } - - public IndifferentAccessMap(IPersistentMap map) { - setMap(map); - } - - public IPersistentMap getMap() { - return _map; - } - - public IPersistentMap setMap(IPersistentMap map) { - _map = map; - return _map; - } - - public int size() { - return ((Map) getMap()).size(); - } - - public int count() { - return size(); - } - - public ISeq seq() { - return getMap().seq(); - } - - @Override - public Object valAt(Object o) { - if (o instanceof Keyword) { - return valAt(((Keyword) o).getName()); - } - return getMap().valAt(o); - } - - @Override - public Object valAt(Object o, Object def) { - Object ret = valAt(o); - if (ret == null) - ret = def; - return ret; - } - - /* IFn */ - @Override - public Object invoke(Object o) { - return valAt(o); - } - - @Override - public Object invoke(Object o, Object notfound) { - return valAt(o, notfound); - } - - /* IPersistentMap */ - /* Naive implementation, but it might be good enough */ - public IPersistentMap assoc(Object k, Object v) { - if (k instanceof Keyword) - return assoc(((Keyword) k).getName(), v); - - return new IndifferentAccessMap(getMap().assoc(k, v)); - } - - public IPersistentMap assocEx(Object k, Object v) { - if (k instanceof Keyword) - return assocEx(((Keyword) k).getName(), v); - - return new IndifferentAccessMap(getMap().assocEx(k, v)); - } - - public IPersistentMap without(Object k) { - if (k instanceof Keyword) - return without(((Keyword) k).getName()); - - return new IndifferentAccessMap(getMap().without(k)); - } - - public boolean containsKey(Object k) { - if (k instanceof Keyword) - return containsKey(((Keyword) k).getName()); - return getMap().containsKey(k); - } - - public IMapEntry entryAt(Object k) { - if (k instanceof Keyword) - return entryAt(((Keyword) k).getName()); - - return getMap().entryAt(k); - } - - public IPersistentCollection cons(Object o) { - return getMap().cons(o); - } - - public IPersistentCollection empty() { - return new IndifferentAccessMap(PersistentArrayMap.EMPTY); - } - - public boolean equiv(Object o) { - return getMap().equiv(o); - } - - public Iterator iterator() { - return getMap().iterator(); - } - - /* Map */ - public boolean containsValue(Object v) { - return ((Map) getMap()).containsValue(v); - } - - public Set entrySet() { - return ((Map) getMap()).entrySet(); - } - - public Object get(Object k) { - return valAt(k); - } - - public boolean isEmpty() { - return ((Map) getMap()).isEmpty(); - } - - public Set keySet() { - return ((Map) getMap()).keySet(); - } - - public Collection values() { - return ((Map) getMap()).values(); - } - - /* Not implemented */ - public void clear() { - throw new UnsupportedOperationException(); - } - - public Object put(Object k, Object v) { - throw new UnsupportedOperationException(); - } - - public void putAll(Map m) { - throw new UnsupportedOperationException(); - } - - public Object remove(Object k) { - throw new UnsupportedOperationException(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/InprocMessaging.java b/jstorm-client/src/main/java/backtype/storm/utils/InprocMessaging.java deleted file mode 100644 index c58c4da24..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/InprocMessaging.java +++ /dev/null @@ -1,42 +0,0 @@ -package backtype.storm.utils; - -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.LinkedBlockingQueue; - -public class InprocMessaging { - private static Map> _queues = new HashMap>(); - private static final Object _lock = new Object(); - private static int port = 1; - - public static int acquireNewPort() { - int ret; - synchronized (_lock) { - ret = port; - port++; - } - return ret; - } - - public static void sendMessage(int port, Object msg) { - getQueue(port).add(msg); - } - - public static Object takeMessage(int port) throws InterruptedException { - return getQueue(port).take(); - } - - public static Object pollMessage(int port) { - return getQueue(port).poll(); - } - - private static LinkedBlockingQueue getQueue(int port) { - synchronized (_lock) { - if (!_queues.containsKey(port)) { - _queues.put(port, new LinkedBlockingQueue()); - } - return _queues.get(port); - } - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/KeyedRoundRobinQueue.java b/jstorm-client/src/main/java/backtype/storm/utils/KeyedRoundRobinQueue.java deleted file mode 100644 index 8e14d1c67..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/KeyedRoundRobinQueue.java +++ /dev/null @@ -1,51 +0,0 @@ -package backtype.storm.utils; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Queue; -import java.util.concurrent.Semaphore; - -public class KeyedRoundRobinQueue { - private final Object _lock = new Object(); - private Semaphore _size = new Semaphore(0); - private Map> _queues = new HashMap>(); - private List _keyOrder = new ArrayList(); - private int _currIndex = 0; - - public void add(Object key, V val) { - synchronized (_lock) { - Queue queue = _queues.get(key); - if (queue == null) { - queue = new LinkedList(); - _queues.put(key, queue); - _keyOrder.add(key); - } - queue.add(val); - } - _size.release(); - } - - public V take() throws InterruptedException { - _size.acquire(); - synchronized (_lock) { - Object key = _keyOrder.get(_currIndex); - Queue queue = _queues.get(key); - V ret = queue.remove(); - if (queue.isEmpty()) { - _keyOrder.remove(_currIndex); - _queues.remove(key); - if (_keyOrder.size() == 0) { - _currIndex = 0; - } else { - _currIndex = _currIndex % _keyOrder.size(); - } - } else { - _currIndex = (_currIndex + 1) % _keyOrder.size(); - } - return ret; - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/ListDelegate.java b/jstorm-client/src/main/java/backtype/storm/utils/ListDelegate.java deleted file mode 100644 index ac9884723..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/ListDelegate.java +++ /dev/null @@ -1,139 +0,0 @@ -package backtype.storm.utils; - -import java.util.Collection; -import java.util.Iterator; -import java.util.ArrayList; -import java.util.List; -import java.util.ListIterator; - -public class ListDelegate implements List { - private List _delegate; - - public ListDelegate() { - _delegate = new ArrayList(); - } - - public void setDelegate(List delegate) { - _delegate = delegate; - } - - public List getDelegate() { - return _delegate; - } - - @Override - public int size() { - return _delegate.size(); - } - - @Override - public boolean isEmpty() { - return _delegate.isEmpty(); - } - - @Override - public boolean contains(Object o) { - return _delegate.contains(o); - } - - @Override - public Iterator iterator() { - return _delegate.iterator(); - } - - @Override - public Object[] toArray() { - return _delegate.toArray(); - } - - @Override - public T[] toArray(T[] ts) { - return _delegate.toArray(ts); - } - - @Override - public boolean add(Object e) { - return _delegate.add(e); - } - - @Override - public boolean remove(Object o) { - return _delegate.remove(o); - } - - @Override - public boolean containsAll(Collection clctn) { - return _delegate.containsAll(clctn); - } - - @Override - public boolean addAll(Collection clctn) { - return _delegate.addAll(clctn); - } - - @Override - public boolean addAll(int i, Collection clctn) { - return _delegate.addAll(i, clctn); - } - - @Override - public boolean removeAll(Collection clctn) { - return _delegate.removeAll(clctn); - } - - @Override - public boolean retainAll(Collection clctn) { - return _delegate.retainAll(clctn); - } - - @Override - public void clear() { - _delegate.clear(); - } - - @Override - public Object get(int i) { - return _delegate.get(i); - } - - @Override - public Object set(int i, Object e) { - return _delegate.set(i, e); - } - - @Override - public void add(int i, Object e) { - _delegate.add(i, e); - } - - @Override - public Object remove(int i) { - return _delegate.remove(i); - } - - @Override - public int indexOf(Object o) { - return _delegate.indexOf(o); - } - - @Override - public int lastIndexOf(Object o) { - return _delegate.lastIndexOf(o); - } - - @Override - public ListIterator listIterator() { - return _delegate.listIterator(); - } - - @Override - public ListIterator listIterator(int i) { - return _delegate.listIterator(i); - } - - @Override - public List subList(int i, int i1) { - return _delegate.subList(i, i1); - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/LocalState.java b/jstorm-client/src/main/java/backtype/storm/utils/LocalState.java deleted file mode 100644 index 6be953487..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/LocalState.java +++ /dev/null @@ -1,78 +0,0 @@ -package backtype.storm.utils; - -import org.apache.commons.io.FileUtils; - -import java.io.File; -import java.util.Map; -import java.util.HashMap; -import java.io.IOException; - -/** - * A simple, durable, atomic K/V database. *Very inefficient*, should only be - * used for occasional reads/writes. Every read/write hits disk. - */ -public class LocalState { - private VersionedStore _vs; - - public LocalState(String backingDir) throws IOException { - _vs = new VersionedStore(backingDir); - } - - public synchronized Map snapshot() throws IOException { - int attempts = 0; - while (true) { - String latestPath = _vs.mostRecentVersionPath(); - if (latestPath == null) - return new HashMap(); - try { - return (Map) Utils.deserialize(FileUtils - .readFileToByteArray(new File(latestPath))); - } catch (IOException e) { - attempts++; - if (attempts >= 10) { - throw e; - } - } - } - } - - public Object get(Object key) throws IOException { - return snapshot().get(key); - } - - public synchronized void put(Object key, Object val) throws IOException { - put(key, val, true); - } - - public synchronized void put(Object key, Object val, boolean cleanup) - throws IOException { - Map curr = snapshot(); - curr.put(key, val); - persist(curr, cleanup); - } - - public synchronized void remove(Object key) throws IOException { - remove(key, true); - } - - public synchronized void remove(Object key, boolean cleanup) - throws IOException { - Map curr = snapshot(); - curr.remove(key); - persist(curr, cleanup); - } - - public synchronized void cleanup(int keepVersions) throws IOException { - _vs.cleanup(keepVersions); - } - - private void persist(Map val, boolean cleanup) - throws IOException { - byte[] toWrite = Utils.serialize(val); - String newPath = _vs.createVersion(); - FileUtils.writeByteArrayToFile(new File(newPath), toWrite); - _vs.succeedVersion(newPath); - if (cleanup) - _vs.cleanup(4); - } -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/backtype/storm/utils/MutableInt.java b/jstorm-client/src/main/java/backtype/storm/utils/MutableInt.java deleted file mode 100644 index 0f41f5cbd..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/MutableInt.java +++ /dev/null @@ -1,26 +0,0 @@ -package backtype.storm.utils; - -public class MutableInt { - int val; - - public MutableInt(int val) { - this.val = val; - } - - public void set(int val) { - this.val = val; - } - - public int get() { - return val; - } - - public int increment() { - return increment(1); - } - - public int increment(int amt) { - val += amt; - return val; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/MutableLong.java b/jstorm-client/src/main/java/backtype/storm/utils/MutableLong.java deleted file mode 100644 index 25b066f30..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/MutableLong.java +++ /dev/null @@ -1,26 +0,0 @@ -package backtype.storm.utils; - -public class MutableLong { - long val; - - public MutableLong(long val) { - this.val = val; - } - - public void set(long val) { - this.val = val; - } - - public long get() { - return val; - } - - public long increment() { - return increment(1); - } - - public long increment(long amt) { - val += amt; - return val; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/MutableObject.java b/jstorm-client/src/main/java/backtype/storm/utils/MutableObject.java deleted file mode 100644 index 12e802b1c..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/MutableObject.java +++ /dev/null @@ -1,21 +0,0 @@ -package backtype.storm.utils; - -public class MutableObject { - Object o = null; - - public MutableObject() { - - } - - public MutableObject(Object o) { - this.o = o; - } - - public void setObject(Object o) { - this.o = o; - } - - public Object getObject() { - return o; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/NimbusClient.java b/jstorm-client/src/main/java/backtype/storm/utils/NimbusClient.java deleted file mode 100644 index 1d5ae3aae..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/NimbusClient.java +++ /dev/null @@ -1,48 +0,0 @@ -package backtype.storm.utils; - -import java.util.Map; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import backtype.storm.generated.Nimbus; -import backtype.storm.security.auth.ThriftClient; - -public class NimbusClient extends ThriftClient { - private Nimbus.Client _client; - private static final Logger LOG = LoggerFactory - .getLogger(NimbusClient.class); - - @SuppressWarnings("unchecked") - public static NimbusClient getConfiguredClient(Map conf) { - try { - // String nimbusHost = (String) conf.get(Config.NIMBUS_HOST); - // int nimbusPort = - // Utils.getInt(conf.get(Config.NIMBUS_THRIFT_PORT)); - // return new NimbusClient(conf, nimbusHost, nimbusPort); - return new NimbusClient(conf); - } catch (Exception ex) { - throw new RuntimeException(ex); - } - } - - private NimbusClient(Map conf) throws Exception { - this(conf, null); - } - - @SuppressWarnings("unchecked") - private NimbusClient(Map conf, Integer timeout) throws Exception { - super(conf, timeout); - flush(); - } - - public Nimbus.Client getClient() { - return _client; - } - - @Override - protected void flush() { - // TODO Auto-generated method stub - _client = new Nimbus.Client(_protocol); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/RegisteredGlobalState.java b/jstorm-client/src/main/java/backtype/storm/utils/RegisteredGlobalState.java deleted file mode 100644 index 74bb7491e..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/RegisteredGlobalState.java +++ /dev/null @@ -1,47 +0,0 @@ -package backtype.storm.utils; - -import java.util.HashMap; -import java.util.UUID; - -/** - * This class is used as part of testing Storm. It is used to keep track of - * "global metrics" in an atomic way. For example, it is used for doing - * fine-grained detection of when a local Storm cluster is idle by tracking the - * number of transferred tuples vs the number of processed tuples. - */ -public class RegisteredGlobalState { - private static HashMap _states = new HashMap(); - private static final Object _lock = new Object(); - - public static Object globalLock() { - return _lock; - } - - public static String registerState(Object init) { - synchronized (_lock) { - String id = UUID.randomUUID().toString(); - _states.put(id, init); - return id; - } - } - - public static void setState(String id, Object init) { - synchronized (_lock) { - _states.put(id, init); - } - } - - public static Object getState(String id) { - synchronized (_lock) { - Object ret = _states.get(id); - // System.out.println("State: " + ret.toString()); - return ret; - } - } - - public static void clearState(String id) { - synchronized (_lock) { - _states.remove(id); - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/RotatingMap.java b/jstorm-client/src/main/java/backtype/storm/utils/RotatingMap.java deleted file mode 100644 index c96663e39..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/RotatingMap.java +++ /dev/null @@ -1,109 +0,0 @@ -package backtype.storm.utils; - -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.Map; -import java.util.Map.Entry; - -import org.apache.log4j.Logger; - -/** - * Expires keys that have not been updated in the configured number of seconds. - * The algorithm used will take between expirationSecs and expirationSecs * (1 + - * 1 / (numBuckets-1)) to actually expire the message. - * - * get, put, remove, containsKey, and size take O(numBuckets) time to run. - * - * - * Please use com.alibaba.jstorm.utils.RotatingMap - */ -@Deprecated -public class RotatingMap { - // this default ensures things expire at most 50% past the expiration time - private static final int DEFAULT_NUM_BUCKETS = 3; - - public static interface ExpiredCallback { - public void expire(K key, V val); - } - - private LinkedList> _buckets; - - private ExpiredCallback _callback; - - public RotatingMap(int numBuckets, ExpiredCallback callback) { - if (numBuckets < 2) { - throw new IllegalArgumentException("numBuckets must be >= 2"); - } - _buckets = new LinkedList>(); - for (int i = 0; i < numBuckets; i++) { - _buckets.add(new HashMap()); - } - - _callback = callback; - } - - public RotatingMap(ExpiredCallback callback) { - this(DEFAULT_NUM_BUCKETS, callback); - } - - public RotatingMap(int numBuckets) { - this(numBuckets, null); - } - - public Map rotate() { - Map dead = _buckets.removeLast(); - _buckets.addFirst(new HashMap()); - if (_callback != null) { - for (Entry entry : dead.entrySet()) { - _callback.expire(entry.getKey(), entry.getValue()); - } - } - return dead; - } - - public boolean containsKey(K key) { - for (HashMap bucket : _buckets) { - if (bucket.containsKey(key)) { - return true; - } - } - return false; - } - - public V get(K key) { - for (HashMap bucket : _buckets) { - if (bucket.containsKey(key)) { - return bucket.get(key); - } - } - return null; - } - - public void put(K key, V value) { - Iterator> it = _buckets.iterator(); - HashMap bucket = it.next(); - bucket.put(key, value); - while (it.hasNext()) { - bucket = it.next(); - bucket.remove(key); - } - } - - public Object remove(K key) { - for (HashMap bucket : _buckets) { - if (bucket.containsKey(key)) { - return bucket.remove(key); - } - } - return null; - } - - public int size() { - int size = 0; - for (HashMap bucket : _buckets) { - size += bucket.size(); - } - return size; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/ServiceRegistry.java b/jstorm-client/src/main/java/backtype/storm/utils/ServiceRegistry.java deleted file mode 100644 index 554d2d0d6..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/ServiceRegistry.java +++ /dev/null @@ -1,30 +0,0 @@ -package backtype.storm.utils; - -import java.util.HashMap; -import java.util.UUID; - -// this class should be combined with RegisteredGlobalState -public class ServiceRegistry { - private static HashMap _services = new HashMap(); - private static final Object _lock = new Object(); - - public static String registerService(Object service) { - synchronized (_lock) { - String id = UUID.randomUUID().toString(); - _services.put(id, service); - return id; - } - } - - public static Object getService(String id) { - synchronized (_lock) { - return _services.get(id); - } - } - - public static void unregisterService(String id) { - synchronized (_lock) { - _services.remove(id); - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/ThreadResourceManager.java b/jstorm-client/src/main/java/backtype/storm/utils/ThreadResourceManager.java deleted file mode 100644 index 0971d0c35..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/ThreadResourceManager.java +++ /dev/null @@ -1,28 +0,0 @@ -package backtype.storm.utils; - -import java.util.concurrent.ConcurrentLinkedQueue; - -public class ThreadResourceManager { - public static interface ResourceFactory { - X makeResource(); - } - - ResourceFactory _factory; - ConcurrentLinkedQueue _resources = new ConcurrentLinkedQueue(); - - public ThreadResourceManager(ResourceFactory factory) { - _factory = factory; - } - - public T acquire() { - T ret = _resources.poll(); - if (ret == null) { - ret = _factory.makeResource(); - } - return ret; - } - - public void release(T resource) { - _resources.add(resource); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/ThriftTopologyUtils.java b/jstorm-client/src/main/java/backtype/storm/utils/ThriftTopologyUtils.java deleted file mode 100644 index 1b52fe386..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/ThriftTopologyUtils.java +++ /dev/null @@ -1,59 +0,0 @@ -package backtype.storm.utils; - -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -import backtype.storm.generated.Bolt; -import backtype.storm.generated.ComponentCommon; -import backtype.storm.generated.SpoutSpec; -import backtype.storm.generated.StateSpoutSpec; -import backtype.storm.generated.StormTopology; - -public class ThriftTopologyUtils { - public static Set getComponentIds(StormTopology topology) { - Set ret = new HashSet(); - for (StormTopology._Fields f : StormTopology.metaDataMap.keySet()) { - Map componentMap = (Map) topology - .getFieldValue(f); - ret.addAll(componentMap.keySet()); - } - return ret; - } - - public static Map getComponents(StormTopology topology) { - Map ret = new HashMap(); - for (StormTopology._Fields f : StormTopology.metaDataMap.keySet()) { - Map componentMap = (Map) topology - .getFieldValue(f); - ret.putAll(componentMap); - } - return ret; - } - - public static ComponentCommon getComponentCommon(StormTopology topology, - String componentId) { - for (StormTopology._Fields f : StormTopology.metaDataMap.keySet()) { - Map componentMap = (Map) topology - .getFieldValue(f); - if (componentMap.containsKey(componentId)) { - Object component = componentMap.get(componentId); - if (component instanceof Bolt) { - return ((Bolt) component).get_common(); - } - if (component instanceof SpoutSpec) { - return ((SpoutSpec) component).get_common(); - } - if (component instanceof StateSpoutSpec) { - return ((StateSpoutSpec) component).get_common(); - } - throw new RuntimeException( - "Unreachable code! No get_common conversion for component " - + component); - } - } - throw new IllegalArgumentException( - "Could not find component common for " + componentId); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/Time.java b/jstorm-client/src/main/java/backtype/storm/utils/Time.java deleted file mode 100644 index e791dc4e1..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/Time.java +++ /dev/null @@ -1,93 +0,0 @@ -package backtype.storm.utils; - -import java.util.concurrent.atomic.AtomicLong; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicBoolean; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class Time { - public static Logger LOG = LoggerFactory.getLogger(Time.class); - - private static AtomicBoolean simulating = new AtomicBoolean(false); - // TODO: should probably use weak references here or something - private static volatile Map threadSleepTimes; - private static final Object sleepTimesLock = new Object(); - - private static AtomicLong simulatedCurrTimeMs; // should this be a thread - // local that's allowed to - // keep advancing? - - public static void startSimulating() { - simulating.set(true); - simulatedCurrTimeMs = new AtomicLong(0); - threadSleepTimes = new ConcurrentHashMap(); - } - - public static void stopSimulating() { - simulating.set(false); - threadSleepTimes = null; - } - - public static boolean isSimulating() { - return simulating.get(); - } - - public static void sleepUntil(long targetTimeMs) - throws InterruptedException { - if (simulating.get()) { - try { - synchronized (sleepTimesLock) { - threadSleepTimes.put(Thread.currentThread(), - new AtomicLong(targetTimeMs)); - } - while (simulatedCurrTimeMs.get() < targetTimeMs) { - Thread.sleep(10); - } - } finally { - synchronized (sleepTimesLock) { - threadSleepTimes.remove(Thread.currentThread()); - } - } - } else { - long sleepTime = targetTimeMs - currentTimeMillis(); - if (sleepTime > 0) - Thread.sleep(sleepTime); - } - } - - public static void sleep(long ms) throws InterruptedException { - sleepUntil(currentTimeMillis() + ms); - } - - public static long currentTimeMillis() { - if (simulating.get()) { - return simulatedCurrTimeMs.get(); - } else { - return System.currentTimeMillis(); - } - } - - public static int currentTimeSecs() { - return (int) (currentTimeMillis() / 1000); - } - - public static void advanceTime(long ms) { - if (!simulating.get()) - throw new IllegalStateException( - "Cannot simulate time unless in simulation mode"); - simulatedCurrTimeMs.set(simulatedCurrTimeMs.get() + ms); - } - - public static boolean isThreadWaiting(Thread t) { - if (!simulating.get()) - throw new IllegalStateException("Must be in simulation mode"); - AtomicLong time; - synchronized (sleepTimesLock) { - time = threadSleepTimes.get(t); - } - return !t.isAlive() || time != null - && currentTimeMillis() < time.longValue(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/TimeCacheMap.java b/jstorm-client/src/main/java/backtype/storm/utils/TimeCacheMap.java deleted file mode 100644 index a1052c453..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/TimeCacheMap.java +++ /dev/null @@ -1,143 +0,0 @@ -package backtype.storm.utils; - -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.Map; -import java.util.Map.Entry; - -/** - * Expires keys that have not been updated in the configured number of seconds. - * The algorithm used will take between expirationSecs and expirationSecs * (1 + - * 1 / (numBuckets-1)) to actually expire the message. - * - * get, put, remove, containsKey, and size take O(numBuckets) time to run. - * - * Please use com.alibaba.jstorm.utils.TimeCacheMap - */ -@Deprecated -public class TimeCacheMap { - // this default ensures things expire at most 50% past the expiration time - private static final int DEFAULT_NUM_BUCKETS = 3; - - @Deprecated - public static interface ExpiredCallback { - public void expire(K key, V val); - } - - private LinkedList> _buckets; - - private final Object _lock = new Object(); - private Thread _cleaner; - private ExpiredCallback _callback; - - public TimeCacheMap(int expirationSecs, int numBuckets, - ExpiredCallback callback) { - if (numBuckets < 2) { - throw new IllegalArgumentException("numBuckets must be >= 2"); - } - _buckets = new LinkedList>(); - for (int i = 0; i < numBuckets; i++) { - _buckets.add(new HashMap()); - } - - _callback = callback; - final long expirationMillis = expirationSecs * 1000L; - final long sleepTime = expirationMillis / (numBuckets - 1); - _cleaner = new Thread(new Runnable() { - public void run() { - try { - while (true) { - Map dead = null; - Time.sleep(sleepTime); - synchronized (_lock) { - dead = _buckets.removeLast(); - _buckets.addFirst(new HashMap()); - } - if (_callback != null) { - for (Entry entry : dead.entrySet()) { - _callback.expire(entry.getKey(), - entry.getValue()); - } - } - } - } catch (InterruptedException ex) { - - } - } - }); - _cleaner.setDaemon(true); - _cleaner.start(); - } - - public TimeCacheMap(int expirationSecs, ExpiredCallback callback) { - this(expirationSecs, DEFAULT_NUM_BUCKETS, callback); - } - - public TimeCacheMap(int expirationSecs) { - this(expirationSecs, DEFAULT_NUM_BUCKETS); - } - - public TimeCacheMap(int expirationSecs, int numBuckets) { - this(expirationSecs, numBuckets, null); - } - - public boolean containsKey(K key) { - synchronized (_lock) { - for (HashMap bucket : _buckets) { - if (bucket.containsKey(key)) { - return true; - } - } - return false; - } - } - - public V get(K key) { - synchronized (_lock) { - for (HashMap bucket : _buckets) { - if (bucket.containsKey(key)) { - return bucket.get(key); - } - } - return null; - } - } - - public void put(K key, V value) { - synchronized (_lock) { - Iterator> it = _buckets.iterator(); - HashMap bucket = it.next(); - bucket.put(key, value); - while (it.hasNext()) { - bucket = it.next(); - bucket.remove(key); - } - } - } - - public Object remove(K key) { - synchronized (_lock) { - for (HashMap bucket : _buckets) { - if (bucket.containsKey(key)) { - return bucket.remove(key); - } - } - return null; - } - } - - public int size() { - synchronized (_lock) { - int size = 0; - for (HashMap bucket : _buckets) { - size += bucket.size(); - } - return size; - } - } - - public void cleanup() { - _cleaner.interrupt(); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/TupleHelpers.java b/jstorm-client/src/main/java/backtype/storm/utils/TupleHelpers.java deleted file mode 100644 index 2f8b5cb36..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/TupleHelpers.java +++ /dev/null @@ -1,16 +0,0 @@ -package backtype.storm.utils; - -import backtype.storm.Constants; -import backtype.storm.tuple.Tuple; - -public class TupleHelpers { - private TupleHelpers() { - - } - - public static boolean isTickTuple(Tuple tuple) { - return tuple.getSourceComponent().equals(Constants.SYSTEM_COMPONENT_ID) - && tuple.getSourceStreamId().equals( - Constants.SYSTEM_TICK_STREAM_ID); - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/Utils.java b/jstorm-client/src/main/java/backtype/storm/utils/Utils.java deleted file mode 100644 index b168a50c8..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/Utils.java +++ /dev/null @@ -1,592 +0,0 @@ -package backtype.storm.utils; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.FileOutputStream; -import java.io.FileReader; -import java.io.IOException; -import java.io.InputStreamReader; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.net.URL; -import java.net.URLClassLoader; -import java.nio.ByteBuffer; -import java.nio.channels.Channels; -import java.nio.channels.WritableByteChannel; -import java.util.ArrayList; -import java.util.Enumeration; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.TreeMap; -import java.util.UUID; - -import org.apache.commons.io.input.ClassLoaderObjectInputStream; -import org.apache.commons.lang.StringUtils; -import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.curator.retry.ExponentialBackoffRetry; -import org.apache.thrift7.TException; -import org.json.simple.JSONValue; -import org.yaml.snakeyaml.Yaml; - -import com.google.gson.Gson; -import com.google.gson.GsonBuilder; - -import backtype.storm.Config; -import backtype.storm.generated.ComponentCommon; -import backtype.storm.generated.ComponentObject; -import backtype.storm.generated.StormTopology; -import clojure.lang.IFn; -import clojure.lang.RT; - -public class Utils { - public static final String DEFAULT_STREAM_ID = "default"; - - public static Object newInstance(String klass) { - try { - Class c = Class.forName(klass); - return c.newInstance(); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - public static byte[] serialize(Object obj) { - try { - ByteArrayOutputStream bos = new ByteArrayOutputStream(); - ObjectOutputStream oos = new ObjectOutputStream(bos); - oos.writeObject(obj); - oos.close(); - return bos.toByteArray(); - } catch (IOException ioe) { - throw new RuntimeException(ioe); - } - } - - public static Object deserialize(byte[] serialized, URLClassLoader loader) { - try { - ByteArrayInputStream bis = new ByteArrayInputStream(serialized); - Object ret = null; - if (loader != null) { - ClassLoaderObjectInputStream cis = new ClassLoaderObjectInputStream( - loader, bis); - ret = cis.readObject(); - cis.close(); - } else { - ObjectInputStream ois = new ObjectInputStream(bis); - ret = ois.readObject(); - ois.close(); - } - return ret; - } catch (IOException ioe) { - throw new RuntimeException(ioe); - } catch (ClassNotFoundException e) { - throw new RuntimeException(e); - } - } - - public static Object deserialize(byte[] serialized) { - return deserialize(serialized, WorkerClassLoader.getInstance()); - } - - public static String to_json(Object m) { - //return JSON.toJSONString(m); - return JSONValue.toJSONString(m); - } - - public static Object from_json(String json) { - if (json == null) { - return null; - } else { - //return JSON.parse(json); - return JSONValue.parse(json); - } - } - - public static String toPrettyJsonString(Object obj) { - Gson gson2 = new GsonBuilder().setPrettyPrinting().create(); - String ret = gson2.toJson(obj); - - return ret; - } - - public static String join(Iterable coll, String sep) { - Iterator it = coll.iterator(); - String ret = ""; - while (it.hasNext()) { - ret = ret + it.next(); - if (it.hasNext()) { - ret = ret + sep; - } - } - return ret; - } - - public static void sleep(long millis) { - try { - Time.sleep(millis); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - - public static List findResources(String name) { - try { - Enumeration resources = Thread.currentThread() - .getContextClassLoader().getResources(name); - List ret = new ArrayList(); - while (resources.hasMoreElements()) { - ret.add(resources.nextElement()); - } - return ret; - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - public static Map findAndReadConfigFile(String name, boolean mustExist) { - try { - HashSet resources = new HashSet(findResources(name)); - if (resources.isEmpty()) { - if (mustExist) - throw new RuntimeException( - "Could not find config file on classpath " + name); - else - return new HashMap(); - } - if (resources.size() > 1) { - throw new RuntimeException( - "Found multiple " - + name - + " resources. You're probably bundling the Storm jars with your topology jar. " - + resources); - } - URL resource = resources.iterator().next(); - Yaml yaml = new Yaml(); - Map ret = (Map) yaml.load(new InputStreamReader(resource - .openStream())); - if (ret == null) - ret = new HashMap(); - - return new HashMap(ret); - - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - public static Map findAndReadConfigFile(String name) { - return findAndReadConfigFile(name, true); - } - - public static Map readDefaultConfig() { - return findAndReadConfigFile("defaults.yaml", true); - } - - public static Map readCommandLineOpts() { - Map ret = new HashMap(); - String commandOptions = System.getProperty("storm.options"); - if (commandOptions != null) { - commandOptions = commandOptions.replaceAll("%%%%", " "); - String[] configs = commandOptions.split(","); - for (String config : configs) { - String[] options = config.split("="); - if (options.length == 2) { - ret.put(options[0], options[1]); - } - } - } - return ret; - } - - public static void replaceLocalDir(Map conf) { - String stormHome = System.getProperty("jstorm.home"); - boolean isEmpty = StringUtils.isBlank(stormHome); - - Map replaceMap = new HashMap(); - - for (Entry entry : conf.entrySet()) { - Object key = entry.getKey(); - Object value = entry.getValue(); - - if (value instanceof String) { - if (StringUtils.isBlank((String)value) == true ) { - continue; - } - - String str = (String)value; - if (isEmpty == true) { - // replace %JSTORM_HOME% as current directory - str = str.replace("%JSTORM_HOME%", "."); - }else { - str = str.replace("%JSTORM_HOME%", stormHome); - } - - - replaceMap.put(key, str); - } - } - - - conf.putAll(replaceMap); - } - - public static Map loadDefinedConf(String confFile) { - File file = new File(confFile); - if (file.exists() == false) { - return findAndReadConfigFile(confFile, true); - } - - Yaml yaml = new Yaml(); - Map ret; - try { - ret = (Map) yaml.load(new FileReader(file)); - } catch (FileNotFoundException e) { - ret = null; - } - if (ret == null) - ret = new HashMap(); - - return new HashMap(ret); - } - - public static Map readStormConfig() { - Map ret = readDefaultConfig(); - String confFile = System.getProperty("storm.conf.file"); - Map storm; - if (StringUtils.isBlank(confFile) == true) { - storm = findAndReadConfigFile("storm.yaml", false); - } else { - storm = loadDefinedConf(confFile); - } - ret.putAll(storm); - ret.putAll(readCommandLineOpts()); - - replaceLocalDir(ret); - return ret; - } - - private static Object normalizeConf(Object conf) { - if (conf == null) - return new HashMap(); - if (conf instanceof Map) { - Map confMap = new HashMap((Map) conf); - for (Object key : confMap.keySet()) { - Object val = confMap.get(key); - confMap.put(key, normalizeConf(val)); - } - return confMap; - } else if (conf instanceof List) { - List confList = new ArrayList((List) conf); - for (int i = 0; i < confList.size(); i++) { - Object val = confList.get(i); - confList.set(i, normalizeConf(val)); - } - return confList; - } else if (conf instanceof Integer) { - return ((Integer) conf).longValue(); - } else if (conf instanceof Float) { - return ((Float) conf).doubleValue(); - } else { - return conf; - } - } - - - - public static boolean isValidConf(Map stormConf) { - return normalizeConf(stormConf).equals( - normalizeConf(Utils.from_json(Utils.to_json(stormConf)))); - } - - public static Object getSetComponentObject(ComponentObject obj, - URLClassLoader loader) { - if (obj.getSetField() == ComponentObject._Fields.SERIALIZED_JAVA) { - return Utils.deserialize(obj.get_serialized_java(), loader); - } else if (obj.getSetField() == ComponentObject._Fields.JAVA_OBJECT) { - return obj.get_java_object(); - } else { - return obj.get_shell(); - } - } - - public static T get(Map m, S key, T def) { - T ret = m.get(key); - if (ret == null) { - ret = def; - } - return ret; - } - - public static List tuple(Object... values) { - List ret = new ArrayList(); - for (Object v : values) { - ret.add(v); - } - return ret; - } - - public static void downloadFromMaster(Map conf, String file, - String localFile) throws IOException, TException { - WritableByteChannel out = null; - NimbusClient client = null; - try { - client = NimbusClient.getConfiguredClient(conf); - String id = client.getClient().beginFileDownload(file); - out = Channels.newChannel(new FileOutputStream(localFile)); - while (true) { - ByteBuffer chunk = client.getClient().downloadChunk(id); - int written = out.write(chunk); - if (written == 0) - break; - } - } finally { - if (out != null) - out.close(); - if (client != null) - client.close(); - } - } - - public static IFn loadClojureFn(String namespace, String name) { - try { - clojure.lang.Compiler.eval(RT.readString("(require '" + namespace - + ")")); - } catch (Exception e) { - // if playing from the repl and defining functions, file won't exist - } - return (IFn) RT.var(namespace, name).deref(); - } - - public static boolean isSystemId(String id) { - return id.startsWith("__"); - } - - public static Map reverseMap(Map map) { - Map ret = new HashMap(); - for (K key : map.keySet()) { - ret.put(map.get(key), key); - } - return ret; - } - - public static ComponentCommon getComponentCommon(StormTopology topology, - String id) { - if (topology.get_spouts().containsKey(id)) { - return topology.get_spouts().get(id).get_common(); - } - if (topology.get_bolts().containsKey(id)) { - return topology.get_bolts().get(id).get_common(); - } - if (topology.get_state_spouts().containsKey(id)) { - return topology.get_state_spouts().get(id).get_common(); - } - throw new IllegalArgumentException("Could not find component with id " - + id); - } - - public static Integer getInt(Object o) { - if (o instanceof Long) { - return ((Long) o).intValue(); - } else if (o instanceof Integer) { - return (Integer) o; - } else if (o instanceof Short) { - return ((Short) o).intValue(); - } else if (o instanceof String) { - return Integer.parseInt(((String) o)); - } else { - throw new IllegalArgumentException("Don't know how to convert " + o - + " to int"); - } - } - - public static Integer getInt(Object o, Integer defaultValue) { - if (null == o) { - return defaultValue; - } - - if(o instanceof Long) { - return ((Long) o ).intValue(); - } else if (o instanceof Integer) { - return (Integer) o; - } else if (o instanceof Short) { - return ((Short) o).intValue(); - } else if (o instanceof String) { - return Integer.parseInt(((String) o)); - } else { - throw new IllegalArgumentException("Don't know how to convert " + o + " to int"); - } - } - - public static long secureRandomLong() { - return UUID.randomUUID().getLeastSignificantBits(); - } - - public static CuratorFramework newCurator(Map conf, List servers, - Object port, String root) { - return newCurator(conf, servers, port, root, null); - } - - public static class BoundedExponentialBackoffRetry extends - ExponentialBackoffRetry { - - protected final int maxRetryInterval; - - public BoundedExponentialBackoffRetry(int baseSleepTimeMs, - int maxRetries, int maxSleepTimeMs) { - super(baseSleepTimeMs, maxRetries); - this.maxRetryInterval = maxSleepTimeMs; - } - - public int getMaxRetryInterval() { - return this.maxRetryInterval; - } - - @Override - public int getSleepTimeMs(int count, long elapsedMs) { - return Math.min(maxRetryInterval, - super.getSleepTimeMs(count, elapsedMs)); - } - - } - - public static CuratorFramework newCurator(Map conf, List servers, - Object port, String root, ZookeeperAuthInfo auth) { - List serverPorts = new ArrayList(); - for (String zkServer : (List) servers) { - serverPorts.add(zkServer + ":" + Utils.getInt(port)); - } - String zkStr = StringUtils.join(serverPorts, ",") + root; - CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory - .builder() - .connectString(zkStr) - .connectionTimeoutMs( - Utils.getInt(conf - .get(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT))) - .sessionTimeoutMs( - Utils.getInt(conf - .get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT))) - .retryPolicy( - new BoundedExponentialBackoffRetry( - Utils.getInt(conf - .get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL)), - Utils.getInt(conf - .get(Config.STORM_ZOOKEEPER_RETRY_TIMES)), - Utils.getInt(conf - .get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING)))); - if (auth != null && auth.scheme != null) { - builder = builder.authorization(auth.scheme, auth.payload); - } - return builder.build(); - } - - public static CuratorFramework newCurator(Map conf, List servers, - Object port) { - return newCurator(conf, servers, port, ""); - } - - public static CuratorFramework newCuratorStarted(Map conf, - List servers, Object port, String root) { - CuratorFramework ret = newCurator(conf, servers, port, root); - ret.start(); - return ret; - } - - public static CuratorFramework newCuratorStarted(Map conf, - List servers, Object port) { - CuratorFramework ret = newCurator(conf, servers, port); - ret.start(); - return ret; - } - - /** - * - (defn integer-divided [sum num-pieces] (let [base (int (/ sum - * num-pieces)) num-inc (mod sum num-pieces) num-bases (- num-pieces - * num-inc)] (if (= num-inc 0) {base num-bases} {base num-bases (inc base) - * num-inc} ))) - * - * @param sum - * @param numPieces - * @return - */ - - public static TreeMap integerDivided(int sum, - int numPieces) { - int base = sum / numPieces; - int numInc = sum % numPieces; - int numBases = numPieces - numInc; - TreeMap ret = new TreeMap(); - ret.put(base, numBases); - if (numInc != 0) { - ret.put(base + 1, numInc); - } - return ret; - } - - public static byte[] toByteArray(ByteBuffer buffer) { - byte[] ret = new byte[buffer.remaining()]; - buffer.get(ret, 0, ret.length); - return ret; - } - - public static boolean exceptionCauseIsInstanceOf(Class klass, - Throwable throwable) { - Throwable t = throwable; - while (t != null) { - if (klass.isInstance(t)) { - return true; - } - t = t.getCause(); - } - return false; - } - - public static List tokenize_path(String path) { - String[] toks = path.split("/"); - java.util.ArrayList rtn = new ArrayList(); - for (String str : toks) { - if (!str.isEmpty()) { - rtn.add(str); - } - } - return rtn; - } - - public static String toks_to_path(List toks) { - StringBuffer buff = new StringBuffer(); - buff.append("/"); - int size = toks.size(); - for (int i = 0; i < size; i++) { - buff.append(toks.get(i)); - if (i < (size - 1)) { - buff.append("/"); - } - - } - return buff.toString(); - } - - public static String normalize_path(String path) { - String rtn = toks_to_path(tokenize_path(path)); - return rtn; - } - - public static String printStack() { - StringBuilder sb = new StringBuilder(); - sb.append("\nCurrent call stack:\n"); - StackTraceElement[] stackElements= Thread.currentThread().getStackTrace(); - for (int i = 2; i < stackElements.length; i++) { - sb.append("\t").append(stackElements[i]).append("\n"); - } - - return sb.toString(); - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/VersionedStore.java b/jstorm-client/src/main/java/backtype/storm/utils/VersionedStore.java deleted file mode 100644 index cd9e7500f..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/VersionedStore.java +++ /dev/null @@ -1,178 +0,0 @@ -package backtype.storm.utils; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.io.File; - -import org.apache.commons.io.FileUtils; - -public class VersionedStore { - private static final String FINISHED_VERSION_SUFFIX = ".version"; - - private String _root; - - public VersionedStore(String path) throws IOException { - _root = path; - mkdirs(_root); - } - - public String getRoot() { - return _root; - } - - public String versionPath(long version) { - return new File(_root, "" + version).getAbsolutePath(); - } - - public String mostRecentVersionPath() throws IOException { - Long v = mostRecentVersion(); - if (v == null) - return null; - return versionPath(v); - } - - public String mostRecentVersionPath(long maxVersion) throws IOException { - Long v = mostRecentVersion(maxVersion); - if (v == null) - return null; - return versionPath(v); - } - - public Long mostRecentVersion() throws IOException { - List all = getAllVersions(); - if (all.size() == 0) - return null; - return all.get(0); - } - - public Long mostRecentVersion(long maxVersion) throws IOException { - List all = getAllVersions(); - for (Long v : all) { - if (v <= maxVersion) - return v; - } - return null; - } - - public String createVersion() throws IOException { - Long mostRecent = mostRecentVersion(); - long version = Time.currentTimeMillis(); - if (mostRecent != null && version <= mostRecent) { - version = mostRecent + 1; - } - return createVersion(version); - } - - public String createVersion(long version) throws IOException { - String ret = versionPath(version); - if (getAllVersions().contains(version)) - throw new RuntimeException( - "Version already exists or data already exists"); - else - return ret; - } - - public void failVersion(String path) throws IOException { - deleteVersion(validateAndGetVersion(path)); - } - - public void deleteVersion(long version) throws IOException { - File versionFile = new File(versionPath(version)); - File tokenFile = new File(tokenPath(version)); - - if (versionFile.exists()) { - FileUtils.forceDelete(versionFile); - } - if (tokenFile.exists()) { - FileUtils.forceDelete(tokenFile); - } - } - - public void succeedVersion(String path) throws IOException { - long version = validateAndGetVersion(path); - // should rewrite this to do a file move - createNewFile(tokenPath(version)); - } - - public void cleanup() throws IOException { - cleanup(-1); - } - - public void cleanup(int versionsToKeep) throws IOException { - List versions = getAllVersions(); - if (versionsToKeep >= 0) { - versions = versions.subList(0, - Math.min(versions.size(), versionsToKeep)); - } - HashSet keepers = new HashSet(versions); - - for (String p : listDir(_root)) { - Long v = parseVersion(p); - if (v != null && !keepers.contains(v)) { - deleteVersion(v); - } - } - } - - /** - * Sorted from most recent to oldest - */ - public List getAllVersions() throws IOException { - List ret = new ArrayList(); - for (String s : listDir(_root)) { - if (s.endsWith(FINISHED_VERSION_SUFFIX)) { - ret.add(validateAndGetVersion(s)); - } - } - Collections.sort(ret); - Collections.reverse(ret); - return ret; - } - - private String tokenPath(long version) { - return new File(_root, "" + version + FINISHED_VERSION_SUFFIX) - .getAbsolutePath(); - } - - private long validateAndGetVersion(String path) { - Long v = parseVersion(path); - if (v == null) - throw new RuntimeException(path + " is not a valid version"); - return v; - } - - private Long parseVersion(String path) { - String name = new File(path).getName(); - if (name.endsWith(FINISHED_VERSION_SUFFIX)) { - name = name.substring(0, - name.length() - FINISHED_VERSION_SUFFIX.length()); - } - try { - return Long.parseLong(name); - } catch (NumberFormatException e) { - return null; - } - } - - private void createNewFile(String path) throws IOException { - new File(path).createNewFile(); - } - - private void mkdirs(String path) throws IOException { - new File(path).mkdirs(); - } - - private List listDir(String dir) throws IOException { - List ret = new ArrayList(); - File[] contents = new File(dir).listFiles(); - if (contents != null) { - for (File f : contents) { - ret.add(f.getAbsolutePath()); - } - } - return ret; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/WindowedTimeThrottler.java b/jstorm-client/src/main/java/backtype/storm/utils/WindowedTimeThrottler.java deleted file mode 100644 index 5d4388466..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/WindowedTimeThrottler.java +++ /dev/null @@ -1,35 +0,0 @@ -package backtype.storm.utils; - -public class WindowedTimeThrottler { - long _windowMillis; - int _maxAmt; - long _windowStartTime; - int _windowEvents = 0; - - public WindowedTimeThrottler(Number windowMillis, Number maxAmt) { - _windowMillis = windowMillis.longValue(); - _maxAmt = maxAmt.intValue(); - _windowStartTime = System.currentTimeMillis(); - } - - public boolean isThrottled() { - resetIfNecessary(); - return _windowEvents >= _maxAmt; - } - - // returns void if the event should continue, false if the event should not - // be done - public void markEvent() { - resetIfNecessary(); - _windowEvents++; - - } - - private void resetIfNecessary() { - long now = System.currentTimeMillis(); - if (now - _windowStartTime >= _windowMillis) { - _windowStartTime = now; - _windowEvents = 0; - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/WorkerClassLoader.java b/jstorm-client/src/main/java/backtype/storm/utils/WorkerClassLoader.java deleted file mode 100644 index a0c707362..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/WorkerClassLoader.java +++ /dev/null @@ -1,156 +0,0 @@ -package backtype.storm.utils; - -import java.net.URL; -import java.net.URLClassLoader; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -import org.apache.log4j.Logger; - -public class WorkerClassLoader extends URLClassLoader { - - public static Logger LOG = Logger.getLogger(WorkerClassLoader.class); - - private ClassLoader defaultClassLoader; - - private ClassLoader JDKClassLoader; - - private boolean isDebug; - - protected static WorkerClassLoader instance; - - protected static boolean enable; - - protected static Map threadContextCache; - - protected WorkerClassLoader(URL[] urls, ClassLoader defaultClassLoader, - ClassLoader JDKClassLoader, boolean isDebug) { - super(urls, JDKClassLoader); - this.defaultClassLoader = defaultClassLoader; - this.JDKClassLoader = JDKClassLoader; - this.isDebug = isDebug; - - // TODO Auto-generated constructor stub - } - - @Override - public Class loadClass(String name) throws ClassNotFoundException { - Class result = null; - try { - result = this.findLoadedClass(name); - - if (result != null) { - return result; - } - - try { - result = JDKClassLoader.loadClass(name); - if (result != null) - return result; - } catch (Exception e) { - - } - - try { - if (name.startsWith("org.apache.log4j") == false - && name.startsWith("backtype.storm") == false - && name.startsWith("com.alibaba.jstorm") == false) { - result = findClass(name); - - if (result != null) { - return result; - } - } - - } catch (Exception e) { - - } - - result = defaultClassLoader.loadClass(name); - return result; - - } finally { - if (result != null) { - ClassLoader resultClassLoader = result.getClassLoader(); - LOG.info("Successfully load class " + name + " by " - + resultClassLoader + ",threadContextLoader:" - + Thread.currentThread().getContextClassLoader()); - } else { - LOG.warn("Failed to load class " + name - + ",threadContextLoader:" - + Thread.currentThread().getContextClassLoader()); - } - - if (isDebug) { - LOG.info(Utils.printStack()); - } - } - - } - - public static WorkerClassLoader mkInstance(URL[] urls, - ClassLoader DefaultClassLoader, ClassLoader JDKClassLoader, - boolean enable, boolean isDebug) { - WorkerClassLoader.enable = enable; - if (enable == false) { - LOG.info("Don't enable UserDefine ClassLoader"); - return null; - } - - synchronized (WorkerClassLoader.class) { - if (instance == null) { - instance = new WorkerClassLoader(urls, DefaultClassLoader, - JDKClassLoader, isDebug); - - threadContextCache = new ConcurrentHashMap(); - } - - } - - LOG.info("Successfully create classloader " + mk_list(urls)); - return instance; - } - - public static WorkerClassLoader getInstance() { - return instance; - } - - public static boolean isEnable() { - return enable; - } - - public static void switchThreadContext() { - if (enable == false) { - return; - } - - Thread thread = Thread.currentThread(); - ClassLoader oldClassLoader = thread.getContextClassLoader(); - threadContextCache.put(thread, oldClassLoader); - thread.setContextClassLoader(instance); - } - - public static void restoreThreadContext() { - if (enable == false) { - return; - } - - Thread thread = Thread.currentThread(); - ClassLoader oldClassLoader = threadContextCache.get(thread); - if (oldClassLoader != null) { - thread.setContextClassLoader(oldClassLoader); - } else { - LOG.info("No context classloader of " + thread.getName()); - } - } - - private static List mk_list(V... args) { - ArrayList rtn = new ArrayList(); - for (V o : args) { - rtn.add(o); - } - return rtn; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/WritableUtils.java b/jstorm-client/src/main/java/backtype/storm/utils/WritableUtils.java deleted file mode 100644 index 9a44a5b5e..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/WritableUtils.java +++ /dev/null @@ -1,377 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/** - * This file originally comes from the Apache Hadoop project. Changes have been made to the file. - * - */ - -package backtype.storm.utils; - -import java.io.*; - -import java.util.zip.GZIPInputStream; -import java.util.zip.GZIPOutputStream; - -public final class WritableUtils { - - public static byte[] readCompressedByteArray(DataInput in) - throws IOException { - int length = in.readInt(); - if (length == -1) - return null; - byte[] buffer = new byte[length]; - in.readFully(buffer); // could/should use readFully(buffer,0,length)? - GZIPInputStream gzi = new GZIPInputStream(new ByteArrayInputStream( - buffer, 0, buffer.length)); - byte[] outbuf = new byte[length]; - ByteArrayOutputStream bos = new ByteArrayOutputStream(); - int len; - while ((len = gzi.read(outbuf, 0, outbuf.length)) != -1) { - bos.write(outbuf, 0, len); - } - byte[] decompressed = bos.toByteArray(); - bos.close(); - gzi.close(); - return decompressed; - } - - public static void skipCompressedByteArray(DataInput in) throws IOException { - int length = in.readInt(); - if (length != -1) { - skipFully(in, length); - } - } - - public static int writeCompressedByteArray(DataOutput out, byte[] bytes) - throws IOException { - if (bytes != null) { - ByteArrayOutputStream bos = new ByteArrayOutputStream(); - GZIPOutputStream gzout = new GZIPOutputStream(bos); - gzout.write(bytes, 0, bytes.length); - gzout.close(); - byte[] buffer = bos.toByteArray(); - int len = buffer.length; - out.writeInt(len); - out.write(buffer, 0, len); - /* debug only! Once we have confidence, can lose this. */ - return ((bytes.length != 0) ? (100 * buffer.length) / bytes.length - : 0); - } else { - out.writeInt(-1); - return -1; - } - } - - /* Ugly utility, maybe someone else can do this better */ - public static String readCompressedString(DataInput in) throws IOException { - byte[] bytes = readCompressedByteArray(in); - if (bytes == null) - return null; - return new String(bytes, "UTF-8"); - } - - public static int writeCompressedString(DataOutput out, String s) - throws IOException { - return writeCompressedByteArray(out, (s != null) ? s.getBytes("UTF-8") - : null); - } - - /* - * - * Write a String as a Network Int n, followed by n Bytes Alternative to 16 - * bit read/writeUTF. Encoding standard is... ? - */ - public static void writeString(DataOutput out, String s) throws IOException { - if (s != null) { - byte[] buffer = s.getBytes("UTF-8"); - int len = buffer.length; - out.writeInt(len); - out.write(buffer, 0, len); - } else { - out.writeInt(-1); - } - } - - /* - * Read a String as a Network Int n, followed by n Bytes Alternative to 16 - * bit read/writeUTF. Encoding standard is... ? - */ - public static String readString(DataInput in) throws IOException { - int length = in.readInt(); - if (length == -1) - return null; - byte[] buffer = new byte[length]; - in.readFully(buffer); // could/should use readFully(buffer,0,length)? - return new String(buffer, "UTF-8"); - } - - /* - * Write a String array as a Nework Int N, followed by Int N Byte Array - * Strings. Could be generalised using introspection. - */ - public static void writeStringArray(DataOutput out, String[] s) - throws IOException { - out.writeInt(s.length); - for (int i = 0; i < s.length; i++) { - writeString(out, s[i]); - } - } - - /* - * Write a String array as a Nework Int N, followed by Int N Byte Array of - * compressed Strings. Handles also null arrays and null values. Could be - * generalised using introspection. - */ - public static void writeCompressedStringArray(DataOutput out, String[] s) - throws IOException { - if (s == null) { - out.writeInt(-1); - return; - } - out.writeInt(s.length); - for (int i = 0; i < s.length; i++) { - writeCompressedString(out, s[i]); - } - } - - /* - * Write a String array as a Nework Int N, followed by Int N Byte Array - * Strings. Could be generalised using introspection. Actually this bit - * couldn't... - */ - public static String[] readStringArray(DataInput in) throws IOException { - int len = in.readInt(); - if (len == -1) - return null; - String[] s = new String[len]; - for (int i = 0; i < len; i++) { - s[i] = readString(in); - } - return s; - } - - /* - * Write a String array as a Nework Int N, followed by Int N Byte Array - * Strings. Could be generalised using introspection. Handles null arrays - * and null values. - */ - public static String[] readCompressedStringArray(DataInput in) - throws IOException { - int len = in.readInt(); - if (len == -1) - return null; - String[] s = new String[len]; - for (int i = 0; i < len; i++) { - s[i] = readCompressedString(in); - } - return s; - } - - /* - * - * Test Utility Method Display Byte Array. - */ - public static void displayByteArray(byte[] record) { - int i; - for (i = 0; i < record.length - 1; i++) { - if (i % 16 == 0) { - System.out.println(); - } - System.out.print(Integer.toHexString(record[i] >> 4 & 0x0F)); - System.out.print(Integer.toHexString(record[i] & 0x0F)); - System.out.print(","); - } - System.out.print(Integer.toHexString(record[i] >> 4 & 0x0F)); - System.out.print(Integer.toHexString(record[i] & 0x0F)); - System.out.println(); - } - - /** - * Serializes an integer to a binary stream with zero-compressed encoding. - * For -120 <= i <= 127, only one byte is used with the actual value. For - * other values of i, the first byte value indicates whether the integer is - * positive or negative, and the number of bytes that follow. If the first - * byte value v is between -121 and -124, the following integer is positive, - * with number of bytes that follow are -(v+120). If the first byte value v - * is between -125 and -128, the following integer is negative, with number - * of bytes that follow are -(v+124). Bytes are stored in the - * high-non-zero-byte-first order. - * - * @param stream - * Binary output stream - * @param i - * Integer to be serialized - * @throws java.io.IOException - */ - public static void writeVInt(DataOutput stream, int i) throws IOException { - writeVLong(stream, i); - } - - /** - * Serializes a long to a binary stream with zero-compressed encoding. For - * -112 <= i <= 127, only one byte is used with the actual value. For other - * values of i, the first byte value indicates whether the long is positive - * or negative, and the number of bytes that follow. If the first byte value - * v is between -113 and -120, the following long is positive, with number - * of bytes that follow are -(v+112). If the first byte value v is between - * -121 and -128, the following long is negative, with number of bytes that - * follow are -(v+120). Bytes are stored in the high-non-zero-byte-first - * order. - * - * @param stream - * Binary output stream - * @param i - * Long to be serialized - * @throws java.io.IOException - */ - public static void writeVLong(DataOutput stream, long i) throws IOException { - if (i >= -112 && i <= 127) { - stream.writeByte((byte) i); - return; - } - - int len = -112; - if (i < 0) { - i ^= -1L; // take one's complement' - len = -120; - } - - long tmp = i; - while (tmp != 0) { - tmp = tmp >> 8; - len--; - } - - stream.writeByte((byte) len); - - len = (len < -120) ? -(len + 120) : -(len + 112); - - for (int idx = len; idx != 0; idx--) { - int shiftbits = (idx - 1) * 8; - long mask = 0xFFL << shiftbits; - stream.writeByte((byte) ((i & mask) >> shiftbits)); - } - } - - /** - * Reads a zero-compressed encoded long from input stream and returns it. - * - * @param stream - * Binary input stream - * @throws java.io.IOException - * @return deserialized long from stream. - */ - public static long readVLong(DataInput stream) throws IOException { - byte firstByte = stream.readByte(); - int len = decodeVIntSize(firstByte); - if (len == 1) { - return firstByte; - } - long i = 0; - for (int idx = 0; idx < len - 1; idx++) { - byte b = stream.readByte(); - i = i << 8; - i = i | (b & 0xFF); - } - return (isNegativeVInt(firstByte) ? (i ^ -1L) : i); - } - - /** - * Reads a zero-compressed encoded integer from input stream and returns it. - * - * @param stream - * Binary input stream - * @throws java.io.IOException - * @return deserialized integer from stream. - */ - public static int readVInt(DataInput stream) throws IOException { - return (int) readVLong(stream); - } - - /** - * Given the first byte of a vint/vlong, determine the sign - * - * @param value - * the first byte - * @return is the value negative - */ - public static boolean isNegativeVInt(byte value) { - return value < -120 || (value >= -112 && value < 0); - } - - /** - * Parse the first byte of a vint/vlong to determine the number of bytes - * - * @param value - * the first byte of the vint/vlong - * @return the total number of bytes (1 to 9) - */ - public static int decodeVIntSize(byte value) { - if (value >= -112) { - return 1; - } else if (value < -120) { - return -119 - value; - } - return -111 - value; - } - - /** - * Get the encoded length if an integer is stored in a variable-length - * format - * - * @return the encoded length - */ - public static int getVIntSize(long i) { - if (i >= -112 && i <= 127) { - return 1; - } - - if (i < 0) { - i ^= -1L; // take one's complement' - } - // find the number of bytes with non-leading zeros - int dataBits = Long.SIZE - Long.numberOfLeadingZeros(i); - // find the number of data bytes + length byte - return (dataBits + 7) / 8 + 1; - } - - /** - * Skip len number of bytes in input streamin - * - * @param in - * input stream - * @param len - * number of bytes to skip - * @throws IOException - * when skipped less number of bytes - */ - public static void skipFully(DataInput in, int len) throws IOException { - int total = 0; - int cur = 0; - - while ((total < len) && ((cur = in.skipBytes(len - total)) > 0)) { - total += cur; - } - - if (total < len) { - throw new IOException("Not able to skip " + len - + " bytes, possibly " + "due to end of input."); - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/ZookeeperAuthInfo.java b/jstorm-client/src/main/java/backtype/storm/utils/ZookeeperAuthInfo.java deleted file mode 100644 index 4c6df4a38..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/ZookeeperAuthInfo.java +++ /dev/null @@ -1,30 +0,0 @@ -package backtype.storm.utils; - -import backtype.storm.Config; -import java.io.UnsupportedEncodingException; -import java.util.Map; - -public class ZookeeperAuthInfo { - public String scheme; - public byte[] payload = null; - - public ZookeeperAuthInfo(Map conf) { - String scheme = (String) conf.get(Config.STORM_ZOOKEEPER_AUTH_SCHEME); - String payload = (String) conf.get(Config.STORM_ZOOKEEPER_AUTH_PAYLOAD); - if (scheme != null) { - this.scheme = scheme; - if (payload != null) { - try { - this.payload = payload.getBytes("UTF-8"); - } catch (UnsupportedEncodingException ex) { - throw new RuntimeException(ex); - } - } - } - } - - public ZookeeperAuthInfo(String scheme, byte[] payload) { - this.scheme = scheme; - this.payload = payload; - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/disruptor/AbstractSequencerExt.java b/jstorm-client/src/main/java/backtype/storm/utils/disruptor/AbstractSequencerExt.java deleted file mode 100644 index 3c4f1f970..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/disruptor/AbstractSequencerExt.java +++ /dev/null @@ -1,21 +0,0 @@ -package backtype.storm.utils.disruptor; - -import com.lmax.disruptor.AbstractSequencer; -import com.lmax.disruptor.WaitStrategy; - -public abstract class AbstractSequencerExt extends AbstractSequencer { - private static boolean waitSleep = true; - - public static boolean isWaitSleep() { - return waitSleep; - } - - public static void setWaitSleep(boolean waitSleep) { - AbstractSequencerExt.waitSleep = waitSleep; - } - - public AbstractSequencerExt(int bufferSize, WaitStrategy waitStrategy) { - super(bufferSize, waitStrategy); - } - -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/disruptor/MultiProducerSequencer.java b/jstorm-client/src/main/java/backtype/storm/utils/disruptor/MultiProducerSequencer.java deleted file mode 100644 index ba4521b69..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/disruptor/MultiProducerSequencer.java +++ /dev/null @@ -1,298 +0,0 @@ -package backtype.storm.utils.disruptor; - - -import java.util.concurrent.locks.LockSupport; - -import sun.misc.Unsafe; - -import com.lmax.disruptor.InsufficientCapacityException; -import com.lmax.disruptor.Sequence; -import com.lmax.disruptor.Sequencer; -import com.lmax.disruptor.WaitStrategy; -import com.lmax.disruptor.util.Util; - - -/** - * Coordinator for claiming sequences for access to a data structure while tracking dependent {@link Sequence}s - * - * Suitable for use for sequencing across multiple publisher threads. - */ -public class MultiProducerSequencer extends AbstractSequencerExt -{ - - - private static final Unsafe UNSAFE = Util.getUnsafe(); - private static final long BASE = UNSAFE.arrayBaseOffset(int[].class); - private static final long SCALE = UNSAFE.arrayIndexScale(int[].class); - - - - private final Sequence gatingSequenceCache = new Sequence(Sequencer.INITIAL_CURSOR_VALUE); - - // availableBuffer tracks the state of each ringbuffer slot - // see below for more details on the approach - private final int[] availableBuffer; - private final int indexMask; - private final int indexShift; - - /** - * Construct a Sequencer with the selected wait strategy and buffer size. - * - * @param bufferSize the size of the buffer that this will sequence over. - * @param waitStrategy for those waiting on sequences. - */ - public MultiProducerSequencer(int bufferSize, final WaitStrategy waitStrategy) - { - super(bufferSize, waitStrategy); - availableBuffer = new int[bufferSize]; - indexMask = bufferSize - 1; - indexShift = Util.log2(bufferSize); - initialiseAvailableBuffer(); - } - - /** - * @see Sequencer#hasAvailableCapacity(int) - */ - @Override - public boolean hasAvailableCapacity(final int requiredCapacity) - { - return hasAvailableCapacity(gatingSequences, requiredCapacity, cursor.get()); - } - - private boolean hasAvailableCapacity(Sequence[] gatingSequences, final int requiredCapacity, long cursorValue) - { - long wrapPoint = (cursorValue + requiredCapacity) - bufferSize; - long cachedGatingSequence = gatingSequenceCache.get(); - - if (wrapPoint > cachedGatingSequence || cachedGatingSequence > cursorValue) - { - long minSequence = Util.getMinimumSequence(gatingSequences, cursorValue); - gatingSequenceCache.set(minSequence); - - if (wrapPoint > minSequence) - { - return false; - } - } - - return true; - } - - /** - * @see Sequencer#claim(long) - */ - @Override - public void claim(long sequence) - { - cursor.set(sequence); - } - - /** - * @see Sequencer#next() - */ - @Override - public long next() - { - return next(1); - } - - /** - * @see Sequencer#next(int) - */ - @Override - public long next(int n) - { - if (n < 1) - { - throw new IllegalArgumentException("n must be > 0"); - } - - long current; - long next; - - do - { - current = cursor.get(); - next = current + n; - - long wrapPoint = next - bufferSize; - long cachedGatingSequence = gatingSequenceCache.get(); - - if (wrapPoint > cachedGatingSequence || cachedGatingSequence > current) - { - long gatingSequence = Util.getMinimumSequence(gatingSequences, current); - - if (wrapPoint > gatingSequence) - { - if (AbstractSequencerExt.isWaitSleep()) { - try { - Thread.sleep(1); - } catch (InterruptedException e) { - } - }else { - LockSupport.parkNanos(1); - } - continue; - } - - gatingSequenceCache.set(gatingSequence); - } - else if (cursor.compareAndSet(current, next)) - { - break; - } - } - while (true); - - return next; - } - - /** - * @see Sequencer#tryNext() - */ - @Override - public long tryNext() throws InsufficientCapacityException - { - return tryNext(1); - } - - /** - * @see Sequencer#tryNext(int) - */ - @Override - public long tryNext(int n) throws InsufficientCapacityException - { - if (n < 1) - { - throw new IllegalArgumentException("n must be > 0"); - } - - long current; - long next; - - do - { - current = cursor.get(); - next = current + n; - - if (!hasAvailableCapacity(gatingSequences, n, current)) - { - throw InsufficientCapacityException.INSTANCE; - } - } - while (!cursor.compareAndSet(current, next)); - - return next; - } - - /** - * @see Sequencer#remainingCapacity() - */ - @Override - public long remainingCapacity() - { - long consumed = Util.getMinimumSequence(gatingSequences, cursor.get()); - long produced = cursor.get(); - return getBufferSize() - (produced - consumed); - } - - private void initialiseAvailableBuffer() - { - for (int i = availableBuffer.length - 1; i != 0; i--) - { - setAvailableBufferValue(i, -1); - } - - setAvailableBufferValue(0, -1); - } - - /** - * @see Sequencer#publish(long) - */ - @Override - public void publish(final long sequence) - { - setAvailable(sequence); - waitStrategy.signalAllWhenBlocking(); - } - - /** - * @see Sequencer#publish(long, long) - */ - @Override - public void publish(long lo, long hi) - { - for (long l = lo; l <= hi; l++) - { - setAvailable(l); - } - waitStrategy.signalAllWhenBlocking(); - } - - /** - * The below methods work on the availableBuffer flag. - * - * The prime reason is to avoid a shared sequence object between publisher threads. - * (Keeping single pointers tracking start and end would require coordination - * between the threads). - * - * -- Firstly we have the constraint that the delta between the cursor and minimum - * gating sequence will never be larger than the buffer size (the code in - * next/tryNext in the Sequence takes care of that). - * -- Given that; take the sequence value and mask off the lower portion of the - * sequence as the index into the buffer (indexMask). (aka modulo operator) - * -- The upper portion of the sequence becomes the value to check for availability. - * ie: it tells us how many times around the ring buffer we've been (aka division) - * -- Because we can't wrap without the gating sequences moving forward (i.e. the - * minimum gating sequence is effectively our last available position in the - * buffer), when we have new data and successfully claimed a slot we can simply - * write over the top. - */ - private void setAvailable(final long sequence) - { - setAvailableBufferValue(calculateIndex(sequence), calculateAvailabilityFlag(sequence)); - } - - private void setAvailableBufferValue(int index, int flag) - { - long bufferAddress = (index * SCALE) + BASE; - UNSAFE.putOrderedInt(availableBuffer, bufferAddress, flag); - } - - /** - * @see Sequencer#isAvailable(long) - */ - @Override - public boolean isAvailable(long sequence) - { - int index = calculateIndex(sequence); - int flag = calculateAvailabilityFlag(sequence); - long bufferAddress = (index * SCALE) + BASE; - return UNSAFE.getIntVolatile(availableBuffer, bufferAddress) == flag; - } - - @Override - public long getHighestPublishedSequence(long lowerBound, long availableSequence) - { - for (long sequence = lowerBound; sequence <= availableSequence; sequence++) - { - if (!isAvailable(sequence)) - { - return sequence - 1; - } - } - - return availableSequence; - } - - private int calculateAvailabilityFlag(final long sequence) - { - return (int) (sequence >>> indexShift); - } - - private int calculateIndex(final long sequence) - { - return ((int) sequence) & indexMask; - } -} - diff --git a/jstorm-client/src/main/java/backtype/storm/utils/disruptor/RingBuffer.java b/jstorm-client/src/main/java/backtype/storm/utils/disruptor/RingBuffer.java deleted file mode 100644 index fcc922d8b..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/disruptor/RingBuffer.java +++ /dev/null @@ -1,1184 +0,0 @@ -package backtype.storm.utils.disruptor; - -import com.lmax.disruptor.BlockingWaitStrategy; -import com.lmax.disruptor.Cursored; -import com.lmax.disruptor.DataProvider; -import com.lmax.disruptor.EventFactory; -import com.lmax.disruptor.EventProcessor; -import com.lmax.disruptor.EventTranslator; -import com.lmax.disruptor.EventTranslatorOneArg; -import com.lmax.disruptor.EventTranslatorThreeArg; -import com.lmax.disruptor.EventTranslatorTwoArg; -import com.lmax.disruptor.EventTranslatorVararg; -import com.lmax.disruptor.InsufficientCapacityException; -import com.lmax.disruptor.Sequence; -import com.lmax.disruptor.SequenceBarrier; -import com.lmax.disruptor.Sequencer; -import com.lmax.disruptor.WaitStrategy; -import com.lmax.disruptor.dsl.ProducerType; -import backtype.storm.utils.disruptor.MultiProducerSequencer; -import backtype.storm.utils.disruptor.SingleProducerSequencer; - -/** - * Ring based store of reusable entries containing the data representing - * an event being exchanged between event producer and {@link EventProcessor}s. - * - * @param implementation storing the data for sharing during exchange or parallel coordination of an event. - */ -public class RingBuffer implements Cursored, DataProvider -{ - public static final long INITIAL_CURSOR_VALUE = -1L; - - private final int indexMask; - private final Object[] entries; - private final int bufferSize; - private final Sequencer sequencer; - - /** - * Construct a RingBuffer with the full option set. - * - * @param eventFactory to newInstance entries for filling the RingBuffer - * @param sequencer sequencer to handle the ordering of events moving through the RingBuffer. - * @throws IllegalArgumentException if bufferSize is less than 1 or not a power of 2 - */ - public RingBuffer(EventFactory eventFactory, - Sequencer sequencer) - { - this.sequencer = sequencer; - this.bufferSize = sequencer.getBufferSize(); - - if (bufferSize < 1) - { - throw new IllegalArgumentException("bufferSize must not be less than 1"); - } - if (Integer.bitCount(bufferSize) != 1) - { - throw new IllegalArgumentException("bufferSize must be a power of 2"); - } - - this.indexMask = bufferSize - 1; - this.entries = new Object[sequencer.getBufferSize()]; - fill(eventFactory); - } - - /** - * Create a new multiple producer RingBuffer with the specified wait strategy. - * - * @see MultiProducerSequencer - * @param factory used to create the events within the ring buffer. - * @param bufferSize number of elements to create within the ring buffer. - * @param waitStrategy used to determine how to wait for new elements to become available. - * @throws IllegalArgumentException if bufferSize is less than 1 or not a power of 2 - */ - public static RingBuffer createMultiProducer(EventFactory factory, - int bufferSize, - WaitStrategy waitStrategy) - { - MultiProducerSequencer sequencer = new MultiProducerSequencer(bufferSize, waitStrategy); - - return new RingBuffer(factory, sequencer); - } - - /** - * Create a new multiple producer RingBuffer using the default wait strategy {@link BlockingWaitStrategy}. - * - * @see MultiProducerSequencer - * @param factory used to create the events within the ring buffer. - * @param bufferSize number of elements to create within the ring buffer. - * @throws IllegalArgumentException if bufferSize is less than 1 or not a power of 2 - */ - public static RingBuffer createMultiProducer(EventFactory factory, int bufferSize) - { - return createMultiProducer(factory, bufferSize, new BlockingWaitStrategy()); - } - - /** - * Create a new single producer RingBuffer with the specified wait strategy. - * - * @see SingleProducerSequencer - * @param factory used to create the events within the ring buffer. - * @param bufferSize number of elements to create within the ring buffer. - * @param waitStrategy used to determine how to wait for new elements to become available. - * @throws IllegalArgumentException if bufferSize is less than 1 or not a power of 2 - */ - public static RingBuffer createSingleProducer(EventFactory factory, - int bufferSize, - WaitStrategy waitStrategy) - { - SingleProducerSequencer sequencer = new SingleProducerSequencer(bufferSize, waitStrategy); - - return new RingBuffer(factory, sequencer); - } - - /** - * Create a new single producer RingBuffer using the default wait strategy {@link BlockingWaitStrategy}. - * - * @see MultiProducerSequencer - * @param factory used to create the events within the ring buffer. - * @param bufferSize number of elements to create within the ring buffer. - * @throws IllegalArgumentException if bufferSize is less than 1 or not a power of 2 - */ - public static RingBuffer createSingleProducer(EventFactory factory, int bufferSize) - { - return createSingleProducer(factory, bufferSize, new BlockingWaitStrategy()); - } - - /** - * Create a new Ring Buffer with the specified producer type (SINGLE or MULTI) - * - * @param producerType producer type to use {@link ProducerType}. - * @param factory used to create events within the ring buffer. - * @param bufferSize number of elements to create within the ring buffer. - * @param waitStrategy used to determine how to wait for new elements to become available. - * @throws IllegalArgumentException if bufferSize is less than 1 or not a power of 2 - */ - public static RingBuffer create(ProducerType producerType, - EventFactory factory, - int bufferSize, - WaitStrategy waitStrategy) - { - switch (producerType) - { - case SINGLE: - return createSingleProducer(factory, bufferSize, waitStrategy); - case MULTI: - return createMultiProducer(factory, bufferSize, waitStrategy); - default: - throw new IllegalStateException(producerType.toString()); - } - } - - /** - *

Get the event for a given sequence in the RingBuffer.

- * - *

This call has 2 uses. Firstly use this call when publishing to a ring buffer. - * After calling {@link RingBuffer#next()} use this call to get hold of the - * preallocated event to fill with data before calling {@link RingBuffer#publish(long)}.

- * - *

Secondly use this call when consuming data from the ring buffer. After calling - * {@link SequenceBarrier#waitFor(long)} call this method with any value greater than - * that your current consumer sequence and less than or equal to the value returned from - * the {@link SequenceBarrier#waitFor(long)} method.

- * - * @param sequence for the event - * @return the event for the given sequence - */ - @SuppressWarnings("unchecked") - public E get(long sequence) - { - return (E)entries[(int)sequence & indexMask]; - } - - /** - * @deprecated Use {@link RingBuffer#get(long)} - */ - @Deprecated - public E getPreallocated(long sequence) - { - return get(sequence); - } - - /** - * @deprecated Use {@link RingBuffer#get(long)} - */ - @Deprecated - public E getPublished(long sequence) - { - return get(sequence); - } - - /** - * Increment and return the next sequence for the ring buffer. Calls of this - * method should ensure that they always publish the sequence afterward. E.g. - *
-     * long sequence = ringBuffer.next();
-     * try {
-     *     Event e = ringBuffer.get(sequence);
-     *     // Do some work with the event.
-     * } finally {
-     *     ringBuffer.publish(sequence);
-     * }
-     * 
- * @see RingBuffer#publish(long) - * @see RingBuffer#get(long) - * @return The next sequence to publish to. - */ - public long next() - { - return sequencer.next(); - } - - /** - * The same functionality as {@link RingBuffer#next()}, but allows the caller to claim - * the next n sequences. - * - * @see Sequencer#next(int) - * @param n number of slots to claim - * @return sequence number of the highest slot claimed - */ - public long next(int n) - { - return sequencer.next(n); - } - - /** - *

Increment and return the next sequence for the ring buffer. Calls of this - * method should ensure that they always publish the sequence afterward. E.g. - *

-     * long sequence = ringBuffer.next();
-     * try {
-     *     Event e = ringBuffer.get(sequence);
-     *     // Do some work with the event.
-     * } finally {
-     *     ringBuffer.publish(sequence);
-     * }
-     * 
- *

This method will not block if there is not space available in the ring - * buffer, instead it will throw an {@link InsufficientCapacityException}. - * - * - * @see RingBuffer#publish(long) - * @see RingBuffer#get(long) - * @return The next sequence to publish to. - * @throws InsufficientCapacityException if the necessary space in the ring buffer is not available - */ - public long tryNext() throws InsufficientCapacityException - { - return sequencer.tryNext(); - } - - /** - * The same functionality as {@link RingBuffer#tryNext()}, but allows the caller to attempt - * to claim the next n sequences. - * - * @param n number of slots to claim - * @return sequence number of the highest slot claimed - * @throws InsufficientCapacityException if the necessary space in the ring buffer is not available - */ - public long tryNext(int n) throws InsufficientCapacityException - { - return sequencer.tryNext(n); - } - - /** - * Resets the cursor to a specific value. This can be applied at any time, but it is worth not - * that it is a racy thing to do and should only be used in controlled circumstances. E.g. during - * initialisation. - * - * @param sequence The sequence to reset too. - * @throws IllegalStateException If any gating sequences have already been specified. - */ - public void resetTo(long sequence) - { - sequencer.claim(sequence); - sequencer.publish(sequence); - } - - /** - * Sets the cursor to a specific sequence and returns the preallocated entry that is stored there. This - * is another deliberately racy call, that should only be done in controlled circumstances, e.g. initialisation. - * - * @param sequence The sequence to claim. - * @return The preallocated event. - */ - public E claimAndGetPreallocated(long sequence) - { - sequencer.claim(sequence); - return get(sequence); - } - - /** - * Determines if a particular entry has been published. - * - * @param sequence The sequence to identify the entry. - * @return If the value has been published or not. - */ - public boolean isPublished(long sequence) - { - return sequencer.isAvailable(sequence); - } - - /** - * Add the specified gating sequences to this instance of the Disruptor. They will - * safely and atomically added to the list of gating sequences. - * - * @param gatingSequences The sequences to add. - */ - public void addGatingSequences(Sequence... gatingSequences) - { - sequencer.addGatingSequences(gatingSequences); - } - - /** - * Get the minimum sequence value from all of the gating sequences - * added to this ringBuffer. - * - * @return The minimum gating sequence or the cursor sequence if - * no sequences have been added. - */ - public long getMinimumGatingSequence() - { - return sequencer.getMinimumSequence(); - } - - /** - * Remove the specified sequence from this ringBuffer. - * - * @param sequence to be removed. - * @return true if this sequence was found, false otherwise. - */ - public boolean removeGatingSequence(Sequence sequence) - { - return sequencer.removeGatingSequence(sequence); - } - - /** - * Create a new SequenceBarrier to be used by an EventProcessor to track which messages - * are available to be read from the ring buffer given a list of sequences to track. - * - * @see SequenceBarrier - * @param sequencesToTrack the additional sequences to track - * @return A sequence barrier that will track the specified sequences. - */ - public SequenceBarrier newBarrier(Sequence... sequencesToTrack) - { - return sequencer.newBarrier(sequencesToTrack); - } - - /** - * Get the current cursor value for the ring buffer. The cursor value is - * the last value that was published, or the highest available sequence - * that can be consumed. - */ - public long getCursor() - { - return sequencer.getCursor(); - } - - /** - * The size of the buffer. - */ - public int getBufferSize() - { - return bufferSize; - } - - /** - * Given specified requiredCapacity determines if that amount of space - * is available. Note, you can not assume that if this method returns true - * that a call to {@link RingBuffer#next()} will not block. Especially true if this - * ring buffer is set up to handle multiple producers. - * - * @param requiredCapacity The capacity to check for. - * @return true If the specified requiredCapacity is available - * false if now. - */ - public boolean hasAvailableCapacity(int requiredCapacity) - { - return sequencer.hasAvailableCapacity(requiredCapacity); - } - - - /** - * Publishes an event to the ring buffer. It handles - * claiming the next sequence, getting the current (uninitialised) - * event from the ring buffer and publishing the claimed sequence - * after translation. - * - * @param translator The user specified translation for the event - */ - public void publishEvent(EventTranslator translator) - { - final long sequence = sequencer.next(); - translateAndPublish(translator, sequence); - } - - /** - * Attempts to publish an event to the ring buffer. It handles - * claiming the next sequence, getting the current (uninitialised) - * event from the ring buffer and publishing the claimed sequence - * after translation. Will return false if specified capacity - * was not available. - * - * @param translator The user specified translation for the event - * @return true if the value was published, false if there was insufficient - * capacity. - */ - public boolean tryPublishEvent(EventTranslator translator) - { - try - { - final long sequence = sequencer.tryNext(); - translateAndPublish(translator, sequence); - return true; - } - catch (InsufficientCapacityException e) - { - return false; - } - } - - /** - * Allows one user supplied argument. - * - * @see #publishEvent(EventTranslator) - * @param translator The user specified translation for the event - * @param arg0 A user supplied argument. - */ - public void publishEvent(EventTranslatorOneArg translator, A arg0) - { - final long sequence = sequencer.next(); - translateAndPublish(translator, sequence, arg0); - } - - /** - * Allows one user supplied argument. - * - * @see #tryPublishEvent(EventTranslator) - * @param translator The user specified translation for the event - * @param arg0 A user supplied argument. - * @return true if the value was published, false if there was insufficient - * capacity. - */ - public boolean tryPublishEvent(EventTranslatorOneArg translator, A arg0) - { - try - { - final long sequence = sequencer.tryNext(); - translateAndPublish(translator, sequence, arg0); - return true; - } - catch (InsufficientCapacityException e) - { - return false; - } - } - - /** - * Allows two user supplied arguments. - * - * @see #publishEvent(EventTranslator) - * @param translator The user specified translation for the event - * @param arg0 A user supplied argument. - * @param arg1 A user supplied argument. - */ - public void publishEvent(EventTranslatorTwoArg translator, A arg0, B arg1) - { - final long sequence = sequencer.next(); - translateAndPublish(translator, sequence, arg0, arg1); - } - - /** - * Allows two user supplied arguments. - * - * @see #tryPublishEvent(EventTranslator) - * @param translator The user specified translation for the event - * @param arg0 A user supplied argument. - * @param arg1 A user supplied argument. - * @return true if the value was published, false if there was insufficient - * capacity. - */ - public boolean tryPublishEvent(EventTranslatorTwoArg translator, A arg0, B arg1) - { - try - { - final long sequence = sequencer.tryNext(); - translateAndPublish(translator, sequence, arg0, arg1); - return true; - } - catch (InsufficientCapacityException e) - { - return false; - } - } - - /** - * Allows three user supplied arguments - * - * @see #publishEvent(EventTranslator) - * @param translator The user specified translation for the event - * @param arg0 A user supplied argument. - * @param arg1 A user supplied argument. - * @param arg2 A user supplied argument. - */ - public void publishEvent(EventTranslatorThreeArg translator, A arg0, B arg1, C arg2) - { - final long sequence = sequencer.next(); - translateAndPublish(translator, sequence, arg0, arg1, arg2); - } - - /** - * Allows three user supplied arguments - * - * @see #publishEvent(EventTranslator) - * @param translator The user specified translation for the event - * @param arg0 A user supplied argument. - * @param arg1 A user supplied argument. - * @param arg2 A user supplied argument. - * @return true if the value was published, false if there was insufficient - * capacity. - */ - public boolean tryPublishEvent(EventTranslatorThreeArg translator, A arg0, B arg1, C arg2) - { - try - { - final long sequence = sequencer.tryNext(); - translateAndPublish(translator, sequence, arg0, arg1, arg2); - return true; - } - catch (InsufficientCapacityException e) - { - return false; - } - } - - /** - * Allows a variable number of user supplied arguments - * - * @see #publishEvent(EventTranslator) - * @param translator The user specified translation for the event - * @param args User supplied arguments. - */ - public void publishEvent(EventTranslatorVararg translator, Object...args) - { - final long sequence = sequencer.next(); - translateAndPublish(translator, sequence, args); - } - - /** - * Allows a variable number of user supplied arguments - * - * @see #publishEvent(EventTranslator) - * @param translator The user specified translation for the event - * @param args User supplied arguments. - * @return true if the value was published, false if there was insufficient - * capacity. - */ - public boolean tryPublishEvent(EventTranslatorVararg translator, Object...args) - { - try - { - final long sequence = sequencer.tryNext(); - translateAndPublish(translator, sequence, args); - return true; - } - catch (InsufficientCapacityException e) - { - return false; - } - } - - - /** - * Publishes multiple events to the ring buffer. It handles - * claiming the next sequence, getting the current (uninitialised) - * event from the ring buffer and publishing the claimed sequence - * after translation. - * - * @param translators The user specified translation for each event - */ - public void publishEvents(EventTranslator[] translators) - { - publishEvents(translators, 0, translators.length); - } - - /** - * Publishes multiple events to the ring buffer. It handles - * claiming the next sequence, getting the current (uninitialised) - * event from the ring buffer and publishing the claimed sequence - * after translation. - * - * @param translators The user specified translation for each event - * @param batchStartsAt The first element of the array which is within the batch. - * @param batchSize The actual size of the batch - */ - public void publishEvents(EventTranslator[] translators, int batchStartsAt, int batchSize) - { - checkBounds(translators, batchStartsAt, batchSize); - final long finalSequence = sequencer.next(batchSize); - translateAndPublishBatch(translators, batchStartsAt, batchSize, finalSequence); - } - - /** - * Attempts to publish multiple events to the ring buffer. It handles - * claiming the next sequence, getting the current (uninitialised) - * event from the ring buffer and publishing the claimed sequence - * after translation. Will return false if specified capacity - * was not available. - * - * @param translators The user specified translation for the event - * @return true if the value was published, false if there was insufficient - * capacity. - */ - public boolean tryPublishEvents(EventTranslator[] translators) - { - return tryPublishEvents(translators, 0, translators.length); - } - - /** - * Attempts to publish multiple events to the ring buffer. It handles - * claiming the next sequence, getting the current (uninitialised) - * event from the ring buffer and publishing the claimed sequence - * after translation. Will return false if specified capacity - * was not available. - * - * @param translators The user specified translation for the event - * @param batchStartsAt The first element of the array which is within the batch. - * @param batchSize The actual size of the batch - * @return true if all the values were published, false if there was insufficient - * capacity. - */ - public boolean tryPublishEvents(EventTranslator[] translators, int batchStartsAt, int batchSize) - { - checkBounds(translators, batchStartsAt, batchSize); - try - { - final long finalSequence = sequencer.tryNext(batchSize); - translateAndPublishBatch(translators, batchStartsAt, batchSize, finalSequence); - return true; - } - catch (InsufficientCapacityException e) - { - return false; - } - } - - /** - * Allows one user supplied argument per event. - * - * @param translator The user specified translation for the event - * @param arg0 A user supplied argument. - * @see #publishEvents(com.lmax.disruptor.EventTranslator[]) - */ - public void publishEvents(EventTranslatorOneArg translator, A[] arg0) - { - publishEvents(translator, 0, arg0.length, arg0); - } - - /** - * Allows one user supplied argument per event. - * - * @param translator The user specified translation for each event - * @param batchStartsAt The first element of the array which is within the batch. - * @param batchSize The actual size of the batch - * @param arg0 An array of user supplied arguments, one element per event. - * @see #publishEvents(EventTranslator[]) - */ - public void publishEvents(EventTranslatorOneArg translator, int batchStartsAt, int batchSize, A[] arg0) - { - checkBounds(arg0, batchStartsAt, batchSize); - final long finalSequence = sequencer.next(batchSize); - translateAndPublishBatch(translator, arg0, batchStartsAt, batchSize, finalSequence); - } - - /** - * Allows one user supplied argument. - * - * @param translator The user specified translation for each event - * @param arg0 An array of user supplied arguments, one element per event. - * @return true if the value was published, false if there was insufficient - * capacity. - * @see #tryPublishEvents(com.lmax.disruptor.EventTranslator[]) - */ - public boolean tryPublishEvents(EventTranslatorOneArg translator, A[] arg0) - { - return tryPublishEvents(translator, 0, arg0.length, arg0); - } - - /** - * Allows one user supplied argument. - * - * @param translator The user specified translation for each event - * @param batchStartsAt The first element of the array which is within the batch. - * @param batchSize The actual size of the batch - * @param arg0 An array of user supplied arguments, one element per event. - * @return true if the value was published, false if there was insufficient - * capacity. - * @see #tryPublishEvents(EventTranslator[]) - */ - public boolean tryPublishEvents(EventTranslatorOneArg translator, int batchStartsAt, int batchSize, A[] arg0) - { - checkBounds(arg0, batchStartsAt, batchSize); - try - { - final long finalSequence = sequencer.tryNext(batchSize); - translateAndPublishBatch(translator, arg0, batchStartsAt, batchSize, finalSequence); - return true; - } - catch (InsufficientCapacityException e) - { - return false; - } - } - - /** - * Allows two user supplied arguments per event. - * - * @param translator The user specified translation for the event - * @param arg0 An array of user supplied arguments, one element per event. - * @param arg1 An array of user supplied arguments, one element per event. - * @see #publishEvents(com.lmax.disruptor.EventTranslator[]) - */ - public void publishEvents(EventTranslatorTwoArg translator, A[] arg0, B[] arg1) - { - publishEvents(translator, 0, arg0.length, arg0, arg1); - } - - /** - * Allows two user supplied arguments per event. - * - * @param translator The user specified translation for the event - * @param batchStartsAt The first element of the array which is within the batch. - * @param batchSize The actual size of the batch. - * @param arg0 An array of user supplied arguments, one element per event. - * @param arg1 An array of user supplied arguments, one element per event. - * @see #publishEvents(EventTranslator[]) - */ - public void publishEvents(EventTranslatorTwoArg translator, int batchStartsAt, int batchSize, A[] arg0, B[] arg1) - { - checkBounds(arg0, arg1, batchStartsAt, batchSize); - final long finalSequence = sequencer.next(batchSize); - translateAndPublishBatch(translator, arg0, arg1, batchStartsAt, batchSize, finalSequence); - } - - /** - * Allows two user supplied arguments per event. - * - * @param translator The user specified translation for the event - * @param arg0 An array of user supplied arguments, one element per event. - * @param arg1 An array of user supplied arguments, one element per event. - * @return true if the value was published, false if there was insufficient - * capacity. - * @see #tryPublishEvents(com.lmax.disruptor.EventTranslator[]) - */ - public boolean tryPublishEvents(EventTranslatorTwoArg translator, A[] arg0, B[] arg1) - { - return tryPublishEvents(translator, 0, arg0.length, arg0, arg1); - } - - /** - * Allows two user supplied arguments per event. - * - * @param translator The user specified translation for the event - * @param batchStartsAt The first element of the array which is within the batch. - * @param batchSize The actual size of the batch. - * @param arg0 An array of user supplied arguments, one element per event. - * @param arg1 An array of user supplied arguments, one element per event. - * @return true if the value was published, false if there was insufficient - * capacity. - * @see #tryPublishEvents(EventTranslator[]) - */ - public boolean tryPublishEvents(EventTranslatorTwoArg translator, int batchStartsAt, int batchSize, A[] arg0, B[] arg1) - { - checkBounds(arg0, arg1, batchStartsAt, batchSize); - try - { - final long finalSequence = sequencer.tryNext(batchSize); - translateAndPublishBatch(translator, arg0, arg1, batchStartsAt, batchSize, finalSequence); - return true; - } - catch (InsufficientCapacityException e) - { - return false; - } - } - - /** - * Allows three user supplied arguments per event. - * - * @param translator The user specified translation for the event - * @param arg0 An array of user supplied arguments, one element per event. - * @param arg1 An array of user supplied arguments, one element per event. - * @param arg2 An array of user supplied arguments, one element per event. - * @see #publishEvents(com.lmax.disruptor.EventTranslator[]) - */ - public void publishEvents(EventTranslatorThreeArg translator, A[] arg0, B[] arg1, C[] arg2) - { - publishEvents(translator, 0, arg0.length, arg0, arg1, arg2); - } - - /** - * Allows three user supplied arguments per event. - * - * @param translator The user specified translation for the event - * @param batchStartsAt The first element of the array which is within the batch. - * @param batchSize The number of elements in the batch. - * @param arg0 An array of user supplied arguments, one element per event. - * @param arg1 An array of user supplied arguments, one element per event. - * @param arg2 An array of user supplied arguments, one element per event. - * @see #publishEvents(EventTranslator[]) - */ - public void publishEvents(EventTranslatorThreeArg translator, int batchStartsAt, int batchSize, A[] arg0, B[] arg1, C[] arg2) - { - checkBounds(arg0, arg1, arg2, batchStartsAt, batchSize); - final long finalSequence = sequencer.next(batchSize); - translateAndPublishBatch(translator, arg0, arg1, arg2, batchStartsAt, batchSize, finalSequence); - } - - /** - * Allows three user supplied arguments per event. - * - * @param translator The user specified translation for the event - * @param arg0 An array of user supplied arguments, one element per event. - * @param arg1 An array of user supplied arguments, one element per event. - * @param arg2 An array of user supplied arguments, one element per event. - * @return true if the value was published, false if there was insufficient - * capacity. - * @see #publishEvents(com.lmax.disruptor.EventTranslator[]) - */ - public boolean tryPublishEvents(EventTranslatorThreeArg translator, A[] arg0, B[] arg1, C[] arg2) - { - return tryPublishEvents(translator, 0, arg0.length, arg0, arg1, arg2); - } - - /** - * Allows three user supplied arguments per event. - * - * @param translator The user specified translation for the event - * @param batchStartsAt The first element of the array which is within the batch. - * @param batchSize The actual size of the batch. - * @param arg0 An array of user supplied arguments, one element per event. - * @param arg1 An array of user supplied arguments, one element per event. - * @param arg2 An array of user supplied arguments, one element per event. - * @return true if the value was published, false if there was insufficient - * capacity. - * @see #publishEvents(EventTranslator[]) - */ - public boolean tryPublishEvents(EventTranslatorThreeArg translator, int batchStartsAt, int batchSize, A[] arg0, B[] arg1, C[] arg2) - { - checkBounds(arg0, arg1, arg2, batchStartsAt, batchSize); - try - { - final long finalSequence = sequencer.tryNext(batchSize); - translateAndPublishBatch(translator, arg0, arg1, arg2, batchStartsAt, batchSize, finalSequence); - return true; - } - catch (InsufficientCapacityException e) - { - return false; - } - } - - /** - * Allows a variable number of user supplied arguments per event. - * - * @param translator The user specified translation for the event - * @param args User supplied arguments, one Object[] per event. - * @see #publishEvents(com.lmax.disruptor.EventTranslator[]) - */ - public void publishEvents(EventTranslatorVararg translator, Object[]... args) - { - publishEvents(translator, 0, args.length, args); - } - - /** - * Allows a variable number of user supplied arguments per event. - * - * @param translator The user specified translation for the event - * @param batchStartsAt The first element of the array which is within the batch. - * @param batchSize The actual size of the batch - * @param args User supplied arguments, one Object[] per event. - * @see #publishEvents(EventTranslator[]) - */ - public void publishEvents(EventTranslatorVararg translator, int batchStartsAt, int batchSize, Object[]... args) - { - checkBounds(batchStartsAt, batchSize, args); - final long finalSequence = sequencer.next(batchSize); - translateAndPublishBatch(translator, batchStartsAt, batchSize, finalSequence, args); - } - - /** - * Allows a variable number of user supplied arguments per event. - * - * @param translator The user specified translation for the event - * @param args User supplied arguments, one Object[] per event. - * @return true if the value was published, false if there was insufficient - * capacity. - * @see #publishEvents(com.lmax.disruptor.EventTranslator[]) - */ - public boolean tryPublishEvents(EventTranslatorVararg translator, Object[]... args) - { - return tryPublishEvents(translator, 0, args.length, args); - } - - /** - * Allows a variable number of user supplied arguments per event. - * - * @param translator The user specified translation for the event - * @param batchStartsAt The first element of the array which is within the batch. - * @param batchSize The actual size of the batch. - * @param args User supplied arguments, one Object[] per event. - * @return true if the value was published, false if there was insufficient - * capacity. - * @see #publishEvents(EventTranslator[]) - */ - public boolean tryPublishEvents(EventTranslatorVararg translator, int batchStartsAt, int batchSize, Object[]... args) - { - checkBounds(args, batchStartsAt, batchSize); - try - { - final long finalSequence = sequencer.tryNext(batchSize); - translateAndPublishBatch(translator, batchStartsAt, batchSize, finalSequence, args); - return true; - } - catch (InsufficientCapacityException e) - { - return false; - } - } - - /** - * Publish the specified sequence. This action marks this particular - * message as being available to be read. - * - * @param sequence the sequence to publish. - */ - public void publish(long sequence) - { - sequencer.publish(sequence); - } - - /** - * Publish the specified sequences. This action marks these particular - * messages as being available to be read. - * - * @see Sequencer#next(int) - * @param lo the lowest sequence number to be published - * @param hi the highest sequence number to be published - */ - public void publish(long lo, long hi) - { - sequencer.publish(lo, hi); - } - - /** - * Get the remaining capacity for this ringBuffer. - * @return The number of slots remaining. - */ - public long remainingCapacity() - { - return sequencer.remainingCapacity(); - } - - private void checkBounds(final EventTranslator[] translators, final int batchStartsAt, final int batchSize) - { - checkBatchSizing(batchStartsAt, batchSize); - batchOverRuns(translators, batchStartsAt, batchSize); - } - - private void checkBatchSizing(int batchStartsAt, int batchSize) - { - if(batchStartsAt < 0 || batchSize < 0) - { - throw new IllegalArgumentException("Both batchStartsAt and batchSize must be positive but got: batchStartsAt " + batchStartsAt + " and batchSize " + batchSize); - } - else if(batchSize > bufferSize) - { - throw new IllegalArgumentException("The ring buffer cannot accommodate " + batchSize + " it only has space for " + bufferSize + " entities."); - } - } - - private void checkBounds(final A[] arg0, final int batchStartsAt, final int batchSize) - { - checkBatchSizing(batchStartsAt, batchSize); - batchOverRuns(arg0, batchStartsAt, batchSize); - } - - private void checkBounds(final A[] arg0, final B[] arg1, final int batchStartsAt, final int batchSize) - { - checkBatchSizing(batchStartsAt, batchSize); - batchOverRuns(arg0, batchStartsAt, batchSize); - batchOverRuns(arg1, batchStartsAt, batchSize); - } - - private void checkBounds(final A[] arg0, final B[] arg1, final C[] arg2, final int batchStartsAt, final int batchSize) - { - checkBatchSizing(batchStartsAt, batchSize); - batchOverRuns(arg0, batchStartsAt, batchSize); - batchOverRuns(arg1, batchStartsAt, batchSize); - batchOverRuns(arg2, batchStartsAt, batchSize); - } - - private void checkBounds(final int batchStartsAt, final int batchSize, final Object[][] args) - { - checkBatchSizing(batchStartsAt, batchSize); - batchOverRuns(args, batchStartsAt, batchSize); - } - - private void batchOverRuns(final A[] arg0, final int batchStartsAt, final int batchSize) - { - if(batchStartsAt + batchSize > arg0.length) - { - throw new IllegalArgumentException("A batchSize of: " + batchSize + - " with batchStatsAt of: " + batchStartsAt + - " will overrun the available number of arguments: " + (arg0.length - batchStartsAt)); - } - } - - private void translateAndPublish(EventTranslator translator, long sequence) - { - try - { - translator.translateTo(get(sequence), sequence); - } - finally - { - sequencer.publish(sequence); - } - } - - private void translateAndPublish(EventTranslatorOneArg translator, long sequence, A arg0) - { - try - { - translator.translateTo(get(sequence), sequence, arg0); - } - finally - { - sequencer.publish(sequence); - } - } - - private void translateAndPublish(EventTranslatorTwoArg translator, long sequence, A arg0, B arg1) - { - try - { - translator.translateTo(get(sequence), sequence, arg0, arg1); - } - finally - { - sequencer.publish(sequence); - } - } - - private void translateAndPublish(EventTranslatorThreeArg translator, long sequence, - A arg0, B arg1, C arg2) - { - try - { - translator.translateTo(get(sequence), sequence, arg0, arg1, arg2); - } - finally - { - sequencer.publish(sequence); - } - } - - private void translateAndPublish(EventTranslatorVararg translator, long sequence, Object...args) - { - try - { - translator.translateTo(get(sequence), sequence, args); - } - finally - { - sequencer.publish(sequence); - } - } - - private void translateAndPublishBatch(final EventTranslator[] translators, int batchStartsAt, - final int batchSize, final long finalSequence) - { - final long initialSequence = finalSequence - (batchSize - 1); - try - { - long sequence = initialSequence; - final int batchEndsAt = batchStartsAt + batchSize; - for (int i = batchStartsAt; i < batchEndsAt; i++) - { - final EventTranslator translator = translators[i]; - translator.translateTo(get(sequence), sequence++); - } - } - finally - { - sequencer.publish(initialSequence, finalSequence); - } - } - - private void translateAndPublishBatch(final EventTranslatorOneArg translator, final A[] arg0, - int batchStartsAt, final int batchSize, final long finalSequence) - { - final long initialSequence = finalSequence - (batchSize - 1); - try - { - long sequence = initialSequence; - final int batchEndsAt = batchStartsAt + batchSize; - for (int i = batchStartsAt; i < batchEndsAt; i++) - { - translator.translateTo(get(sequence), sequence++, arg0[i]); - } - } - finally - { - sequencer.publish(initialSequence, finalSequence); - } - } - - private void translateAndPublishBatch(final EventTranslatorTwoArg translator, final A[] arg0, - final B[] arg1, int batchStartsAt, int batchSize, - final long finalSequence) - { - final long initialSequence = finalSequence - (batchSize - 1); - try - { - long sequence = initialSequence; - final int batchEndsAt = batchStartsAt + batchSize; - for (int i = batchStartsAt; i < batchEndsAt; i++) - { - translator.translateTo(get(sequence), sequence++, arg0[i], arg1[i]); - } - } - finally - { - sequencer.publish(initialSequence, finalSequence); - } - } - - private void translateAndPublishBatch(final EventTranslatorThreeArg translator, - final A[] arg0, final B[] arg1, final C[] arg2, int batchStartsAt, - final int batchSize, final long finalSequence) - { - final long initialSequence = finalSequence - (batchSize - 1); - try - { - long sequence = initialSequence; - final int batchEndsAt = batchStartsAt + batchSize; - for (int i = batchStartsAt; i < batchEndsAt; i++) - { - translator.translateTo(get(sequence), sequence++, arg0[i], arg1[i], arg2[i]); - } - } - finally - { - sequencer.publish(initialSequence, finalSequence); - } - } - - private void translateAndPublishBatch(final EventTranslatorVararg translator, int batchStartsAt, - final int batchSize, final long finalSequence, final Object[][] args) - { - final long initialSequence = finalSequence - (batchSize - 1); - try - { - long sequence = initialSequence; - final int batchEndsAt = batchStartsAt + batchSize; - for (int i = batchStartsAt; i < batchEndsAt; i++) - { - translator.translateTo(get(sequence), sequence++, args[i]); - } - } - finally - { - sequencer.publish(initialSequence, finalSequence); - } - } - - private void fill(EventFactory eventFactory) - { - for (int i = 0; i < entries.length; i++) - { - entries[i] = eventFactory.newInstance(); - } - } -} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/disruptor/SingleProducerSequencer.java b/jstorm-client/src/main/java/backtype/storm/utils/disruptor/SingleProducerSequencer.java deleted file mode 100644 index 422a6d4f6..000000000 --- a/jstorm-client/src/main/java/backtype/storm/utils/disruptor/SingleProducerSequencer.java +++ /dev/null @@ -1,199 +0,0 @@ -package backtype.storm.utils.disruptor; - -import java.util.concurrent.locks.LockSupport; - -import com.lmax.disruptor.InsufficientCapacityException; -import com.lmax.disruptor.Sequence; -import com.lmax.disruptor.Sequencer; -import com.lmax.disruptor.WaitStrategy; -import com.lmax.disruptor.util.Util; - - -/** - *

Coordinator for claiming sequences for access to a data structure while tracking dependent {@link Sequence}s.

- * - *

Generally not safe for use from multiple threads as it does not implement any barriers.

- */ -public class SingleProducerSequencer extends AbstractSequencerExt -{ - @SuppressWarnings("unused") - private static class Padding - { - /** Set to -1 as sequence starting point */ - public long nextValue = -1L, cachedValue = -1L, p2, p3, p4, p5, p6, p7; - } - - private final Padding pad = new Padding(); - - /** - * Construct a Sequencer with the selected wait strategy and buffer size. - * - * @param bufferSize the size of the buffer that this will sequence over. - * @param waitStrategy for those waiting on sequences. - */ - public SingleProducerSequencer(int bufferSize, final WaitStrategy waitStrategy) - { - super(bufferSize, waitStrategy); - } - - /** - * @see Sequencer#hasAvailableCapacity(int) - */ - @Override - public boolean hasAvailableCapacity(final int requiredCapacity) - { - long nextValue = pad.nextValue; - - long wrapPoint = (nextValue + requiredCapacity) - bufferSize; - long cachedGatingSequence = pad.cachedValue; - - if (wrapPoint > cachedGatingSequence || cachedGatingSequence > nextValue) - { - long minSequence = Util.getMinimumSequence(gatingSequences, nextValue); - pad.cachedValue = minSequence; - - if (wrapPoint > minSequence) - { - return false; - } - } - - return true; - } - - /** - * @see Sequencer#next() - */ - @Override - public long next() - { - return next(1); - } - - /** - * @see Sequencer#next(int) - */ - @Override - public long next(int n) - { - if (n < 1) - { - throw new IllegalArgumentException("n must be > 0"); - } - - long nextValue = pad.nextValue; - - long nextSequence = nextValue + n; - long wrapPoint = nextSequence - bufferSize; - long cachedGatingSequence = pad.cachedValue; - - if (wrapPoint > cachedGatingSequence || cachedGatingSequence > nextValue) - { - long minSequence; - while (wrapPoint > (minSequence = Util.getMinimumSequence(gatingSequences, nextValue))) - { - if (AbstractSequencerExt.isWaitSleep()) { - try { - Thread.sleep(1); - } catch (InterruptedException e) { - } - }else { - LockSupport.parkNanos(1); - } - } - - pad.cachedValue = minSequence; - } - - pad.nextValue = nextSequence; - - return nextSequence; - } - - /** - * @see Sequencer#tryNext() - */ - @Override - public long tryNext() throws InsufficientCapacityException - { - return tryNext(1); - } - - /** - * @see Sequencer#tryNext(int) - */ - @Override - public long tryNext(int n) throws InsufficientCapacityException - { - if (n < 1) - { - throw new IllegalArgumentException("n must be > 0"); - } - - if (!hasAvailableCapacity(n)) - { - throw InsufficientCapacityException.INSTANCE; - } - - long nextSequence = pad.nextValue += n; - - return nextSequence; - } - - /** - * @see Sequencer#remainingCapacity() - */ - @Override - public long remainingCapacity() - { - long nextValue = pad.nextValue; - - long consumed = Util.getMinimumSequence(gatingSequences, nextValue); - long produced = nextValue; - return getBufferSize() - (produced - consumed); - } - - /** - * @see Sequencer#claim(long) - */ - @Override - public void claim(long sequence) - { - pad.nextValue = sequence; - } - - /** - * @see Sequencer#publish(long) - */ - @Override - public void publish(long sequence) - { - cursor.set(sequence); - waitStrategy.signalAllWhenBlocking(); - } - - /** - * @see Sequencer#publish(long, long) - */ - @Override - public void publish(long lo, long hi) - { - publish(hi); - } - - /** - * @see Sequencer#isAvailable(long) - */ - @Override - public boolean isAvailable(long sequence) - { - return sequence <= cursor.get(); - } - - @Override - public long getHighestPublishedSequence(long lowerBound, long availableSequence) - { - return availableSequence; - } -} - diff --git a/jstorm-client/src/main/java/storm/trident/JoinType.java b/jstorm-client/src/main/java/storm/trident/JoinType.java deleted file mode 100644 index 30169d42e..000000000 --- a/jstorm-client/src/main/java/storm/trident/JoinType.java +++ /dev/null @@ -1,13 +0,0 @@ -package storm.trident; - -import java.util.Arrays; -import java.util.List; - -public enum JoinType { - INNER, - OUTER; - - public static List mixed(JoinType... types) { - return Arrays.asList(types); - } -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/storm/trident/TridentState.java b/jstorm-client/src/main/java/storm/trident/TridentState.java deleted file mode 100644 index c6771d5a6..000000000 --- a/jstorm-client/src/main/java/storm/trident/TridentState.java +++ /dev/null @@ -1,23 +0,0 @@ -package storm.trident; - -import storm.trident.planner.Node; - - -public class TridentState { - TridentTopology _topology; - Node _node; - - protected TridentState(TridentTopology topology, Node node) { - _topology = topology; - _node = node; - } - - public Stream newValuesStream() { - return new Stream(_topology, _node.name, _node); - } - - public TridentState parallelismHint(int parallelism) { - _node.parallelismHint = parallelism; - return this; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/drpc/ReturnResultsReducer.java b/jstorm-client/src/main/java/storm/trident/drpc/ReturnResultsReducer.java deleted file mode 100644 index e89719ec5..000000000 --- a/jstorm-client/src/main/java/storm/trident/drpc/ReturnResultsReducer.java +++ /dev/null @@ -1,96 +0,0 @@ -package storm.trident.drpc; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.thrift7.TException; - -import storm.trident.drpc.ReturnResultsReducer.ReturnResultsState; -import storm.trident.operation.MultiReducer; -import storm.trident.operation.TridentCollector; -import storm.trident.operation.TridentMultiReducerContext; -import storm.trident.tuple.TridentTuple; -import backtype.storm.Config; -import backtype.storm.drpc.DRPCInvocationsClient; -import backtype.storm.generated.DistributedRPCInvocations; -import backtype.storm.utils.ServiceRegistry; -import backtype.storm.utils.Utils; - - -public class ReturnResultsReducer implements MultiReducer { - public static class ReturnResultsState { - List results = new ArrayList(); - String returnInfo; - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this); - } - } - boolean local; - - Map _clients = new HashMap(); - - - @Override - public void prepare(Map conf, TridentMultiReducerContext context) { - local = conf.get(Config.STORM_CLUSTER_MODE).equals("local"); - } - - @Override - public ReturnResultsState init(TridentCollector collector) { - return new ReturnResultsState(); - } - - @Override - public void execute(ReturnResultsState state, int streamIndex, TridentTuple input, TridentCollector collector) { - if(streamIndex==0) { - state.returnInfo = input.getString(0); - } else { - state.results.add(input); - } - } - - @Override - public void complete(ReturnResultsState state, TridentCollector collector) { - // only one of the multireducers will receive the tuples - if(state.returnInfo!=null) { - String result = Utils.to_json(state.results); - Map retMap = (Map) Utils.from_json(state.returnInfo); - final String host = (String) retMap.get("host"); - final int port = Utils.getInt(retMap.get("port")); - String id = (String) retMap.get("id"); - DistributedRPCInvocations.Iface client; - if(local) { - client = (DistributedRPCInvocations.Iface) ServiceRegistry.getService(host); - } else { - List server = new ArrayList() {{ - add(host); - add(port); - }}; - - if(!_clients.containsKey(server)) { - _clients.put(server, new DRPCInvocationsClient(host, port)); - } - client = _clients.get(server); - } - - try { - client.result(id, result); - } catch(TException e) { - collector.reportError(e); - } - } - } - - @Override - public void cleanup() { - for(DRPCInvocationsClient c: _clients.values()) { - c.close(); - } - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/fluent/ChainedFullAggregatorDeclarer.java b/jstorm-client/src/main/java/storm/trident/fluent/ChainedFullAggregatorDeclarer.java deleted file mode 100644 index 84436a66c..000000000 --- a/jstorm-client/src/main/java/storm/trident/fluent/ChainedFullAggregatorDeclarer.java +++ /dev/null @@ -1,15 +0,0 @@ -package storm.trident.fluent; - -import backtype.storm.tuple.Fields; -import storm.trident.operation.Aggregator; -import storm.trident.operation.CombinerAggregator; -import storm.trident.operation.ReducerAggregator; - -public interface ChainedFullAggregatorDeclarer extends IChainedAggregatorDeclarer { - ChainedFullAggregatorDeclarer aggregate(Aggregator agg, Fields functionFields); - ChainedFullAggregatorDeclarer aggregate(Fields inputFields, Aggregator agg, Fields functionFields); - ChainedFullAggregatorDeclarer aggregate(CombinerAggregator agg, Fields functionFields); - ChainedFullAggregatorDeclarer aggregate(Fields inputFields, CombinerAggregator agg, Fields functionFields); - ChainedFullAggregatorDeclarer aggregate(ReducerAggregator agg, Fields functionFields); - ChainedFullAggregatorDeclarer aggregate(Fields inputFields, ReducerAggregator agg, Fields functionFields); -} diff --git a/jstorm-client/src/main/java/storm/trident/fluent/ChainedPartitionAggregatorDeclarer.java b/jstorm-client/src/main/java/storm/trident/fluent/ChainedPartitionAggregatorDeclarer.java deleted file mode 100644 index 00e2c5a67..000000000 --- a/jstorm-client/src/main/java/storm/trident/fluent/ChainedPartitionAggregatorDeclarer.java +++ /dev/null @@ -1,15 +0,0 @@ -package storm.trident.fluent; - -import backtype.storm.tuple.Fields; -import storm.trident.operation.Aggregator; -import storm.trident.operation.CombinerAggregator; -import storm.trident.operation.ReducerAggregator; - -public interface ChainedPartitionAggregatorDeclarer extends IChainedAggregatorDeclarer { - ChainedPartitionAggregatorDeclarer partitionAggregate(Aggregator agg, Fields functionFields); - ChainedPartitionAggregatorDeclarer partitionAggregate(Fields inputFields, Aggregator agg, Fields functionFields); - ChainedPartitionAggregatorDeclarer partitionAggregate(CombinerAggregator agg, Fields functionFields); - ChainedPartitionAggregatorDeclarer partitionAggregate(Fields inputFields, CombinerAggregator agg, Fields functionFields); - ChainedPartitionAggregatorDeclarer partitionAggregate(ReducerAggregator agg, Fields functionFields); - ChainedPartitionAggregatorDeclarer partitionAggregate(Fields inputFields, ReducerAggregator agg, Fields functionFields); -} diff --git a/jstorm-client/src/main/java/storm/trident/fluent/GlobalAggregationScheme.java b/jstorm-client/src/main/java/storm/trident/fluent/GlobalAggregationScheme.java deleted file mode 100644 index 96f15e91d..000000000 --- a/jstorm-client/src/main/java/storm/trident/fluent/GlobalAggregationScheme.java +++ /dev/null @@ -1,9 +0,0 @@ -package storm.trident.fluent; - -import storm.trident.operation.impl.SingleEmitAggregator.BatchToPartition; - - -public interface GlobalAggregationScheme { - IAggregatableStream aggPartition(S stream); // how to partition for second stage of aggregation - BatchToPartition singleEmitPartitioner(); // return null if it's not single emit -} diff --git a/jstorm-client/src/main/java/storm/trident/fluent/IAggregatableStream.java b/jstorm-client/src/main/java/storm/trident/fluent/IAggregatableStream.java deleted file mode 100644 index e10852e40..000000000 --- a/jstorm-client/src/main/java/storm/trident/fluent/IAggregatableStream.java +++ /dev/null @@ -1,14 +0,0 @@ -package storm.trident.fluent; - -import backtype.storm.tuple.Fields; -import storm.trident.Stream; -import storm.trident.operation.Aggregator; -import storm.trident.operation.Function; -import storm.trident.operation.impl.SingleEmitAggregator.BatchToPartition; - -public interface IAggregatableStream { - IAggregatableStream each(Fields inputFields, Function function, Fields functionFields); - IAggregatableStream partitionAggregate(Fields inputFields, Aggregator agg, Fields functionFields); - Stream toStream(); - Fields getOutputFields(); -} diff --git a/jstorm-client/src/main/java/storm/trident/fluent/IChainedAggregatorDeclarer.java b/jstorm-client/src/main/java/storm/trident/fluent/IChainedAggregatorDeclarer.java deleted file mode 100644 index a42dfbe57..000000000 --- a/jstorm-client/src/main/java/storm/trident/fluent/IChainedAggregatorDeclarer.java +++ /dev/null @@ -1,7 +0,0 @@ -package storm.trident.fluent; - -import storm.trident.Stream; - -public interface IChainedAggregatorDeclarer { - Stream chainEnd(); -} diff --git a/jstorm-client/src/main/java/storm/trident/fluent/UniqueIdGen.java b/jstorm-client/src/main/java/storm/trident/fluent/UniqueIdGen.java deleted file mode 100644 index 64ad62195..000000000 --- a/jstorm-client/src/main/java/storm/trident/fluent/UniqueIdGen.java +++ /dev/null @@ -1,17 +0,0 @@ -package storm.trident.fluent; - -public class UniqueIdGen { - int _streamCounter = 0; - - public String getUniqueStreamId() { - _streamCounter++; - return "s" + _streamCounter; - } - - int _stateCounter = 0; - - public String getUniqueStateId() { - _stateCounter++; - return "state" + _stateCounter; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/graph/Group.java b/jstorm-client/src/main/java/storm/trident/graph/Group.java deleted file mode 100644 index c329ad6a9..000000000 --- a/jstorm-client/src/main/java/storm/trident/graph/Group.java +++ /dev/null @@ -1,89 +0,0 @@ -package storm.trident.graph; - -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.UUID; - -import org.jgrapht.DirectedGraph; - -import storm.trident.planner.Node; -import storm.trident.util.IndexedEdge; -import storm.trident.util.TridentUtils; - - -public class Group { - public Set nodes = new HashSet(); - private DirectedGraph graph; - private String id; - - public Group(DirectedGraph graph, List nodes) { - init(graph); - this.nodes.addAll(nodes); - this.graph = graph; - } - - public Group(DirectedGraph graph, Node n) { - this(graph, Arrays.asList(n)); - } - - public Group(Group g1, Group g2) { - init(g1.graph); - nodes.addAll(g1.nodes); - nodes.addAll(g2.nodes); - } - - private void init(DirectedGraph graph) { - this.graph = graph; - this.id = UUID.randomUUID().toString(); - } - - public Set outgoingNodes() { - Set ret = new HashSet(); - for(Node n: nodes) { - ret.addAll(TridentUtils.getChildren(graph, n)); - } - return ret; - } - - public Set incomingNodes() { - Set ret = new HashSet(); - for(Node n: nodes) { - ret.addAll(TridentUtils.getParents(graph, n)); - } - return ret; - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + ((id == null) ? 0 : id.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - Group other = (Group) obj; - if (id == null) { - if (other.id != null) - return false; - } else if (!id.equals(other.id)) - return false; - return true; - } - - - - @Override - public String toString() { - return nodes.toString(); - } -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/Aggregator.java b/jstorm-client/src/main/java/storm/trident/operation/Aggregator.java deleted file mode 100644 index 518170356..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/Aggregator.java +++ /dev/null @@ -1,9 +0,0 @@ -package storm.trident.operation; - -import storm.trident.tuple.TridentTuple; - -public interface Aggregator extends Operation { - T init(Object batchId, TridentCollector collector); - void aggregate(T val, TridentTuple tuple, TridentCollector collector); - void complete(T val, TridentCollector collector); -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/Assembly.java b/jstorm-client/src/main/java/storm/trident/operation/Assembly.java deleted file mode 100644 index 17aaca27b..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/Assembly.java +++ /dev/null @@ -1,8 +0,0 @@ -package storm.trident.operation; - -import storm.trident.Stream; - - -public interface Assembly { - Stream apply(Stream input); -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/BaseAggregator.java b/jstorm-client/src/main/java/storm/trident/operation/BaseAggregator.java deleted file mode 100644 index c97b84ff5..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/BaseAggregator.java +++ /dev/null @@ -1,6 +0,0 @@ -package storm.trident.operation; - - -public abstract class BaseAggregator extends BaseOperation implements Aggregator { - -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/BaseFilter.java b/jstorm-client/src/main/java/storm/trident/operation/BaseFilter.java deleted file mode 100644 index d629d0d55..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/BaseFilter.java +++ /dev/null @@ -1,6 +0,0 @@ -package storm.trident.operation; - - -public abstract class BaseFilter extends BaseOperation implements Filter { - -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/BaseFunction.java b/jstorm-client/src/main/java/storm/trident/operation/BaseFunction.java deleted file mode 100644 index 8ff6b055b..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/BaseFunction.java +++ /dev/null @@ -1,6 +0,0 @@ -package storm.trident.operation; - - -public abstract class BaseFunction extends BaseOperation implements Function { - -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/BaseMultiReducer.java b/jstorm-client/src/main/java/storm/trident/operation/BaseMultiReducer.java deleted file mode 100644 index 328205d56..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/BaseMultiReducer.java +++ /dev/null @@ -1,16 +0,0 @@ -package storm.trident.operation; - -import java.util.Map; - -public abstract class BaseMultiReducer implements MultiReducer { - - @Override - public void prepare(Map conf, TridentMultiReducerContext context) { - } - - - @Override - public void cleanup() { - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/BaseOperation.java b/jstorm-client/src/main/java/storm/trident/operation/BaseOperation.java deleted file mode 100644 index df6166d6e..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/BaseOperation.java +++ /dev/null @@ -1,15 +0,0 @@ -package storm.trident.operation; - -import java.util.Map; - -public class BaseOperation implements Operation { - - @Override - public void prepare(Map conf, TridentOperationContext context) { - } - - @Override - public void cleanup() { - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/CombinerAggregator.java b/jstorm-client/src/main/java/storm/trident/operation/CombinerAggregator.java deleted file mode 100644 index 03933c9ec..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/CombinerAggregator.java +++ /dev/null @@ -1,12 +0,0 @@ -package storm.trident.operation; - -import java.io.Serializable; -import storm.trident.tuple.TridentTuple; - -// doesn't manipulate tuples (lists of stuff) so that things like aggregating into -// cassandra is cleaner (don't need lists everywhere, just store the single value there) -public interface CombinerAggregator extends Serializable { - T init(TridentTuple tuple); - T combine(T val1, T val2); - T zero(); -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/EachOperation.java b/jstorm-client/src/main/java/storm/trident/operation/EachOperation.java deleted file mode 100644 index b56fe962b..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/EachOperation.java +++ /dev/null @@ -1,5 +0,0 @@ -package storm.trident.operation; - -public interface EachOperation extends Operation { - -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/Filter.java b/jstorm-client/src/main/java/storm/trident/operation/Filter.java deleted file mode 100644 index ea7cbb609..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/Filter.java +++ /dev/null @@ -1,8 +0,0 @@ -package storm.trident.operation; - -import storm.trident.tuple.TridentTuple; - - -public interface Filter extends EachOperation { - boolean isKeep(TridentTuple tuple); -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/Function.java b/jstorm-client/src/main/java/storm/trident/operation/Function.java deleted file mode 100644 index b58a29d72..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/Function.java +++ /dev/null @@ -1,7 +0,0 @@ -package storm.trident.operation; - -import storm.trident.tuple.TridentTuple; - -public interface Function extends EachOperation { - void execute(TridentTuple tuple, TridentCollector collector); -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/GroupedMultiReducer.java b/jstorm-client/src/main/java/storm/trident/operation/GroupedMultiReducer.java deleted file mode 100644 index 9223cf7d6..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/GroupedMultiReducer.java +++ /dev/null @@ -1,14 +0,0 @@ -package storm.trident.operation; - -import java.io.Serializable; -import java.util.Map; -import storm.trident.tuple.TridentTuple; - - -public interface GroupedMultiReducer extends Serializable { - void prepare(Map conf, TridentMultiReducerContext context); - T init(TridentCollector collector, TridentTuple group); - void execute(T state, int streamIndex, TridentTuple group, TridentTuple input, TridentCollector collector); - void complete(T state, TridentTuple group, TridentCollector collector); - void cleanup(); -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/MultiReducer.java b/jstorm-client/src/main/java/storm/trident/operation/MultiReducer.java deleted file mode 100644 index 520f4b96d..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/MultiReducer.java +++ /dev/null @@ -1,14 +0,0 @@ -package storm.trident.operation; - -import java.io.Serializable; -import java.util.Map; -import storm.trident.tuple.TridentTuple; - - -public interface MultiReducer extends Serializable { - void prepare(Map conf, TridentMultiReducerContext context); - T init(TridentCollector collector); - void execute(T state, int streamIndex, TridentTuple input, TridentCollector collector); - void complete(T state, TridentCollector collector); - void cleanup(); -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/Operation.java b/jstorm-client/src/main/java/storm/trident/operation/Operation.java deleted file mode 100644 index f67281e4c..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/Operation.java +++ /dev/null @@ -1,9 +0,0 @@ -package storm.trident.operation; - -import java.io.Serializable; -import java.util.Map; - -public interface Operation extends Serializable { - void prepare(Map conf, TridentOperationContext context); - void cleanup(); -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/ReducerAggregator.java b/jstorm-client/src/main/java/storm/trident/operation/ReducerAggregator.java deleted file mode 100644 index 3b4efca95..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/ReducerAggregator.java +++ /dev/null @@ -1,9 +0,0 @@ -package storm.trident.operation; - -import java.io.Serializable; -import storm.trident.tuple.TridentTuple; - -public interface ReducerAggregator extends Serializable { - T init(); - T reduce(T curr, TridentTuple tuple); -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/TridentCollector.java b/jstorm-client/src/main/java/storm/trident/operation/TridentCollector.java deleted file mode 100644 index b1a74d11e..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/TridentCollector.java +++ /dev/null @@ -1,9 +0,0 @@ -package storm.trident.operation; - -import java.util.List; - - -public interface TridentCollector { - void emit(List values); - void reportError(Throwable t); -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/TridentMultiReducerContext.java b/jstorm-client/src/main/java/storm/trident/operation/TridentMultiReducerContext.java deleted file mode 100644 index fe0ff0493..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/TridentMultiReducerContext.java +++ /dev/null @@ -1,19 +0,0 @@ -package storm.trident.operation; - -import backtype.storm.tuple.Fields; -import java.util.List; -import storm.trident.tuple.TridentTuple; -import storm.trident.tuple.TridentTupleView.ProjectionFactory; - - -public class TridentMultiReducerContext { - List _factories; - - public TridentMultiReducerContext(List factories) { - _factories = factories; - } - - public ProjectionFactory makeProjectionFactory(int streamIndex, Fields fields) { - return new ProjectionFactory(_factories.get(streamIndex), fields); - } -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/TridentOperationContext.java b/jstorm-client/src/main/java/storm/trident/operation/TridentOperationContext.java deleted file mode 100644 index 3693125e4..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/TridentOperationContext.java +++ /dev/null @@ -1,48 +0,0 @@ -package storm.trident.operation; - -import backtype.storm.metric.api.CombinedMetric; -import backtype.storm.metric.api.ICombiner; -import backtype.storm.metric.api.IMetric; -import backtype.storm.metric.api.IReducer; -import backtype.storm.metric.api.ReducedMetric; -import backtype.storm.task.IMetricsContext; -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Fields; -import storm.trident.tuple.TridentTuple; -import storm.trident.tuple.TridentTupleView.ProjectionFactory; - -public class TridentOperationContext implements IMetricsContext{ - TridentTuple.Factory _factory; - TopologyContext _topoContext; - - public TridentOperationContext(TopologyContext topoContext, TridentTuple.Factory factory) { - _factory = factory; - _topoContext = topoContext; - } - - public TridentOperationContext(TridentOperationContext parent, TridentTuple.Factory factory) { - this(parent._topoContext, factory); - } - - public ProjectionFactory makeProjectionFactory(Fields fields) { - return new ProjectionFactory(_factory, fields); - } - - public int numPartitions() { - return _topoContext.getComponentTasks(_topoContext.getThisComponentId()).size(); - } - - public int getPartitionIndex() { - return _topoContext.getThisTaskIndex(); - } - - public T registerMetric(String name, T metric, int timeBucketSizeInSecs) { - return _topoContext.registerMetric(name, metric, timeBucketSizeInSecs); - } - public ReducedMetric registerMetric(String name, IReducer reducer, int timeBucketSizeInSecs) { - return _topoContext.registerMetric(name, new ReducedMetric(reducer), timeBucketSizeInSecs); - } - public CombinedMetric registerMetric(String name, ICombiner combiner, int timeBucketSizeInSecs) { - return _topoContext.registerMetric(name, new CombinedMetric(combiner), timeBucketSizeInSecs); - } -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/builtin/Count.java b/jstorm-client/src/main/java/storm/trident/operation/builtin/Count.java deleted file mode 100644 index e40177e9e..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/builtin/Count.java +++ /dev/null @@ -1,24 +0,0 @@ -package storm.trident.operation.builtin; - -import storm.trident.operation.CombinerAggregator; -import storm.trident.tuple.TridentTuple; - - -public class Count implements CombinerAggregator { - - @Override - public Long init(TridentTuple tuple) { - return 1L; - } - - @Override - public Long combine(Long val1, Long val2) { - return val1 + val2; - } - - @Override - public Long zero() { - return 0L; - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/builtin/Debug.java b/jstorm-client/src/main/java/storm/trident/operation/builtin/Debug.java deleted file mode 100644 index 34e905cb3..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/builtin/Debug.java +++ /dev/null @@ -1,22 +0,0 @@ -package storm.trident.operation.builtin; - -import storm.trident.operation.BaseFilter; -import storm.trident.tuple.TridentTuple; - -public class Debug extends BaseFilter { - private final String name; - - public Debug() { - name = "DEBUG: "; - } - - public Debug(String name) { - this.name = "DEBUG(" + name + "): "; - } - - @Override - public boolean isKeep(TridentTuple tuple) { - System.out.println(name + tuple.toString()); - return true; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/builtin/Equals.java b/jstorm-client/src/main/java/storm/trident/operation/builtin/Equals.java deleted file mode 100644 index c53cfddbe..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/builtin/Equals.java +++ /dev/null @@ -1,26 +0,0 @@ -package storm.trident.operation.builtin; - -import storm.trident.operation.BaseFilter; -import storm.trident.tuple.TridentTuple; - - -public class Equals extends BaseFilter { - - @Override - public boolean isKeep(TridentTuple tuple) { - for(int i=0; i { - @Override - public List batchRetrieve(ReadOnlyMapState map, List keys) { - return map.multiGet((List) keys); - } - - @Override - public void execute(TridentTuple tuple, Object result, TridentCollector collector) { - collector.emit(new Values(result)); - } -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/builtin/Negate.java b/jstorm-client/src/main/java/storm/trident/operation/builtin/Negate.java deleted file mode 100644 index 7a48477e2..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/builtin/Negate.java +++ /dev/null @@ -1,31 +0,0 @@ -package storm.trident.operation.builtin; - -import java.util.Map; -import storm.trident.operation.Filter; -import storm.trident.operation.TridentOperationContext; -import storm.trident.tuple.TridentTuple; - -public class Negate implements Filter { - - Filter _delegate; - - public Negate(Filter delegate) { - _delegate = delegate; - } - - @Override - public boolean isKeep(TridentTuple tuple) { - return !_delegate.isKeep(tuple); - } - - @Override - public void prepare(Map conf, TridentOperationContext context) { - _delegate.prepare(conf, context); - } - - @Override - public void cleanup() { - _delegate.cleanup(); - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/builtin/SnapshotGet.java b/jstorm-client/src/main/java/storm/trident/operation/builtin/SnapshotGet.java deleted file mode 100644 index fbc328610..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/builtin/SnapshotGet.java +++ /dev/null @@ -1,27 +0,0 @@ -package storm.trident.operation.builtin; - -import backtype.storm.tuple.Values; -import java.util.ArrayList; -import java.util.List; -import storm.trident.operation.TridentCollector; -import storm.trident.state.BaseQueryFunction; -import storm.trident.state.snapshot.ReadOnlySnapshottable; -import storm.trident.tuple.TridentTuple; - -public class SnapshotGet extends BaseQueryFunction { - - @Override - public List batchRetrieve(ReadOnlySnapshottable state, List args) { - List ret = new ArrayList(args.size()); - Object snapshot = state.get(); - for(int i=0; i { - - @Override - public Number init(TridentTuple tuple) { - return (Number) tuple.getValue(0); - } - - @Override - public Number combine(Number val1, Number val2) { - return Numbers.add(val1, val2); - } - - @Override - public Number zero() { - return 0; - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/builtin/TupleCollectionGet.java b/jstorm-client/src/main/java/storm/trident/operation/builtin/TupleCollectionGet.java deleted file mode 100644 index 6302e023b..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/builtin/TupleCollectionGet.java +++ /dev/null @@ -1,29 +0,0 @@ -package storm.trident.operation.builtin; - -import storm.trident.state.ITupleCollection; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import storm.trident.operation.TridentCollector; -import storm.trident.state.BaseQueryFunction; -import storm.trident.state.State; -import storm.trident.tuple.TridentTuple; - -public class TupleCollectionGet extends BaseQueryFunction>> { - - @Override - public List>> batchRetrieve(State state, List args) { - List>> ret = new ArrayList(args.size()); - for(int i=0; i> tuplesIterator, TridentCollector collector) { - while(tuplesIterator.hasNext()) { - collector.emit(tuplesIterator.next()); - } - } -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/impl/CaptureCollector.java b/jstorm-client/src/main/java/storm/trident/operation/impl/CaptureCollector.java deleted file mode 100644 index 9fe441939..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/impl/CaptureCollector.java +++ /dev/null @@ -1,25 +0,0 @@ -package storm.trident.operation.impl; - -import java.util.ArrayList; -import java.util.List; -import storm.trident.operation.TridentCollector; - -public class CaptureCollector implements TridentCollector { - public List> captured = new ArrayList(); - - TridentCollector _coll; - - public void setCollector(TridentCollector coll) { - _coll = coll; - } - - @Override - public void emit(List values) { - this.captured.add(values); - } - - @Override - public void reportError(Throwable t) { - _coll.reportError(t); - } -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/impl/ChainedResult.java b/jstorm-client/src/main/java/storm/trident/operation/impl/ChainedResult.java deleted file mode 100644 index a35df3aa9..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/impl/ChainedResult.java +++ /dev/null @@ -1,36 +0,0 @@ -package storm.trident.operation.impl; - -import org.apache.commons.lang.builder.ToStringBuilder; -import storm.trident.operation.TridentCollector; - - -//for ChainedAggregator -public class ChainedResult { - Object[] objs; - TridentCollector[] collectors; - - public ChainedResult(TridentCollector collector, int size) { - objs = new Object[size]; - collectors = new TridentCollector[size]; - for(int i=0; i1) { - for(TridentCollector c: collectors) { - ((CaptureCollector) c).setCollector(collector); - } - } - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(objs); - } -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/impl/CombinerAggStateUpdater.java b/jstorm-client/src/main/java/storm/trident/operation/impl/CombinerAggStateUpdater.java deleted file mode 100644 index 97a9b9dd7..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/impl/CombinerAggStateUpdater.java +++ /dev/null @@ -1,39 +0,0 @@ -package storm.trident.operation.impl; - -import backtype.storm.tuple.Values; -import java.util.List; -import java.util.Map; -import storm.trident.operation.CombinerAggregator; -import storm.trident.operation.TridentCollector; -import storm.trident.operation.TridentOperationContext; -import storm.trident.state.CombinerValueUpdater; -import storm.trident.state.StateUpdater; -import storm.trident.state.snapshot.Snapshottable; -import storm.trident.tuple.TridentTuple; - -public class CombinerAggStateUpdater implements StateUpdater { - CombinerAggregator _agg; - - public CombinerAggStateUpdater(CombinerAggregator agg) { - _agg = agg; - } - - - @Override - public void updateState(Snapshottable state, List tuples, TridentCollector collector) { - if(tuples.size()!=1) { - throw new IllegalArgumentException("Combiner state updater should receive a single tuple. Received: " + tuples.toString()); - } - Object newVal = state.update(new CombinerValueUpdater(_agg, tuples.get(0).getValue(0))); - collector.emit(new Values(newVal)); - } - - @Override - public void prepare(Map conf, TridentOperationContext context) { - } - - @Override - public void cleanup() { - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/impl/CombinerAggregatorCombineImpl.java b/jstorm-client/src/main/java/storm/trident/operation/impl/CombinerAggregatorCombineImpl.java deleted file mode 100644 index d9d00e544..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/impl/CombinerAggregatorCombineImpl.java +++ /dev/null @@ -1,44 +0,0 @@ -package storm.trident.operation.impl; - -import backtype.storm.tuple.Values; -import java.util.Map; -import storm.trident.operation.Aggregator; -import storm.trident.operation.CombinerAggregator; -import storm.trident.operation.TridentCollector; -import storm.trident.operation.TridentOperationContext; -import storm.trident.tuple.TridentTuple; - -public class CombinerAggregatorCombineImpl implements Aggregator { - CombinerAggregator _agg; - - public CombinerAggregatorCombineImpl(CombinerAggregator agg) { - _agg = agg; - } - - public void prepare(Map conf, TridentOperationContext context) { - - } - - public Result init(Object batchId, TridentCollector collector) { - Result ret = new Result(); - ret.obj = _agg.zero(); - return ret; - } - - public void aggregate(Result val, TridentTuple tuple, TridentCollector collector) { - Object v = tuple.getValue(0); - if(val.obj==null) { - val.obj = v; - } else { - val.obj = _agg.combine(val.obj, v); - } - } - - public void complete(Result val, TridentCollector collector) { - collector.emit(new Values(val.obj)); - } - - public void cleanup() { - - } -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/impl/CombinerAggregatorInitImpl.java b/jstorm-client/src/main/java/storm/trident/operation/impl/CombinerAggregatorInitImpl.java deleted file mode 100644 index 9020094d2..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/impl/CombinerAggregatorInitImpl.java +++ /dev/null @@ -1,32 +0,0 @@ -package storm.trident.operation.impl; - -import backtype.storm.tuple.Values; -import java.util.Map; -import storm.trident.operation.CombinerAggregator; -import storm.trident.operation.Function; -import storm.trident.operation.TridentCollector; -import storm.trident.operation.TridentOperationContext; -import storm.trident.tuple.TridentTuple; - -public class CombinerAggregatorInitImpl implements Function { - - CombinerAggregator _agg; - - public CombinerAggregatorInitImpl(CombinerAggregator agg) { - _agg = agg; - } - - @Override - public void execute(TridentTuple tuple, TridentCollector collector) { - collector.emit(new Values(_agg.init(tuple))); - } - - @Override - public void prepare(Map conf, TridentOperationContext context) { - } - - @Override - public void cleanup() { - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/impl/FilterExecutor.java b/jstorm-client/src/main/java/storm/trident/operation/impl/FilterExecutor.java deleted file mode 100644 index 2b96834e1..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/impl/FilterExecutor.java +++ /dev/null @@ -1,36 +0,0 @@ -package storm.trident.operation.impl; - -import java.util.Map; -import storm.trident.operation.Filter; -import storm.trident.operation.Function; -import storm.trident.operation.TridentCollector; -import storm.trident.operation.TridentOperationContext; -import storm.trident.tuple.TridentTuple; - -// works by emitting null to the collector. since the planner knows this is an ADD node with -// no new output fields, it just passes the tuple forward -public class FilterExecutor implements Function { - Filter _filter; - - public FilterExecutor(Filter filter) { - _filter = filter; - } - - @Override - public void execute(TridentTuple tuple, TridentCollector collector) { - if(_filter.isKeep(tuple)) { - collector.emit(null); - } - } - - @Override - public void prepare(Map conf, TridentOperationContext context) { - _filter.prepare(conf, context); - } - - @Override - public void cleanup() { - _filter.cleanup(); - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/impl/GlobalBatchToPartition.java b/jstorm-client/src/main/java/storm/trident/operation/impl/GlobalBatchToPartition.java deleted file mode 100644 index 3bf52b3a3..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/impl/GlobalBatchToPartition.java +++ /dev/null @@ -1,12 +0,0 @@ -package storm.trident.operation.impl; - - -public class GlobalBatchToPartition implements SingleEmitAggregator.BatchToPartition { - - @Override - public int partitionIndex(Object batchId, int numPartitions) { - // TODO: take away knowledge of storm's internals here - return 0; - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/impl/GroupCollector.java b/jstorm-client/src/main/java/storm/trident/operation/impl/GroupCollector.java deleted file mode 100644 index b997217d1..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/impl/GroupCollector.java +++ /dev/null @@ -1,31 +0,0 @@ -package storm.trident.operation.impl; - -import java.util.List; -import storm.trident.operation.TridentCollector; -import storm.trident.tuple.ComboList; - -public class GroupCollector implements TridentCollector { - public List currGroup; - - ComboList.Factory _factory; - TridentCollector _collector; - - public GroupCollector(TridentCollector collector, ComboList.Factory factory) { - _factory = factory; - _collector = collector; - } - - @Override - public void emit(List values) { - List[] delegates = new List[2]; - delegates[0] = currGroup; - delegates[1] = values; - _collector.emit(_factory.create(delegates)); - } - - @Override - public void reportError(Throwable t) { - _collector.reportError(t); - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/impl/IdentityMultiReducer.java b/jstorm-client/src/main/java/storm/trident/operation/impl/IdentityMultiReducer.java deleted file mode 100644 index f482ec428..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/impl/IdentityMultiReducer.java +++ /dev/null @@ -1,34 +0,0 @@ -package storm.trident.operation.impl; - -import java.util.Map; -import storm.trident.operation.MultiReducer; -import storm.trident.operation.TridentCollector; -import storm.trident.operation.TridentMultiReducerContext; -import storm.trident.tuple.TridentTuple; - - -public class IdentityMultiReducer implements MultiReducer { - - @Override - public void prepare(Map conf, TridentMultiReducerContext context) { - } - - @Override - public Object init(TridentCollector collector) { - return null; - } - - @Override - public void execute(Object state, int streamIndex, TridentTuple input, TridentCollector collector) { - collector.emit(input); - } - - @Override - public void complete(Object state, TridentCollector collector) { - } - - @Override - public void cleanup() { - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/impl/IndexHashBatchToPartition.java b/jstorm-client/src/main/java/storm/trident/operation/impl/IndexHashBatchToPartition.java deleted file mode 100644 index 779c4b8b5..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/impl/IndexHashBatchToPartition.java +++ /dev/null @@ -1,12 +0,0 @@ -package storm.trident.operation.impl; - -import storm.trident.partition.IndexHashGrouping; - -public class IndexHashBatchToPartition implements SingleEmitAggregator.BatchToPartition { - - @Override - public int partitionIndex(Object batchId, int numPartitions) { - return IndexHashGrouping.objectToIndex(batchId, numPartitions); - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/impl/ReducerAggStateUpdater.java b/jstorm-client/src/main/java/storm/trident/operation/impl/ReducerAggStateUpdater.java deleted file mode 100644 index 647d30f24..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/impl/ReducerAggStateUpdater.java +++ /dev/null @@ -1,36 +0,0 @@ -package storm.trident.operation.impl; - -import backtype.storm.tuple.Values; -import java.util.List; -import java.util.Map; -import storm.trident.operation.ReducerAggregator; -import storm.trident.operation.TridentCollector; -import storm.trident.operation.TridentOperationContext; -import storm.trident.state.ReducerValueUpdater; -import storm.trident.state.StateUpdater; -import storm.trident.state.snapshot.Snapshottable; -import storm.trident.tuple.TridentTuple; - -public class ReducerAggStateUpdater implements StateUpdater { - ReducerAggregator _agg; - - public ReducerAggStateUpdater(ReducerAggregator agg) { - _agg = agg; - } - - - @Override - public void updateState(Snapshottable state, List tuples, TridentCollector collector) { - Object newVal = state.update(new ReducerValueUpdater(_agg, tuples)); - collector.emit(new Values(newVal)); - } - - @Override - public void prepare(Map conf, TridentOperationContext context) { - } - - @Override - public void cleanup() { - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/impl/ReducerAggregatorImpl.java b/jstorm-client/src/main/java/storm/trident/operation/impl/ReducerAggregatorImpl.java deleted file mode 100644 index c0477629c..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/impl/ReducerAggregatorImpl.java +++ /dev/null @@ -1,39 +0,0 @@ -package storm.trident.operation.impl; - -import backtype.storm.tuple.Values; -import java.util.Map; -import storm.trident.operation.Aggregator; -import storm.trident.operation.ReducerAggregator; -import storm.trident.operation.TridentCollector; -import storm.trident.operation.TridentOperationContext; -import storm.trident.tuple.TridentTuple; - -public class ReducerAggregatorImpl implements Aggregator { - ReducerAggregator _agg; - - public ReducerAggregatorImpl(ReducerAggregator agg) { - _agg = agg; - } - - public void prepare(Map conf, TridentOperationContext context) { - - } - - public Result init(Object batchId, TridentCollector collector) { - Result ret = new Result(); - ret.obj = _agg.init(); - return ret; - } - - public void aggregate(Result val, TridentTuple tuple, TridentCollector collector) { - val.obj = _agg.reduce(val.obj, tuple); - } - - public void complete(Result val, TridentCollector collector) { - collector.emit(new Values(val.obj)); - } - - public void cleanup() { - - } -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/impl/Result.java b/jstorm-client/src/main/java/storm/trident/operation/impl/Result.java deleted file mode 100644 index 3748a7a35..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/impl/Result.java +++ /dev/null @@ -1,10 +0,0 @@ -package storm.trident.operation.impl; - -public class Result { - public Object obj; - - @Override - public String toString() { - return "" + obj; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/impl/SingleEmitAggregator.java b/jstorm-client/src/main/java/storm/trident/operation/impl/SingleEmitAggregator.java deleted file mode 100644 index 4be7c456e..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/impl/SingleEmitAggregator.java +++ /dev/null @@ -1,78 +0,0 @@ -package storm.trident.operation.impl; - -import java.io.Serializable; -import java.util.Map; -import storm.trident.operation.Aggregator; -import storm.trident.operation.TridentCollector; -import storm.trident.operation.TridentOperationContext; -import storm.trident.operation.impl.SingleEmitAggregator.SingleEmitState; -import storm.trident.tuple.TridentTuple; - - -public class SingleEmitAggregator implements Aggregator { - public static interface BatchToPartition extends Serializable { - int partitionIndex(Object batchId, int numPartitions); - } - - static class SingleEmitState { - boolean received = false; - Object state; - Object batchId; - - public SingleEmitState(Object batchId) { - this.batchId = batchId; - } - } - - Aggregator _agg; - BatchToPartition _batchToPartition; - - public SingleEmitAggregator(Aggregator agg, BatchToPartition batchToPartition) { - _agg = agg; - _batchToPartition = batchToPartition; - } - - - @Override - public SingleEmitState init(Object batchId, TridentCollector collector) { - return new SingleEmitState(batchId); - } - - @Override - public void aggregate(SingleEmitState val, TridentTuple tuple, TridentCollector collector) { - if(!val.received) { - val.state = _agg.init(val.batchId, collector); - val.received = true; - } - _agg.aggregate(val.state, tuple, collector); - } - - @Override - public void complete(SingleEmitState val, TridentCollector collector) { - if(!val.received) { - if(this.myPartitionIndex == _batchToPartition.partitionIndex(val.batchId, this.totalPartitions)) { - val.state = _agg.init(val.batchId, collector); - _agg.complete(val.state, collector); - } - } else { - _agg.complete(val.state, collector); - } - } - - int myPartitionIndex; - int totalPartitions; - - @Override - public void prepare(Map conf, TridentOperationContext context) { - _agg.prepare(conf, context); - this.myPartitionIndex = context.getPartitionIndex(); - this.totalPartitions = context.numPartitions(); - } - - @Override - public void cleanup() { - _agg.cleanup(); - } - - -} diff --git a/jstorm-client/src/main/java/storm/trident/operation/impl/TrueFilter.java b/jstorm-client/src/main/java/storm/trident/operation/impl/TrueFilter.java deleted file mode 100644 index 6e9d15c3b..000000000 --- a/jstorm-client/src/main/java/storm/trident/operation/impl/TrueFilter.java +++ /dev/null @@ -1,23 +0,0 @@ -package storm.trident.operation.impl; - -import java.util.Map; -import storm.trident.operation.Filter; -import storm.trident.operation.TridentOperationContext; -import storm.trident.tuple.TridentTuple; - -public class TrueFilter implements Filter { - - @Override - public boolean isKeep(TridentTuple tuple) { - return true; - } - - @Override - public void prepare(Map conf, TridentOperationContext context) { - } - - @Override - public void cleanup() { - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/partition/GlobalGrouping.java b/jstorm-client/src/main/java/storm/trident/partition/GlobalGrouping.java deleted file mode 100644 index 0270bf498..000000000 --- a/jstorm-client/src/main/java/storm/trident/partition/GlobalGrouping.java +++ /dev/null @@ -1,28 +0,0 @@ -package storm.trident.partition; - -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.grouping.CustomStreamGrouping; -import backtype.storm.task.WorkerTopologyContext; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -public class GlobalGrouping implements CustomStreamGrouping { - - List target; - - - @Override - public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List targets) { - List sorted = new ArrayList(targets); - Collections.sort(sorted); - target = Arrays.asList(sorted.get(0)); - } - - @Override - public List chooseTasks(int i, List list) { - return target; - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/partition/IdentityGrouping.java b/jstorm-client/src/main/java/storm/trident/partition/IdentityGrouping.java deleted file mode 100644 index ccb9d6e37..000000000 --- a/jstorm-client/src/main/java/storm/trident/partition/IdentityGrouping.java +++ /dev/null @@ -1,44 +0,0 @@ -package storm.trident.partition; - -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.grouping.CustomStreamGrouping; -import backtype.storm.task.WorkerTopologyContext; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - - -public class IdentityGrouping implements CustomStreamGrouping { - - List ret = new ArrayList(); - Map> _precomputed = new HashMap(); - - @Override - public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List tasks) { - List sourceTasks = new ArrayList(context.getComponentTasks(stream.get_componentId())); - Collections.sort(sourceTasks); - if(sourceTasks.size()!=tasks.size()) { - throw new RuntimeException("Can only do an identity grouping when source and target have same number of tasks"); - } - tasks = new ArrayList(tasks); - Collections.sort(tasks); - for(int i=0; i chooseTasks(int task, List values) { - List ret = _precomputed.get(task); - if(ret==null) { - throw new RuntimeException("Tuple emitted by task that's not part of this component. Should be impossible"); - } - return ret; - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/partition/IndexHashGrouping.java b/jstorm-client/src/main/java/storm/trident/partition/IndexHashGrouping.java deleted file mode 100644 index 69c36aceb..000000000 --- a/jstorm-client/src/main/java/storm/trident/partition/IndexHashGrouping.java +++ /dev/null @@ -1,36 +0,0 @@ -package storm.trident.partition; - -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.grouping.CustomStreamGrouping; -import backtype.storm.task.WorkerTopologyContext; -import java.util.Arrays; -import java.util.List; - -public class IndexHashGrouping implements CustomStreamGrouping { - public static int objectToIndex(Object val, int numPartitions) { - if(val==null) return 0; - else { - return Math.abs(val.hashCode() % numPartitions); - } - } - - int _index; - List _targets; - - public IndexHashGrouping(int index) { - _index = index; - } - - - @Override - public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List targetTasks) { - _targets = targetTasks; - } - - @Override - public List chooseTasks(int fromTask, List values) { - int i = objectToIndex(values.get(_index), _targets.size()); - return Arrays.asList(_targets.get(i)); - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/planner/BridgeReceiver.java b/jstorm-client/src/main/java/storm/trident/planner/BridgeReceiver.java deleted file mode 100644 index b596d54c6..000000000 --- a/jstorm-client/src/main/java/storm/trident/planner/BridgeReceiver.java +++ /dev/null @@ -1,21 +0,0 @@ -package storm.trident.planner; - -import backtype.storm.coordination.BatchOutputCollector; -import storm.trident.tuple.ConsList; -import storm.trident.tuple.TridentTuple; - - -public class BridgeReceiver implements TupleReceiver { - - BatchOutputCollector _collector; - - public BridgeReceiver(BatchOutputCollector collector) { - _collector = collector; - } - - @Override - public void execute(ProcessorContext context, String streamId, TridentTuple tuple) { - _collector.emit(streamId, new ConsList(context.batchId, tuple)); - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/planner/Node.java b/jstorm-client/src/main/java/storm/trident/planner/Node.java deleted file mode 100644 index 1a0e29dd1..000000000 --- a/jstorm-client/src/main/java/storm/trident/planner/Node.java +++ /dev/null @@ -1,63 +0,0 @@ -package storm.trident.planner; - -import backtype.storm.tuple.Fields; - -import java.io.Serializable; -import java.util.UUID; -import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; - -public class Node implements Serializable { - private static AtomicInteger INDEX = new AtomicInteger(0); - - private String nodeId; - - public String name = null; - public Fields allOutputFields; - public String streamId; - public Integer parallelismHint = null; - public NodeStateInfo stateInfo = null; - public int creationIndex; - - public Node(String streamId, String name, Fields allOutputFields) { - this.nodeId = UUID.randomUUID().toString(); - this.allOutputFields = allOutputFields; - this.streamId = streamId; - this.name = name; - this.creationIndex = INDEX.incrementAndGet(); - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + ((nodeId == null) ? 0 : nodeId.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - Node other = (Node) obj; - if (nodeId == null) { - if (other.nodeId != null) - return false; - } else if (!nodeId.equals(other.nodeId)) - return false; - return true; - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this, - ToStringStyle.MULTI_LINE_STYLE); - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/planner/NodeStateInfo.java b/jstorm-client/src/main/java/storm/trident/planner/NodeStateInfo.java deleted file mode 100644 index a045eef5f..000000000 --- a/jstorm-client/src/main/java/storm/trident/planner/NodeStateInfo.java +++ /dev/null @@ -1,14 +0,0 @@ -package storm.trident.planner; - -import java.io.Serializable; -import storm.trident.state.StateSpec; - -public class NodeStateInfo implements Serializable { - public String id; - public StateSpec spec; - - public NodeStateInfo(String id, StateSpec spec) { - this.id = id; - this.spec = spec; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/planner/PartitionNode.java b/jstorm-client/src/main/java/storm/trident/planner/PartitionNode.java deleted file mode 100644 index fdde1338b..000000000 --- a/jstorm-client/src/main/java/storm/trident/planner/PartitionNode.java +++ /dev/null @@ -1,35 +0,0 @@ -package storm.trident.planner; - -import backtype.storm.generated.Grouping; -import backtype.storm.tuple.Fields; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.util.ArrayList; -import java.util.List; -import storm.trident.util.TridentUtils; - - -public class PartitionNode extends Node { - public transient Grouping thriftGrouping; - - //has the streamid/outputFields of the node it's doing the partitioning on - public PartitionNode(String streamId, String name, Fields allOutputFields, Grouping grouping) { - super(streamId, name, allOutputFields); - this.thriftGrouping = grouping; - } - - private void writeObject(ObjectOutputStream oos) throws IOException { - oos.defaultWriteObject(); - byte[] ser = TridentUtils.thriftSerialize(thriftGrouping); - oos.writeInt(ser.length); - oos.write(ser); - } - - private void readObject(ObjectInputStream ois) throws ClassNotFoundException, IOException { - ois.defaultReadObject(); - byte[] ser = new byte[ois.readInt()]; - ois.readFully(ser); - this.thriftGrouping = TridentUtils.thriftDeserialize(Grouping.class, ser); - } -} diff --git a/jstorm-client/src/main/java/storm/trident/planner/ProcessorContext.java b/jstorm-client/src/main/java/storm/trident/planner/ProcessorContext.java deleted file mode 100644 index dc8bb6aa8..000000000 --- a/jstorm-client/src/main/java/storm/trident/planner/ProcessorContext.java +++ /dev/null @@ -1,12 +0,0 @@ -package storm.trident.planner; - - -public class ProcessorContext { - public Object batchId; - public Object[] state; - - public ProcessorContext(Object batchId, Object[] state) { - this.batchId = batchId; - this.state = state; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/planner/ProcessorNode.java b/jstorm-client/src/main/java/storm/trident/planner/ProcessorNode.java deleted file mode 100644 index c0e09aadb..000000000 --- a/jstorm-client/src/main/java/storm/trident/planner/ProcessorNode.java +++ /dev/null @@ -1,16 +0,0 @@ -package storm.trident.planner; - -import backtype.storm.tuple.Fields; - -public class ProcessorNode extends Node { - - public boolean committer; // for partitionpersist - public TridentProcessor processor; - public Fields selfOutFields; - - public ProcessorNode(String streamId, String name, Fields allOutputFields, Fields selfOutFields, TridentProcessor processor) { - super(streamId, name, allOutputFields); - this.processor = processor; - this.selfOutFields = selfOutFields; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/planner/SpoutNode.java b/jstorm-client/src/main/java/storm/trident/planner/SpoutNode.java deleted file mode 100644 index 1432c4311..000000000 --- a/jstorm-client/src/main/java/storm/trident/planner/SpoutNode.java +++ /dev/null @@ -1,22 +0,0 @@ -package storm.trident.planner; - -import backtype.storm.tuple.Fields; - - -public class SpoutNode extends Node { - public static enum SpoutType { - DRPC, - BATCH - } - - public Object spout; - public String txId; //where state is stored in zookeeper (only for batch spout types) - public SpoutType type; - - public SpoutNode(String streamId, Fields allOutputFields, String txid, Object spout, SpoutType type) { - super(streamId, null, allOutputFields); - this.txId = txid; - this.spout = spout; - this.type = type; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/planner/TridentProcessor.java b/jstorm-client/src/main/java/storm/trident/planner/TridentProcessor.java deleted file mode 100644 index 866d05898..000000000 --- a/jstorm-client/src/main/java/storm/trident/planner/TridentProcessor.java +++ /dev/null @@ -1,23 +0,0 @@ -package storm.trident.planner; - -import backtype.storm.task.TopologyContext; -import java.io.Serializable; -import java.util.Map; -import storm.trident.planner.processor.TridentContext; -import storm.trident.tuple.TridentTuple.Factory; - -public interface TridentProcessor extends Serializable, TupleReceiver { - - // imperative that don't emit any tuples from here, since output factory cannot be gotten until - // preparation is done, therefore, receivers won't be ready to receive tuples yet - // can't emit tuples from here anyway, since it's not within a batch context (which is only - // startBatch, execute, and finishBatch - void prepare(Map conf, TopologyContext context, TridentContext tridentContext); - void cleanup(); - - void startBatch(ProcessorContext processorContext); - - void finishBatch(ProcessorContext processorContext); - - Factory getOutputFactory(); -} diff --git a/jstorm-client/src/main/java/storm/trident/planner/TupleReceiver.java b/jstorm-client/src/main/java/storm/trident/planner/TupleReceiver.java deleted file mode 100644 index a2fc148e2..000000000 --- a/jstorm-client/src/main/java/storm/trident/planner/TupleReceiver.java +++ /dev/null @@ -1,10 +0,0 @@ -package storm.trident.planner; - -import storm.trident.tuple.TridentTuple; - - -public interface TupleReceiver { - //streaId indicates where tuple came from - void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple); - -} diff --git a/jstorm-client/src/main/java/storm/trident/planner/processor/AggregateProcessor.java b/jstorm-client/src/main/java/storm/trident/planner/processor/AggregateProcessor.java deleted file mode 100644 index ce6279071..000000000 --- a/jstorm-client/src/main/java/storm/trident/planner/processor/AggregateProcessor.java +++ /dev/null @@ -1,67 +0,0 @@ -package storm.trident.planner.processor; - -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Fields; -import java.util.List; -import java.util.Map; -import storm.trident.operation.Aggregator; -import storm.trident.operation.TridentOperationContext; -import storm.trident.planner.ProcessorContext; -import storm.trident.planner.TridentProcessor; -import storm.trident.tuple.TridentTuple; -import storm.trident.tuple.TridentTuple.Factory; -import storm.trident.tuple.TridentTupleView.ProjectionFactory; - - -public class AggregateProcessor implements TridentProcessor { - Aggregator _agg; - TridentContext _context; - FreshCollector _collector; - Fields _inputFields; - ProjectionFactory _projection; - - public AggregateProcessor(Fields inputFields, Aggregator agg) { - _agg = agg; - _inputFields = inputFields; - } - - @Override - public void prepare(Map conf, TopologyContext context, TridentContext tridentContext) { - List parents = tridentContext.getParentTupleFactories(); - if(parents.size()!=1) { - throw new RuntimeException("Aggregate operation can only have one parent"); - } - _context = tridentContext; - _collector = new FreshCollector(tridentContext); - _projection = new ProjectionFactory(parents.get(0), _inputFields); - _agg.prepare(conf, new TridentOperationContext(context, _projection)); - } - - @Override - public void cleanup() { - _agg.cleanup(); - } - - @Override - public void startBatch(ProcessorContext processorContext) { - _collector.setContext(processorContext); - processorContext.state[_context.getStateIndex()] = _agg.init(processorContext.batchId, _collector); - } - - @Override - public void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple) { - _collector.setContext(processorContext); - _agg.aggregate(processorContext.state[_context.getStateIndex()], _projection.create(tuple), _collector); - } - - @Override - public void finishBatch(ProcessorContext processorContext) { - _collector.setContext(processorContext); - _agg.complete(processorContext.state[_context.getStateIndex()], _collector); - } - - @Override - public Factory getOutputFactory() { - return _collector.getOutputFactory(); - } -} diff --git a/jstorm-client/src/main/java/storm/trident/planner/processor/AppendCollector.java b/jstorm-client/src/main/java/storm/trident/planner/processor/AppendCollector.java deleted file mode 100644 index 92932cb9a..000000000 --- a/jstorm-client/src/main/java/storm/trident/planner/processor/AppendCollector.java +++ /dev/null @@ -1,45 +0,0 @@ -package storm.trident.planner.processor; - -import java.util.List; -import storm.trident.operation.TridentCollector; -import storm.trident.planner.ProcessorContext; -import storm.trident.planner.TupleReceiver; -import storm.trident.tuple.TridentTuple; -import storm.trident.tuple.TridentTuple.Factory; -import storm.trident.tuple.TridentTupleView; -import storm.trident.tuple.TridentTupleView.OperationOutputFactory; - - -public class AppendCollector implements TridentCollector { - OperationOutputFactory _factory; - TridentContext _triContext; - TridentTuple tuple; - ProcessorContext context; - - public AppendCollector(TridentContext context) { - _triContext = context; - _factory = new OperationOutputFactory(context.getParentTupleFactories().get(0), context.getSelfOutputFields()); - } - - public void setContext(ProcessorContext pc, TridentTuple t) { - this.context = pc; - this.tuple = t; - } - - @Override - public void emit(List values) { - TridentTuple toEmit = _factory.create((TridentTupleView) tuple, values); - for(TupleReceiver r: _triContext.getReceivers()) { - r.execute(context, _triContext.getOutStreamId(), toEmit); - } - } - - @Override - public void reportError(Throwable t) { - _triContext.getDelegateCollector().reportError(t); - } - - public Factory getOutputFactory() { - return _factory; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/planner/processor/EachProcessor.java b/jstorm-client/src/main/java/storm/trident/planner/processor/EachProcessor.java deleted file mode 100644 index 7b217de1c..000000000 --- a/jstorm-client/src/main/java/storm/trident/planner/processor/EachProcessor.java +++ /dev/null @@ -1,63 +0,0 @@ -package storm.trident.planner.processor; - -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Fields; -import java.util.List; -import java.util.Map; -import storm.trident.operation.Function; -import storm.trident.operation.TridentOperationContext; -import storm.trident.planner.ProcessorContext; -import storm.trident.planner.TridentProcessor; -import storm.trident.tuple.TridentTuple; -import storm.trident.tuple.TridentTuple.Factory; -import storm.trident.tuple.TridentTupleView.ProjectionFactory; - - -public class EachProcessor implements TridentProcessor { - Function _function; - TridentContext _context; - AppendCollector _collector; - Fields _inputFields; - ProjectionFactory _projection; - - public EachProcessor(Fields inputFields, Function function) { - _function = function; - _inputFields = inputFields; - } - - @Override - public void prepare(Map conf, TopologyContext context, TridentContext tridentContext) { - List parents = tridentContext.getParentTupleFactories(); - if(parents.size()!=1) { - throw new RuntimeException("Each operation can only have one parent"); - } - _context = tridentContext; - _collector = new AppendCollector(tridentContext); - _projection = new ProjectionFactory(parents.get(0), _inputFields); - _function.prepare(conf, new TridentOperationContext(context, _projection)); - } - - @Override - public void cleanup() { - _function.cleanup(); - } - - @Override - public void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple) { - _collector.setContext(processorContext, tuple); - _function.execute(_projection.create(tuple), _collector); - } - - @Override - public void startBatch(ProcessorContext processorContext) { - } - - @Override - public void finishBatch(ProcessorContext processorContext) { - } - - @Override - public Factory getOutputFactory() { - return _collector.getOutputFactory(); - } -} diff --git a/jstorm-client/src/main/java/storm/trident/planner/processor/FreshCollector.java b/jstorm-client/src/main/java/storm/trident/planner/processor/FreshCollector.java deleted file mode 100644 index 1fb3aa6d3..000000000 --- a/jstorm-client/src/main/java/storm/trident/planner/processor/FreshCollector.java +++ /dev/null @@ -1,42 +0,0 @@ -package storm.trident.planner.processor; - -import java.util.List; -import storm.trident.operation.TridentCollector; -import storm.trident.planner.ProcessorContext; -import storm.trident.planner.TupleReceiver; -import storm.trident.tuple.TridentTuple; -import storm.trident.tuple.TridentTuple.Factory; -import storm.trident.tuple.TridentTupleView.FreshOutputFactory; - - -public class FreshCollector implements TridentCollector { - FreshOutputFactory _factory; - TridentContext _triContext; - ProcessorContext context; - - public FreshCollector(TridentContext context) { - _triContext = context; - _factory = new FreshOutputFactory(context.getSelfOutputFields()); - } - - public void setContext(ProcessorContext pc) { - this.context = pc; - } - - @Override - public void emit(List values) { - TridentTuple toEmit = _factory.create(values); - for(TupleReceiver r: _triContext.getReceivers()) { - r.execute(context, _triContext.getOutStreamId(), toEmit); - } - } - - @Override - public void reportError(Throwable t) { - _triContext.getDelegateCollector().reportError(t); - } - - public Factory getOutputFactory() { - return _factory; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/planner/processor/ProjectedProcessor.java b/jstorm-client/src/main/java/storm/trident/planner/processor/ProjectedProcessor.java deleted file mode 100644 index c6d34e519..000000000 --- a/jstorm-client/src/main/java/storm/trident/planner/processor/ProjectedProcessor.java +++ /dev/null @@ -1,56 +0,0 @@ -package storm.trident.planner.processor; - -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Fields; -import java.util.Map; -import storm.trident.planner.ProcessorContext; -import storm.trident.planner.TridentProcessor; -import storm.trident.planner.TupleReceiver; -import storm.trident.tuple.TridentTuple; -import storm.trident.tuple.TridentTuple.Factory; -import storm.trident.tuple.TridentTupleView.ProjectionFactory; - - -public class ProjectedProcessor implements TridentProcessor { - Fields _projectFields; - ProjectionFactory _factory; - TridentContext _context; - - public ProjectedProcessor(Fields projectFields) { - _projectFields = projectFields; - } - - @Override - public void prepare(Map conf, TopologyContext context, TridentContext tridentContext) { - if(tridentContext.getParentTupleFactories().size()!=1) { - throw new RuntimeException("Projection processor can only have one parent"); - } - _context = tridentContext; - _factory = new ProjectionFactory(tridentContext.getParentTupleFactories().get(0), _projectFields); - } - - @Override - public void cleanup() { - } - - @Override - public void startBatch(ProcessorContext processorContext) { - } - - @Override - public void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple) { - TridentTuple toEmit = _factory.create(tuple); - for(TupleReceiver r: _context.getReceivers()) { - r.execute(processorContext, _context.getOutStreamId(), toEmit); - } - } - - @Override - public void finishBatch(ProcessorContext processorContext) { - } - - @Override - public Factory getOutputFactory() { - return _factory; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/planner/processor/TridentContext.java b/jstorm-client/src/main/java/storm/trident/planner/processor/TridentContext.java deleted file mode 100644 index ae39768c1..000000000 --- a/jstorm-client/src/main/java/storm/trident/planner/processor/TridentContext.java +++ /dev/null @@ -1,59 +0,0 @@ -package storm.trident.planner.processor; - -import backtype.storm.coordination.BatchOutputCollector; -import backtype.storm.tuple.Fields; -import java.util.List; -import storm.trident.planner.TupleReceiver; -import storm.trident.tuple.TridentTuple.Factory; - - -public class TridentContext { - Fields selfFields; - List parentFactories; - List parentStreams; - List receivers; - String outStreamId; - int stateIndex; - BatchOutputCollector collector; - - public TridentContext(Fields selfFields, List parentFactories, - List parentStreams, List receivers, - String outStreamId, int stateIndex, BatchOutputCollector collector) { - this.selfFields = selfFields; - this.parentFactories = parentFactories; - this.parentStreams = parentStreams; - this.receivers = receivers; - this.outStreamId = outStreamId; - this.stateIndex = stateIndex; - this.collector = collector; - } - - public List getParentTupleFactories() { - return parentFactories; - } - - public Fields getSelfOutputFields() { - return selfFields; - } - - public List getParentStreams() { - return parentStreams; - } - - public List getReceivers() { - return receivers; - } - - public String getOutStreamId() { - return outStreamId; - } - - public int getStateIndex() { - return stateIndex; - } - - //for reporting errors - public BatchOutputCollector getDelegateCollector() { - return collector; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/spout/BatchSpoutExecutor.java b/jstorm-client/src/main/java/storm/trident/spout/BatchSpoutExecutor.java deleted file mode 100644 index 4e5fc99f7..000000000 --- a/jstorm-client/src/main/java/storm/trident/spout/BatchSpoutExecutor.java +++ /dev/null @@ -1,75 +0,0 @@ -package storm.trident.spout; - -import backtype.storm.task.TopologyContext; -import storm.trident.topology.TransactionAttempt; -import backtype.storm.tuple.Fields; -import java.util.Map; -import storm.trident.operation.TridentCollector; - -public class BatchSpoutExecutor implements ITridentSpout { - public static class EmptyCoordinator implements BatchCoordinator { - @Override - public Object initializeTransaction(long txid, Object prevMetadata, Object currMetadata) { - return null; - } - - @Override - public void close() { - } - - @Override - public void success(long txid) { - } - - @Override - public boolean isReady(long txid) { - return true; - } - } - - public class BatchSpoutEmitter implements Emitter { - - @Override - public void emitBatch(TransactionAttempt tx, Object coordinatorMeta, TridentCollector collector) { - _spout.emitBatch(tx.getTransactionId(), collector); - } - - @Override - public void success(TransactionAttempt tx) { - _spout.ack(tx.getTransactionId()); - } - - @Override - public void close() { - _spout.close(); - } - } - - IBatchSpout _spout; - - public BatchSpoutExecutor(IBatchSpout spout) { - _spout = spout; - } - - @Override - public BatchCoordinator getCoordinator(String txStateId, Map conf, TopologyContext context) { - return new EmptyCoordinator(); - } - - @Override - public Emitter getEmitter(String txStateId, Map conf, TopologyContext context) { - _spout.open(conf, context); - return new BatchSpoutEmitter(); - } - - @Override - public Map getComponentConfiguration() { - return _spout.getComponentConfiguration(); - } - - @Override - public Fields getOutputFields() { - return _spout.getOutputFields(); - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/spout/IBatchID.java b/jstorm-client/src/main/java/storm/trident/spout/IBatchID.java deleted file mode 100644 index e41d8e284..000000000 --- a/jstorm-client/src/main/java/storm/trident/spout/IBatchID.java +++ /dev/null @@ -1,7 +0,0 @@ -package storm.trident.spout; - - -public interface IBatchID { - Object getId(); - int getAttemptId(); -} diff --git a/jstorm-client/src/main/java/storm/trident/spout/IBatchSpout.java b/jstorm-client/src/main/java/storm/trident/spout/IBatchSpout.java deleted file mode 100644 index 80340267e..000000000 --- a/jstorm-client/src/main/java/storm/trident/spout/IBatchSpout.java +++ /dev/null @@ -1,16 +0,0 @@ -package storm.trident.spout; - -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Fields; -import java.io.Serializable; -import java.util.Map; -import storm.trident.operation.TridentCollector; - -public interface IBatchSpout extends Serializable { - void open(Map conf, TopologyContext context); - void emitBatch(long batchId, TridentCollector collector); - void ack(long batchId); - void close(); - Map getComponentConfiguration(); - Fields getOutputFields(); -} diff --git a/jstorm-client/src/main/java/storm/trident/spout/ICommitterTridentSpout.java b/jstorm-client/src/main/java/storm/trident/spout/ICommitterTridentSpout.java deleted file mode 100644 index e163c774e..000000000 --- a/jstorm-client/src/main/java/storm/trident/spout/ICommitterTridentSpout.java +++ /dev/null @@ -1,14 +0,0 @@ -package storm.trident.spout; - -import backtype.storm.task.TopologyContext; -import storm.trident.topology.TransactionAttempt; -import java.util.Map; - -public interface ICommitterTridentSpout extends ITridentSpout { - public interface Emitter extends ITridentSpout.Emitter { - void commit(TransactionAttempt attempt); - } - - @Override - public Emitter getEmitter(String txStateId, Map conf, TopologyContext context); -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/storm/trident/spout/IOpaquePartitionedTridentSpout.java b/jstorm-client/src/main/java/storm/trident/spout/IOpaquePartitionedTridentSpout.java deleted file mode 100644 index 7a0e25431..000000000 --- a/jstorm-client/src/main/java/storm/trident/spout/IOpaquePartitionedTridentSpout.java +++ /dev/null @@ -1,45 +0,0 @@ -package storm.trident.spout; - -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Fields; -import java.io.Serializable; -import java.util.List; -import java.util.Map; -import storm.trident.operation.TridentCollector; -import storm.trident.topology.TransactionAttempt; - -/** - * This defines a transactional spout which does *not* necessarily - * replay the same batch every time it emits a batch for a transaction id. - * - */ -public interface IOpaquePartitionedTridentSpout extends Serializable { - public interface Coordinator { - boolean isReady(long txid); - Partitions getPartitionsForBatch(); - void close(); - } - - public interface Emitter { - /** - * Emit a batch of tuples for a partition/transaction. - * - * Return the metadata describing this batch that will be used as lastPartitionMeta - * for defining the parameters of the next batch. - */ - M emitPartitionBatch(TransactionAttempt tx, TridentCollector collector, Partition partition, M lastPartitionMeta); - - /** - * This method is called when this task is responsible for a new set of partitions. Should be used - * to manage things like connections to brokers. - */ - void refreshPartitions(List partitionResponsibilities); - List getOrderedPartitions(Partitions allPartitionInfo); - void close(); - } - - Emitter getEmitter(Map conf, TopologyContext context); - Coordinator getCoordinator(Map conf, TopologyContext context); - Map getComponentConfiguration(); - Fields getOutputFields(); -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/storm/trident/spout/ISpoutPartition.java b/jstorm-client/src/main/java/storm/trident/spout/ISpoutPartition.java deleted file mode 100644 index 38a561fff..000000000 --- a/jstorm-client/src/main/java/storm/trident/spout/ISpoutPartition.java +++ /dev/null @@ -1,8 +0,0 @@ -package storm.trident.spout; - -public interface ISpoutPartition { - /** - * This is used as a Zookeeper node path for storing metadata. - */ - String getId(); -} diff --git a/jstorm-client/src/main/java/storm/trident/spout/RichSpoutBatchId.java b/jstorm-client/src/main/java/storm/trident/spout/RichSpoutBatchId.java deleted file mode 100644 index 1340d210f..000000000 --- a/jstorm-client/src/main/java/storm/trident/spout/RichSpoutBatchId.java +++ /dev/null @@ -1,32 +0,0 @@ -package storm.trident.spout; - -public class RichSpoutBatchId implements IBatchID { - long _id; - - public RichSpoutBatchId(long id) { - _id = id; - } - - @Override - public Object getId() { - // this is to distinguish from TransactionAttempt - return this; - } - - @Override - public int getAttemptId() { - return 0; // each drpc request is always a single attempt - } - - @Override - public int hashCode() { - return ((Long) _id).hashCode(); - } - - @Override - public boolean equals(Object o) { - if(!(o instanceof RichSpoutBatchId)) return false; - RichSpoutBatchId other = (RichSpoutBatchId) o; - return _id == other._id; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/spout/RichSpoutBatchIdSerializer.java b/jstorm-client/src/main/java/storm/trident/spout/RichSpoutBatchIdSerializer.java deleted file mode 100644 index d544fa75d..000000000 --- a/jstorm-client/src/main/java/storm/trident/spout/RichSpoutBatchIdSerializer.java +++ /dev/null @@ -1,21 +0,0 @@ -package storm.trident.spout; - -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.Serializer; -import com.esotericsoftware.kryo.io.Input; -import com.esotericsoftware.kryo.io.Output; - - -public class RichSpoutBatchIdSerializer extends Serializer { - - @Override - public void write(Kryo kryo, Output output, RichSpoutBatchId id) { - output.writeLong(id._id); - } - - @Override - public RichSpoutBatchId read(Kryo kryo, Input input, Class type) { - long l = input.readLong(); - return new RichSpoutBatchId(l); - } -} diff --git a/jstorm-client/src/main/java/storm/trident/spout/TridentSpoutCoordinator.java b/jstorm-client/src/main/java/storm/trident/spout/TridentSpoutCoordinator.java deleted file mode 100644 index 7095f363c..000000000 --- a/jstorm-client/src/main/java/storm/trident/spout/TridentSpoutCoordinator.java +++ /dev/null @@ -1,78 +0,0 @@ -package storm.trident.spout; - -import backtype.storm.Config; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.BasicOutputCollector; -import backtype.storm.topology.IBasicBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import storm.trident.topology.TransactionAttempt; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; -import java.util.Map; -import org.apache.log4j.Logger; -import storm.trident.topology.MasterBatchCoordinator; -import storm.trident.topology.state.RotatingTransactionalState; -import storm.trident.topology.state.TransactionalState; - - -public class TridentSpoutCoordinator implements IBasicBolt { - public static final Logger LOG = Logger.getLogger(TridentSpoutCoordinator.class); - private static final String META_DIR = "meta"; - - ITridentSpout _spout; - ITridentSpout.BatchCoordinator _coord; - RotatingTransactionalState _state; - TransactionalState _underlyingState; - String _id; - - - public TridentSpoutCoordinator(String id, ITridentSpout spout) { - _spout = spout; - _id = id; - } - - @Override - public void prepare(Map conf, TopologyContext context) { - _coord = _spout.getCoordinator(_id, conf, context); - _underlyingState = TransactionalState.newCoordinatorState(conf, _id); - _state = new RotatingTransactionalState(_underlyingState, META_DIR); - } - - @Override - public void execute(Tuple tuple, BasicOutputCollector collector) { - TransactionAttempt attempt = (TransactionAttempt) tuple.getValue(0); - - if(tuple.getSourceStreamId().equals(MasterBatchCoordinator.SUCCESS_STREAM_ID)) { - _state.cleanupBefore(attempt.getTransactionId()); - _coord.success(attempt.getTransactionId()); - } else if (tuple.getSourceStreamId().equals(MasterBatchCoordinator.COMMIT_STREAM_ID)) { - // Do nothing. - } else { - long txid = attempt.getTransactionId(); - Object prevMeta = _state.getPreviousState(txid); - Object meta = _coord.initializeTransaction(txid, prevMeta, _state.getState(txid)); - _state.overrideState(txid, meta); - collector.emit(MasterBatchCoordinator.BATCH_STREAM_ID, new Values(attempt, meta)); - } - - } - - @Override - public void cleanup() { - _coord.close(); - _underlyingState.close(); - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declareStream(MasterBatchCoordinator.BATCH_STREAM_ID, new Fields("tx", "metadata")); - } - - @Override - public Map getComponentConfiguration() { - Config ret = new Config(); - ret.setMaxTaskParallelism(1); - return ret; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/state/BaseQueryFunction.java b/jstorm-client/src/main/java/storm/trident/state/BaseQueryFunction.java deleted file mode 100644 index 49be20e1d..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/BaseQueryFunction.java +++ /dev/null @@ -1,8 +0,0 @@ -package storm.trident.state; - -import storm.trident.operation.BaseOperation; - - -public abstract class BaseQueryFunction extends BaseOperation implements QueryFunction { - -} diff --git a/jstorm-client/src/main/java/storm/trident/state/BaseStateUpdater.java b/jstorm-client/src/main/java/storm/trident/state/BaseStateUpdater.java deleted file mode 100644 index 292c59f7d..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/BaseStateUpdater.java +++ /dev/null @@ -1,8 +0,0 @@ -package storm.trident.state; - -import storm.trident.operation.BaseOperation; - - -public abstract class BaseStateUpdater extends BaseOperation implements StateUpdater { - -} diff --git a/jstorm-client/src/main/java/storm/trident/state/CombinerValueUpdater.java b/jstorm-client/src/main/java/storm/trident/state/CombinerValueUpdater.java deleted file mode 100644 index ea2248f9c..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/CombinerValueUpdater.java +++ /dev/null @@ -1,19 +0,0 @@ -package storm.trident.state; - -import storm.trident.operation.CombinerAggregator; - -public class CombinerValueUpdater implements ValueUpdater { - Object arg; - CombinerAggregator agg; - - public CombinerValueUpdater(CombinerAggregator agg, Object arg) { - this.agg = agg; - this.arg = arg; - } - - @Override - public Object update(Object stored) { - if(stored==null) return arg; - else return agg.combine(stored, arg); - } -} diff --git a/jstorm-client/src/main/java/storm/trident/state/ITupleCollection.java b/jstorm-client/src/main/java/storm/trident/state/ITupleCollection.java deleted file mode 100644 index 23c1253f6..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/ITupleCollection.java +++ /dev/null @@ -1,9 +0,0 @@ -package storm.trident.state; - -import java.util.Iterator; -import java.util.List; - -/* Container of a collection of tuples */ -public interface ITupleCollection { - public Iterator> getTuples(); -} diff --git a/jstorm-client/src/main/java/storm/trident/state/JSONNonTransactionalSerializer.java b/jstorm-client/src/main/java/storm/trident/state/JSONNonTransactionalSerializer.java deleted file mode 100644 index a289ddb57..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/JSONNonTransactionalSerializer.java +++ /dev/null @@ -1,28 +0,0 @@ -package storm.trident.state; - -import java.io.UnsupportedEncodingException; - -import backtype.storm.utils.Utils; - - -public class JSONNonTransactionalSerializer implements Serializer { - - @Override - public byte[] serialize(Object obj) { - try { - return Utils.to_json(obj).getBytes("UTF-8"); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException(e); - } - } - - @Override - public Object deserialize(byte[] b) { - try { - return Utils.from_json(new String(b, "UTF-8")); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException(e); - } - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/state/JSONOpaqueSerializer.java b/jstorm-client/src/main/java/storm/trident/state/JSONOpaqueSerializer.java deleted file mode 100644 index a96d4574d..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/JSONOpaqueSerializer.java +++ /dev/null @@ -1,36 +0,0 @@ -package storm.trident.state; - -import java.io.UnsupportedEncodingException; -import java.util.ArrayList; -import java.util.List; - -import backtype.storm.utils.Utils; - - -public class JSONOpaqueSerializer implements Serializer { - - @Override - public byte[] serialize(OpaqueValue obj) { - List toSer = new ArrayList(3); - toSer.add(obj.currTxid); - toSer.add(obj.curr); - toSer.add(obj.prev); - try { - return Utils.to_json(toSer).getBytes("UTF-8"); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException(e); - } - } - - @Override - public OpaqueValue deserialize(byte[] b) { - try { - String s = new String(b, "UTF-8"); - List deser = (List) Utils.from_json(s); - return new OpaqueValue((Long) deser.get(0), deser.get(1), deser.get(2)); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException(e); - } - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/state/JSONTransactionalSerializer.java b/jstorm-client/src/main/java/storm/trident/state/JSONTransactionalSerializer.java deleted file mode 100644 index a0df3af48..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/JSONTransactionalSerializer.java +++ /dev/null @@ -1,34 +0,0 @@ -package storm.trident.state; - -import java.io.UnsupportedEncodingException; -import java.util.ArrayList; -import java.util.List; - -import backtype.storm.utils.Utils; - - -public class JSONTransactionalSerializer implements Serializer { - @Override - public byte[] serialize(TransactionalValue obj) { - List toSer = new ArrayList(2); - toSer.add(obj.getTxid()); - toSer.add(obj.getVal()); - try { - return Utils.to_json(toSer).getBytes("UTF-8"); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException(e); - } - } - - @Override - public TransactionalValue deserialize(byte[] b) { - try { - String s = new String(b, "UTF-8"); - List deser = (List) Utils.from_json(s); - return new TransactionalValue((Long) deser.get(0), deser.get(1)); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException(e); - } - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/state/OpaqueValue.java b/jstorm-client/src/main/java/storm/trident/state/OpaqueValue.java deleted file mode 100644 index 14a39d458..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/OpaqueValue.java +++ /dev/null @@ -1,58 +0,0 @@ -package storm.trident.state; - -import org.apache.commons.lang.builder.ToStringBuilder; - -public class OpaqueValue { - Long currTxid; - T prev; - T curr; - - public OpaqueValue(Long currTxid, T val, T prev) { - this.curr = val; - this.currTxid = currTxid; - this.prev = prev; - } - - public OpaqueValue(Long currTxid, T val) { - this(currTxid, val, null); - } - - public OpaqueValue update(Long batchTxid, T newVal) { - T prev; - if(batchTxid==null || (this.currTxid < batchTxid)) { - prev = this.curr; - } else if(batchTxid.equals(this.currTxid)){ - prev = this.prev; - } else { - throw new RuntimeException("Current batch (" + batchTxid + ") is behind state's batch: " + this.toString()); - } - return new OpaqueValue(batchTxid, newVal, prev); - } - - public T get(Long batchTxid) { - if(batchTxid==null || (this.currTxid < batchTxid)) { - return curr; - } else if(batchTxid.equals(this.currTxid)){ - return prev; - } else { - throw new RuntimeException("Current batch (" + batchTxid + ") is behind state's batch: " + this.toString()); - } - } - - public T getCurr() { - return curr; - } - - public Long getCurrTxid() { - return currTxid; - } - - public T getPrev() { - return prev; - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this); - } -} diff --git a/jstorm-client/src/main/java/storm/trident/state/QueryFunction.java b/jstorm-client/src/main/java/storm/trident/state/QueryFunction.java deleted file mode 100644 index 38eb41ff9..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/QueryFunction.java +++ /dev/null @@ -1,11 +0,0 @@ -package storm.trident.state; - -import java.util.List; -import storm.trident.operation.EachOperation; -import storm.trident.operation.TridentCollector; -import storm.trident.tuple.TridentTuple; - -public interface QueryFunction extends EachOperation { - List batchRetrieve(S state, List args); - void execute(TridentTuple tuple, T result, TridentCollector collector); -} diff --git a/jstorm-client/src/main/java/storm/trident/state/ReadOnlyState.java b/jstorm-client/src/main/java/storm/trident/state/ReadOnlyState.java deleted file mode 100644 index f8c62f08b..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/ReadOnlyState.java +++ /dev/null @@ -1,14 +0,0 @@ -package storm.trident.state; - -public class ReadOnlyState implements State { - - @Override - public void beginCommit(Long txid) { - throw new UnsupportedOperationException("This state is read-only and does not support updates"); - } - - @Override - public void commit(Long txid) { - throw new UnsupportedOperationException("This state is read-only and does not support updates"); - } -} diff --git a/jstorm-client/src/main/java/storm/trident/state/ReducerValueUpdater.java b/jstorm-client/src/main/java/storm/trident/state/ReducerValueUpdater.java deleted file mode 100644 index 2ba3aecdc..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/ReducerValueUpdater.java +++ /dev/null @@ -1,24 +0,0 @@ -package storm.trident.state; - -import java.util.List; -import storm.trident.operation.ReducerAggregator; -import storm.trident.tuple.TridentTuple; - -public class ReducerValueUpdater implements ValueUpdater { - List tuples; - ReducerAggregator agg; - - public ReducerValueUpdater(ReducerAggregator agg, List tuples) { - this.agg = agg; - this.tuples = tuples; - } - - @Override - public Object update(Object stored) { - Object ret = (stored == null) ? this.agg.init() : stored; - for(TridentTuple t: tuples) { - ret = this.agg.reduce(ret, t); - } - return ret; - } -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/storm/trident/state/Serializer.java b/jstorm-client/src/main/java/storm/trident/state/Serializer.java deleted file mode 100644 index 9f91a3831..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/Serializer.java +++ /dev/null @@ -1,9 +0,0 @@ -package storm.trident.state; - -import java.io.Serializable; - - -public interface Serializer extends Serializable { - byte[] serialize(T obj); - T deserialize(byte[] b); -} diff --git a/jstorm-client/src/main/java/storm/trident/state/State.java b/jstorm-client/src/main/java/storm/trident/state/State.java deleted file mode 100644 index 93f7255d8..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/State.java +++ /dev/null @@ -1,22 +0,0 @@ -package storm.trident.state; - -/** - * There's 3 different kinds of state: - * - * 1. non-transactional: ignores commits, updates are permanent. no rollback. a cassandra incrementing state would be like this - * 2. repeat-transactional: idempotent as long as all batches for a txid are identical - * 3. opaque-transactional: the most general kind of state. updates are always done - * based on the previous version of the value if the current commit = latest stored commit - * Idempotent even if the batch for a txid can change. - * - * repeat transactional is idempotent for transactional spouts - * opaque transactional is idempotent for opaque or transactional spouts - * - * Trident should log warnings when state is idempotent but updates will not be idempotent - * because of spout - */ -// retrieving is encapsulated in Retrieval interface -public interface State { - void beginCommit(Long txid); // can be null for things like partitionPersist occuring off a DRPC stream - void commit(Long txid); -} diff --git a/jstorm-client/src/main/java/storm/trident/state/StateFactory.java b/jstorm-client/src/main/java/storm/trident/state/StateFactory.java deleted file mode 100644 index a77321b0a..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/StateFactory.java +++ /dev/null @@ -1,9 +0,0 @@ -package storm.trident.state; - -import backtype.storm.task.IMetricsContext; -import java.io.Serializable; -import java.util.Map; - -public interface StateFactory extends Serializable { - State makeState(Map conf, IMetricsContext metrics, int partitionIndex, int numPartitions); -} diff --git a/jstorm-client/src/main/java/storm/trident/state/StateSpec.java b/jstorm-client/src/main/java/storm/trident/state/StateSpec.java deleted file mode 100644 index 569311eb9..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/StateSpec.java +++ /dev/null @@ -1,13 +0,0 @@ -package storm.trident.state; - -import java.io.Serializable; - - -public class StateSpec implements Serializable { - public StateFactory stateFactory; - public Integer requiredNumPartitions = null; - - public StateSpec(StateFactory stateFactory) { - this.stateFactory = stateFactory; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/state/StateType.java b/jstorm-client/src/main/java/storm/trident/state/StateType.java deleted file mode 100644 index f77ec9ded..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/StateType.java +++ /dev/null @@ -1,8 +0,0 @@ -package storm.trident.state; - - -public enum StateType { - NON_TRANSACTIONAL, - TRANSACTIONAL, - OPAQUE -} diff --git a/jstorm-client/src/main/java/storm/trident/state/StateUpdater.java b/jstorm-client/src/main/java/storm/trident/state/StateUpdater.java deleted file mode 100644 index 7a1f19ca8..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/StateUpdater.java +++ /dev/null @@ -1,16 +0,0 @@ -package storm.trident.state; - -import java.util.List; -import storm.trident.operation.Operation; -import storm.trident.operation.TridentCollector; -import storm.trident.tuple.TridentTuple; - - -public interface StateUpdater extends Operation { - // maybe it needs a start phase (where it can do a retrieval, an update phase, and then a finish phase...? - // shouldn't really be a one-at-a-time interface, since we have all the tuples already? - // TOOD: used for the new values stream - // the list is needed to be able to get reduceragg and combineragg persistentaggregate - // for grouped streams working efficiently - void updateState(S state, List tuples, TridentCollector collector); -} diff --git a/jstorm-client/src/main/java/storm/trident/state/TransactionalValue.java b/jstorm-client/src/main/java/storm/trident/state/TransactionalValue.java deleted file mode 100644 index 933a0eaa0..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/TransactionalValue.java +++ /dev/null @@ -1,27 +0,0 @@ -package storm.trident.state; - -import org.apache.commons.lang.builder.ToStringBuilder; - - -public class TransactionalValue { - T val; - Long txid; - - public TransactionalValue(Long txid, T val) { - this.val = val; - this.txid = txid; - } - - public T getVal() { - return val; - } - - public Long getTxid() { - return txid; - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this); - } -} diff --git a/jstorm-client/src/main/java/storm/trident/state/ValueUpdater.java b/jstorm-client/src/main/java/storm/trident/state/ValueUpdater.java deleted file mode 100644 index 466a921de..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/ValueUpdater.java +++ /dev/null @@ -1,6 +0,0 @@ -package storm.trident.state; - - -public interface ValueUpdater { - T update(T stored); -} diff --git a/jstorm-client/src/main/java/storm/trident/state/map/CachedBatchReadsMap.java b/jstorm-client/src/main/java/storm/trident/state/map/CachedBatchReadsMap.java deleted file mode 100644 index 7d8c4423c..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/map/CachedBatchReadsMap.java +++ /dev/null @@ -1,63 +0,0 @@ -package storm.trident.state.map; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import storm.trident.state.ValueUpdater; - - -public class CachedBatchReadsMap { - public static class RetVal { - public boolean cached; - public T val; - - public RetVal(T v, boolean c) { - val = v; - cached = c; - } - } - - Map, T> _cached = new HashMap, T>(); - - public IBackingMap _delegate; - - public CachedBatchReadsMap(IBackingMap delegate) { - _delegate = delegate; - } - - public void reset() { - _cached.clear(); - } - - public List> multiGet(List> keys) { - // TODO: can optimize further by only querying backing map for keys not in the cache - List vals = _delegate.multiGet(keys); - List> ret = new ArrayList(vals.size()); - for(int i=0; i key = keys.get(i); - if(_cached.containsKey(key)) { - ret.add(new RetVal(_cached.get(key), true)); - } else { - ret.add(new RetVal(vals.get(i), false)); - } - } - return ret; - } - - public void multiPut(List> keys, List vals) { - _delegate.multiPut(keys, vals); - cache(keys, vals); - } - - private void cache(List> keys, List vals) { - for(int i=0; i key = keys.get(i); - T val = vals.get(i); - _cached.put(key, val); - } - } - - - -} diff --git a/jstorm-client/src/main/java/storm/trident/state/map/CachedMap.java b/jstorm-client/src/main/java/storm/trident/state/map/CachedMap.java deleted file mode 100644 index f7ad64601..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/map/CachedMap.java +++ /dev/null @@ -1,62 +0,0 @@ -package storm.trident.state.map; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import storm.trident.util.LRUMap; - -/** - * Useful to layer over a map that communicates with a database. you generally layer opaque map over this over your database store - * @author nathan - * @param - */ -public class CachedMap implements IBackingMap { - LRUMap, T> _cache; - IBackingMap _delegate; - - public CachedMap(IBackingMap delegate, int cacheSize) { - _cache = new LRUMap, T>(cacheSize); - _delegate = delegate; - } - - @Override - public List multiGet(List> keys) { - Map, T> results = new HashMap, T>(); - List> toGet = new ArrayList>(); - for(List key: keys) { - if(_cache.containsKey(key)) { - results.put(key, _cache.get(key)); - } else { - toGet.add(key); - } - } - - List fetchedVals = _delegate.multiGet(toGet); - for(int i=0; i key = toGet.get(i); - T val = fetchedVals.get(i); - _cache.put(key, val); - results.put(key, val); - } - - List ret = new ArrayList(keys.size()); - for(List key: keys) { - ret.add(results.get(key)); - } - return ret; - } - - @Override - public void multiPut(List> keys, List values) { - cache(keys, values); - _delegate.multiPut(keys, values); - } - - private void cache(List> keys, List values) { - for(int i=0; i { - List multiGet(List> keys); - void multiPut(List> keys, List vals); -} diff --git a/jstorm-client/src/main/java/storm/trident/state/map/MapReducerAggStateUpdater.java b/jstorm-client/src/main/java/storm/trident/state/map/MapReducerAggStateUpdater.java deleted file mode 100644 index f7c227bcd..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/map/MapReducerAggStateUpdater.java +++ /dev/null @@ -1,75 +0,0 @@ -package storm.trident.state.map; - -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Values; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import storm.trident.operation.ReducerAggregator; -import storm.trident.operation.TridentCollector; -import storm.trident.operation.TridentOperationContext; -import storm.trident.state.ReducerValueUpdater; -import storm.trident.state.StateUpdater; -import storm.trident.state.ValueUpdater; -import storm.trident.tuple.ComboList; -import storm.trident.tuple.TridentTuple; -import storm.trident.tuple.TridentTupleView.ProjectionFactory; - -public class MapReducerAggStateUpdater implements StateUpdater { - ReducerAggregator _agg; - Fields _groupFields; - Fields _inputFields; - ProjectionFactory _groupFactory; - ProjectionFactory _inputFactory; - ComboList.Factory _factory; - - - public MapReducerAggStateUpdater(ReducerAggregator agg, Fields groupFields, Fields inputFields) { - _agg = agg; - _groupFields = groupFields; - _inputFields = inputFields; - _factory = new ComboList.Factory(groupFields.size(), 1); - } - - - @Override - public void updateState(MapState map, List tuples, TridentCollector collector) { - Map, List> grouped = new HashMap(); - - //List> groups = new ArrayList>(tuples.size()); - //List values = new ArrayList(tuples.size()); - for(TridentTuple t: tuples) { - List group = _groupFactory.create(t); - List groupTuples = grouped.get(group); - if(groupTuples==null) { - groupTuples = new ArrayList(); - grouped.put(group, groupTuples); - } - groupTuples.add(_inputFactory.create(t)); - } - List> uniqueGroups = new ArrayList(grouped.keySet()); - List updaters = new ArrayList(uniqueGroups.size()); - for(List group: uniqueGroups) { - updaters.add(new ReducerValueUpdater(_agg, grouped.get(group))); - } - List results = map.multiUpdate(uniqueGroups, updaters); - - for(int i=0; i group = uniqueGroups.get(i); - Object result = results.get(i); - collector.emit(_factory.create(new List[] {group, new Values(result) })); - } - } - - @Override - public void prepare(Map conf, TridentOperationContext context) { - _groupFactory = context.makeProjectionFactory(_groupFields); - _inputFactory = context.makeProjectionFactory(_inputFields); - } - - @Override - public void cleanup() { - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/state/map/MapState.java b/jstorm-client/src/main/java/storm/trident/state/map/MapState.java deleted file mode 100644 index 78901d94d..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/map/MapState.java +++ /dev/null @@ -1,9 +0,0 @@ -package storm.trident.state.map; - -import java.util.List; -import storm.trident.state.ValueUpdater; - -public interface MapState extends ReadOnlyMapState { - List multiUpdate(List> keys, List updaters); - void multiPut(List> keys, List vals); -} diff --git a/jstorm-client/src/main/java/storm/trident/state/map/MicroBatchIBackingMap.java b/jstorm-client/src/main/java/storm/trident/state/map/MicroBatchIBackingMap.java deleted file mode 100644 index 2f356b135..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/map/MicroBatchIBackingMap.java +++ /dev/null @@ -1,68 +0,0 @@ -package storm.trident.state.map; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.LinkedList; -import java.util.List; - -public class MicroBatchIBackingMap implements IBackingMap { - IBackingMap _delegate; - Options _options; - - - public static class Options implements Serializable { - public int maxMultiGetBatchSize = 0; // 0 means delegate batch size = trident batch size. - public int maxMultiPutBatchSize = 0; - } - - public MicroBatchIBackingMap(final Options options, final IBackingMap delegate) { - _options = options; - _delegate = delegate; - assert options.maxMultiPutBatchSize >= 0; - assert options.maxMultiGetBatchSize >= 0; - } - - @Override - public void multiPut(final List> keys, final List values) { - int thisBatchSize; - if(_options.maxMultiPutBatchSize == 0) { thisBatchSize = keys.size(); } - else { thisBatchSize = _options.maxMultiPutBatchSize; } - - LinkedList> keysTodo = new LinkedList>(keys); - LinkedList valuesTodo = new LinkedList(values); - - while(!keysTodo.isEmpty()) { - List> keysBatch = new ArrayList>(thisBatchSize); - List valuesBatch = new ArrayList(thisBatchSize); - for(int i=0; i multiGet(final List> keys) { - int thisBatchSize; - if(_options.maxMultiGetBatchSize == 0) { thisBatchSize = keys.size(); } - else { thisBatchSize = _options.maxMultiGetBatchSize; } - - LinkedList> keysTodo = new LinkedList>(keys); - - List ret = new ArrayList(keys.size()); - - while(!keysTodo.isEmpty()) { - List> keysBatch = new ArrayList>(thisBatchSize); - for(int i=0; i retSubset = _delegate.multiGet(keysBatch); - ret.addAll(retSubset); - } - - return ret; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/state/map/NonTransactionalMap.java b/jstorm-client/src/main/java/storm/trident/state/map/NonTransactionalMap.java deleted file mode 100644 index 3a140b5d8..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/map/NonTransactionalMap.java +++ /dev/null @@ -1,50 +0,0 @@ -package storm.trident.state.map; - -import storm.trident.state.ValueUpdater; - -import java.util.ArrayList; -import java.util.List; - - -public class NonTransactionalMap implements MapState { - public static MapState build(IBackingMap backing) { - return new NonTransactionalMap(backing); - } - - IBackingMap _backing; - - protected NonTransactionalMap(IBackingMap backing) { - _backing = backing; - } - - @Override - public List multiGet(List> keys) { - return _backing.multiGet(keys); - } - - @Override - public List multiUpdate(List> keys, List updaters) { - List curr = _backing.multiGet(keys); - List ret = new ArrayList(curr.size()); - for(int i=0; i updater = updaters.get(i); - ret.add(updater.update(currVal)); - } - _backing.multiPut(keys, ret); - return ret; - } - - @Override - public void multiPut(List> keys, List vals) { - _backing.multiPut(keys, vals); - } - - @Override - public void beginCommit(Long txid) { - } - - @Override - public void commit(Long txid) { - } -} diff --git a/jstorm-client/src/main/java/storm/trident/state/map/ReadOnlyMapState.java b/jstorm-client/src/main/java/storm/trident/state/map/ReadOnlyMapState.java deleted file mode 100644 index 5a519c4ef..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/map/ReadOnlyMapState.java +++ /dev/null @@ -1,9 +0,0 @@ -package storm.trident.state.map; - -import java.util.List; -import storm.trident.state.State; - -public interface ReadOnlyMapState extends State { - // certain states might only accept one-tuple keys - those should just throw an error - List multiGet(List> keys); -} diff --git a/jstorm-client/src/main/java/storm/trident/state/map/RemovableMapState.java b/jstorm-client/src/main/java/storm/trident/state/map/RemovableMapState.java deleted file mode 100644 index cf34f05e8..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/map/RemovableMapState.java +++ /dev/null @@ -1,8 +0,0 @@ -package storm.trident.state.map; - -import java.util.List; -import storm.trident.state.State; - -public interface RemovableMapState extends State { - void multiRemove(List> keys); -} diff --git a/jstorm-client/src/main/java/storm/trident/state/map/SnapshottableMap.java b/jstorm-client/src/main/java/storm/trident/state/map/SnapshottableMap.java deleted file mode 100644 index f42a5c983..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/map/SnapshottableMap.java +++ /dev/null @@ -1,59 +0,0 @@ -package storm.trident.state.map; - -import java.util.Arrays; -import java.util.List; -import storm.trident.state.ValueUpdater; -import storm.trident.state.snapshot.Snapshottable; - - -public class SnapshottableMap implements MapState, Snapshottable { - MapState _delegate; - List> _keys; - - public SnapshottableMap(MapState delegate, List snapshotKey) { - _delegate = delegate; - _keys = Arrays.asList(snapshotKey); - } - - @Override - public List multiGet(List> keys) { - return _delegate.multiGet(keys); - } - - @Override - public List multiUpdate(List> keys, List updaters) { - return _delegate.multiUpdate(keys, updaters); - } - - @Override - public void multiPut(List> keys, List vals) { - _delegate.multiPut(keys, vals); - } - - @Override - public void beginCommit(Long txid) { - _delegate.beginCommit(txid); - } - - @Override - public void commit(Long txid) { - _delegate.commit(txid); - } - - @Override - public T get() { - return multiGet(_keys).get(0); - } - - @Override - public T update(ValueUpdater updater) { - List updaters = Arrays.asList(updater); - return multiUpdate(_keys, updaters).get(0); - } - - @Override - public void set(T o) { - multiPut(_keys, Arrays.asList(o)); - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/state/snapshot/ReadOnlySnapshottable.java b/jstorm-client/src/main/java/storm/trident/state/snapshot/ReadOnlySnapshottable.java deleted file mode 100644 index 2064a9868..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/snapshot/ReadOnlySnapshottable.java +++ /dev/null @@ -1,7 +0,0 @@ -package storm.trident.state.snapshot; - -import storm.trident.state.State; - -public interface ReadOnlySnapshottable extends State { - T get(); -} diff --git a/jstorm-client/src/main/java/storm/trident/state/snapshot/Snapshottable.java b/jstorm-client/src/main/java/storm/trident/state/snapshot/Snapshottable.java deleted file mode 100644 index f2164858f..000000000 --- a/jstorm-client/src/main/java/storm/trident/state/snapshot/Snapshottable.java +++ /dev/null @@ -1,10 +0,0 @@ -package storm.trident.state.snapshot; - -import storm.trident.state.ValueUpdater; - - -// used by Stream#persistentAggregate -public interface Snapshottable extends ReadOnlySnapshottable { - T update(ValueUpdater updater); - void set(T o); -} diff --git a/jstorm-client/src/main/java/storm/trident/testing/CountAsAggregator.java b/jstorm-client/src/main/java/storm/trident/testing/CountAsAggregator.java deleted file mode 100644 index 52f482f0c..000000000 --- a/jstorm-client/src/main/java/storm/trident/testing/CountAsAggregator.java +++ /dev/null @@ -1,30 +0,0 @@ -package storm.trident.testing; - -import backtype.storm.tuple.Values; -import storm.trident.operation.BaseAggregator; -import storm.trident.operation.TridentCollector; -import storm.trident.tuple.TridentTuple; - - -public class CountAsAggregator extends BaseAggregator { - - static class State { - long count = 0; - } - - @Override - public State init(Object batchId, TridentCollector collector) { - return new State(); - } - - @Override - public void aggregate(State state, TridentTuple tuple, TridentCollector collector) { - state.count++; - } - - @Override - public void complete(State state, TridentCollector collector) { - collector.emit(new Values(state.count)); - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/testing/FeederCommitterBatchSpout.java b/jstorm-client/src/main/java/storm/trident/testing/FeederCommitterBatchSpout.java deleted file mode 100644 index d105c0cdb..000000000 --- a/jstorm-client/src/main/java/storm/trident/testing/FeederCommitterBatchSpout.java +++ /dev/null @@ -1,79 +0,0 @@ -package storm.trident.testing; - -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Fields; -import java.util.List; -import java.util.Map; -import storm.trident.operation.TridentCollector; -import storm.trident.spout.ICommitterTridentSpout; -import storm.trident.spout.ITridentSpout; -import storm.trident.topology.TransactionAttempt; - - -public class FeederCommitterBatchSpout implements ICommitterTridentSpout, IFeeder { - - FeederBatchSpout _spout; - - public FeederCommitterBatchSpout(List fields) { - _spout = new FeederBatchSpout(fields); - } - - public void setWaitToEmit(boolean trueIfWait) { - _spout.setWaitToEmit(trueIfWait); - } - - static class CommitterEmitter implements ICommitterTridentSpout.Emitter { - ITridentSpout.Emitter _emitter; - - - public CommitterEmitter(ITridentSpout.Emitter e) { - _emitter = e; - } - - @Override - public void commit(TransactionAttempt attempt) { - } - - @Override - public void emitBatch(TransactionAttempt tx, Object coordinatorMeta, TridentCollector collector) { - _emitter.emitBatch(tx, coordinatorMeta, collector); - } - - @Override - public void success(TransactionAttempt tx) { - _emitter.success(tx); - } - - @Override - public void close() { - _emitter.close(); - } - - } - - @Override - public Emitter getEmitter(String txStateId, Map conf, TopologyContext context) { - return new CommitterEmitter(_spout.getEmitter(txStateId, conf, context)); - } - - @Override - public BatchCoordinator getCoordinator(String txStateId, Map conf, TopologyContext context) { - return _spout.getCoordinator(txStateId, conf, context); - } - - @Override - public Fields getOutputFields() { - return _spout.getOutputFields(); - } - - @Override - public Map getComponentConfiguration() { - return _spout.getComponentConfiguration(); - } - - @Override - public void feed(Object tuples) { - _spout.feed(tuples); - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/testing/FixedBatchSpout.java b/jstorm-client/src/main/java/storm/trident/testing/FixedBatchSpout.java deleted file mode 100644 index 6e32c1a5d..000000000 --- a/jstorm-client/src/main/java/storm/trident/testing/FixedBatchSpout.java +++ /dev/null @@ -1,80 +0,0 @@ -package storm.trident.testing; - -import backtype.storm.Config; -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Fields; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.HashMap; - -import storm.trident.operation.TridentCollector; -import storm.trident.spout.IBatchSpout; - - -public class FixedBatchSpout implements IBatchSpout { - - Fields fields; - List[] outputs; - int maxBatchSize; - HashMap>> batches = new HashMap>>(); - - public FixedBatchSpout(Fields fields, int maxBatchSize, List... outputs) { - this.fields = fields; - this.outputs = outputs; - this.maxBatchSize = maxBatchSize; - } - - int index = 0; - boolean cycle = false; - - public void setCycle(boolean cycle) { - this.cycle = cycle; - } - - @Override - public void open(Map conf, TopologyContext context) { - index = 0; - } - - @Override - public void emitBatch(long batchId, TridentCollector collector) { - List> batch = this.batches.get(batchId); - if(batch == null){ - batch = new ArrayList>(); - if(index>=outputs.length && cycle) { - index = 0; - } - for(int i=0; index < outputs.length && i < maxBatchSize; index++, i++) { - batch.add(outputs[index]); - } - this.batches.put(batchId, batch); - } - for(List list : batch){ - collector.emit(list); - } - } - - @Override - public void ack(long batchId) { - this.batches.remove(batchId); - } - - @Override - public void close() { - } - - @Override - public Map getComponentConfiguration() { - Config conf = new Config(); - conf.setMaxTaskParallelism(1); - return conf; - } - - @Override - public Fields getOutputFields() { - return fields; - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/testing/IFeeder.java b/jstorm-client/src/main/java/storm/trident/testing/IFeeder.java deleted file mode 100644 index eaf02bb51..000000000 --- a/jstorm-client/src/main/java/storm/trident/testing/IFeeder.java +++ /dev/null @@ -1,6 +0,0 @@ -package storm.trident.testing; - - -public interface IFeeder { - void feed(Object tuples); -} diff --git a/jstorm-client/src/main/java/storm/trident/testing/MemoryBackingMap.java b/jstorm-client/src/main/java/storm/trident/testing/MemoryBackingMap.java deleted file mode 100644 index e222ba6c2..000000000 --- a/jstorm-client/src/main/java/storm/trident/testing/MemoryBackingMap.java +++ /dev/null @@ -1,30 +0,0 @@ -package storm.trident.testing; - -import storm.trident.state.map.IBackingMap; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class MemoryBackingMap implements IBackingMap { - Map _vals = new HashMap(); - - @Override - public List multiGet(List> keys) { - List ret = new ArrayList(); - for(List key: keys) { - ret.add(_vals.get(key)); - } - return ret; - } - - @Override - public void multiPut(List> keys, List vals) { - for(int i=0; i 0) { - collector.emit(new Values(word)); - } - } - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/testing/StringLength.java b/jstorm-client/src/main/java/storm/trident/testing/StringLength.java deleted file mode 100644 index f99a5c73a..000000000 --- a/jstorm-client/src/main/java/storm/trident/testing/StringLength.java +++ /dev/null @@ -1,15 +0,0 @@ -package storm.trident.testing; - -import backtype.storm.tuple.Values; -import storm.trident.operation.BaseFunction; -import storm.trident.operation.TridentCollector; -import storm.trident.tuple.TridentTuple; - -public class StringLength extends BaseFunction { - - @Override - public void execute(TridentTuple tuple, TridentCollector collector) { - collector.emit(new Values(tuple.getString(0).length())); - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/testing/TrueFilter.java b/jstorm-client/src/main/java/storm/trident/testing/TrueFilter.java deleted file mode 100644 index 6912063bb..000000000 --- a/jstorm-client/src/main/java/storm/trident/testing/TrueFilter.java +++ /dev/null @@ -1,13 +0,0 @@ -package storm.trident.testing; - -import storm.trident.operation.BaseFilter; -import storm.trident.tuple.TridentTuple; - -public class TrueFilter extends BaseFilter { - - @Override - public boolean isKeep(TridentTuple tuple) { - return true; - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/testing/TuplifyArgs.java b/jstorm-client/src/main/java/storm/trident/testing/TuplifyArgs.java deleted file mode 100644 index 764e51ea6..000000000 --- a/jstorm-client/src/main/java/storm/trident/testing/TuplifyArgs.java +++ /dev/null @@ -1,21 +0,0 @@ -package storm.trident.testing; - -import java.util.List; - -import storm.trident.operation.BaseFunction; -import storm.trident.operation.TridentCollector; -import storm.trident.tuple.TridentTuple; -import backtype.storm.utils.Utils; - -public class TuplifyArgs extends BaseFunction { - - @Override - public void execute(TridentTuple input, TridentCollector collector) { - String args = input.getString(0); - List> tuples = (List) Utils.from_json(args); - for(List tuple: tuples) { - collector.emit(tuple); - } - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/topology/BatchInfo.java b/jstorm-client/src/main/java/storm/trident/topology/BatchInfo.java deleted file mode 100644 index a3e3076aa..000000000 --- a/jstorm-client/src/main/java/storm/trident/topology/BatchInfo.java +++ /dev/null @@ -1,16 +0,0 @@ -package storm.trident.topology; - -import storm.trident.spout.IBatchID; - - -public class BatchInfo { - public IBatchID batchId; - public Object state; - public String batchGroup; - - public BatchInfo(String batchGroup, IBatchID batchId, Object state) { - this.batchGroup = batchGroup; - this.batchId = batchId; - this.state = state; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/topology/ITridentBatchBolt.java b/jstorm-client/src/main/java/storm/trident/topology/ITridentBatchBolt.java deleted file mode 100644 index b6f60ce30..000000000 --- a/jstorm-client/src/main/java/storm/trident/topology/ITridentBatchBolt.java +++ /dev/null @@ -1,15 +0,0 @@ -package storm.trident.topology; - -import backtype.storm.coordination.BatchOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IComponent; -import backtype.storm.tuple.Tuple; -import java.util.Map; - -public interface ITridentBatchBolt extends IComponent { - void prepare(Map conf, TopologyContext context, BatchOutputCollector collector); - void execute(BatchInfo batchInfo, Tuple tuple); - void finishBatch(BatchInfo batchInfo); - Object initBatchState(String batchGroup, Object batchId); - void cleanup(); -} diff --git a/jstorm-client/src/main/java/storm/trident/topology/MasterBatchCoordinator.java b/jstorm-client/src/main/java/storm/trident/topology/MasterBatchCoordinator.java deleted file mode 100644 index 201696e00..000000000 --- a/jstorm-client/src/main/java/storm/trident/topology/MasterBatchCoordinator.java +++ /dev/null @@ -1,317 +0,0 @@ -package storm.trident.topology; - -import backtype.storm.Config; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseRichSpout; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Values; -import backtype.storm.utils.WindowedTimeThrottler; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.TreeMap; -import java.util.Random; -import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.log4j.Logger; -import storm.trident.spout.ITridentSpout; -import storm.trident.spout.ICommitterTridentSpout; -import storm.trident.topology.state.TransactionalState; - -public class MasterBatchCoordinator extends BaseRichSpout { - public static final Logger LOG = Logger.getLogger(MasterBatchCoordinator.class); - - public static final long INIT_TXID = 1L; - - - public static final String BATCH_STREAM_ID = "$batch"; - public static final String COMMIT_STREAM_ID = "$commit"; - public static final String SUCCESS_STREAM_ID = "$success"; - - private static final String CURRENT_TX = "currtx"; - private static final String CURRENT_ATTEMPTS = "currattempts"; - - private static enum Operation { - ACK, - FAIL, - NEXTTUPLE - } - - private List _states = new ArrayList(); - - TreeMap _activeTx = new TreeMap(); - TreeMap _attemptIds; - - private SpoutOutputCollector _collector; - Long _currTransaction; - int _maxTransactionActive; - - List _coordinators = new ArrayList(); - - - List _managedSpoutIds; - List _spouts; - WindowedTimeThrottler _throttler; - - boolean _active = true; - - AtomicBoolean failedOccur = new AtomicBoolean(false); - - public MasterBatchCoordinator(List spoutIds, List spouts) { - if(spoutIds.isEmpty()) { - throw new IllegalArgumentException("Must manage at least one spout"); - } - _managedSpoutIds = spoutIds; - _spouts = spouts; - } - - public List getManagedSpoutIds(){ - return _managedSpoutIds; - } - - @Override - public void activate() { - _active = true; - } - - @Override - public void deactivate() { - _active = false; - } - - @Override - public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { - _throttler = new WindowedTimeThrottler((Number)conf.get(Config.TOPOLOGY_TRIDENT_BATCH_EMIT_INTERVAL_MILLIS), 1); - for(String spoutId: _managedSpoutIds) { - _states.add(TransactionalState.newCoordinatorState(conf, spoutId)); - } - _currTransaction = getStoredCurrTransaction(); - - _collector = collector; - Number active = (Number) conf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING); - if(active==null) { - _maxTransactionActive = 1; - } else { - _maxTransactionActive = active.intValue(); - } - _attemptIds = getStoredCurrAttempts(_currTransaction, _maxTransactionActive); - - - for(int i=0; i<_spouts.size(); i++) { - String txId = _managedSpoutIds.get(i); - _coordinators.add(_spouts.get(i).getCoordinator(txId, conf, context)); - } - } - - @Override - public void close() { - for(TransactionalState state: _states) { - state.close(); - } - } - - @Override - public void nextTuple() { - sync(Operation.NEXTTUPLE, null); - } - - @Override - public void ack(Object msgId) { - sync(Operation.ACK, (TransactionAttempt) msgId); - } - - @Override - public void fail(Object msgId) { - sync(Operation.FAIL, (TransactionAttempt) msgId); - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - // in partitioned example, in case an emitter task receives a later transaction than it's emitted so far, - // when it sees the earlier txid it should know to emit nothing - declarer.declareStream(BATCH_STREAM_ID, new Fields("tx")); - declarer.declareStream(COMMIT_STREAM_ID, new Fields("tx")); - declarer.declareStream(SUCCESS_STREAM_ID, new Fields("tx")); - } - - synchronized private void sync(Operation op, TransactionAttempt attempt) { - TransactionStatus status; - long txid; - - switch (op) { - case FAIL: - // Remove the failed one and the items whose id is higher than the failed one. - // Then those ones will be retried when nextTuple. - txid = attempt.getTransactionId(); - status = _activeTx.remove(txid); - if(status!=null && status.attempt.equals(attempt)) { - _activeTx.tailMap(txid).clear(); - } - break; - - case ACK: - txid = attempt.getTransactionId(); - status = _activeTx.get(txid); - if(status!=null && attempt.equals(status.attempt)) { - if(status.status==AttemptStatus.PROCESSING ) { - status.status = AttemptStatus.PROCESSED; - } else if(status.status==AttemptStatus.COMMITTING) { - status.status = AttemptStatus.COMMITTED; - } - } - break; - - case NEXTTUPLE: - // note that sometimes the tuples active may be less than max_spout_pending, e.g. - // max_spout_pending = 3 - // tx 1, 2, 3 active, tx 2 is acked. there won't be a commit for tx 2 (because tx 1 isn't committed yet), - // and there won't be a batch for tx 4 because there's max_spout_pending tx active - status = _activeTx.get(_currTransaction); - if (status!=null) { - if(status.status == AttemptStatus.PROCESSED) { - status.status = AttemptStatus.COMMITTING; - _collector.emit(COMMIT_STREAM_ID, new Values(status.attempt), status.attempt); - } else if (status.status == AttemptStatus.COMMITTED) { - _activeTx.remove(status.attempt.getTransactionId()); - _attemptIds.remove(status.attempt.getTransactionId()); - _collector.emit(SUCCESS_STREAM_ID, new Values(status.attempt)); - _currTransaction = nextTransactionId(status.attempt.getTransactionId()); - for(TransactionalState state: _states) { - state.setData(CURRENT_TX, _currTransaction); - } - } - } - - if(_active) { - if(_activeTx.size() < _maxTransactionActive) { - Long curr = _currTransaction; - for(int i=0; i<_maxTransactionActive; i++) { - if(batchDelay()) { - break; - } - - if(isReady(curr)) { - if(!_activeTx.containsKey(curr)) { - // by using a monotonically increasing attempt id, downstream tasks - // can be memory efficient by clearing out state for old attempts - // as soon as they see a higher attempt id for a transaction - Integer attemptId = _attemptIds.get(curr); - if(attemptId==null) { - attemptId = 0; - } else { - attemptId++; - } - _attemptIds.put(curr, attemptId); - for(TransactionalState state: _states) { - state.setData(CURRENT_ATTEMPTS, _attemptIds); - } - - TransactionAttempt currAttempt = new TransactionAttempt(curr, attemptId); - _activeTx.put(curr, new TransactionStatus(currAttempt)); - _collector.emit(BATCH_STREAM_ID, new Values(currAttempt), currAttempt); - _throttler.markEvent(); - break; - } - } - curr = nextTransactionId(curr); - } - } else { - // Do nothing - } - } - break; - - default: - LOG.warn("Unknow Operation code=" + op); - break; - } - } - - private boolean isReady(long txid) { - //TODO: make this strategy configurable?... right now it goes if anyone is ready - for(ITridentSpout.BatchCoordinator coord: _coordinators) { - if(coord.isReady(txid)) return true; - } - return false; - } - - private boolean batchDelay() { - return _throttler.isThrottled(); - } - - @Override - public Map getComponentConfiguration() { - Config ret = new Config(); - ret.setMaxTaskParallelism(1); - ret.registerSerialization(TransactionAttempt.class); - return ret; - } - - private static enum AttemptStatus { - PROCESSING, - PROCESSED, - COMMITTING, - COMMITTED - } - - private static class TransactionStatus { - TransactionAttempt attempt; - AttemptStatus status; - - public TransactionStatus(TransactionAttempt attempt) { - this.attempt = attempt; - this.status = AttemptStatus.PROCESSING; - } - - @Override - public String toString() { - return attempt.toString() + " <" + status.toString() + ">"; - } - } - - - private Long nextTransactionId(Long id) { - return id + 1; - } - - private Long getStoredCurrTransaction() { - Long ret = INIT_TXID; - for(TransactionalState state: _states) { - Long curr = (Long) state.getData(CURRENT_TX); - if(curr!=null && curr.compareTo(ret) > 0) { - ret = curr; - } - } - return ret; - } - - private TreeMap getStoredCurrAttempts(long currTransaction, int maxBatches) { - TreeMap ret = new TreeMap(); - for(TransactionalState state: _states) { - Map attempts = (Map) state.getData(CURRENT_ATTEMPTS); - if(attempts==null) attempts = new HashMap(); - for(Entry e: attempts.entrySet()) { - // this is because json doesn't allow numbers as keys... - // TODO: replace json with a better form of encoding - Number txidObj; - if(e.getKey() instanceof String) { - txidObj = Long.parseLong((String) e.getKey()); - } else { - txidObj = (Number) e.getKey(); - } - long txid = ((Number) txidObj).longValue(); - int attemptId = ((Number) e.getValue()).intValue(); - Integer curr = ret.get(txid); - if(curr==null || attemptId > curr) { - ret.put(txid, attemptId); - } - } - } - ret.headMap(currTransaction).clear(); - ret.tailMap(currTransaction + maxBatches - 1).clear(); - return ret; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/topology/TransactionAttempt.java b/jstorm-client/src/main/java/storm/trident/topology/TransactionAttempt.java deleted file mode 100644 index b2ea32861..000000000 --- a/jstorm-client/src/main/java/storm/trident/topology/TransactionAttempt.java +++ /dev/null @@ -1,49 +0,0 @@ -package storm.trident.topology; - -import storm.trident.spout.IBatchID; - - -public class TransactionAttempt implements IBatchID { - Long _txid; - int _attemptId; - - - // for kryo compatibility - public TransactionAttempt() { - - } - - public TransactionAttempt(Long txid, int attemptId) { - _txid = txid; - _attemptId = attemptId; - } - - public Long getTransactionId() { - return _txid; - } - - public Object getId() { - return _txid; - } - - public int getAttemptId() { - return _attemptId; - } - - @Override - public int hashCode() { - return _txid.hashCode(); - } - - @Override - public boolean equals(Object o) { - if(!(o instanceof TransactionAttempt)) return false; - TransactionAttempt other = (TransactionAttempt) o; - return _txid.equals(other._txid) && _attemptId == other._attemptId; - } - - @Override - public String toString() { - return "" + _txid + ":" + _attemptId; - } -} \ No newline at end of file diff --git a/jstorm-client/src/main/java/storm/trident/topology/state/RotatingTransactionalState.java b/jstorm-client/src/main/java/storm/trident/topology/state/RotatingTransactionalState.java deleted file mode 100644 index 9f22cc763..000000000 --- a/jstorm-client/src/main/java/storm/trident/topology/state/RotatingTransactionalState.java +++ /dev/null @@ -1,130 +0,0 @@ -package storm.trident.topology.state; - -import backtype.storm.utils.Utils; -import org.apache.zookeeper.KeeperException; - -import java.util.HashSet; -import java.util.List; -import java.util.SortedMap; -import java.util.TreeMap; - -public class RotatingTransactionalState { - public static interface StateInitializer { - Object init(long txid, Object lastState); - } - - private TransactionalState _state; - private String _subdir; - - private TreeMap _curr = new TreeMap(); - - public RotatingTransactionalState(TransactionalState state, String subdir) { - _state = state; - _subdir = subdir; - state.mkdir(subdir); - sync(); - } - - - public Object getLastState() { - if(_curr.isEmpty()) return null; - else return _curr.lastEntry().getValue(); - } - - public void overrideState(long txid, Object state) { - _state.setData(txPath(txid), state); - _curr.put(txid, state); - } - - public void removeState(long txid) { - if(_curr.containsKey(txid)) { - _curr.remove(txid); - _state.delete(txPath(txid)); - } - } - - public Object getState(long txid) { - return _curr.get(txid); - } - - public Object getState(long txid, StateInitializer init) { - if(!_curr.containsKey(txid)) { - SortedMap prevMap = _curr.headMap(txid); - SortedMap afterMap = _curr.tailMap(txid); - - Long prev = null; - if(!prevMap.isEmpty()) prev = prevMap.lastKey(); - - Object data; - if(afterMap.isEmpty()) { - Object prevData; - if(prev!=null) { - prevData = _curr.get(prev); - } else { - prevData = null; - } - data = init.init(txid, prevData); - } else { - data = null; - } - _curr.put(txid, data); - _state.setData(txPath(txid), data); - } - return _curr.get(txid); - } - - public Object getPreviousState(long txid) { - SortedMap prevMap = _curr.headMap(txid); - if(prevMap.isEmpty()) return null; - else return prevMap.get(prevMap.lastKey()); - } - - public boolean hasCache(long txid) { - return _curr.containsKey(txid); - } - - /** - * Returns null if it was created, the value otherwise. - */ - public Object getStateOrCreate(long txid, StateInitializer init) { - if(_curr.containsKey(txid)) { - return _curr.get(txid); - } else { - getState(txid, init); - return null; - } - } - - public void cleanupBefore(long txid) { - SortedMap toDelete = _curr.headMap(txid); - for(long tx: new HashSet(toDelete.keySet())) { - _curr.remove(tx); - try { - _state.delete(txPath(tx)); - } catch(RuntimeException e) { - // Ignore NoNodeExists exceptions because when sync() it may populate _curr with stale data since - // zookeeper reads are eventually consistent. - if(!Utils.exceptionCauseIsInstanceOf(KeeperException.NoNodeException.class, e)) { - throw e; - } - } - } - } - - private void sync() { - List txids = _state.list(_subdir); - for(String txid_s: txids) { - Object data = _state.getData(txPath(txid_s)); - _curr.put(Long.parseLong(txid_s), data); - } - } - - private String txPath(long tx) { - return txPath("" + tx); - } - - private String txPath(String tx) { - return _subdir + "/" + tx; - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/topology/state/TransactionalState.java b/jstorm-client/src/main/java/storm/trident/topology/state/TransactionalState.java deleted file mode 100644 index 44d42829c..000000000 --- a/jstorm-client/src/main/java/storm/trident/topology/state/TransactionalState.java +++ /dev/null @@ -1,119 +0,0 @@ -package storm.trident.topology.state; - - -import java.io.UnsupportedEncodingException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.curator.framework.CuratorFramework; -import org.apache.zookeeper.CreateMode; -import org.apache.zookeeper.KeeperException; - -import backtype.storm.Config; -import backtype.storm.utils.Utils; - -public class TransactionalState { - CuratorFramework _curator; - - public static TransactionalState newUserState(Map conf, String id) { - return new TransactionalState(conf, id, "user"); - } - - public static TransactionalState newCoordinatorState(Map conf, String id) { - return new TransactionalState(conf, id, "coordinator"); - } - - protected TransactionalState(Map conf, String id, String subroot) { - try { - conf = new HashMap(conf); - String rootDir = conf.get(Config.TRANSACTIONAL_ZOOKEEPER_ROOT) + "/" + id + "/" + subroot; - List servers = (List) getWithBackup(conf, Config.TRANSACTIONAL_ZOOKEEPER_SERVERS, Config.STORM_ZOOKEEPER_SERVERS); - Object port = getWithBackup(conf, Config.TRANSACTIONAL_ZOOKEEPER_PORT, Config.STORM_ZOOKEEPER_PORT); - CuratorFramework initter = Utils.newCuratorStarted(conf, servers, port); - try { - initter.create().creatingParentsIfNeeded().forPath(rootDir); - } catch(KeeperException.NodeExistsException e) { - - } - - initter.close(); - - _curator = Utils.newCuratorStarted(conf, servers, port, rootDir); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - public void setData(String path, Object obj) { - path = "/" + path; - byte[] ser; - try { - ser = Utils.to_json(obj).getBytes("UTF-8"); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException(e); - } - try { - if(_curator.checkExists().forPath(path)!=null) { - _curator.setData().forPath(path, ser); - } else { - _curator.create() - .creatingParentsIfNeeded() - .withMode(CreateMode.PERSISTENT) - .forPath(path, ser); - } - } catch(Exception e) { - throw new RuntimeException(e); - } - } - - public void delete(String path) { - path = "/" + path; - try { - _curator.delete().forPath(path); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - public List list(String path) { - path = "/" + path; - try { - if(_curator.checkExists().forPath(path)==null) { - return new ArrayList(); - } else { - return _curator.getChildren().forPath(path); - } - } catch(Exception e) { - throw new RuntimeException(e); - } - } - - public void mkdir(String path) { - setData(path, 7); - } - - public Object getData(String path) { - path = "/" + path; - try { - if(_curator.checkExists().forPath(path)!=null) { - return Utils.from_json(new String(_curator.getData().forPath(path), "UTF-8")); - } else { - return null; - } - } catch(Exception e) { - throw new RuntimeException(e); - } - } - - public void close() { - _curator.close(); - } - - private Object getWithBackup(Map amap, Object primary, Object backup) { - Object ret = amap.get(primary); - if(ret==null) return amap.get(backup); - return ret; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/tuple/ComboList.java b/jstorm-client/src/main/java/storm/trident/tuple/ComboList.java deleted file mode 100644 index 022157965..000000000 --- a/jstorm-client/src/main/java/storm/trident/tuple/ComboList.java +++ /dev/null @@ -1,75 +0,0 @@ -package storm.trident.tuple; - -import java.io.Serializable; -import java.util.AbstractList; -import java.util.List; -import org.apache.commons.lang.builder.ToStringBuilder; - - -public class ComboList extends AbstractList { - public static class Factory implements Serializable { - Pointer[] index; - int[] sizes; - - public Factory(int... sizes) { - this.sizes = sizes; - int total = 0; - for(int size: sizes) { - total+=size; - } - index = new Pointer[total]; - int i=0; - int j=0; - for(int size: sizes) { - for(int z=0; z { - List _elems; - Object _first; - - public ConsList(Object o, List elems) { - _elems = elems; - _first = o; - } - - @Override - public Object get(int i) { - if(i==0) return _first; - else { - return _elems.get(i - 1); - } - } - - @Override - public int size() { - return _elems.size() + 1; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/tuple/TridentTuple.java b/jstorm-client/src/main/java/storm/trident/tuple/TridentTuple.java deleted file mode 100644 index 9159ce79d..000000000 --- a/jstorm-client/src/main/java/storm/trident/tuple/TridentTuple.java +++ /dev/null @@ -1,17 +0,0 @@ -package storm.trident.tuple; - -import backtype.storm.tuple.ITuple; - -import java.io.Serializable; -import java.util.List; -import java.util.Map; - -public interface TridentTuple extends ITuple, List { - - public static interface Factory extends Serializable { - Map getFieldIndex(); - List getOutputFields(); - int numDelegates(); - } - -} diff --git a/jstorm-client/src/main/java/storm/trident/tuple/ValuePointer.java b/jstorm-client/src/main/java/storm/trident/tuple/ValuePointer.java deleted file mode 100644 index 401261ec5..000000000 --- a/jstorm-client/src/main/java/storm/trident/tuple/ValuePointer.java +++ /dev/null @@ -1,43 +0,0 @@ -package storm.trident.tuple; - -import backtype.storm.tuple.Fields; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.commons.lang.builder.ToStringBuilder; - -public class ValuePointer { - public static Map buildFieldIndex(ValuePointer[] pointers) { - Map ret = new HashMap(); - for(ValuePointer ptr: pointers) { - ret.put(ptr.field, ptr); - } - return ret; - } - - public static ValuePointer[] buildIndex(Fields fieldsOrder, Map pointers) { - if(fieldsOrder.size()!=pointers.size()) { - throw new IllegalArgumentException("Fields order must be same length as pointers map"); - } - ValuePointer[] ret = new ValuePointer[pointers.size()]; - for(int i=0; i implements Comparable, Serializable { - public T source; - public T target; - public int index; - - public IndexedEdge(T source, T target, int index) { - this.source = source; - this.target = target; - this.index = index; - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + index; - result = prime * result + ((source == null) ? 0 : source.hashCode()); - result = prime * result + ((target == null) ? 0 : target.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - IndexedEdge other = (IndexedEdge) obj; - if (index != other.index) - return false; - if (source == null) { - if (other.source != null) - return false; - } else if (!source.equals(other.source)) - return false; - if (target == null) { - if (other.target != null) - return false; - } else if (!target.equals(other.target)) - return false; - return true; - } - - @Override - public int compareTo(Object t) { - IndexedEdge other = (IndexedEdge) t; - return index - other.index; - } -} diff --git a/jstorm-client/src/main/java/storm/trident/util/LRUMap.java b/jstorm-client/src/main/java/storm/trident/util/LRUMap.java deleted file mode 100644 index 8d1a9a347..000000000 --- a/jstorm-client/src/main/java/storm/trident/util/LRUMap.java +++ /dev/null @@ -1,18 +0,0 @@ -package storm.trident.util; - -import java.util.LinkedHashMap; -import java.util.Map; - -public class LRUMap extends LinkedHashMap { - private int _maxSize; - - public LRUMap(int maxSize) { - super(maxSize + 1, 1.0f, true); - _maxSize = maxSize; - } - - @Override - protected boolean removeEldestEntry(final Map.Entry eldest) { - return size() > _maxSize; - } -} diff --git a/jstorm-client/src/main/py/storm/DistributedRPC-remote b/jstorm-client/src/main/py/storm/DistributedRPC-remote deleted file mode 100644 index 9b7ebd84e..000000000 --- a/jstorm-client/src/main/py/storm/DistributedRPC-remote +++ /dev/null @@ -1,85 +0,0 @@ -#!/usr/bin/env python -# -# Autogenerated by Thrift Compiler (0.7.0) -# -# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING -# - -import sys -import pprint -from urlparse import urlparse -from thrift.transport import TTransport -from thrift.transport import TSocket -from thrift.transport import THttpClient -from thrift.protocol import TBinaryProtocol - -import DistributedRPC -from ttypes import * - -if len(sys.argv) <= 1 or sys.argv[1] == '--help': - print '' - print 'Usage: ' + sys.argv[0] + ' [-h host:port] [-u url] [-f[ramed]] function [arg1 [arg2...]]' - print '' - print 'Functions:' - print ' string execute(string functionName, string funcArgs)' - print '' - sys.exit(0) - -pp = pprint.PrettyPrinter(indent = 2) -host = 'localhost' -port = 9090 -uri = '' -framed = False -http = False -argi = 1 - -if sys.argv[argi] == '-h': - parts = sys.argv[argi+1].split(':') - host = parts[0] - port = int(parts[1]) - argi += 2 - -if sys.argv[argi] == '-u': - url = urlparse(sys.argv[argi+1]) - parts = url[1].split(':') - host = parts[0] - if len(parts) > 1: - port = int(parts[1]) - else: - port = 80 - uri = url[2] - if url[4]: - uri += '?%s' % url[4] - http = True - argi += 2 - -if sys.argv[argi] == '-f' or sys.argv[argi] == '-framed': - framed = True - argi += 1 - -cmd = sys.argv[argi] -args = sys.argv[argi+1:] - -if http: - transport = THttpClient.THttpClient(host, port, uri) -else: - socket = TSocket.TSocket(host, port) - if framed: - transport = TTransport.TFramedTransport(socket) - else: - transport = TTransport.TBufferedTransport(socket) -protocol = TBinaryProtocol.TBinaryProtocol(transport) -client = DistributedRPC.Client(protocol) -transport.open() - -if cmd == 'execute': - if len(args) != 2: - print 'execute requires 2 args' - sys.exit(1) - pp.pprint(client.execute(args[0],args[1],)) - -else: - print 'Unrecognized method %s' % cmd - sys.exit(1) - -transport.close() diff --git a/jstorm-client/src/main/py/storm/DistributedRPCInvocations-remote b/jstorm-client/src/main/py/storm/DistributedRPCInvocations-remote deleted file mode 100644 index 5235dfee9..000000000 --- a/jstorm-client/src/main/py/storm/DistributedRPCInvocations-remote +++ /dev/null @@ -1,99 +0,0 @@ -#!/usr/bin/env python -# -# Autogenerated by Thrift Compiler (0.7.0) -# -# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING -# - -import sys -import pprint -from urlparse import urlparse -from thrift.transport import TTransport -from thrift.transport import TSocket -from thrift.transport import THttpClient -from thrift.protocol import TBinaryProtocol - -import DistributedRPCInvocations -from ttypes import * - -if len(sys.argv) <= 1 or sys.argv[1] == '--help': - print '' - print 'Usage: ' + sys.argv[0] + ' [-h host:port] [-u url] [-f[ramed]] function [arg1 [arg2...]]' - print '' - print 'Functions:' - print ' void result(string id, string result)' - print ' DRPCRequest fetchRequest(string functionName)' - print ' void failRequest(string id)' - print '' - sys.exit(0) - -pp = pprint.PrettyPrinter(indent = 2) -host = 'localhost' -port = 9090 -uri = '' -framed = False -http = False -argi = 1 - -if sys.argv[argi] == '-h': - parts = sys.argv[argi+1].split(':') - host = parts[0] - port = int(parts[1]) - argi += 2 - -if sys.argv[argi] == '-u': - url = urlparse(sys.argv[argi+1]) - parts = url[1].split(':') - host = parts[0] - if len(parts) > 1: - port = int(parts[1]) - else: - port = 80 - uri = url[2] - if url[4]: - uri += '?%s' % url[4] - http = True - argi += 2 - -if sys.argv[argi] == '-f' or sys.argv[argi] == '-framed': - framed = True - argi += 1 - -cmd = sys.argv[argi] -args = sys.argv[argi+1:] - -if http: - transport = THttpClient.THttpClient(host, port, uri) -else: - socket = TSocket.TSocket(host, port) - if framed: - transport = TTransport.TFramedTransport(socket) - else: - transport = TTransport.TBufferedTransport(socket) -protocol = TBinaryProtocol.TBinaryProtocol(transport) -client = DistributedRPCInvocations.Client(protocol) -transport.open() - -if cmd == 'result': - if len(args) != 2: - print 'result requires 2 args' - sys.exit(1) - pp.pprint(client.result(args[0],args[1],)) - -elif cmd == 'fetchRequest': - if len(args) != 1: - print 'fetchRequest requires 1 args' - sys.exit(1) - pp.pprint(client.fetchRequest(args[0],)) - -elif cmd == 'failRequest': - if len(args) != 1: - print 'failRequest requires 1 args' - sys.exit(1) - pp.pprint(client.failRequest(args[0],)) - -else: - print 'Unrecognized method %s' % cmd - sys.exit(1) - -transport.close() diff --git a/jstorm-client/src/main/py/storm/Nimbus-remote b/jstorm-client/src/main/py/storm/Nimbus-remote deleted file mode 100644 index 1fe705144..000000000 --- a/jstorm-client/src/main/py/storm/Nimbus-remote +++ /dev/null @@ -1,232 +0,0 @@ -#!/usr/bin/env python -# -# Autogenerated by Thrift Compiler (0.7.0) -# -# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING -# - -import sys -import pprint -from urlparse import urlparse -from thrift.transport import TTransport -from thrift.transport import TSocket -from thrift.transport import THttpClient -from thrift.protocol import TBinaryProtocol - -import Nimbus -from ttypes import * - -if len(sys.argv) <= 1 or sys.argv[1] == '--help': - print '' - print 'Usage: ' + sys.argv[0] + ' [-h host:port] [-u url] [-f[ramed]] function [arg1 [arg2...]]' - print '' - print 'Functions:' - print ' void submitTopology(string name, string uploadedJarLocation, string jsonConf, StormTopology topology)' - print ' void submitTopologyWithOpts(string name, string uploadedJarLocation, string jsonConf, StormTopology topology, SubmitOptions options)' - print ' void killTopology(string name)' - print ' void killTopologyWithOpts(string name, KillOptions options)' - print ' void activate(string name)' - print ' void deactivate(string name)' - print ' void rebalance(string name, RebalanceOptions options)' - print ' void metricMonitor(string name, MonitorOptions options)' - print ' void beginLibUpload(string libName)' - print ' string beginFileUpload()' - print ' void uploadChunk(string location, string chunk)' - print ' void finishFileUpload(string location)' - print ' string beginFileDownload(string file)' - print ' string downloadChunk(string id)' - print ' string getNimbusConf()' - print ' ClusterSummary getClusterInfo()' - print ' TopologyInfo getTopologyInfo(string id)' - print ' SupervisorWorkers getSupervisorWorkers(string host)' - print ' string getTopologyConf(string id)' - print ' StormTopology getTopology(string id)' - print ' StormTopology getUserTopology(string id)' - print ' TopologyMetricInfo getTopologyMetric(string id)' - print '' - sys.exit(0) - -pp = pprint.PrettyPrinter(indent = 2) -host = 'localhost' -port = 9090 -uri = '' -framed = False -http = False -argi = 1 - -if sys.argv[argi] == '-h': - parts = sys.argv[argi+1].split(':') - host = parts[0] - port = int(parts[1]) - argi += 2 - -if sys.argv[argi] == '-u': - url = urlparse(sys.argv[argi+1]) - parts = url[1].split(':') - host = parts[0] - if len(parts) > 1: - port = int(parts[1]) - else: - port = 80 - uri = url[2] - if url[4]: - uri += '?%s' % url[4] - http = True - argi += 2 - -if sys.argv[argi] == '-f' or sys.argv[argi] == '-framed': - framed = True - argi += 1 - -cmd = sys.argv[argi] -args = sys.argv[argi+1:] - -if http: - transport = THttpClient.THttpClient(host, port, uri) -else: - socket = TSocket.TSocket(host, port) - if framed: - transport = TTransport.TFramedTransport(socket) - else: - transport = TTransport.TBufferedTransport(socket) -protocol = TBinaryProtocol.TBinaryProtocol(transport) -client = Nimbus.Client(protocol) -transport.open() - -if cmd == 'submitTopology': - if len(args) != 4: - print 'submitTopology requires 4 args' - sys.exit(1) - pp.pprint(client.submitTopology(args[0],args[1],args[2],eval(args[3]),)) - -elif cmd == 'submitTopologyWithOpts': - if len(args) != 5: - print 'submitTopologyWithOpts requires 5 args' - sys.exit(1) - pp.pprint(client.submitTopologyWithOpts(args[0],args[1],args[2],eval(args[3]),eval(args[4]),)) - -elif cmd == 'killTopology': - if len(args) != 1: - print 'killTopology requires 1 args' - sys.exit(1) - pp.pprint(client.killTopology(args[0],)) - -elif cmd == 'killTopologyWithOpts': - if len(args) != 2: - print 'killTopologyWithOpts requires 2 args' - sys.exit(1) - pp.pprint(client.killTopologyWithOpts(args[0],eval(args[1]),)) - -elif cmd == 'activate': - if len(args) != 1: - print 'activate requires 1 args' - sys.exit(1) - pp.pprint(client.activate(args[0],)) - -elif cmd == 'deactivate': - if len(args) != 1: - print 'deactivate requires 1 args' - sys.exit(1) - pp.pprint(client.deactivate(args[0],)) - -elif cmd == 'rebalance': - if len(args) != 2: - print 'rebalance requires 2 args' - sys.exit(1) - pp.pprint(client.rebalance(args[0],eval(args[1]),)) - -elif cmd == 'metricMonitor': - if len(args) != 2: - print 'metricMonitor requires 2 args' - sys.exit(1) - pp.pprint(client.metricMonitor(args[0],eval(args[1]),)) - -elif cmd == 'beginLibUpload': - if len(args) != 1: - print 'beginLibUpload requires 1 args' - sys.exit(1) - pp.pprint(client.beginLibUpload(args[0],)) - -elif cmd == 'beginFileUpload': - if len(args) != 0: - print 'beginFileUpload requires 0 args' - sys.exit(1) - pp.pprint(client.beginFileUpload()) - -elif cmd == 'uploadChunk': - if len(args) != 2: - print 'uploadChunk requires 2 args' - sys.exit(1) - pp.pprint(client.uploadChunk(args[0],args[1],)) - -elif cmd == 'finishFileUpload': - if len(args) != 1: - print 'finishFileUpload requires 1 args' - sys.exit(1) - pp.pprint(client.finishFileUpload(args[0],)) - -elif cmd == 'beginFileDownload': - if len(args) != 1: - print 'beginFileDownload requires 1 args' - sys.exit(1) - pp.pprint(client.beginFileDownload(args[0],)) - -elif cmd == 'downloadChunk': - if len(args) != 1: - print 'downloadChunk requires 1 args' - sys.exit(1) - pp.pprint(client.downloadChunk(args[0],)) - -elif cmd == 'getNimbusConf': - if len(args) != 0: - print 'getNimbusConf requires 0 args' - sys.exit(1) - pp.pprint(client.getNimbusConf()) - -elif cmd == 'getClusterInfo': - if len(args) != 0: - print 'getClusterInfo requires 0 args' - sys.exit(1) - pp.pprint(client.getClusterInfo()) - -elif cmd == 'getTopologyInfo': - if len(args) != 1: - print 'getTopologyInfo requires 1 args' - sys.exit(1) - pp.pprint(client.getTopologyInfo(args[0],)) - -elif cmd == 'getSupervisorWorkers': - if len(args) != 1: - print 'getSupervisorWorkers requires 1 args' - sys.exit(1) - pp.pprint(client.getSupervisorWorkers(args[0],)) - -elif cmd == 'getTopologyConf': - if len(args) != 1: - print 'getTopologyConf requires 1 args' - sys.exit(1) - pp.pprint(client.getTopologyConf(args[0],)) - -elif cmd == 'getTopology': - if len(args) != 1: - print 'getTopology requires 1 args' - sys.exit(1) - pp.pprint(client.getTopology(args[0],)) - -elif cmd == 'getUserTopology': - if len(args) != 1: - print 'getUserTopology requires 1 args' - sys.exit(1) - pp.pprint(client.getUserTopology(args[0],)) - -elif cmd == 'getTopologyMetric': - if len(args) != 1: - print 'getTopologyMetric requires 1 args' - sys.exit(1) - pp.pprint(client.getTopologyMetric(args[0],)) - -else: - print 'Unrecognized method %s' % cmd - sys.exit(1) - -transport.close() diff --git a/jstorm-client/src/main/py/storm/Nimbus.py b/jstorm-client/src/main/py/storm/Nimbus.py deleted file mode 100644 index 9f382a5ac..000000000 --- a/jstorm-client/src/main/py/storm/Nimbus.py +++ /dev/null @@ -1,4149 +0,0 @@ -# -# Autogenerated by Thrift Compiler (0.7.0) -# -# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING -# - -from thrift.Thrift import * -from ttypes import * -from thrift.Thrift import TProcessor -from thrift.transport import TTransport -from thrift.protocol import TBinaryProtocol, TProtocol -try: - from thrift.protocol import fastbinary -except: - fastbinary = None - - -class Iface: - def submitTopology(self, name, uploadedJarLocation, jsonConf, topology): - """ - Parameters: - - name - - uploadedJarLocation - - jsonConf - - topology - """ - pass - - def submitTopologyWithOpts(self, name, uploadedJarLocation, jsonConf, topology, options): - """ - Parameters: - - name - - uploadedJarLocation - - jsonConf - - topology - - options - """ - pass - - def killTopology(self, name): - """ - Parameters: - - name - """ - pass - - def killTopologyWithOpts(self, name, options): - """ - Parameters: - - name - - options - """ - pass - - def activate(self, name): - """ - Parameters: - - name - """ - pass - - def deactivate(self, name): - """ - Parameters: - - name - """ - pass - - def rebalance(self, name, options): - """ - Parameters: - - name - - options - """ - pass - - def metricMonitor(self, name, options): - """ - Parameters: - - name - - options - """ - pass - - def beginLibUpload(self, libName): - """ - Parameters: - - libName - """ - pass - - def beginFileUpload(self, ): - pass - - def uploadChunk(self, location, chunk): - """ - Parameters: - - location - - chunk - """ - pass - - def finishFileUpload(self, location): - """ - Parameters: - - location - """ - pass - - def beginFileDownload(self, file): - """ - Parameters: - - file - """ - pass - - def downloadChunk(self, id): - """ - Parameters: - - id - """ - pass - - def getNimbusConf(self, ): - pass - - def getClusterInfo(self, ): - pass - - def getTopologyInfo(self, id): - """ - Parameters: - - id - """ - pass - - def getSupervisorWorkers(self, host): - """ - Parameters: - - host - """ - pass - - def getTopologyConf(self, id): - """ - Parameters: - - id - """ - pass - - def getTopology(self, id): - """ - Parameters: - - id - """ - pass - - def getUserTopology(self, id): - """ - Parameters: - - id - """ - pass - - def getTopologyMetric(self, id): - """ - Parameters: - - id - """ - pass - - -class Client(Iface): - def __init__(self, iprot, oprot=None): - self._iprot = self._oprot = iprot - if oprot is not None: - self._oprot = oprot - self._seqid = 0 - - def submitTopology(self, name, uploadedJarLocation, jsonConf, topology): - """ - Parameters: - - name - - uploadedJarLocation - - jsonConf - - topology - """ - self.send_submitTopology(name, uploadedJarLocation, jsonConf, topology) - self.recv_submitTopology() - - def send_submitTopology(self, name, uploadedJarLocation, jsonConf, topology): - self._oprot.writeMessageBegin('submitTopology', TMessageType.CALL, self._seqid) - args = submitTopology_args() - args.name = name - args.uploadedJarLocation = uploadedJarLocation - args.jsonConf = jsonConf - args.topology = topology - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_submitTopology(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = submitTopology_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - if result.e is not None: - raise result.e - if result.ite is not None: - raise result.ite - if result.tae is not None: - raise result.tae - return - - def submitTopologyWithOpts(self, name, uploadedJarLocation, jsonConf, topology, options): - """ - Parameters: - - name - - uploadedJarLocation - - jsonConf - - topology - - options - """ - self.send_submitTopologyWithOpts(name, uploadedJarLocation, jsonConf, topology, options) - self.recv_submitTopologyWithOpts() - - def send_submitTopologyWithOpts(self, name, uploadedJarLocation, jsonConf, topology, options): - self._oprot.writeMessageBegin('submitTopologyWithOpts', TMessageType.CALL, self._seqid) - args = submitTopologyWithOpts_args() - args.name = name - args.uploadedJarLocation = uploadedJarLocation - args.jsonConf = jsonConf - args.topology = topology - args.options = options - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_submitTopologyWithOpts(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = submitTopologyWithOpts_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - if result.e is not None: - raise result.e - if result.ite is not None: - raise result.ite - if result.tae is not None: - raise result.tae - return - - def killTopology(self, name): - """ - Parameters: - - name - """ - self.send_killTopology(name) - self.recv_killTopology() - - def send_killTopology(self, name): - self._oprot.writeMessageBegin('killTopology', TMessageType.CALL, self._seqid) - args = killTopology_args() - args.name = name - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_killTopology(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = killTopology_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - if result.e is not None: - raise result.e - return - - def killTopologyWithOpts(self, name, options): - """ - Parameters: - - name - - options - """ - self.send_killTopologyWithOpts(name, options) - self.recv_killTopologyWithOpts() - - def send_killTopologyWithOpts(self, name, options): - self._oprot.writeMessageBegin('killTopologyWithOpts', TMessageType.CALL, self._seqid) - args = killTopologyWithOpts_args() - args.name = name - args.options = options - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_killTopologyWithOpts(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = killTopologyWithOpts_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - if result.e is not None: - raise result.e - return - - def activate(self, name): - """ - Parameters: - - name - """ - self.send_activate(name) - self.recv_activate() - - def send_activate(self, name): - self._oprot.writeMessageBegin('activate', TMessageType.CALL, self._seqid) - args = activate_args() - args.name = name - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_activate(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = activate_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - if result.e is not None: - raise result.e - return - - def deactivate(self, name): - """ - Parameters: - - name - """ - self.send_deactivate(name) - self.recv_deactivate() - - def send_deactivate(self, name): - self._oprot.writeMessageBegin('deactivate', TMessageType.CALL, self._seqid) - args = deactivate_args() - args.name = name - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_deactivate(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = deactivate_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - if result.e is not None: - raise result.e - return - - def rebalance(self, name, options): - """ - Parameters: - - name - - options - """ - self.send_rebalance(name, options) - self.recv_rebalance() - - def send_rebalance(self, name, options): - self._oprot.writeMessageBegin('rebalance', TMessageType.CALL, self._seqid) - args = rebalance_args() - args.name = name - args.options = options - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_rebalance(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = rebalance_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - if result.e is not None: - raise result.e - if result.ite is not None: - raise result.ite - return - - def metricMonitor(self, name, options): - """ - Parameters: - - name - - options - """ - self.send_metricMonitor(name, options) - self.recv_metricMonitor() - - def send_metricMonitor(self, name, options): - self._oprot.writeMessageBegin('metricMonitor', TMessageType.CALL, self._seqid) - args = metricMonitor_args() - args.name = name - args.options = options - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_metricMonitor(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = metricMonitor_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - if result.e is not None: - raise result.e - return - - def beginLibUpload(self, libName): - """ - Parameters: - - libName - """ - self.send_beginLibUpload(libName) - self.recv_beginLibUpload() - - def send_beginLibUpload(self, libName): - self._oprot.writeMessageBegin('beginLibUpload', TMessageType.CALL, self._seqid) - args = beginLibUpload_args() - args.libName = libName - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_beginLibUpload(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = beginLibUpload_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - return - - def beginFileUpload(self, ): - self.send_beginFileUpload() - return self.recv_beginFileUpload() - - def send_beginFileUpload(self, ): - self._oprot.writeMessageBegin('beginFileUpload', TMessageType.CALL, self._seqid) - args = beginFileUpload_args() - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_beginFileUpload(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = beginFileUpload_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - if result.success is not None: - return result.success - raise TApplicationException(TApplicationException.MISSING_RESULT, "beginFileUpload failed: unknown result"); - - def uploadChunk(self, location, chunk): - """ - Parameters: - - location - - chunk - """ - self.send_uploadChunk(location, chunk) - self.recv_uploadChunk() - - def send_uploadChunk(self, location, chunk): - self._oprot.writeMessageBegin('uploadChunk', TMessageType.CALL, self._seqid) - args = uploadChunk_args() - args.location = location - args.chunk = chunk - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_uploadChunk(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = uploadChunk_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - return - - def finishFileUpload(self, location): - """ - Parameters: - - location - """ - self.send_finishFileUpload(location) - self.recv_finishFileUpload() - - def send_finishFileUpload(self, location): - self._oprot.writeMessageBegin('finishFileUpload', TMessageType.CALL, self._seqid) - args = finishFileUpload_args() - args.location = location - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_finishFileUpload(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = finishFileUpload_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - return - - def beginFileDownload(self, file): - """ - Parameters: - - file - """ - self.send_beginFileDownload(file) - return self.recv_beginFileDownload() - - def send_beginFileDownload(self, file): - self._oprot.writeMessageBegin('beginFileDownload', TMessageType.CALL, self._seqid) - args = beginFileDownload_args() - args.file = file - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_beginFileDownload(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = beginFileDownload_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - if result.success is not None: - return result.success - raise TApplicationException(TApplicationException.MISSING_RESULT, "beginFileDownload failed: unknown result"); - - def downloadChunk(self, id): - """ - Parameters: - - id - """ - self.send_downloadChunk(id) - return self.recv_downloadChunk() - - def send_downloadChunk(self, id): - self._oprot.writeMessageBegin('downloadChunk', TMessageType.CALL, self._seqid) - args = downloadChunk_args() - args.id = id - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_downloadChunk(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = downloadChunk_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - if result.success is not None: - return result.success - raise TApplicationException(TApplicationException.MISSING_RESULT, "downloadChunk failed: unknown result"); - - def getNimbusConf(self, ): - self.send_getNimbusConf() - return self.recv_getNimbusConf() - - def send_getNimbusConf(self, ): - self._oprot.writeMessageBegin('getNimbusConf', TMessageType.CALL, self._seqid) - args = getNimbusConf_args() - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_getNimbusConf(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = getNimbusConf_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - if result.success is not None: - return result.success - raise TApplicationException(TApplicationException.MISSING_RESULT, "getNimbusConf failed: unknown result"); - - def getClusterInfo(self, ): - self.send_getClusterInfo() - return self.recv_getClusterInfo() - - def send_getClusterInfo(self, ): - self._oprot.writeMessageBegin('getClusterInfo', TMessageType.CALL, self._seqid) - args = getClusterInfo_args() - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_getClusterInfo(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = getClusterInfo_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - if result.success is not None: - return result.success - raise TApplicationException(TApplicationException.MISSING_RESULT, "getClusterInfo failed: unknown result"); - - def getTopologyInfo(self, id): - """ - Parameters: - - id - """ - self.send_getTopologyInfo(id) - return self.recv_getTopologyInfo() - - def send_getTopologyInfo(self, id): - self._oprot.writeMessageBegin('getTopologyInfo', TMessageType.CALL, self._seqid) - args = getTopologyInfo_args() - args.id = id - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_getTopologyInfo(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = getTopologyInfo_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - if result.success is not None: - return result.success - if result.e is not None: - raise result.e - raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyInfo failed: unknown result"); - - def getSupervisorWorkers(self, host): - """ - Parameters: - - host - """ - self.send_getSupervisorWorkers(host) - return self.recv_getSupervisorWorkers() - - def send_getSupervisorWorkers(self, host): - self._oprot.writeMessageBegin('getSupervisorWorkers', TMessageType.CALL, self._seqid) - args = getSupervisorWorkers_args() - args.host = host - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_getSupervisorWorkers(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = getSupervisorWorkers_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - if result.success is not None: - return result.success - if result.e is not None: - raise result.e - raise TApplicationException(TApplicationException.MISSING_RESULT, "getSupervisorWorkers failed: unknown result"); - - def getTopologyConf(self, id): - """ - Parameters: - - id - """ - self.send_getTopologyConf(id) - return self.recv_getTopologyConf() - - def send_getTopologyConf(self, id): - self._oprot.writeMessageBegin('getTopologyConf', TMessageType.CALL, self._seqid) - args = getTopologyConf_args() - args.id = id - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_getTopologyConf(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = getTopologyConf_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - if result.success is not None: - return result.success - if result.e is not None: - raise result.e - raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyConf failed: unknown result"); - - def getTopology(self, id): - """ - Parameters: - - id - """ - self.send_getTopology(id) - return self.recv_getTopology() - - def send_getTopology(self, id): - self._oprot.writeMessageBegin('getTopology', TMessageType.CALL, self._seqid) - args = getTopology_args() - args.id = id - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_getTopology(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = getTopology_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - if result.success is not None: - return result.success - if result.e is not None: - raise result.e - raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopology failed: unknown result"); - - def getUserTopology(self, id): - """ - Parameters: - - id - """ - self.send_getUserTopology(id) - return self.recv_getUserTopology() - - def send_getUserTopology(self, id): - self._oprot.writeMessageBegin('getUserTopology', TMessageType.CALL, self._seqid) - args = getUserTopology_args() - args.id = id - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_getUserTopology(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = getUserTopology_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - if result.success is not None: - return result.success - if result.e is not None: - raise result.e - raise TApplicationException(TApplicationException.MISSING_RESULT, "getUserTopology failed: unknown result"); - - def getTopologyMetric(self, id): - """ - Parameters: - - id - """ - self.send_getTopologyMetric(id) - return self.recv_getTopologyMetric() - - def send_getTopologyMetric(self, id): - self._oprot.writeMessageBegin('getTopologyMetric', TMessageType.CALL, self._seqid) - args = getTopologyMetric_args() - args.id = id - args.write(self._oprot) - self._oprot.writeMessageEnd() - self._oprot.trans.flush() - - def recv_getTopologyMetric(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() - if mtype == TMessageType.EXCEPTION: - x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() - raise x - result = getTopologyMetric_result() - result.read(self._iprot) - self._iprot.readMessageEnd() - if result.success is not None: - return result.success - if result.e is not None: - raise result.e - raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyMetric failed: unknown result"); - - -class Processor(Iface, TProcessor): - def __init__(self, handler): - self._handler = handler - self._processMap = {} - self._processMap["submitTopology"] = Processor.process_submitTopology - self._processMap["submitTopologyWithOpts"] = Processor.process_submitTopologyWithOpts - self._processMap["killTopology"] = Processor.process_killTopology - self._processMap["killTopologyWithOpts"] = Processor.process_killTopologyWithOpts - self._processMap["activate"] = Processor.process_activate - self._processMap["deactivate"] = Processor.process_deactivate - self._processMap["rebalance"] = Processor.process_rebalance - self._processMap["metricMonitor"] = Processor.process_metricMonitor - self._processMap["beginLibUpload"] = Processor.process_beginLibUpload - self._processMap["beginFileUpload"] = Processor.process_beginFileUpload - self._processMap["uploadChunk"] = Processor.process_uploadChunk - self._processMap["finishFileUpload"] = Processor.process_finishFileUpload - self._processMap["beginFileDownload"] = Processor.process_beginFileDownload - self._processMap["downloadChunk"] = Processor.process_downloadChunk - self._processMap["getNimbusConf"] = Processor.process_getNimbusConf - self._processMap["getClusterInfo"] = Processor.process_getClusterInfo - self._processMap["getTopologyInfo"] = Processor.process_getTopologyInfo - self._processMap["getSupervisorWorkers"] = Processor.process_getSupervisorWorkers - self._processMap["getTopologyConf"] = Processor.process_getTopologyConf - self._processMap["getTopology"] = Processor.process_getTopology - self._processMap["getUserTopology"] = Processor.process_getUserTopology - self._processMap["getTopologyMetric"] = Processor.process_getTopologyMetric - - def process(self, iprot, oprot): - (name, type, seqid) = iprot.readMessageBegin() - if name not in self._processMap: - iprot.skip(TType.STRUCT) - iprot.readMessageEnd() - x = TApplicationException(TApplicationException.UNKNOWN_METHOD, 'Unknown function %s' % (name)) - oprot.writeMessageBegin(name, TMessageType.EXCEPTION, seqid) - x.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - return - else: - self._processMap[name](self, seqid, iprot, oprot) - return True - - def process_submitTopology(self, seqid, iprot, oprot): - args = submitTopology_args() - args.read(iprot) - iprot.readMessageEnd() - result = submitTopology_result() - try: - self._handler.submitTopology(args.name, args.uploadedJarLocation, args.jsonConf, args.topology) - except AlreadyAliveException, e: - result.e = e - except InvalidTopologyException, ite: - result.ite = ite - except TopologyAssignException, tae: - result.tae = tae - oprot.writeMessageBegin("submitTopology", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_submitTopologyWithOpts(self, seqid, iprot, oprot): - args = submitTopologyWithOpts_args() - args.read(iprot) - iprot.readMessageEnd() - result = submitTopologyWithOpts_result() - try: - self._handler.submitTopologyWithOpts(args.name, args.uploadedJarLocation, args.jsonConf, args.topology, args.options) - except AlreadyAliveException, e: - result.e = e - except InvalidTopologyException, ite: - result.ite = ite - except TopologyAssignException, tae: - result.tae = tae - oprot.writeMessageBegin("submitTopologyWithOpts", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_killTopology(self, seqid, iprot, oprot): - args = killTopology_args() - args.read(iprot) - iprot.readMessageEnd() - result = killTopology_result() - try: - self._handler.killTopology(args.name) - except NotAliveException, e: - result.e = e - oprot.writeMessageBegin("killTopology", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_killTopologyWithOpts(self, seqid, iprot, oprot): - args = killTopologyWithOpts_args() - args.read(iprot) - iprot.readMessageEnd() - result = killTopologyWithOpts_result() - try: - self._handler.killTopologyWithOpts(args.name, args.options) - except NotAliveException, e: - result.e = e - oprot.writeMessageBegin("killTopologyWithOpts", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_activate(self, seqid, iprot, oprot): - args = activate_args() - args.read(iprot) - iprot.readMessageEnd() - result = activate_result() - try: - self._handler.activate(args.name) - except NotAliveException, e: - result.e = e - oprot.writeMessageBegin("activate", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_deactivate(self, seqid, iprot, oprot): - args = deactivate_args() - args.read(iprot) - iprot.readMessageEnd() - result = deactivate_result() - try: - self._handler.deactivate(args.name) - except NotAliveException, e: - result.e = e - oprot.writeMessageBegin("deactivate", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_rebalance(self, seqid, iprot, oprot): - args = rebalance_args() - args.read(iprot) - iprot.readMessageEnd() - result = rebalance_result() - try: - self._handler.rebalance(args.name, args.options) - except NotAliveException, e: - result.e = e - except InvalidTopologyException, ite: - result.ite = ite - oprot.writeMessageBegin("rebalance", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_metricMonitor(self, seqid, iprot, oprot): - args = metricMonitor_args() - args.read(iprot) - iprot.readMessageEnd() - result = metricMonitor_result() - try: - self._handler.metricMonitor(args.name, args.options) - except NotAliveException, e: - result.e = e - oprot.writeMessageBegin("metricMonitor", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_beginLibUpload(self, seqid, iprot, oprot): - args = beginLibUpload_args() - args.read(iprot) - iprot.readMessageEnd() - result = beginLibUpload_result() - self._handler.beginLibUpload(args.libName) - oprot.writeMessageBegin("beginLibUpload", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_beginFileUpload(self, seqid, iprot, oprot): - args = beginFileUpload_args() - args.read(iprot) - iprot.readMessageEnd() - result = beginFileUpload_result() - result.success = self._handler.beginFileUpload() - oprot.writeMessageBegin("beginFileUpload", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_uploadChunk(self, seqid, iprot, oprot): - args = uploadChunk_args() - args.read(iprot) - iprot.readMessageEnd() - result = uploadChunk_result() - self._handler.uploadChunk(args.location, args.chunk) - oprot.writeMessageBegin("uploadChunk", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_finishFileUpload(self, seqid, iprot, oprot): - args = finishFileUpload_args() - args.read(iprot) - iprot.readMessageEnd() - result = finishFileUpload_result() - self._handler.finishFileUpload(args.location) - oprot.writeMessageBegin("finishFileUpload", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_beginFileDownload(self, seqid, iprot, oprot): - args = beginFileDownload_args() - args.read(iprot) - iprot.readMessageEnd() - result = beginFileDownload_result() - result.success = self._handler.beginFileDownload(args.file) - oprot.writeMessageBegin("beginFileDownload", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_downloadChunk(self, seqid, iprot, oprot): - args = downloadChunk_args() - args.read(iprot) - iprot.readMessageEnd() - result = downloadChunk_result() - result.success = self._handler.downloadChunk(args.id) - oprot.writeMessageBegin("downloadChunk", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_getNimbusConf(self, seqid, iprot, oprot): - args = getNimbusConf_args() - args.read(iprot) - iprot.readMessageEnd() - result = getNimbusConf_result() - result.success = self._handler.getNimbusConf() - oprot.writeMessageBegin("getNimbusConf", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_getClusterInfo(self, seqid, iprot, oprot): - args = getClusterInfo_args() - args.read(iprot) - iprot.readMessageEnd() - result = getClusterInfo_result() - result.success = self._handler.getClusterInfo() - oprot.writeMessageBegin("getClusterInfo", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_getTopologyInfo(self, seqid, iprot, oprot): - args = getTopologyInfo_args() - args.read(iprot) - iprot.readMessageEnd() - result = getTopologyInfo_result() - try: - result.success = self._handler.getTopologyInfo(args.id) - except NotAliveException, e: - result.e = e - oprot.writeMessageBegin("getTopologyInfo", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_getSupervisorWorkers(self, seqid, iprot, oprot): - args = getSupervisorWorkers_args() - args.read(iprot) - iprot.readMessageEnd() - result = getSupervisorWorkers_result() - try: - result.success = self._handler.getSupervisorWorkers(args.host) - except NotAliveException, e: - result.e = e - oprot.writeMessageBegin("getSupervisorWorkers", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_getTopologyConf(self, seqid, iprot, oprot): - args = getTopologyConf_args() - args.read(iprot) - iprot.readMessageEnd() - result = getTopologyConf_result() - try: - result.success = self._handler.getTopologyConf(args.id) - except NotAliveException, e: - result.e = e - oprot.writeMessageBegin("getTopologyConf", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_getTopology(self, seqid, iprot, oprot): - args = getTopology_args() - args.read(iprot) - iprot.readMessageEnd() - result = getTopology_result() - try: - result.success = self._handler.getTopology(args.id) - except NotAliveException, e: - result.e = e - oprot.writeMessageBegin("getTopology", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_getUserTopology(self, seqid, iprot, oprot): - args = getUserTopology_args() - args.read(iprot) - iprot.readMessageEnd() - result = getUserTopology_result() - try: - result.success = self._handler.getUserTopology(args.id) - except NotAliveException, e: - result.e = e - oprot.writeMessageBegin("getUserTopology", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - def process_getTopologyMetric(self, seqid, iprot, oprot): - args = getTopologyMetric_args() - args.read(iprot) - iprot.readMessageEnd() - result = getTopologyMetric_result() - try: - result.success = self._handler.getTopologyMetric(args.id) - except NotAliveException, e: - result.e = e - oprot.writeMessageBegin("getTopologyMetric", TMessageType.REPLY, seqid) - result.write(oprot) - oprot.writeMessageEnd() - oprot.trans.flush() - - -# HELPER FUNCTIONS AND STRUCTURES - -class submitTopology_args: - """ - Attributes: - - name - - uploadedJarLocation - - jsonConf - - topology - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'name', None, None, ), # 1 - (2, TType.STRING, 'uploadedJarLocation', None, None, ), # 2 - (3, TType.STRING, 'jsonConf', None, None, ), # 3 - (4, TType.STRUCT, 'topology', (StormTopology, StormTopology.thrift_spec), None, ), # 4 - ) - - def __hash__(self): - return 0 + hash(self.name) + hash(self.uploadedJarLocation) + hash(self.jsonConf) + hash(self.topology) - - def __init__(self, name=None, uploadedJarLocation=None, jsonConf=None, topology=None,): - self.name = name - self.uploadedJarLocation = uploadedJarLocation - self.jsonConf = jsonConf - self.topology = topology - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.name = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.uploadedJarLocation = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRING: - self.jsonConf = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.topology = StormTopology() - self.topology.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('submitTopology_args') - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 1) - oprot.writeString(self.name.encode('utf-8')) - oprot.writeFieldEnd() - if self.uploadedJarLocation is not None: - oprot.writeFieldBegin('uploadedJarLocation', TType.STRING, 2) - oprot.writeString(self.uploadedJarLocation.encode('utf-8')) - oprot.writeFieldEnd() - if self.jsonConf is not None: - oprot.writeFieldBegin('jsonConf', TType.STRING, 3) - oprot.writeString(self.jsonConf.encode('utf-8')) - oprot.writeFieldEnd() - if self.topology is not None: - oprot.writeFieldBegin('topology', TType.STRUCT, 4) - self.topology.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class submitTopology_result: - """ - Attributes: - - e - - ite - - tae - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'e', (AlreadyAliveException, AlreadyAliveException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'ite', (InvalidTopologyException, InvalidTopologyException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'tae', (TopologyAssignException, TopologyAssignException.thrift_spec), None, ), # 3 - ) - - def __hash__(self): - return 0 + hash(self.e) + hash(self.ite) + hash(self.tae) - - def __init__(self, e=None, ite=None, tae=None,): - self.e = e - self.ite = ite - self.tae = tae - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRUCT: - self.e = AlreadyAliveException() - self.e.read(iprot) - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.ite = InvalidTopologyException() - self.ite.read(iprot) - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.tae = TopologyAssignException() - self.tae.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('submitTopology_result') - if self.e is not None: - oprot.writeFieldBegin('e', TType.STRUCT, 1) - self.e.write(oprot) - oprot.writeFieldEnd() - if self.ite is not None: - oprot.writeFieldBegin('ite', TType.STRUCT, 2) - self.ite.write(oprot) - oprot.writeFieldEnd() - if self.tae is not None: - oprot.writeFieldBegin('tae', TType.STRUCT, 3) - self.tae.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class submitTopologyWithOpts_args: - """ - Attributes: - - name - - uploadedJarLocation - - jsonConf - - topology - - options - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'name', None, None, ), # 1 - (2, TType.STRING, 'uploadedJarLocation', None, None, ), # 2 - (3, TType.STRING, 'jsonConf', None, None, ), # 3 - (4, TType.STRUCT, 'topology', (StormTopology, StormTopology.thrift_spec), None, ), # 4 - (5, TType.STRUCT, 'options', (SubmitOptions, SubmitOptions.thrift_spec), None, ), # 5 - ) - - def __hash__(self): - return 0 + hash(self.name) + hash(self.uploadedJarLocation) + hash(self.jsonConf) + hash(self.topology) + hash(self.options) - - def __init__(self, name=None, uploadedJarLocation=None, jsonConf=None, topology=None, options=None,): - self.name = name - self.uploadedJarLocation = uploadedJarLocation - self.jsonConf = jsonConf - self.topology = topology - self.options = options - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.name = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.uploadedJarLocation = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRING: - self.jsonConf = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.topology = StormTopology() - self.topology.read(iprot) - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.STRUCT: - self.options = SubmitOptions() - self.options.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('submitTopologyWithOpts_args') - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 1) - oprot.writeString(self.name.encode('utf-8')) - oprot.writeFieldEnd() - if self.uploadedJarLocation is not None: - oprot.writeFieldBegin('uploadedJarLocation', TType.STRING, 2) - oprot.writeString(self.uploadedJarLocation.encode('utf-8')) - oprot.writeFieldEnd() - if self.jsonConf is not None: - oprot.writeFieldBegin('jsonConf', TType.STRING, 3) - oprot.writeString(self.jsonConf.encode('utf-8')) - oprot.writeFieldEnd() - if self.topology is not None: - oprot.writeFieldBegin('topology', TType.STRUCT, 4) - self.topology.write(oprot) - oprot.writeFieldEnd() - if self.options is not None: - oprot.writeFieldBegin('options', TType.STRUCT, 5) - self.options.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class submitTopologyWithOpts_result: - """ - Attributes: - - e - - ite - - tae - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'e', (AlreadyAliveException, AlreadyAliveException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'ite', (InvalidTopologyException, InvalidTopologyException.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'tae', (TopologyAssignException, TopologyAssignException.thrift_spec), None, ), # 3 - ) - - def __hash__(self): - return 0 + hash(self.e) + hash(self.ite) + hash(self.tae) - - def __init__(self, e=None, ite=None, tae=None,): - self.e = e - self.ite = ite - self.tae = tae - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRUCT: - self.e = AlreadyAliveException() - self.e.read(iprot) - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.ite = InvalidTopologyException() - self.ite.read(iprot) - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.tae = TopologyAssignException() - self.tae.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('submitTopologyWithOpts_result') - if self.e is not None: - oprot.writeFieldBegin('e', TType.STRUCT, 1) - self.e.write(oprot) - oprot.writeFieldEnd() - if self.ite is not None: - oprot.writeFieldBegin('ite', TType.STRUCT, 2) - self.ite.write(oprot) - oprot.writeFieldEnd() - if self.tae is not None: - oprot.writeFieldBegin('tae', TType.STRUCT, 3) - self.tae.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class killTopology_args: - """ - Attributes: - - name - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'name', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.name) - - def __init__(self, name=None,): - self.name = name - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.name = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('killTopology_args') - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 1) - oprot.writeString(self.name.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class killTopology_result: - """ - Attributes: - - e - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.e) - - def __init__(self, e=None,): - self.e = e - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRUCT: - self.e = NotAliveException() - self.e.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('killTopology_result') - if self.e is not None: - oprot.writeFieldBegin('e', TType.STRUCT, 1) - self.e.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class killTopologyWithOpts_args: - """ - Attributes: - - name - - options - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'name', None, None, ), # 1 - (2, TType.STRUCT, 'options', (KillOptions, KillOptions.thrift_spec), None, ), # 2 - ) - - def __hash__(self): - return 0 + hash(self.name) + hash(self.options) - - def __init__(self, name=None, options=None,): - self.name = name - self.options = options - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.name = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.options = KillOptions() - self.options.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('killTopologyWithOpts_args') - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 1) - oprot.writeString(self.name.encode('utf-8')) - oprot.writeFieldEnd() - if self.options is not None: - oprot.writeFieldBegin('options', TType.STRUCT, 2) - self.options.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class killTopologyWithOpts_result: - """ - Attributes: - - e - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.e) - - def __init__(self, e=None,): - self.e = e - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRUCT: - self.e = NotAliveException() - self.e.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('killTopologyWithOpts_result') - if self.e is not None: - oprot.writeFieldBegin('e', TType.STRUCT, 1) - self.e.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class activate_args: - """ - Attributes: - - name - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'name', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.name) - - def __init__(self, name=None,): - self.name = name - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.name = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('activate_args') - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 1) - oprot.writeString(self.name.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class activate_result: - """ - Attributes: - - e - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.e) - - def __init__(self, e=None,): - self.e = e - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRUCT: - self.e = NotAliveException() - self.e.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('activate_result') - if self.e is not None: - oprot.writeFieldBegin('e', TType.STRUCT, 1) - self.e.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class deactivate_args: - """ - Attributes: - - name - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'name', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.name) - - def __init__(self, name=None,): - self.name = name - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.name = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('deactivate_args') - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 1) - oprot.writeString(self.name.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class deactivate_result: - """ - Attributes: - - e - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.e) - - def __init__(self, e=None,): - self.e = e - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRUCT: - self.e = NotAliveException() - self.e.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('deactivate_result') - if self.e is not None: - oprot.writeFieldBegin('e', TType.STRUCT, 1) - self.e.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class rebalance_args: - """ - Attributes: - - name - - options - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'name', None, None, ), # 1 - (2, TType.STRUCT, 'options', (RebalanceOptions, RebalanceOptions.thrift_spec), None, ), # 2 - ) - - def __hash__(self): - return 0 + hash(self.name) + hash(self.options) - - def __init__(self, name=None, options=None,): - self.name = name - self.options = options - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.name = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.options = RebalanceOptions() - self.options.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('rebalance_args') - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 1) - oprot.writeString(self.name.encode('utf-8')) - oprot.writeFieldEnd() - if self.options is not None: - oprot.writeFieldBegin('options', TType.STRUCT, 2) - self.options.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class rebalance_result: - """ - Attributes: - - e - - ite - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'ite', (InvalidTopologyException, InvalidTopologyException.thrift_spec), None, ), # 2 - ) - - def __hash__(self): - return 0 + hash(self.e) + hash(self.ite) - - def __init__(self, e=None, ite=None,): - self.e = e - self.ite = ite - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRUCT: - self.e = NotAliveException() - self.e.read(iprot) - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.ite = InvalidTopologyException() - self.ite.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('rebalance_result') - if self.e is not None: - oprot.writeFieldBegin('e', TType.STRUCT, 1) - self.e.write(oprot) - oprot.writeFieldEnd() - if self.ite is not None: - oprot.writeFieldBegin('ite', TType.STRUCT, 2) - self.ite.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class metricMonitor_args: - """ - Attributes: - - name - - options - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'name', None, None, ), # 1 - (2, TType.STRUCT, 'options', (MonitorOptions, MonitorOptions.thrift_spec), None, ), # 2 - ) - - def __hash__(self): - return 0 + hash(self.name) + hash(self.options) - - def __init__(self, name=None, options=None,): - self.name = name - self.options = options - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.name = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.options = MonitorOptions() - self.options.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('metricMonitor_args') - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 1) - oprot.writeString(self.name.encode('utf-8')) - oprot.writeFieldEnd() - if self.options is not None: - oprot.writeFieldBegin('options', TType.STRUCT, 2) - self.options.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class metricMonitor_result: - """ - Attributes: - - e - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.e) - - def __init__(self, e=None,): - self.e = e - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRUCT: - self.e = NotAliveException() - self.e.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('metricMonitor_result') - if self.e is not None: - oprot.writeFieldBegin('e', TType.STRUCT, 1) - self.e.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class beginLibUpload_args: - """ - Attributes: - - libName - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'libName', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.libName) - - def __init__(self, libName=None,): - self.libName = libName - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.libName = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('beginLibUpload_args') - if self.libName is not None: - oprot.writeFieldBegin('libName', TType.STRING, 1) - oprot.writeString(self.libName.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class beginLibUpload_result: - - thrift_spec = ( - ) - - def __hash__(self): - return 0 - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('beginLibUpload_result') - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class beginFileUpload_args: - - thrift_spec = ( - ) - - def __hash__(self): - return 0 - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('beginFileUpload_args') - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class beginFileUpload_result: - """ - Attributes: - - success - """ - - thrift_spec = ( - (0, TType.STRING, 'success', None, None, ), # 0 - ) - - def __hash__(self): - return 0 + hash(self.success) - - def __init__(self, success=None,): - self.success = success - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 0: - if ftype == TType.STRING: - self.success = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('beginFileUpload_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.STRING, 0) - oprot.writeString(self.success.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class uploadChunk_args: - """ - Attributes: - - location - - chunk - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'location', None, None, ), # 1 - (2, TType.STRING, 'chunk', None, None, ), # 2 - ) - - def __hash__(self): - return 0 + hash(self.location) + hash(self.chunk) - - def __init__(self, location=None, chunk=None,): - self.location = location - self.chunk = chunk - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.location = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.chunk = iprot.readString(); - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('uploadChunk_args') - if self.location is not None: - oprot.writeFieldBegin('location', TType.STRING, 1) - oprot.writeString(self.location.encode('utf-8')) - oprot.writeFieldEnd() - if self.chunk is not None: - oprot.writeFieldBegin('chunk', TType.STRING, 2) - oprot.writeString(self.chunk) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class uploadChunk_result: - - thrift_spec = ( - ) - - def __hash__(self): - return 0 - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('uploadChunk_result') - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class finishFileUpload_args: - """ - Attributes: - - location - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'location', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.location) - - def __init__(self, location=None,): - self.location = location - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.location = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('finishFileUpload_args') - if self.location is not None: - oprot.writeFieldBegin('location', TType.STRING, 1) - oprot.writeString(self.location.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class finishFileUpload_result: - - thrift_spec = ( - ) - - def __hash__(self): - return 0 - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('finishFileUpload_result') - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class beginFileDownload_args: - """ - Attributes: - - file - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'file', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.file) - - def __init__(self, file=None,): - self.file = file - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.file = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('beginFileDownload_args') - if self.file is not None: - oprot.writeFieldBegin('file', TType.STRING, 1) - oprot.writeString(self.file.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class beginFileDownload_result: - """ - Attributes: - - success - """ - - thrift_spec = ( - (0, TType.STRING, 'success', None, None, ), # 0 - ) - - def __hash__(self): - return 0 + hash(self.success) - - def __init__(self, success=None,): - self.success = success - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 0: - if ftype == TType.STRING: - self.success = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('beginFileDownload_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.STRING, 0) - oprot.writeString(self.success.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class downloadChunk_args: - """ - Attributes: - - id - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'id', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.id) - - def __init__(self, id=None,): - self.id = id - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.id = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('downloadChunk_args') - if self.id is not None: - oprot.writeFieldBegin('id', TType.STRING, 1) - oprot.writeString(self.id.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class downloadChunk_result: - """ - Attributes: - - success - """ - - thrift_spec = ( - (0, TType.STRING, 'success', None, None, ), # 0 - ) - - def __hash__(self): - return 0 + hash(self.success) - - def __init__(self, success=None,): - self.success = success - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 0: - if ftype == TType.STRING: - self.success = iprot.readString(); - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('downloadChunk_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.STRING, 0) - oprot.writeString(self.success) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class getNimbusConf_args: - - thrift_spec = ( - ) - - def __hash__(self): - return 0 - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('getNimbusConf_args') - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class getNimbusConf_result: - """ - Attributes: - - success - """ - - thrift_spec = ( - (0, TType.STRING, 'success', None, None, ), # 0 - ) - - def __hash__(self): - return 0 + hash(self.success) - - def __init__(self, success=None,): - self.success = success - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 0: - if ftype == TType.STRING: - self.success = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('getNimbusConf_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.STRING, 0) - oprot.writeString(self.success.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class getClusterInfo_args: - - thrift_spec = ( - ) - - def __hash__(self): - return 0 - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('getClusterInfo_args') - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class getClusterInfo_result: - """ - Attributes: - - success - """ - - thrift_spec = ( - (0, TType.STRUCT, 'success', (ClusterSummary, ClusterSummary.thrift_spec), None, ), # 0 - ) - - def __hash__(self): - return 0 + hash(self.success) - - def __init__(self, success=None,): - self.success = success - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 0: - if ftype == TType.STRUCT: - self.success = ClusterSummary() - self.success.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('getClusterInfo_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class getTopologyInfo_args: - """ - Attributes: - - id - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'id', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.id) - - def __init__(self, id=None,): - self.id = id - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.id = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('getTopologyInfo_args') - if self.id is not None: - oprot.writeFieldBegin('id', TType.STRING, 1) - oprot.writeString(self.id.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class getTopologyInfo_result: - """ - Attributes: - - success - - e - """ - - thrift_spec = ( - (0, TType.STRUCT, 'success', (TopologyInfo, TopologyInfo.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.success) + hash(self.e) - - def __init__(self, success=None, e=None,): - self.success = success - self.e = e - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 0: - if ftype == TType.STRUCT: - self.success = TopologyInfo() - self.success.read(iprot) - else: - iprot.skip(ftype) - elif fid == 1: - if ftype == TType.STRUCT: - self.e = NotAliveException() - self.e.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('getTopologyInfo_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) - oprot.writeFieldEnd() - if self.e is not None: - oprot.writeFieldBegin('e', TType.STRUCT, 1) - self.e.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class getSupervisorWorkers_args: - """ - Attributes: - - host - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'host', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.host) - - def __init__(self, host=None,): - self.host = host - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.host = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('getSupervisorWorkers_args') - if self.host is not None: - oprot.writeFieldBegin('host', TType.STRING, 1) - oprot.writeString(self.host.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class getSupervisorWorkers_result: - """ - Attributes: - - success - - e - """ - - thrift_spec = ( - (0, TType.STRUCT, 'success', (SupervisorWorkers, SupervisorWorkers.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.success) + hash(self.e) - - def __init__(self, success=None, e=None,): - self.success = success - self.e = e - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 0: - if ftype == TType.STRUCT: - self.success = SupervisorWorkers() - self.success.read(iprot) - else: - iprot.skip(ftype) - elif fid == 1: - if ftype == TType.STRUCT: - self.e = NotAliveException() - self.e.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('getSupervisorWorkers_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) - oprot.writeFieldEnd() - if self.e is not None: - oprot.writeFieldBegin('e', TType.STRUCT, 1) - self.e.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class getTopologyConf_args: - """ - Attributes: - - id - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'id', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.id) - - def __init__(self, id=None,): - self.id = id - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.id = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('getTopologyConf_args') - if self.id is not None: - oprot.writeFieldBegin('id', TType.STRING, 1) - oprot.writeString(self.id.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class getTopologyConf_result: - """ - Attributes: - - success - - e - """ - - thrift_spec = ( - (0, TType.STRING, 'success', None, None, ), # 0 - (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.success) + hash(self.e) - - def __init__(self, success=None, e=None,): - self.success = success - self.e = e - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 0: - if ftype == TType.STRING: - self.success = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 1: - if ftype == TType.STRUCT: - self.e = NotAliveException() - self.e.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('getTopologyConf_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.STRING, 0) - oprot.writeString(self.success.encode('utf-8')) - oprot.writeFieldEnd() - if self.e is not None: - oprot.writeFieldBegin('e', TType.STRUCT, 1) - self.e.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class getTopology_args: - """ - Attributes: - - id - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'id', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.id) - - def __init__(self, id=None,): - self.id = id - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.id = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('getTopology_args') - if self.id is not None: - oprot.writeFieldBegin('id', TType.STRING, 1) - oprot.writeString(self.id.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class getTopology_result: - """ - Attributes: - - success - - e - """ - - thrift_spec = ( - (0, TType.STRUCT, 'success', (StormTopology, StormTopology.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.success) + hash(self.e) - - def __init__(self, success=None, e=None,): - self.success = success - self.e = e - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 0: - if ftype == TType.STRUCT: - self.success = StormTopology() - self.success.read(iprot) - else: - iprot.skip(ftype) - elif fid == 1: - if ftype == TType.STRUCT: - self.e = NotAliveException() - self.e.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('getTopology_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) - oprot.writeFieldEnd() - if self.e is not None: - oprot.writeFieldBegin('e', TType.STRUCT, 1) - self.e.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class getUserTopology_args: - """ - Attributes: - - id - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'id', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.id) - - def __init__(self, id=None,): - self.id = id - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.id = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('getUserTopology_args') - if self.id is not None: - oprot.writeFieldBegin('id', TType.STRING, 1) - oprot.writeString(self.id.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class getUserTopology_result: - """ - Attributes: - - success - - e - """ - - thrift_spec = ( - (0, TType.STRUCT, 'success', (StormTopology, StormTopology.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.success) + hash(self.e) - - def __init__(self, success=None, e=None,): - self.success = success - self.e = e - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 0: - if ftype == TType.STRUCT: - self.success = StormTopology() - self.success.read(iprot) - else: - iprot.skip(ftype) - elif fid == 1: - if ftype == TType.STRUCT: - self.e = NotAliveException() - self.e.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('getUserTopology_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) - oprot.writeFieldEnd() - if self.e is not None: - oprot.writeFieldBegin('e', TType.STRUCT, 1) - self.e.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class getTopologyMetric_args: - """ - Attributes: - - id - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'id', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.id) - - def __init__(self, id=None,): - self.id = id - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.id = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('getTopologyMetric_args') - if self.id is not None: - oprot.writeFieldBegin('id', TType.STRING, 1) - oprot.writeString(self.id.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class getTopologyMetric_result: - """ - Attributes: - - success - - e - """ - - thrift_spec = ( - (0, TType.STRUCT, 'success', (TopologyMetricInfo, TopologyMetricInfo.thrift_spec), None, ), # 0 - (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.success) + hash(self.e) - - def __init__(self, success=None, e=None,): - self.success = success - self.e = e - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 0: - if ftype == TType.STRUCT: - self.success = TopologyMetricInfo() - self.success.read(iprot) - else: - iprot.skip(ftype) - elif fid == 1: - if ftype == TType.STRUCT: - self.e = NotAliveException() - self.e.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('getTopologyMetric_result') - if self.success is not None: - oprot.writeFieldBegin('success', TType.STRUCT, 0) - self.success.write(oprot) - oprot.writeFieldEnd() - if self.e is not None: - oprot.writeFieldBegin('e', TType.STRUCT, 1) - self.e.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) diff --git a/jstorm-client/src/main/py/storm/constants.py b/jstorm-client/src/main/py/storm/constants.py deleted file mode 100644 index 732b3680d..000000000 --- a/jstorm-client/src/main/py/storm/constants.py +++ /dev/null @@ -1,9 +0,0 @@ -# -# Autogenerated by Thrift Compiler (0.7.0) -# -# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING -# - -from thrift.Thrift import * -from ttypes import * - diff --git a/jstorm-client/src/main/py/storm/ttypes.py b/jstorm-client/src/main/py/storm/ttypes.py deleted file mode 100644 index 0e9ba3eea..000000000 --- a/jstorm-client/src/main/py/storm/ttypes.py +++ /dev/null @@ -1,4254 +0,0 @@ -# -# Autogenerated by Thrift Compiler (0.7.0) -# -# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING -# - -from thrift.Thrift import * - -from thrift.transport import TTransport -from thrift.protocol import TBinaryProtocol, TProtocol -try: - from thrift.protocol import fastbinary -except: - fastbinary = None - - -class TopologyInitialStatus: - ACTIVE = 1 - INACTIVE = 2 - - _VALUES_TO_NAMES = { - 1: "ACTIVE", - 2: "INACTIVE", - } - - _NAMES_TO_VALUES = { - "ACTIVE": 1, - "INACTIVE": 2, - } - - -class JavaObjectArg: - """ - Attributes: - - int_arg - - long_arg - - string_arg - - bool_arg - - binary_arg - - double_arg - """ - - thrift_spec = ( - None, # 0 - (1, TType.I32, 'int_arg', None, None, ), # 1 - (2, TType.I64, 'long_arg', None, None, ), # 2 - (3, TType.STRING, 'string_arg', None, None, ), # 3 - (4, TType.BOOL, 'bool_arg', None, None, ), # 4 - (5, TType.STRING, 'binary_arg', None, None, ), # 5 - (6, TType.DOUBLE, 'double_arg', None, None, ), # 6 - ) - - def __hash__(self): - return 0 + hash(self.int_arg) + hash(self.long_arg) + hash(self.string_arg) + hash(self.bool_arg) + hash(self.binary_arg) + hash(self.double_arg) - - def __init__(self, int_arg=None, long_arg=None, string_arg=None, bool_arg=None, binary_arg=None, double_arg=None,): - self.int_arg = int_arg - self.long_arg = long_arg - self.string_arg = string_arg - self.bool_arg = bool_arg - self.binary_arg = binary_arg - self.double_arg = double_arg - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.I32: - self.int_arg = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.I64: - self.long_arg = iprot.readI64(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRING: - self.string_arg = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.BOOL: - self.bool_arg = iprot.readBool(); - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.STRING: - self.binary_arg = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 6: - if ftype == TType.DOUBLE: - self.double_arg = iprot.readDouble(); - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('JavaObjectArg') - if self.int_arg is not None: - oprot.writeFieldBegin('int_arg', TType.I32, 1) - oprot.writeI32(self.int_arg) - oprot.writeFieldEnd() - if self.long_arg is not None: - oprot.writeFieldBegin('long_arg', TType.I64, 2) - oprot.writeI64(self.long_arg) - oprot.writeFieldEnd() - if self.string_arg is not None: - oprot.writeFieldBegin('string_arg', TType.STRING, 3) - oprot.writeString(self.string_arg.encode('utf-8')) - oprot.writeFieldEnd() - if self.bool_arg is not None: - oprot.writeFieldBegin('bool_arg', TType.BOOL, 4) - oprot.writeBool(self.bool_arg) - oprot.writeFieldEnd() - if self.binary_arg is not None: - oprot.writeFieldBegin('binary_arg', TType.STRING, 5) - oprot.writeString(self.binary_arg) - oprot.writeFieldEnd() - if self.double_arg is not None: - oprot.writeFieldBegin('double_arg', TType.DOUBLE, 6) - oprot.writeDouble(self.double_arg) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class JavaObject: - """ - Attributes: - - full_class_name - - args_list - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'full_class_name', None, None, ), # 1 - (2, TType.LIST, 'args_list', (TType.STRUCT,(JavaObjectArg, JavaObjectArg.thrift_spec)), None, ), # 2 - ) - - def __hash__(self): - return 0 + hash(self.full_class_name) + hash(self.args_list) - - def __init__(self, full_class_name=None, args_list=None,): - self.full_class_name = full_class_name - self.args_list = args_list - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.full_class_name = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.LIST: - self.args_list = [] - (_etype3, _size0) = iprot.readListBegin() - for _i4 in xrange(_size0): - _elem5 = JavaObjectArg() - _elem5.read(iprot) - self.args_list.append(_elem5) - iprot.readListEnd() - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('JavaObject') - if self.full_class_name is not None: - oprot.writeFieldBegin('full_class_name', TType.STRING, 1) - oprot.writeString(self.full_class_name.encode('utf-8')) - oprot.writeFieldEnd() - if self.args_list is not None: - oprot.writeFieldBegin('args_list', TType.LIST, 2) - oprot.writeListBegin(TType.STRUCT, len(self.args_list)) - for iter6 in self.args_list: - iter6.write(oprot) - oprot.writeListEnd() - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.full_class_name is None: - raise TProtocol.TProtocolException(message='Required field full_class_name is unset!') - if self.args_list is None: - raise TProtocol.TProtocolException(message='Required field args_list is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class NullStruct: - - thrift_spec = ( - ) - - def __hash__(self): - return 0 - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('NullStruct') - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class GlobalStreamId: - """ - Attributes: - - componentId - - streamId - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'componentId', None, None, ), # 1 - (2, TType.STRING, 'streamId', None, None, ), # 2 - ) - - def __hash__(self): - return 0 + hash(self.componentId) + hash(self.streamId) - - def __init__(self, componentId=None, streamId=None,): - self.componentId = componentId - self.streamId = streamId - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.componentId = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.streamId = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('GlobalStreamId') - if self.componentId is not None: - oprot.writeFieldBegin('componentId', TType.STRING, 1) - oprot.writeString(self.componentId.encode('utf-8')) - oprot.writeFieldEnd() - if self.streamId is not None: - oprot.writeFieldBegin('streamId', TType.STRING, 2) - oprot.writeString(self.streamId.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.componentId is None: - raise TProtocol.TProtocolException(message='Required field componentId is unset!') - if self.streamId is None: - raise TProtocol.TProtocolException(message='Required field streamId is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class Grouping: - """ - Attributes: - - fields - - shuffle - - all - - none - - direct - - custom_object - - custom_serialized - - local_or_shuffle - - localFirst - """ - - thrift_spec = ( - None, # 0 - (1, TType.LIST, 'fields', (TType.STRING,None), None, ), # 1 - (2, TType.STRUCT, 'shuffle', (NullStruct, NullStruct.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'all', (NullStruct, NullStruct.thrift_spec), None, ), # 3 - (4, TType.STRUCT, 'none', (NullStruct, NullStruct.thrift_spec), None, ), # 4 - (5, TType.STRUCT, 'direct', (NullStruct, NullStruct.thrift_spec), None, ), # 5 - (6, TType.STRUCT, 'custom_object', (JavaObject, JavaObject.thrift_spec), None, ), # 6 - (7, TType.STRING, 'custom_serialized', None, None, ), # 7 - (8, TType.STRUCT, 'local_or_shuffle', (NullStruct, NullStruct.thrift_spec), None, ), # 8 - (9, TType.STRUCT, 'localFirst', (NullStruct, NullStruct.thrift_spec), None, ), # 9 - ) - - def __hash__(self): - return 0 + hash(self.fields) + hash(self.shuffle) + hash(self.all) + hash(self.none) + hash(self.direct) + hash(self.custom_object) + hash(self.custom_serialized) + hash(self.local_or_shuffle) + hash(self.localFirst) - - def __init__(self, fields=None, shuffle=None, all=None, none=None, direct=None, custom_object=None, custom_serialized=None, local_or_shuffle=None, localFirst=None,): - self.fields = fields - self.shuffle = shuffle - self.all = all - self.none = none - self.direct = direct - self.custom_object = custom_object - self.custom_serialized = custom_serialized - self.local_or_shuffle = local_or_shuffle - self.localFirst = localFirst - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.LIST: - self.fields = [] - (_etype10, _size7) = iprot.readListBegin() - for _i11 in xrange(_size7): - _elem12 = iprot.readString().decode('utf-8') - self.fields.append(_elem12) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.shuffle = NullStruct() - self.shuffle.read(iprot) - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.all = NullStruct() - self.all.read(iprot) - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRUCT: - self.none = NullStruct() - self.none.read(iprot) - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.STRUCT: - self.direct = NullStruct() - self.direct.read(iprot) - else: - iprot.skip(ftype) - elif fid == 6: - if ftype == TType.STRUCT: - self.custom_object = JavaObject() - self.custom_object.read(iprot) - else: - iprot.skip(ftype) - elif fid == 7: - if ftype == TType.STRING: - self.custom_serialized = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 8: - if ftype == TType.STRUCT: - self.local_or_shuffle = NullStruct() - self.local_or_shuffle.read(iprot) - else: - iprot.skip(ftype) - elif fid == 9: - if ftype == TType.STRUCT: - self.localFirst = NullStruct() - self.localFirst.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('Grouping') - if self.fields is not None: - oprot.writeFieldBegin('fields', TType.LIST, 1) - oprot.writeListBegin(TType.STRING, len(self.fields)) - for iter13 in self.fields: - oprot.writeString(iter13.encode('utf-8')) - oprot.writeListEnd() - oprot.writeFieldEnd() - if self.shuffle is not None: - oprot.writeFieldBegin('shuffle', TType.STRUCT, 2) - self.shuffle.write(oprot) - oprot.writeFieldEnd() - if self.all is not None: - oprot.writeFieldBegin('all', TType.STRUCT, 3) - self.all.write(oprot) - oprot.writeFieldEnd() - if self.none is not None: - oprot.writeFieldBegin('none', TType.STRUCT, 4) - self.none.write(oprot) - oprot.writeFieldEnd() - if self.direct is not None: - oprot.writeFieldBegin('direct', TType.STRUCT, 5) - self.direct.write(oprot) - oprot.writeFieldEnd() - if self.custom_object is not None: - oprot.writeFieldBegin('custom_object', TType.STRUCT, 6) - self.custom_object.write(oprot) - oprot.writeFieldEnd() - if self.custom_serialized is not None: - oprot.writeFieldBegin('custom_serialized', TType.STRING, 7) - oprot.writeString(self.custom_serialized) - oprot.writeFieldEnd() - if self.local_or_shuffle is not None: - oprot.writeFieldBegin('local_or_shuffle', TType.STRUCT, 8) - self.local_or_shuffle.write(oprot) - oprot.writeFieldEnd() - if self.localFirst is not None: - oprot.writeFieldBegin('localFirst', TType.STRUCT, 9) - self.localFirst.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class StreamInfo: - """ - Attributes: - - output_fields - - direct - """ - - thrift_spec = ( - None, # 0 - (1, TType.LIST, 'output_fields', (TType.STRING,None), None, ), # 1 - (2, TType.BOOL, 'direct', None, None, ), # 2 - ) - - def __hash__(self): - return 0 + hash(self.output_fields) + hash(self.direct) - - def __init__(self, output_fields=None, direct=None,): - self.output_fields = output_fields - self.direct = direct - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.LIST: - self.output_fields = [] - (_etype17, _size14) = iprot.readListBegin() - for _i18 in xrange(_size14): - _elem19 = iprot.readString().decode('utf-8') - self.output_fields.append(_elem19) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.BOOL: - self.direct = iprot.readBool(); - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('StreamInfo') - if self.output_fields is not None: - oprot.writeFieldBegin('output_fields', TType.LIST, 1) - oprot.writeListBegin(TType.STRING, len(self.output_fields)) - for iter20 in self.output_fields: - oprot.writeString(iter20.encode('utf-8')) - oprot.writeListEnd() - oprot.writeFieldEnd() - if self.direct is not None: - oprot.writeFieldBegin('direct', TType.BOOL, 2) - oprot.writeBool(self.direct) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.output_fields is None: - raise TProtocol.TProtocolException(message='Required field output_fields is unset!') - if self.direct is None: - raise TProtocol.TProtocolException(message='Required field direct is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class ShellComponent: - """ - Attributes: - - execution_command - - script - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'execution_command', None, None, ), # 1 - (2, TType.STRING, 'script', None, None, ), # 2 - ) - - def __hash__(self): - return 0 + hash(self.execution_command) + hash(self.script) - - def __init__(self, execution_command=None, script=None,): - self.execution_command = execution_command - self.script = script - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.execution_command = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.script = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('ShellComponent') - if self.execution_command is not None: - oprot.writeFieldBegin('execution_command', TType.STRING, 1) - oprot.writeString(self.execution_command.encode('utf-8')) - oprot.writeFieldEnd() - if self.script is not None: - oprot.writeFieldBegin('script', TType.STRING, 2) - oprot.writeString(self.script.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class ComponentObject: - """ - Attributes: - - serialized_java - - shell - - java_object - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'serialized_java', None, None, ), # 1 - (2, TType.STRUCT, 'shell', (ShellComponent, ShellComponent.thrift_spec), None, ), # 2 - (3, TType.STRUCT, 'java_object', (JavaObject, JavaObject.thrift_spec), None, ), # 3 - ) - - def __hash__(self): - return 0 + hash(self.serialized_java) + hash(self.shell) + hash(self.java_object) - - def __init__(self, serialized_java=None, shell=None, java_object=None,): - self.serialized_java = serialized_java - self.shell = shell - self.java_object = java_object - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.serialized_java = iprot.readString(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.shell = ShellComponent() - self.shell.read(iprot) - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRUCT: - self.java_object = JavaObject() - self.java_object.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('ComponentObject') - if self.serialized_java is not None: - oprot.writeFieldBegin('serialized_java', TType.STRING, 1) - oprot.writeString(self.serialized_java) - oprot.writeFieldEnd() - if self.shell is not None: - oprot.writeFieldBegin('shell', TType.STRUCT, 2) - self.shell.write(oprot) - oprot.writeFieldEnd() - if self.java_object is not None: - oprot.writeFieldBegin('java_object', TType.STRUCT, 3) - self.java_object.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class ComponentCommon: - """ - Attributes: - - inputs - - streams - - parallelism_hint - - json_conf - """ - - thrift_spec = ( - None, # 0 - (1, TType.MAP, 'inputs', (TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.STRUCT,(Grouping, Grouping.thrift_spec)), None, ), # 1 - (2, TType.MAP, 'streams', (TType.STRING,None,TType.STRUCT,(StreamInfo, StreamInfo.thrift_spec)), None, ), # 2 - (3, TType.I32, 'parallelism_hint', None, None, ), # 3 - (4, TType.STRING, 'json_conf', None, None, ), # 4 - ) - - def __hash__(self): - return 0 + hash(self.inputs) + hash(self.streams) + hash(self.parallelism_hint) + hash(self.json_conf) - - def __init__(self, inputs=None, streams=None, parallelism_hint=None, json_conf=None,): - self.inputs = inputs - self.streams = streams - self.parallelism_hint = parallelism_hint - self.json_conf = json_conf - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.MAP: - self.inputs = {} - (_ktype22, _vtype23, _size21 ) = iprot.readMapBegin() - for _i25 in xrange(_size21): - _key26 = GlobalStreamId() - _key26.read(iprot) - _val27 = Grouping() - _val27.read(iprot) - self.inputs[_key26] = _val27 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.MAP: - self.streams = {} - (_ktype29, _vtype30, _size28 ) = iprot.readMapBegin() - for _i32 in xrange(_size28): - _key33 = iprot.readString().decode('utf-8') - _val34 = StreamInfo() - _val34.read(iprot) - self.streams[_key33] = _val34 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.I32: - self.parallelism_hint = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRING: - self.json_conf = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('ComponentCommon') - if self.inputs is not None: - oprot.writeFieldBegin('inputs', TType.MAP, 1) - oprot.writeMapBegin(TType.STRUCT, TType.STRUCT, len(self.inputs)) - for kiter35,viter36 in self.inputs.items(): - kiter35.write(oprot) - viter36.write(oprot) - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.streams is not None: - oprot.writeFieldBegin('streams', TType.MAP, 2) - oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.streams)) - for kiter37,viter38 in self.streams.items(): - oprot.writeString(kiter37.encode('utf-8')) - viter38.write(oprot) - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.parallelism_hint is not None: - oprot.writeFieldBegin('parallelism_hint', TType.I32, 3) - oprot.writeI32(self.parallelism_hint) - oprot.writeFieldEnd() - if self.json_conf is not None: - oprot.writeFieldBegin('json_conf', TType.STRING, 4) - oprot.writeString(self.json_conf.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.inputs is None: - raise TProtocol.TProtocolException(message='Required field inputs is unset!') - if self.streams is None: - raise TProtocol.TProtocolException(message='Required field streams is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class SpoutSpec: - """ - Attributes: - - spout_object - - common - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'spout_object', (ComponentObject, ComponentObject.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'common', (ComponentCommon, ComponentCommon.thrift_spec), None, ), # 2 - ) - - def __hash__(self): - return 0 + hash(self.spout_object) + hash(self.common) - - def __init__(self, spout_object=None, common=None,): - self.spout_object = spout_object - self.common = common - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRUCT: - self.spout_object = ComponentObject() - self.spout_object.read(iprot) - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.common = ComponentCommon() - self.common.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('SpoutSpec') - if self.spout_object is not None: - oprot.writeFieldBegin('spout_object', TType.STRUCT, 1) - self.spout_object.write(oprot) - oprot.writeFieldEnd() - if self.common is not None: - oprot.writeFieldBegin('common', TType.STRUCT, 2) - self.common.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.spout_object is None: - raise TProtocol.TProtocolException(message='Required field spout_object is unset!') - if self.common is None: - raise TProtocol.TProtocolException(message='Required field common is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class Bolt: - """ - Attributes: - - bolt_object - - common - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'bolt_object', (ComponentObject, ComponentObject.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'common', (ComponentCommon, ComponentCommon.thrift_spec), None, ), # 2 - ) - - def __hash__(self): - return 0 + hash(self.bolt_object) + hash(self.common) - - def __init__(self, bolt_object=None, common=None,): - self.bolt_object = bolt_object - self.common = common - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRUCT: - self.bolt_object = ComponentObject() - self.bolt_object.read(iprot) - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.common = ComponentCommon() - self.common.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('Bolt') - if self.bolt_object is not None: - oprot.writeFieldBegin('bolt_object', TType.STRUCT, 1) - self.bolt_object.write(oprot) - oprot.writeFieldEnd() - if self.common is not None: - oprot.writeFieldBegin('common', TType.STRUCT, 2) - self.common.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.bolt_object is None: - raise TProtocol.TProtocolException(message='Required field bolt_object is unset!') - if self.common is None: - raise TProtocol.TProtocolException(message='Required field common is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class StateSpoutSpec: - """ - Attributes: - - state_spout_object - - common - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'state_spout_object', (ComponentObject, ComponentObject.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'common', (ComponentCommon, ComponentCommon.thrift_spec), None, ), # 2 - ) - - def __hash__(self): - return 0 + hash(self.state_spout_object) + hash(self.common) - - def __init__(self, state_spout_object=None, common=None,): - self.state_spout_object = state_spout_object - self.common = common - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRUCT: - self.state_spout_object = ComponentObject() - self.state_spout_object.read(iprot) - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.common = ComponentCommon() - self.common.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('StateSpoutSpec') - if self.state_spout_object is not None: - oprot.writeFieldBegin('state_spout_object', TType.STRUCT, 1) - self.state_spout_object.write(oprot) - oprot.writeFieldEnd() - if self.common is not None: - oprot.writeFieldBegin('common', TType.STRUCT, 2) - self.common.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.state_spout_object is None: - raise TProtocol.TProtocolException(message='Required field state_spout_object is unset!') - if self.common is None: - raise TProtocol.TProtocolException(message='Required field common is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class StormTopology: - """ - Attributes: - - spouts - - bolts - - state_spouts - """ - - thrift_spec = ( - None, # 0 - (1, TType.MAP, 'spouts', (TType.STRING,None,TType.STRUCT,(SpoutSpec, SpoutSpec.thrift_spec)), None, ), # 1 - (2, TType.MAP, 'bolts', (TType.STRING,None,TType.STRUCT,(Bolt, Bolt.thrift_spec)), None, ), # 2 - (3, TType.MAP, 'state_spouts', (TType.STRING,None,TType.STRUCT,(StateSpoutSpec, StateSpoutSpec.thrift_spec)), None, ), # 3 - ) - - def __hash__(self): - return 0 + hash(self.spouts) + hash(self.bolts) + hash(self.state_spouts) - - def __init__(self, spouts=None, bolts=None, state_spouts=None,): - self.spouts = spouts - self.bolts = bolts - self.state_spouts = state_spouts - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.MAP: - self.spouts = {} - (_ktype40, _vtype41, _size39 ) = iprot.readMapBegin() - for _i43 in xrange(_size39): - _key44 = iprot.readString().decode('utf-8') - _val45 = SpoutSpec() - _val45.read(iprot) - self.spouts[_key44] = _val45 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.MAP: - self.bolts = {} - (_ktype47, _vtype48, _size46 ) = iprot.readMapBegin() - for _i50 in xrange(_size46): - _key51 = iprot.readString().decode('utf-8') - _val52 = Bolt() - _val52.read(iprot) - self.bolts[_key51] = _val52 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.MAP: - self.state_spouts = {} - (_ktype54, _vtype55, _size53 ) = iprot.readMapBegin() - for _i57 in xrange(_size53): - _key58 = iprot.readString().decode('utf-8') - _val59 = StateSpoutSpec() - _val59.read(iprot) - self.state_spouts[_key58] = _val59 - iprot.readMapEnd() - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('StormTopology') - if self.spouts is not None: - oprot.writeFieldBegin('spouts', TType.MAP, 1) - oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.spouts)) - for kiter60,viter61 in self.spouts.items(): - oprot.writeString(kiter60.encode('utf-8')) - viter61.write(oprot) - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.bolts is not None: - oprot.writeFieldBegin('bolts', TType.MAP, 2) - oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.bolts)) - for kiter62,viter63 in self.bolts.items(): - oprot.writeString(kiter62.encode('utf-8')) - viter63.write(oprot) - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.state_spouts is not None: - oprot.writeFieldBegin('state_spouts', TType.MAP, 3) - oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.state_spouts)) - for kiter64,viter65 in self.state_spouts.items(): - oprot.writeString(kiter64.encode('utf-8')) - viter65.write(oprot) - oprot.writeMapEnd() - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.spouts is None: - raise TProtocol.TProtocolException(message='Required field spouts is unset!') - if self.bolts is None: - raise TProtocol.TProtocolException(message='Required field bolts is unset!') - if self.state_spouts is None: - raise TProtocol.TProtocolException(message='Required field state_spouts is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class TopologyAssignException(Exception): - """ - Attributes: - - msg - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'msg', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.msg) - - def __init__(self, msg=None,): - self.msg = msg - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.msg = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('TopologyAssignException') - if self.msg is not None: - oprot.writeFieldBegin('msg', TType.STRING, 1) - oprot.writeString(self.msg.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.msg is None: - raise TProtocol.TProtocolException(message='Required field msg is unset!') - return - - - def __str__(self): - return repr(self) - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class AlreadyAliveException(Exception): - """ - Attributes: - - msg - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'msg', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.msg) - - def __init__(self, msg=None,): - self.msg = msg - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.msg = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('AlreadyAliveException') - if self.msg is not None: - oprot.writeFieldBegin('msg', TType.STRING, 1) - oprot.writeString(self.msg.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.msg is None: - raise TProtocol.TProtocolException(message='Required field msg is unset!') - return - - - def __str__(self): - return repr(self) - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class NotAliveException(Exception): - """ - Attributes: - - msg - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'msg', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.msg) - - def __init__(self, msg=None,): - self.msg = msg - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.msg = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('NotAliveException') - if self.msg is not None: - oprot.writeFieldBegin('msg', TType.STRING, 1) - oprot.writeString(self.msg.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.msg is None: - raise TProtocol.TProtocolException(message='Required field msg is unset!') - return - - - def __str__(self): - return repr(self) - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class InvalidTopologyException(Exception): - """ - Attributes: - - msg - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'msg', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.msg) - - def __init__(self, msg=None,): - self.msg = msg - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.msg = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('InvalidTopologyException') - if self.msg is not None: - oprot.writeFieldBegin('msg', TType.STRING, 1) - oprot.writeString(self.msg.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.msg is None: - raise TProtocol.TProtocolException(message='Required field msg is unset!') - return - - - def __str__(self): - return repr(self) - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class TopologySummary: - """ - Attributes: - - id - - name - - status - - uptime_secs - - num_tasks - - num_workers - - error_info - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'id', None, None, ), # 1 - (2, TType.STRING, 'name', None, None, ), # 2 - (3, TType.STRING, 'status', None, None, ), # 3 - (4, TType.I32, 'uptime_secs', None, None, ), # 4 - (5, TType.I32, 'num_tasks', None, None, ), # 5 - (6, TType.I32, 'num_workers', None, None, ), # 6 - (7, TType.STRING, 'error_info', None, None, ), # 7 - ) - - def __hash__(self): - return 0 + hash(self.id) + hash(self.name) + hash(self.status) + hash(self.uptime_secs) + hash(self.num_tasks) + hash(self.num_workers) + hash(self.error_info) - - def __init__(self, id=None, name=None, status=None, uptime_secs=None, num_tasks=None, num_workers=None, error_info=None,): - self.id = id - self.name = name - self.status = status - self.uptime_secs = uptime_secs - self.num_tasks = num_tasks - self.num_workers = num_workers - self.error_info = error_info - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.id = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.name = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRING: - self.status = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.I32: - self.uptime_secs = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.I32: - self.num_tasks = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 6: - if ftype == TType.I32: - self.num_workers = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 7: - if ftype == TType.STRING: - self.error_info = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('TopologySummary') - if self.id is not None: - oprot.writeFieldBegin('id', TType.STRING, 1) - oprot.writeString(self.id.encode('utf-8')) - oprot.writeFieldEnd() - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 2) - oprot.writeString(self.name.encode('utf-8')) - oprot.writeFieldEnd() - if self.status is not None: - oprot.writeFieldBegin('status', TType.STRING, 3) - oprot.writeString(self.status.encode('utf-8')) - oprot.writeFieldEnd() - if self.uptime_secs is not None: - oprot.writeFieldBegin('uptime_secs', TType.I32, 4) - oprot.writeI32(self.uptime_secs) - oprot.writeFieldEnd() - if self.num_tasks is not None: - oprot.writeFieldBegin('num_tasks', TType.I32, 5) - oprot.writeI32(self.num_tasks) - oprot.writeFieldEnd() - if self.num_workers is not None: - oprot.writeFieldBegin('num_workers', TType.I32, 6) - oprot.writeI32(self.num_workers) - oprot.writeFieldEnd() - if self.error_info is not None: - oprot.writeFieldBegin('error_info', TType.STRING, 7) - oprot.writeString(self.error_info.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.id is None: - raise TProtocol.TProtocolException(message='Required field id is unset!') - if self.name is None: - raise TProtocol.TProtocolException(message='Required field name is unset!') - if self.status is None: - raise TProtocol.TProtocolException(message='Required field status is unset!') - if self.uptime_secs is None: - raise TProtocol.TProtocolException(message='Required field uptime_secs is unset!') - if self.num_tasks is None: - raise TProtocol.TProtocolException(message='Required field num_tasks is unset!') - if self.num_workers is None: - raise TProtocol.TProtocolException(message='Required field num_workers is unset!') - if self.error_info is None: - raise TProtocol.TProtocolException(message='Required field error_info is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class SupervisorSummary: - """ - Attributes: - - host - - supervisor_id - - uptime_secs - - num_workers - - num_used_workers - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'host', None, None, ), # 1 - (2, TType.STRING, 'supervisor_id', None, None, ), # 2 - (3, TType.I32, 'uptime_secs', None, None, ), # 3 - (4, TType.I32, 'num_workers', None, None, ), # 4 - (5, TType.I32, 'num_used_workers', None, None, ), # 5 - ) - - def __hash__(self): - return 0 + hash(self.host) + hash(self.supervisor_id) + hash(self.uptime_secs) + hash(self.num_workers) + hash(self.num_used_workers) - - def __init__(self, host=None, supervisor_id=None, uptime_secs=None, num_workers=None, num_used_workers=None,): - self.host = host - self.supervisor_id = supervisor_id - self.uptime_secs = uptime_secs - self.num_workers = num_workers - self.num_used_workers = num_used_workers - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.host = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.supervisor_id = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.I32: - self.uptime_secs = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.I32: - self.num_workers = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.I32: - self.num_used_workers = iprot.readI32(); - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('SupervisorSummary') - if self.host is not None: - oprot.writeFieldBegin('host', TType.STRING, 1) - oprot.writeString(self.host.encode('utf-8')) - oprot.writeFieldEnd() - if self.supervisor_id is not None: - oprot.writeFieldBegin('supervisor_id', TType.STRING, 2) - oprot.writeString(self.supervisor_id.encode('utf-8')) - oprot.writeFieldEnd() - if self.uptime_secs is not None: - oprot.writeFieldBegin('uptime_secs', TType.I32, 3) - oprot.writeI32(self.uptime_secs) - oprot.writeFieldEnd() - if self.num_workers is not None: - oprot.writeFieldBegin('num_workers', TType.I32, 4) - oprot.writeI32(self.num_workers) - oprot.writeFieldEnd() - if self.num_used_workers is not None: - oprot.writeFieldBegin('num_used_workers', TType.I32, 5) - oprot.writeI32(self.num_used_workers) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.host is None: - raise TProtocol.TProtocolException(message='Required field host is unset!') - if self.supervisor_id is None: - raise TProtocol.TProtocolException(message='Required field supervisor_id is unset!') - if self.uptime_secs is None: - raise TProtocol.TProtocolException(message='Required field uptime_secs is unset!') - if self.num_workers is None: - raise TProtocol.TProtocolException(message='Required field num_workers is unset!') - if self.num_used_workers is None: - raise TProtocol.TProtocolException(message='Required field num_used_workers is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class ClusterSummary: - """ - Attributes: - - supervisors - - nimbus_uptime_secs - - topologies - - version - """ - - thrift_spec = ( - None, # 0 - (1, TType.LIST, 'supervisors', (TType.STRUCT,(SupervisorSummary, SupervisorSummary.thrift_spec)), None, ), # 1 - (2, TType.I32, 'nimbus_uptime_secs', None, None, ), # 2 - (3, TType.LIST, 'topologies', (TType.STRUCT,(TopologySummary, TopologySummary.thrift_spec)), None, ), # 3 - (4, TType.STRING, 'version', None, None, ), # 4 - ) - - def __hash__(self): - return 0 + hash(self.supervisors) + hash(self.nimbus_uptime_secs) + hash(self.topologies) + hash(self.version) - - def __init__(self, supervisors=None, nimbus_uptime_secs=None, topologies=None, version=None,): - self.supervisors = supervisors - self.nimbus_uptime_secs = nimbus_uptime_secs - self.topologies = topologies - self.version = version - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.LIST: - self.supervisors = [] - (_etype69, _size66) = iprot.readListBegin() - for _i70 in xrange(_size66): - _elem71 = SupervisorSummary() - _elem71.read(iprot) - self.supervisors.append(_elem71) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.I32: - self.nimbus_uptime_secs = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.LIST: - self.topologies = [] - (_etype75, _size72) = iprot.readListBegin() - for _i76 in xrange(_size72): - _elem77 = TopologySummary() - _elem77.read(iprot) - self.topologies.append(_elem77) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.STRING: - self.version = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('ClusterSummary') - if self.supervisors is not None: - oprot.writeFieldBegin('supervisors', TType.LIST, 1) - oprot.writeListBegin(TType.STRUCT, len(self.supervisors)) - for iter78 in self.supervisors: - iter78.write(oprot) - oprot.writeListEnd() - oprot.writeFieldEnd() - if self.nimbus_uptime_secs is not None: - oprot.writeFieldBegin('nimbus_uptime_secs', TType.I32, 2) - oprot.writeI32(self.nimbus_uptime_secs) - oprot.writeFieldEnd() - if self.topologies is not None: - oprot.writeFieldBegin('topologies', TType.LIST, 3) - oprot.writeListBegin(TType.STRUCT, len(self.topologies)) - for iter79 in self.topologies: - iter79.write(oprot) - oprot.writeListEnd() - oprot.writeFieldEnd() - if self.version is not None: - oprot.writeFieldBegin('version', TType.STRING, 4) - oprot.writeString(self.version.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.supervisors is None: - raise TProtocol.TProtocolException(message='Required field supervisors is unset!') - if self.nimbus_uptime_secs is None: - raise TProtocol.TProtocolException(message='Required field nimbus_uptime_secs is unset!') - if self.topologies is None: - raise TProtocol.TProtocolException(message='Required field topologies is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class ErrorInfo: - """ - Attributes: - - error - - error_time_secs - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'error', None, None, ), # 1 - (2, TType.I32, 'error_time_secs', None, None, ), # 2 - ) - - def __hash__(self): - return 0 + hash(self.error) + hash(self.error_time_secs) - - def __init__(self, error=None, error_time_secs=None,): - self.error = error - self.error_time_secs = error_time_secs - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.error = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.I32: - self.error_time_secs = iprot.readI32(); - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('ErrorInfo') - if self.error is not None: - oprot.writeFieldBegin('error', TType.STRING, 1) - oprot.writeString(self.error.encode('utf-8')) - oprot.writeFieldEnd() - if self.error_time_secs is not None: - oprot.writeFieldBegin('error_time_secs', TType.I32, 2) - oprot.writeI32(self.error_time_secs) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.error is None: - raise TProtocol.TProtocolException(message='Required field error is unset!') - if self.error_time_secs is None: - raise TProtocol.TProtocolException(message='Required field error_time_secs is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class BoltStats: - """ - Attributes: - - acked - - failed - - process_ms_avg - - executed - - execute_ms_avg - """ - - thrift_spec = ( - None, # 0 - (1, TType.MAP, 'acked', (TType.STRING,None,TType.MAP,(TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.I64,None)), None, ), # 1 - (2, TType.MAP, 'failed', (TType.STRING,None,TType.MAP,(TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.I64,None)), None, ), # 2 - (3, TType.MAP, 'process_ms_avg', (TType.STRING,None,TType.MAP,(TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.DOUBLE,None)), None, ), # 3 - (4, TType.MAP, 'executed', (TType.STRING,None,TType.MAP,(TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.I64,None)), None, ), # 4 - (5, TType.MAP, 'execute_ms_avg', (TType.STRING,None,TType.MAP,(TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.DOUBLE,None)), None, ), # 5 - ) - - def __hash__(self): - return 0 + hash(self.acked) + hash(self.failed) + hash(self.process_ms_avg) + hash(self.executed) + hash(self.execute_ms_avg) - - def __init__(self, acked=None, failed=None, process_ms_avg=None, executed=None, execute_ms_avg=None,): - self.acked = acked - self.failed = failed - self.process_ms_avg = process_ms_avg - self.executed = executed - self.execute_ms_avg = execute_ms_avg - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.MAP: - self.acked = {} - (_ktype81, _vtype82, _size80 ) = iprot.readMapBegin() - for _i84 in xrange(_size80): - _key85 = iprot.readString().decode('utf-8') - _val86 = {} - (_ktype88, _vtype89, _size87 ) = iprot.readMapBegin() - for _i91 in xrange(_size87): - _key92 = GlobalStreamId() - _key92.read(iprot) - _val93 = iprot.readI64(); - _val86[_key92] = _val93 - iprot.readMapEnd() - self.acked[_key85] = _val86 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.MAP: - self.failed = {} - (_ktype95, _vtype96, _size94 ) = iprot.readMapBegin() - for _i98 in xrange(_size94): - _key99 = iprot.readString().decode('utf-8') - _val100 = {} - (_ktype102, _vtype103, _size101 ) = iprot.readMapBegin() - for _i105 in xrange(_size101): - _key106 = GlobalStreamId() - _key106.read(iprot) - _val107 = iprot.readI64(); - _val100[_key106] = _val107 - iprot.readMapEnd() - self.failed[_key99] = _val100 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.MAP: - self.process_ms_avg = {} - (_ktype109, _vtype110, _size108 ) = iprot.readMapBegin() - for _i112 in xrange(_size108): - _key113 = iprot.readString().decode('utf-8') - _val114 = {} - (_ktype116, _vtype117, _size115 ) = iprot.readMapBegin() - for _i119 in xrange(_size115): - _key120 = GlobalStreamId() - _key120.read(iprot) - _val121 = iprot.readDouble(); - _val114[_key120] = _val121 - iprot.readMapEnd() - self.process_ms_avg[_key113] = _val114 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.MAP: - self.executed = {} - (_ktype123, _vtype124, _size122 ) = iprot.readMapBegin() - for _i126 in xrange(_size122): - _key127 = iprot.readString().decode('utf-8') - _val128 = {} - (_ktype130, _vtype131, _size129 ) = iprot.readMapBegin() - for _i133 in xrange(_size129): - _key134 = GlobalStreamId() - _key134.read(iprot) - _val135 = iprot.readI64(); - _val128[_key134] = _val135 - iprot.readMapEnd() - self.executed[_key127] = _val128 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.MAP: - self.execute_ms_avg = {} - (_ktype137, _vtype138, _size136 ) = iprot.readMapBegin() - for _i140 in xrange(_size136): - _key141 = iprot.readString().decode('utf-8') - _val142 = {} - (_ktype144, _vtype145, _size143 ) = iprot.readMapBegin() - for _i147 in xrange(_size143): - _key148 = GlobalStreamId() - _key148.read(iprot) - _val149 = iprot.readDouble(); - _val142[_key148] = _val149 - iprot.readMapEnd() - self.execute_ms_avg[_key141] = _val142 - iprot.readMapEnd() - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('BoltStats') - if self.acked is not None: - oprot.writeFieldBegin('acked', TType.MAP, 1) - oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.acked)) - for kiter150,viter151 in self.acked.items(): - oprot.writeString(kiter150.encode('utf-8')) - oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter151)) - for kiter152,viter153 in viter151.items(): - kiter152.write(oprot) - oprot.writeI64(viter153) - oprot.writeMapEnd() - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.failed is not None: - oprot.writeFieldBegin('failed', TType.MAP, 2) - oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.failed)) - for kiter154,viter155 in self.failed.items(): - oprot.writeString(kiter154.encode('utf-8')) - oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter155)) - for kiter156,viter157 in viter155.items(): - kiter156.write(oprot) - oprot.writeI64(viter157) - oprot.writeMapEnd() - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.process_ms_avg is not None: - oprot.writeFieldBegin('process_ms_avg', TType.MAP, 3) - oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.process_ms_avg)) - for kiter158,viter159 in self.process_ms_avg.items(): - oprot.writeString(kiter158.encode('utf-8')) - oprot.writeMapBegin(TType.STRUCT, TType.DOUBLE, len(viter159)) - for kiter160,viter161 in viter159.items(): - kiter160.write(oprot) - oprot.writeDouble(viter161) - oprot.writeMapEnd() - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.executed is not None: - oprot.writeFieldBegin('executed', TType.MAP, 4) - oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.executed)) - for kiter162,viter163 in self.executed.items(): - oprot.writeString(kiter162.encode('utf-8')) - oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter163)) - for kiter164,viter165 in viter163.items(): - kiter164.write(oprot) - oprot.writeI64(viter165) - oprot.writeMapEnd() - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.execute_ms_avg is not None: - oprot.writeFieldBegin('execute_ms_avg', TType.MAP, 5) - oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.execute_ms_avg)) - for kiter166,viter167 in self.execute_ms_avg.items(): - oprot.writeString(kiter166.encode('utf-8')) - oprot.writeMapBegin(TType.STRUCT, TType.DOUBLE, len(viter167)) - for kiter168,viter169 in viter167.items(): - kiter168.write(oprot) - oprot.writeDouble(viter169) - oprot.writeMapEnd() - oprot.writeMapEnd() - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.acked is None: - raise TProtocol.TProtocolException(message='Required field acked is unset!') - if self.failed is None: - raise TProtocol.TProtocolException(message='Required field failed is unset!') - if self.process_ms_avg is None: - raise TProtocol.TProtocolException(message='Required field process_ms_avg is unset!') - if self.executed is None: - raise TProtocol.TProtocolException(message='Required field executed is unset!') - if self.execute_ms_avg is None: - raise TProtocol.TProtocolException(message='Required field execute_ms_avg is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class SpoutStats: - """ - Attributes: - - acked - - failed - - complete_ms_avg - """ - - thrift_spec = ( - None, # 0 - (1, TType.MAP, 'acked', (TType.STRING,None,TType.MAP,(TType.STRING,None,TType.I64,None)), None, ), # 1 - (2, TType.MAP, 'failed', (TType.STRING,None,TType.MAP,(TType.STRING,None,TType.I64,None)), None, ), # 2 - (3, TType.MAP, 'complete_ms_avg', (TType.STRING,None,TType.MAP,(TType.STRING,None,TType.DOUBLE,None)), None, ), # 3 - ) - - def __hash__(self): - return 0 + hash(self.acked) + hash(self.failed) + hash(self.complete_ms_avg) - - def __init__(self, acked=None, failed=None, complete_ms_avg=None,): - self.acked = acked - self.failed = failed - self.complete_ms_avg = complete_ms_avg - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.MAP: - self.acked = {} - (_ktype171, _vtype172, _size170 ) = iprot.readMapBegin() - for _i174 in xrange(_size170): - _key175 = iprot.readString().decode('utf-8') - _val176 = {} - (_ktype178, _vtype179, _size177 ) = iprot.readMapBegin() - for _i181 in xrange(_size177): - _key182 = iprot.readString().decode('utf-8') - _val183 = iprot.readI64(); - _val176[_key182] = _val183 - iprot.readMapEnd() - self.acked[_key175] = _val176 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.MAP: - self.failed = {} - (_ktype185, _vtype186, _size184 ) = iprot.readMapBegin() - for _i188 in xrange(_size184): - _key189 = iprot.readString().decode('utf-8') - _val190 = {} - (_ktype192, _vtype193, _size191 ) = iprot.readMapBegin() - for _i195 in xrange(_size191): - _key196 = iprot.readString().decode('utf-8') - _val197 = iprot.readI64(); - _val190[_key196] = _val197 - iprot.readMapEnd() - self.failed[_key189] = _val190 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.MAP: - self.complete_ms_avg = {} - (_ktype199, _vtype200, _size198 ) = iprot.readMapBegin() - for _i202 in xrange(_size198): - _key203 = iprot.readString().decode('utf-8') - _val204 = {} - (_ktype206, _vtype207, _size205 ) = iprot.readMapBegin() - for _i209 in xrange(_size205): - _key210 = iprot.readString().decode('utf-8') - _val211 = iprot.readDouble(); - _val204[_key210] = _val211 - iprot.readMapEnd() - self.complete_ms_avg[_key203] = _val204 - iprot.readMapEnd() - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('SpoutStats') - if self.acked is not None: - oprot.writeFieldBegin('acked', TType.MAP, 1) - oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.acked)) - for kiter212,viter213 in self.acked.items(): - oprot.writeString(kiter212.encode('utf-8')) - oprot.writeMapBegin(TType.STRING, TType.I64, len(viter213)) - for kiter214,viter215 in viter213.items(): - oprot.writeString(kiter214.encode('utf-8')) - oprot.writeI64(viter215) - oprot.writeMapEnd() - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.failed is not None: - oprot.writeFieldBegin('failed', TType.MAP, 2) - oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.failed)) - for kiter216,viter217 in self.failed.items(): - oprot.writeString(kiter216.encode('utf-8')) - oprot.writeMapBegin(TType.STRING, TType.I64, len(viter217)) - for kiter218,viter219 in viter217.items(): - oprot.writeString(kiter218.encode('utf-8')) - oprot.writeI64(viter219) - oprot.writeMapEnd() - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.complete_ms_avg is not None: - oprot.writeFieldBegin('complete_ms_avg', TType.MAP, 3) - oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.complete_ms_avg)) - for kiter220,viter221 in self.complete_ms_avg.items(): - oprot.writeString(kiter220.encode('utf-8')) - oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(viter221)) - for kiter222,viter223 in viter221.items(): - oprot.writeString(kiter222.encode('utf-8')) - oprot.writeDouble(viter223) - oprot.writeMapEnd() - oprot.writeMapEnd() - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.acked is None: - raise TProtocol.TProtocolException(message='Required field acked is unset!') - if self.failed is None: - raise TProtocol.TProtocolException(message='Required field failed is unset!') - if self.complete_ms_avg is None: - raise TProtocol.TProtocolException(message='Required field complete_ms_avg is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class ExecutorSpecificStats: - """ - Attributes: - - bolt - - spout - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'bolt', (BoltStats, BoltStats.thrift_spec), None, ), # 1 - (2, TType.STRUCT, 'spout', (SpoutStats, SpoutStats.thrift_spec), None, ), # 2 - ) - - def __hash__(self): - return 0 + hash(self.bolt) + hash(self.spout) - - def __init__(self, bolt=None, spout=None,): - self.bolt = bolt - self.spout = spout - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRUCT: - self.bolt = BoltStats() - self.bolt.read(iprot) - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRUCT: - self.spout = SpoutStats() - self.spout.read(iprot) - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('ExecutorSpecificStats') - if self.bolt is not None: - oprot.writeFieldBegin('bolt', TType.STRUCT, 1) - self.bolt.write(oprot) - oprot.writeFieldEnd() - if self.spout is not None: - oprot.writeFieldBegin('spout', TType.STRUCT, 2) - self.spout.write(oprot) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class TaskStats: - """ - Attributes: - - emitted - - send_tps - - recv_tps - - acked - - failed - - process_ms_avg - """ - - thrift_spec = ( - None, # 0 - (1, TType.MAP, 'emitted', (TType.STRING,None,TType.MAP,(TType.STRING,None,TType.I64,None)), None, ), # 1 - (2, TType.MAP, 'send_tps', (TType.STRING,None,TType.MAP,(TType.STRING,None,TType.DOUBLE,None)), None, ), # 2 - (3, TType.MAP, 'recv_tps', (TType.STRING,None,TType.MAP,(TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.DOUBLE,None)), None, ), # 3 - (4, TType.MAP, 'acked', (TType.STRING,None,TType.MAP,(TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.I64,None)), None, ), # 4 - (5, TType.MAP, 'failed', (TType.STRING,None,TType.MAP,(TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.I64,None)), None, ), # 5 - (6, TType.MAP, 'process_ms_avg', (TType.STRING,None,TType.MAP,(TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.DOUBLE,None)), None, ), # 6 - ) - - def __hash__(self): - return 0 + hash(self.emitted) + hash(self.send_tps) + hash(self.recv_tps) + hash(self.acked) + hash(self.failed) + hash(self.process_ms_avg) - - def __init__(self, emitted=None, send_tps=None, recv_tps=None, acked=None, failed=None, process_ms_avg=None,): - self.emitted = emitted - self.send_tps = send_tps - self.recv_tps = recv_tps - self.acked = acked - self.failed = failed - self.process_ms_avg = process_ms_avg - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.MAP: - self.emitted = {} - (_ktype225, _vtype226, _size224 ) = iprot.readMapBegin() - for _i228 in xrange(_size224): - _key229 = iprot.readString().decode('utf-8') - _val230 = {} - (_ktype232, _vtype233, _size231 ) = iprot.readMapBegin() - for _i235 in xrange(_size231): - _key236 = iprot.readString().decode('utf-8') - _val237 = iprot.readI64(); - _val230[_key236] = _val237 - iprot.readMapEnd() - self.emitted[_key229] = _val230 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.MAP: - self.send_tps = {} - (_ktype239, _vtype240, _size238 ) = iprot.readMapBegin() - for _i242 in xrange(_size238): - _key243 = iprot.readString().decode('utf-8') - _val244 = {} - (_ktype246, _vtype247, _size245 ) = iprot.readMapBegin() - for _i249 in xrange(_size245): - _key250 = iprot.readString().decode('utf-8') - _val251 = iprot.readDouble(); - _val244[_key250] = _val251 - iprot.readMapEnd() - self.send_tps[_key243] = _val244 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.MAP: - self.recv_tps = {} - (_ktype253, _vtype254, _size252 ) = iprot.readMapBegin() - for _i256 in xrange(_size252): - _key257 = iprot.readString().decode('utf-8') - _val258 = {} - (_ktype260, _vtype261, _size259 ) = iprot.readMapBegin() - for _i263 in xrange(_size259): - _key264 = GlobalStreamId() - _key264.read(iprot) - _val265 = iprot.readDouble(); - _val258[_key264] = _val265 - iprot.readMapEnd() - self.recv_tps[_key257] = _val258 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.MAP: - self.acked = {} - (_ktype267, _vtype268, _size266 ) = iprot.readMapBegin() - for _i270 in xrange(_size266): - _key271 = iprot.readString().decode('utf-8') - _val272 = {} - (_ktype274, _vtype275, _size273 ) = iprot.readMapBegin() - for _i277 in xrange(_size273): - _key278 = GlobalStreamId() - _key278.read(iprot) - _val279 = iprot.readI64(); - _val272[_key278] = _val279 - iprot.readMapEnd() - self.acked[_key271] = _val272 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.MAP: - self.failed = {} - (_ktype281, _vtype282, _size280 ) = iprot.readMapBegin() - for _i284 in xrange(_size280): - _key285 = iprot.readString().decode('utf-8') - _val286 = {} - (_ktype288, _vtype289, _size287 ) = iprot.readMapBegin() - for _i291 in xrange(_size287): - _key292 = GlobalStreamId() - _key292.read(iprot) - _val293 = iprot.readI64(); - _val286[_key292] = _val293 - iprot.readMapEnd() - self.failed[_key285] = _val286 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 6: - if ftype == TType.MAP: - self.process_ms_avg = {} - (_ktype295, _vtype296, _size294 ) = iprot.readMapBegin() - for _i298 in xrange(_size294): - _key299 = iprot.readString().decode('utf-8') - _val300 = {} - (_ktype302, _vtype303, _size301 ) = iprot.readMapBegin() - for _i305 in xrange(_size301): - _key306 = GlobalStreamId() - _key306.read(iprot) - _val307 = iprot.readDouble(); - _val300[_key306] = _val307 - iprot.readMapEnd() - self.process_ms_avg[_key299] = _val300 - iprot.readMapEnd() - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('TaskStats') - if self.emitted is not None: - oprot.writeFieldBegin('emitted', TType.MAP, 1) - oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.emitted)) - for kiter308,viter309 in self.emitted.items(): - oprot.writeString(kiter308.encode('utf-8')) - oprot.writeMapBegin(TType.STRING, TType.I64, len(viter309)) - for kiter310,viter311 in viter309.items(): - oprot.writeString(kiter310.encode('utf-8')) - oprot.writeI64(viter311) - oprot.writeMapEnd() - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.send_tps is not None: - oprot.writeFieldBegin('send_tps', TType.MAP, 2) - oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.send_tps)) - for kiter312,viter313 in self.send_tps.items(): - oprot.writeString(kiter312.encode('utf-8')) - oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(viter313)) - for kiter314,viter315 in viter313.items(): - oprot.writeString(kiter314.encode('utf-8')) - oprot.writeDouble(viter315) - oprot.writeMapEnd() - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.recv_tps is not None: - oprot.writeFieldBegin('recv_tps', TType.MAP, 3) - oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.recv_tps)) - for kiter316,viter317 in self.recv_tps.items(): - oprot.writeString(kiter316.encode('utf-8')) - oprot.writeMapBegin(TType.STRUCT, TType.DOUBLE, len(viter317)) - for kiter318,viter319 in viter317.items(): - kiter318.write(oprot) - oprot.writeDouble(viter319) - oprot.writeMapEnd() - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.acked is not None: - oprot.writeFieldBegin('acked', TType.MAP, 4) - oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.acked)) - for kiter320,viter321 in self.acked.items(): - oprot.writeString(kiter320.encode('utf-8')) - oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter321)) - for kiter322,viter323 in viter321.items(): - kiter322.write(oprot) - oprot.writeI64(viter323) - oprot.writeMapEnd() - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.failed is not None: - oprot.writeFieldBegin('failed', TType.MAP, 5) - oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.failed)) - for kiter324,viter325 in self.failed.items(): - oprot.writeString(kiter324.encode('utf-8')) - oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter325)) - for kiter326,viter327 in viter325.items(): - kiter326.write(oprot) - oprot.writeI64(viter327) - oprot.writeMapEnd() - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.process_ms_avg is not None: - oprot.writeFieldBegin('process_ms_avg', TType.MAP, 6) - oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.process_ms_avg)) - for kiter328,viter329 in self.process_ms_avg.items(): - oprot.writeString(kiter328.encode('utf-8')) - oprot.writeMapBegin(TType.STRUCT, TType.DOUBLE, len(viter329)) - for kiter330,viter331 in viter329.items(): - kiter330.write(oprot) - oprot.writeDouble(viter331) - oprot.writeMapEnd() - oprot.writeMapEnd() - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.emitted is None: - raise TProtocol.TProtocolException(message='Required field emitted is unset!') - if self.send_tps is None: - raise TProtocol.TProtocolException(message='Required field send_tps is unset!') - if self.recv_tps is None: - raise TProtocol.TProtocolException(message='Required field recv_tps is unset!') - if self.acked is None: - raise TProtocol.TProtocolException(message='Required field acked is unset!') - if self.failed is None: - raise TProtocol.TProtocolException(message='Required field failed is unset!') - if self.process_ms_avg is None: - raise TProtocol.TProtocolException(message='Required field process_ms_avg is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class ExecutorInfo: - """ - Attributes: - - task_start - - task_end - """ - - thrift_spec = ( - None, # 0 - (1, TType.I32, 'task_start', None, None, ), # 1 - (2, TType.I32, 'task_end', None, None, ), # 2 - ) - - def __hash__(self): - return 0 + hash(self.task_start) + hash(self.task_end) - - def __init__(self, task_start=None, task_end=None,): - self.task_start = task_start - self.task_end = task_end - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.I32: - self.task_start = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.I32: - self.task_end = iprot.readI32(); - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('ExecutorInfo') - if self.task_start is not None: - oprot.writeFieldBegin('task_start', TType.I32, 1) - oprot.writeI32(self.task_start) - oprot.writeFieldEnd() - if self.task_end is not None: - oprot.writeFieldBegin('task_end', TType.I32, 2) - oprot.writeI32(self.task_end) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.task_start is None: - raise TProtocol.TProtocolException(message='Required field task_start is unset!') - if self.task_end is None: - raise TProtocol.TProtocolException(message='Required field task_end is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class TaskSummary: - """ - Attributes: - - task_id - - component_id - - host - - port - - uptime_secs - - errors - - stats - - component_type - """ - - thrift_spec = ( - None, # 0 - (1, TType.I32, 'task_id', None, None, ), # 1 - (2, TType.STRING, 'component_id', None, None, ), # 2 - (3, TType.STRING, 'host', None, None, ), # 3 - (4, TType.I32, 'port', None, None, ), # 4 - (5, TType.I32, 'uptime_secs', None, None, ), # 5 - (6, TType.LIST, 'errors', (TType.STRUCT,(ErrorInfo, ErrorInfo.thrift_spec)), None, ), # 6 - (7, TType.STRUCT, 'stats', (TaskStats, TaskStats.thrift_spec), None, ), # 7 - (8, TType.STRING, 'component_type', None, None, ), # 8 - ) - - def __hash__(self): - return 0 + hash(self.task_id) + hash(self.component_id) + hash(self.host) + hash(self.port) + hash(self.uptime_secs) + hash(self.errors) + hash(self.stats) + hash(self.component_type) - - def __init__(self, task_id=None, component_id=None, host=None, port=None, uptime_secs=None, errors=None, stats=None, component_type=None,): - self.task_id = task_id - self.component_id = component_id - self.host = host - self.port = port - self.uptime_secs = uptime_secs - self.errors = errors - self.stats = stats - self.component_type = component_type - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.I32: - self.task_id = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.component_id = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.STRING: - self.host = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.I32: - self.port = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.I32: - self.uptime_secs = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 6: - if ftype == TType.LIST: - self.errors = [] - (_etype335, _size332) = iprot.readListBegin() - for _i336 in xrange(_size332): - _elem337 = ErrorInfo() - _elem337.read(iprot) - self.errors.append(_elem337) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 7: - if ftype == TType.STRUCT: - self.stats = TaskStats() - self.stats.read(iprot) - else: - iprot.skip(ftype) - elif fid == 8: - if ftype == TType.STRING: - self.component_type = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('TaskSummary') - if self.task_id is not None: - oprot.writeFieldBegin('task_id', TType.I32, 1) - oprot.writeI32(self.task_id) - oprot.writeFieldEnd() - if self.component_id is not None: - oprot.writeFieldBegin('component_id', TType.STRING, 2) - oprot.writeString(self.component_id.encode('utf-8')) - oprot.writeFieldEnd() - if self.host is not None: - oprot.writeFieldBegin('host', TType.STRING, 3) - oprot.writeString(self.host.encode('utf-8')) - oprot.writeFieldEnd() - if self.port is not None: - oprot.writeFieldBegin('port', TType.I32, 4) - oprot.writeI32(self.port) - oprot.writeFieldEnd() - if self.uptime_secs is not None: - oprot.writeFieldBegin('uptime_secs', TType.I32, 5) - oprot.writeI32(self.uptime_secs) - oprot.writeFieldEnd() - if self.errors is not None: - oprot.writeFieldBegin('errors', TType.LIST, 6) - oprot.writeListBegin(TType.STRUCT, len(self.errors)) - for iter338 in self.errors: - iter338.write(oprot) - oprot.writeListEnd() - oprot.writeFieldEnd() - if self.stats is not None: - oprot.writeFieldBegin('stats', TType.STRUCT, 7) - self.stats.write(oprot) - oprot.writeFieldEnd() - if self.component_type is not None: - oprot.writeFieldBegin('component_type', TType.STRING, 8) - oprot.writeString(self.component_type.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.task_id is None: - raise TProtocol.TProtocolException(message='Required field task_id is unset!') - if self.component_id is None: - raise TProtocol.TProtocolException(message='Required field component_id is unset!') - if self.host is None: - raise TProtocol.TProtocolException(message='Required field host is unset!') - if self.port is None: - raise TProtocol.TProtocolException(message='Required field port is unset!') - if self.uptime_secs is None: - raise TProtocol.TProtocolException(message='Required field uptime_secs is unset!') - if self.errors is None: - raise TProtocol.TProtocolException(message='Required field errors is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class WorkerSummary: - """ - Attributes: - - port - - topology - - tasks - """ - - thrift_spec = ( - None, # 0 - (1, TType.I32, 'port', None, None, ), # 1 - (2, TType.STRING, 'topology', None, None, ), # 2 - (3, TType.LIST, 'tasks', (TType.STRUCT,(TaskSummary, TaskSummary.thrift_spec)), None, ), # 3 - ) - - def __hash__(self): - return 0 + hash(self.port) + hash(self.topology) + hash(self.tasks) - - def __init__(self, port=None, topology=None, tasks=None,): - self.port = port - self.topology = topology - self.tasks = tasks - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.I32: - self.port = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.topology = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.LIST: - self.tasks = [] - (_etype342, _size339) = iprot.readListBegin() - for _i343 in xrange(_size339): - _elem344 = TaskSummary() - _elem344.read(iprot) - self.tasks.append(_elem344) - iprot.readListEnd() - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('WorkerSummary') - if self.port is not None: - oprot.writeFieldBegin('port', TType.I32, 1) - oprot.writeI32(self.port) - oprot.writeFieldEnd() - if self.topology is not None: - oprot.writeFieldBegin('topology', TType.STRING, 2) - oprot.writeString(self.topology.encode('utf-8')) - oprot.writeFieldEnd() - if self.tasks is not None: - oprot.writeFieldBegin('tasks', TType.LIST, 3) - oprot.writeListBegin(TType.STRUCT, len(self.tasks)) - for iter345 in self.tasks: - iter345.write(oprot) - oprot.writeListEnd() - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.port is None: - raise TProtocol.TProtocolException(message='Required field port is unset!') - if self.topology is None: - raise TProtocol.TProtocolException(message='Required field topology is unset!') - if self.tasks is None: - raise TProtocol.TProtocolException(message='Required field tasks is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class TopologyInfo: - """ - Attributes: - - id - - name - - uptime_secs - - workers - - status - - tasks - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'id', None, None, ), # 1 - (2, TType.STRING, 'name', None, None, ), # 2 - (3, TType.I32, 'uptime_secs', None, None, ), # 3 - (4, TType.LIST, 'workers', (TType.STRUCT,(WorkerSummary, WorkerSummary.thrift_spec)), None, ), # 4 - (5, TType.STRING, 'status', None, None, ), # 5 - (6, TType.LIST, 'tasks', (TType.STRUCT,(TaskSummary, TaskSummary.thrift_spec)), None, ), # 6 - ) - - def __hash__(self): - return 0 + hash(self.id) + hash(self.name) + hash(self.uptime_secs) + hash(self.workers) + hash(self.status) + hash(self.tasks) - - def __init__(self, id=None, name=None, uptime_secs=None, workers=None, status=None, tasks=None,): - self.id = id - self.name = name - self.uptime_secs = uptime_secs - self.workers = workers - self.status = status - self.tasks = tasks - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.id = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.name = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.I32: - self.uptime_secs = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.LIST: - self.workers = [] - (_etype349, _size346) = iprot.readListBegin() - for _i350 in xrange(_size346): - _elem351 = WorkerSummary() - _elem351.read(iprot) - self.workers.append(_elem351) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.STRING: - self.status = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 6: - if ftype == TType.LIST: - self.tasks = [] - (_etype355, _size352) = iprot.readListBegin() - for _i356 in xrange(_size352): - _elem357 = TaskSummary() - _elem357.read(iprot) - self.tasks.append(_elem357) - iprot.readListEnd() - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('TopologyInfo') - if self.id is not None: - oprot.writeFieldBegin('id', TType.STRING, 1) - oprot.writeString(self.id.encode('utf-8')) - oprot.writeFieldEnd() - if self.name is not None: - oprot.writeFieldBegin('name', TType.STRING, 2) - oprot.writeString(self.name.encode('utf-8')) - oprot.writeFieldEnd() - if self.uptime_secs is not None: - oprot.writeFieldBegin('uptime_secs', TType.I32, 3) - oprot.writeI32(self.uptime_secs) - oprot.writeFieldEnd() - if self.workers is not None: - oprot.writeFieldBegin('workers', TType.LIST, 4) - oprot.writeListBegin(TType.STRUCT, len(self.workers)) - for iter358 in self.workers: - iter358.write(oprot) - oprot.writeListEnd() - oprot.writeFieldEnd() - if self.status is not None: - oprot.writeFieldBegin('status', TType.STRING, 5) - oprot.writeString(self.status.encode('utf-8')) - oprot.writeFieldEnd() - if self.tasks is not None: - oprot.writeFieldBegin('tasks', TType.LIST, 6) - oprot.writeListBegin(TType.STRUCT, len(self.tasks)) - for iter359 in self.tasks: - iter359.write(oprot) - oprot.writeListEnd() - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.id is None: - raise TProtocol.TProtocolException(message='Required field id is unset!') - if self.name is None: - raise TProtocol.TProtocolException(message='Required field name is unset!') - if self.uptime_secs is None: - raise TProtocol.TProtocolException(message='Required field uptime_secs is unset!') - if self.workers is None: - raise TProtocol.TProtocolException(message='Required field workers is unset!') - if self.status is None: - raise TProtocol.TProtocolException(message='Required field status is unset!') - if self.tasks is None: - raise TProtocol.TProtocolException(message='Required field tasks is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class SupervisorWorkers: - """ - Attributes: - - supervisor - - workers - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRUCT, 'supervisor', (SupervisorSummary, SupervisorSummary.thrift_spec), None, ), # 1 - (2, TType.LIST, 'workers', (TType.STRUCT,(WorkerSummary, WorkerSummary.thrift_spec)), None, ), # 2 - ) - - def __hash__(self): - return 0 + hash(self.supervisor) + hash(self.workers) - - def __init__(self, supervisor=None, workers=None,): - self.supervisor = supervisor - self.workers = workers - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRUCT: - self.supervisor = SupervisorSummary() - self.supervisor.read(iprot) - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.LIST: - self.workers = [] - (_etype363, _size360) = iprot.readListBegin() - for _i364 in xrange(_size360): - _elem365 = WorkerSummary() - _elem365.read(iprot) - self.workers.append(_elem365) - iprot.readListEnd() - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('SupervisorWorkers') - if self.supervisor is not None: - oprot.writeFieldBegin('supervisor', TType.STRUCT, 1) - self.supervisor.write(oprot) - oprot.writeFieldEnd() - if self.workers is not None: - oprot.writeFieldBegin('workers', TType.LIST, 2) - oprot.writeListBegin(TType.STRUCT, len(self.workers)) - for iter366 in self.workers: - iter366.write(oprot) - oprot.writeListEnd() - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.supervisor is None: - raise TProtocol.TProtocolException(message='Required field supervisor is unset!') - if self.workers is None: - raise TProtocol.TProtocolException(message='Required field workers is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class KillOptions: - """ - Attributes: - - wait_secs - """ - - thrift_spec = ( - None, # 0 - (1, TType.I32, 'wait_secs', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.wait_secs) - - def __init__(self, wait_secs=None,): - self.wait_secs = wait_secs - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.I32: - self.wait_secs = iprot.readI32(); - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('KillOptions') - if self.wait_secs is not None: - oprot.writeFieldBegin('wait_secs', TType.I32, 1) - oprot.writeI32(self.wait_secs) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class RebalanceOptions: - """ - Attributes: - - wait_secs - - num_workers - """ - - thrift_spec = ( - None, # 0 - (1, TType.I32, 'wait_secs', None, None, ), # 1 - (2, TType.I32, 'num_workers', None, None, ), # 2 - ) - - def __hash__(self): - return 0 + hash(self.wait_secs) + hash(self.num_workers) - - def __init__(self, wait_secs=None, num_workers=None,): - self.wait_secs = wait_secs - self.num_workers = num_workers - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.I32: - self.wait_secs = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.I32: - self.num_workers = iprot.readI32(); - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('RebalanceOptions') - if self.wait_secs is not None: - oprot.writeFieldBegin('wait_secs', TType.I32, 1) - oprot.writeI32(self.wait_secs) - oprot.writeFieldEnd() - if self.num_workers is not None: - oprot.writeFieldBegin('num_workers', TType.I32, 2) - oprot.writeI32(self.num_workers) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class SubmitOptions: - """ - Attributes: - - initial_status - """ - - thrift_spec = ( - None, # 0 - (1, TType.I32, 'initial_status', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.initial_status) - - def __init__(self, initial_status=None,): - self.initial_status = initial_status - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.I32: - self.initial_status = iprot.readI32(); - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('SubmitOptions') - if self.initial_status is not None: - oprot.writeFieldBegin('initial_status', TType.I32, 1) - oprot.writeI32(self.initial_status) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.initial_status is None: - raise TProtocol.TProtocolException(message='Required field initial_status is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class MonitorOptions: - """ - Attributes: - - isEnable - """ - - thrift_spec = ( - None, # 0 - (1, TType.BOOL, 'isEnable', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.isEnable) - - def __init__(self, isEnable=None,): - self.isEnable = isEnable - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.BOOL: - self.isEnable = iprot.readBool(); - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('MonitorOptions') - if self.isEnable is not None: - oprot.writeFieldBegin('isEnable', TType.BOOL, 1) - oprot.writeBool(self.isEnable) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class TaskMetricData: - """ - Attributes: - - task_id - - component_id - - gauge - - counter - - meter - - timer - - histogram - """ - - thrift_spec = ( - None, # 0 - (1, TType.I32, 'task_id', None, None, ), # 1 - (2, TType.STRING, 'component_id', None, None, ), # 2 - (3, TType.MAP, 'gauge', (TType.STRING,None,TType.DOUBLE,None), None, ), # 3 - (4, TType.MAP, 'counter', (TType.STRING,None,TType.DOUBLE,None), None, ), # 4 - (5, TType.MAP, 'meter', (TType.STRING,None,TType.DOUBLE,None), None, ), # 5 - (6, TType.MAP, 'timer', (TType.STRING,None,TType.DOUBLE,None), None, ), # 6 - (7, TType.MAP, 'histogram', (TType.STRING,None,TType.DOUBLE,None), None, ), # 7 - ) - - def __hash__(self): - return 0 + hash(self.task_id) + hash(self.component_id) + hash(self.gauge) + hash(self.counter) + hash(self.meter) + hash(self.timer) + hash(self.histogram) - - def __init__(self, task_id=None, component_id=None, gauge=None, counter=None, meter=None, timer=None, histogram=None,): - self.task_id = task_id - self.component_id = component_id - self.gauge = gauge - self.counter = counter - self.meter = meter - self.timer = timer - self.histogram = histogram - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.I32: - self.task_id = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.component_id = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.MAP: - self.gauge = {} - (_ktype368, _vtype369, _size367 ) = iprot.readMapBegin() - for _i371 in xrange(_size367): - _key372 = iprot.readString().decode('utf-8') - _val373 = iprot.readDouble(); - self.gauge[_key372] = _val373 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.MAP: - self.counter = {} - (_ktype375, _vtype376, _size374 ) = iprot.readMapBegin() - for _i378 in xrange(_size374): - _key379 = iprot.readString().decode('utf-8') - _val380 = iprot.readDouble(); - self.counter[_key379] = _val380 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.MAP: - self.meter = {} - (_ktype382, _vtype383, _size381 ) = iprot.readMapBegin() - for _i385 in xrange(_size381): - _key386 = iprot.readString().decode('utf-8') - _val387 = iprot.readDouble(); - self.meter[_key386] = _val387 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 6: - if ftype == TType.MAP: - self.timer = {} - (_ktype389, _vtype390, _size388 ) = iprot.readMapBegin() - for _i392 in xrange(_size388): - _key393 = iprot.readString().decode('utf-8') - _val394 = iprot.readDouble(); - self.timer[_key393] = _val394 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 7: - if ftype == TType.MAP: - self.histogram = {} - (_ktype396, _vtype397, _size395 ) = iprot.readMapBegin() - for _i399 in xrange(_size395): - _key400 = iprot.readString().decode('utf-8') - _val401 = iprot.readDouble(); - self.histogram[_key400] = _val401 - iprot.readMapEnd() - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('TaskMetricData') - if self.task_id is not None: - oprot.writeFieldBegin('task_id', TType.I32, 1) - oprot.writeI32(self.task_id) - oprot.writeFieldEnd() - if self.component_id is not None: - oprot.writeFieldBegin('component_id', TType.STRING, 2) - oprot.writeString(self.component_id.encode('utf-8')) - oprot.writeFieldEnd() - if self.gauge is not None: - oprot.writeFieldBegin('gauge', TType.MAP, 3) - oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(self.gauge)) - for kiter402,viter403 in self.gauge.items(): - oprot.writeString(kiter402.encode('utf-8')) - oprot.writeDouble(viter403) - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.counter is not None: - oprot.writeFieldBegin('counter', TType.MAP, 4) - oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(self.counter)) - for kiter404,viter405 in self.counter.items(): - oprot.writeString(kiter404.encode('utf-8')) - oprot.writeDouble(viter405) - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.meter is not None: - oprot.writeFieldBegin('meter', TType.MAP, 5) - oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(self.meter)) - for kiter406,viter407 in self.meter.items(): - oprot.writeString(kiter406.encode('utf-8')) - oprot.writeDouble(viter407) - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.timer is not None: - oprot.writeFieldBegin('timer', TType.MAP, 6) - oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(self.timer)) - for kiter408,viter409 in self.timer.items(): - oprot.writeString(kiter408.encode('utf-8')) - oprot.writeDouble(viter409) - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.histogram is not None: - oprot.writeFieldBegin('histogram', TType.MAP, 7) - oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(self.histogram)) - for kiter410,viter411 in self.histogram.items(): - oprot.writeString(kiter410.encode('utf-8')) - oprot.writeDouble(viter411) - oprot.writeMapEnd() - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.task_id is None: - raise TProtocol.TProtocolException(message='Required field task_id is unset!') - if self.component_id is None: - raise TProtocol.TProtocolException(message='Required field component_id is unset!') - if self.gauge is None: - raise TProtocol.TProtocolException(message='Required field gauge is unset!') - if self.counter is None: - raise TProtocol.TProtocolException(message='Required field counter is unset!') - if self.meter is None: - raise TProtocol.TProtocolException(message='Required field meter is unset!') - if self.timer is None: - raise TProtocol.TProtocolException(message='Required field timer is unset!') - if self.histogram is None: - raise TProtocol.TProtocolException(message='Required field histogram is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class WorkerMetricData: - """ - Attributes: - - hostname - - port - - gauge - - counter - - meter - - timer - - histogram - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'hostname', None, None, ), # 1 - (2, TType.I32, 'port', None, None, ), # 2 - (3, TType.MAP, 'gauge', (TType.STRING,None,TType.DOUBLE,None), None, ), # 3 - (4, TType.MAP, 'counter', (TType.STRING,None,TType.DOUBLE,None), None, ), # 4 - (5, TType.MAP, 'meter', (TType.STRING,None,TType.DOUBLE,None), None, ), # 5 - (6, TType.MAP, 'timer', (TType.STRING,None,TType.DOUBLE,None), None, ), # 6 - (7, TType.MAP, 'histogram', (TType.STRING,None,TType.DOUBLE,None), None, ), # 7 - ) - - def __hash__(self): - return 0 + hash(self.hostname) + hash(self.port) + hash(self.gauge) + hash(self.counter) + hash(self.meter) + hash(self.timer) + hash(self.histogram) - - def __init__(self, hostname=None, port=None, gauge=None, counter=None, meter=None, timer=None, histogram=None,): - self.hostname = hostname - self.port = port - self.gauge = gauge - self.counter = counter - self.meter = meter - self.timer = timer - self.histogram = histogram - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.hostname = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.I32: - self.port = iprot.readI32(); - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.MAP: - self.gauge = {} - (_ktype413, _vtype414, _size412 ) = iprot.readMapBegin() - for _i416 in xrange(_size412): - _key417 = iprot.readString().decode('utf-8') - _val418 = iprot.readDouble(); - self.gauge[_key417] = _val418 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 4: - if ftype == TType.MAP: - self.counter = {} - (_ktype420, _vtype421, _size419 ) = iprot.readMapBegin() - for _i423 in xrange(_size419): - _key424 = iprot.readString().decode('utf-8') - _val425 = iprot.readDouble(); - self.counter[_key424] = _val425 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 5: - if ftype == TType.MAP: - self.meter = {} - (_ktype427, _vtype428, _size426 ) = iprot.readMapBegin() - for _i430 in xrange(_size426): - _key431 = iprot.readString().decode('utf-8') - _val432 = iprot.readDouble(); - self.meter[_key431] = _val432 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 6: - if ftype == TType.MAP: - self.timer = {} - (_ktype434, _vtype435, _size433 ) = iprot.readMapBegin() - for _i437 in xrange(_size433): - _key438 = iprot.readString().decode('utf-8') - _val439 = iprot.readDouble(); - self.timer[_key438] = _val439 - iprot.readMapEnd() - else: - iprot.skip(ftype) - elif fid == 7: - if ftype == TType.MAP: - self.histogram = {} - (_ktype441, _vtype442, _size440 ) = iprot.readMapBegin() - for _i444 in xrange(_size440): - _key445 = iprot.readString().decode('utf-8') - _val446 = iprot.readDouble(); - self.histogram[_key445] = _val446 - iprot.readMapEnd() - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('WorkerMetricData') - if self.hostname is not None: - oprot.writeFieldBegin('hostname', TType.STRING, 1) - oprot.writeString(self.hostname.encode('utf-8')) - oprot.writeFieldEnd() - if self.port is not None: - oprot.writeFieldBegin('port', TType.I32, 2) - oprot.writeI32(self.port) - oprot.writeFieldEnd() - if self.gauge is not None: - oprot.writeFieldBegin('gauge', TType.MAP, 3) - oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(self.gauge)) - for kiter447,viter448 in self.gauge.items(): - oprot.writeString(kiter447.encode('utf-8')) - oprot.writeDouble(viter448) - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.counter is not None: - oprot.writeFieldBegin('counter', TType.MAP, 4) - oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(self.counter)) - for kiter449,viter450 in self.counter.items(): - oprot.writeString(kiter449.encode('utf-8')) - oprot.writeDouble(viter450) - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.meter is not None: - oprot.writeFieldBegin('meter', TType.MAP, 5) - oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(self.meter)) - for kiter451,viter452 in self.meter.items(): - oprot.writeString(kiter451.encode('utf-8')) - oprot.writeDouble(viter452) - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.timer is not None: - oprot.writeFieldBegin('timer', TType.MAP, 6) - oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(self.timer)) - for kiter453,viter454 in self.timer.items(): - oprot.writeString(kiter453.encode('utf-8')) - oprot.writeDouble(viter454) - oprot.writeMapEnd() - oprot.writeFieldEnd() - if self.histogram is not None: - oprot.writeFieldBegin('histogram', TType.MAP, 7) - oprot.writeMapBegin(TType.STRING, TType.DOUBLE, len(self.histogram)) - for kiter455,viter456 in self.histogram.items(): - oprot.writeString(kiter455.encode('utf-8')) - oprot.writeDouble(viter456) - oprot.writeMapEnd() - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.hostname is None: - raise TProtocol.TProtocolException(message='Required field hostname is unset!') - if self.port is None: - raise TProtocol.TProtocolException(message='Required field port is unset!') - if self.gauge is None: - raise TProtocol.TProtocolException(message='Required field gauge is unset!') - if self.counter is None: - raise TProtocol.TProtocolException(message='Required field counter is unset!') - if self.meter is None: - raise TProtocol.TProtocolException(message='Required field meter is unset!') - if self.timer is None: - raise TProtocol.TProtocolException(message='Required field timer is unset!') - if self.histogram is None: - raise TProtocol.TProtocolException(message='Required field histogram is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class TopologyMetricInfo: - """ - Attributes: - - topology_id - - task_metric_list - - worker_metric_list - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'topology_id', None, None, ), # 1 - (2, TType.LIST, 'task_metric_list', (TType.STRUCT,(TaskMetricData, TaskMetricData.thrift_spec)), None, ), # 2 - (3, TType.LIST, 'worker_metric_list', (TType.STRUCT,(WorkerMetricData, WorkerMetricData.thrift_spec)), None, ), # 3 - ) - - def __hash__(self): - return 0 + hash(self.topology_id) + hash(self.task_metric_list) + hash(self.worker_metric_list) - - def __init__(self, topology_id=None, task_metric_list=None, worker_metric_list=None,): - self.topology_id = topology_id - self.task_metric_list = task_metric_list - self.worker_metric_list = worker_metric_list - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.topology_id = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.LIST: - self.task_metric_list = [] - (_etype460, _size457) = iprot.readListBegin() - for _i461 in xrange(_size457): - _elem462 = TaskMetricData() - _elem462.read(iprot) - self.task_metric_list.append(_elem462) - iprot.readListEnd() - else: - iprot.skip(ftype) - elif fid == 3: - if ftype == TType.LIST: - self.worker_metric_list = [] - (_etype466, _size463) = iprot.readListBegin() - for _i467 in xrange(_size463): - _elem468 = WorkerMetricData() - _elem468.read(iprot) - self.worker_metric_list.append(_elem468) - iprot.readListEnd() - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('TopologyMetricInfo') - if self.topology_id is not None: - oprot.writeFieldBegin('topology_id', TType.STRING, 1) - oprot.writeString(self.topology_id.encode('utf-8')) - oprot.writeFieldEnd() - if self.task_metric_list is not None: - oprot.writeFieldBegin('task_metric_list', TType.LIST, 2) - oprot.writeListBegin(TType.STRUCT, len(self.task_metric_list)) - for iter469 in self.task_metric_list: - iter469.write(oprot) - oprot.writeListEnd() - oprot.writeFieldEnd() - if self.worker_metric_list is not None: - oprot.writeFieldBegin('worker_metric_list', TType.LIST, 3) - oprot.writeListBegin(TType.STRUCT, len(self.worker_metric_list)) - for iter470 in self.worker_metric_list: - iter470.write(oprot) - oprot.writeListEnd() - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.topology_id is None: - raise TProtocol.TProtocolException(message='Required field topology_id is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class DRPCRequest: - """ - Attributes: - - func_args - - request_id - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'func_args', None, None, ), # 1 - (2, TType.STRING, 'request_id', None, None, ), # 2 - ) - - def __hash__(self): - return 0 + hash(self.func_args) + hash(self.request_id) - - def __init__(self, func_args=None, request_id=None,): - self.func_args = func_args - self.request_id = request_id - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.func_args = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - elif fid == 2: - if ftype == TType.STRING: - self.request_id = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('DRPCRequest') - if self.func_args is not None: - oprot.writeFieldBegin('func_args', TType.STRING, 1) - oprot.writeString(self.func_args.encode('utf-8')) - oprot.writeFieldEnd() - if self.request_id is not None: - oprot.writeFieldBegin('request_id', TType.STRING, 2) - oprot.writeString(self.request_id.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.func_args is None: - raise TProtocol.TProtocolException(message='Required field func_args is unset!') - if self.request_id is None: - raise TProtocol.TProtocolException(message='Required field request_id is unset!') - return - - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) - -class DRPCExecutionException(Exception): - """ - Attributes: - - msg - """ - - thrift_spec = ( - None, # 0 - (1, TType.STRING, 'msg', None, None, ), # 1 - ) - - def __hash__(self): - return 0 + hash(self.msg) - - def __init__(self, msg=None,): - self.msg = msg - - def read(self, iprot): - if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: - fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) - return - iprot.readStructBegin() - while True: - (fname, ftype, fid) = iprot.readFieldBegin() - if ftype == TType.STOP: - break - if fid == 1: - if ftype == TType.STRING: - self.msg = iprot.readString().decode('utf-8') - else: - iprot.skip(ftype) - else: - iprot.skip(ftype) - iprot.readFieldEnd() - iprot.readStructEnd() - - def write(self, oprot): - if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: - oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) - return - oprot.writeStructBegin('DRPCExecutionException') - if self.msg is not None: - oprot.writeFieldBegin('msg', TType.STRING, 1) - oprot.writeString(self.msg.encode('utf-8')) - oprot.writeFieldEnd() - oprot.writeFieldStop() - oprot.writeStructEnd() - - def validate(self): - if self.msg is None: - raise TProtocol.TProtocolException(message='Required field msg is unset!') - return - - - def __str__(self): - return repr(self) - - def __repr__(self): - L = ['%s=%r' % (key, value) - for key, value in self.__dict__.iteritems()] - return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) - - def __eq__(self, other): - return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ - - def __ne__(self, other): - return not (self == other) diff --git a/jstorm-client/src/test/java/com/alibaba/jstorm/util/queue/DisruptorTest.java b/jstorm-client/src/test/java/com/alibaba/jstorm/util/queue/DisruptorTest.java deleted file mode 100644 index c6f4479ef..000000000 --- a/jstorm-client/src/test/java/com/alibaba/jstorm/util/queue/DisruptorTest.java +++ /dev/null @@ -1,545 +0,0 @@ -package com.alibaba.jstorm.util.queue; - -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; - -import junit.framework.Assert; - -import org.apache.log4j.Logger; -import org.junit.Test; - -import backtype.storm.utils.DisruptorQueue; - -import com.lmax.disruptor.BlockingWaitStrategy; -import com.lmax.disruptor.EventHandler; -import com.lmax.disruptor.InsufficientCapacityException; -import com.lmax.disruptor.dsl.ProducerType; - -public class DisruptorTest { - - static { - DisruptorQueue.setUseSleep(true); - DisruptorQueue.setLimited(true); - } - - private int count = 100000000; - private int buffer_size = 8 * 1024; - - private Logger logger = Logger.getLogger(DisruptorTest.class); - - @Test - public void testMultipleConsume() { - final DisruptorQueue disruptorQueue = createQueue("test", - ProducerType.MULTI, 1024); - - // new Thread(new Runnable() { - // - // @Override - // public void run() { - // System.out.println("Begin to produce item"); - // JStormUtils.sleepMs(1000); - // - // for (int i = 0; i < 1000000; i++) { - // disruptorQueue.publish(Integer.valueOf(i)); - // } - // - // System.out.println("Finish produce item"); - // } - // }).start(); - // - // - // new Thread(new Runnable() { - // - // @Override - // public void run() { - // while(true) { - // disruptorQueue.consumeBatchWhenAvailable(new EventHandler() { - // - // @Override - // public void onEvent(Object event, long sequence, - // boolean endOfBatch) throws Exception { - // - // System.out.println("Consumer 1:" + (Integer)event); - // } - // - // }); - // } - // - // } - // }).start(); - // - // new Thread(new Runnable() { - // - // @Override - // public void run() { - // while(true) { - // disruptorQueue.consumeBatchWhenAvailable(new EventHandler() { - // - // @Override - // public void onEvent(Object event, long sequence, - // boolean endOfBatch) throws Exception { - // - // System.out.println("Consumer 2:" + (Integer)event); - // } - // - // }); - // } - // - // } - // }).start(); - // - // JStormUtils.sleepMs(100000); - } - - private final static int TIMEOUT = 5; // MS - private final static int PRODUCER_NUM = 4; - - @Test - public void testLaterStartConsumer() throws InterruptedException { - System.out - .println("!!!!!!!!!!!!!!!Begin testLaterStartConsumer!!!!!!!!!!"); - final AtomicBoolean messageConsumed = new AtomicBoolean(false); - - // Set queue length to 1, so that the RingBuffer can be easily full - // to trigger consumer blocking - DisruptorQueue queue = createQueue("consumerHang", ProducerType.MULTI, - 2); - push(queue, 1); - Runnable producer = new Producer(queue); - Runnable consumer = new Consumer(queue, new EventHandler() { - long count = 0; - - @Override - public void onEvent(Object obj, long sequence, boolean endOfBatch) - throws Exception { - - messageConsumed.set(true); - System.out.println("Consume " + count++); - } - }); - - run(producer, 0, 0, consumer, 50); - Assert.assertTrue( - "disruptor message is never consumed due to consumer thread hangs", - messageConsumed.get()); - - System.out - .println("!!!!!!!!!!!!!!!!End testLaterStartConsumer!!!!!!!!!!"); - } - - @Test - public void testBeforeStartConsumer() throws InterruptedException { - System.out - .println("!!!!!!!!!!!!Begin testBeforeStartConsumer!!!!!!!!!"); - final AtomicBoolean messageConsumed = new AtomicBoolean(false); - - // Set queue length to 1, so that the RingBuffer can be easily full - // to trigger consumer blocking - DisruptorQueue queue = createQueue("consumerHang", ProducerType.MULTI, - 2); - queue.consumerStarted(); - push(queue, 1); - Runnable producer = new Producer(queue); - Runnable consumer = new Consumer(queue, new EventHandler() { - long count = 0; - - @Override - public void onEvent(Object obj, long sequence, boolean endOfBatch) - throws Exception { - - messageConsumed.set(true); - System.out.println("Consume " + count++); - } - }); - - run(producer, 0, 0, consumer, 50); - Assert.assertTrue( - "disruptor message is never consumed due to consumer thread hangs", - messageConsumed.get()); - - System.out - .println("!!!!!!!!!!!!!End testBeforeStartConsumer!!!!!!!!!!"); - } - - @Test - public void testSingleProducer() throws InterruptedException { - System.out - .println("!!!!!!!!!!!!!!Begin testSingleProducer!!!!!!!!!!!!!!"); - final AtomicBoolean messageConsumed = new AtomicBoolean(false); - - // Set queue length to 1, so that the RingBuffer can be easily full - // to trigger consumer blocking - DisruptorQueue queue = createQueue("consumerHang", ProducerType.SINGLE, - 1); - push(queue, 1); - Runnable producer = new Producer(queue); - Runnable consumer = new Consumer(queue, new EventHandler() { - long count = 0; - - @Override - public void onEvent(Object obj, long sequence, boolean endOfBatch) - throws Exception { - - messageConsumed.set(true); - System.out.println("Consume " + count++); - } - }); - - run(producer, 0, 0, consumer, 50); - Assert.assertTrue( - "disruptor message is never consumed due to consumer thread hangs", - messageConsumed.get()); - - System.out - .println("!!!!!!!!!!!!!!End testSingleProducer!!!!!!!!!!!!!!"); - } - - public static AtomicLong produceNum = new AtomicLong(0); - public static AtomicLong consumerNum = new AtomicLong(0); - - public static EventHandlerTest handler = new EventHandlerTest(); - - public static void resetNum() { - produceNum.set(0); - consumerNum.set(0); - handler.reset(); - - } - - @Test - public void testMessageDisorder() throws InterruptedException { - - System.out - .println("!!!!!!!!!!!!!!!!Begin testMessageDisorder!!!!!!!!!!"); - // Set queue length to bigger enough - DisruptorQueue queue = createQueue("messageOrder", ProducerType.MULTI, - 128); - - queue.publish("1"); - - Runnable producer = new Producer(queue); - - final Object[] result = new Object[1]; - Runnable consumer = new Consumer(queue, new EventHandler() { - private boolean head = true; - private Map lastIdMap = new HashMap(); - - @Override - public void onEvent(Object obj, long sequence, boolean endOfBatch) - throws Exception { - consumerNum.incrementAndGet(); - if (head) { - head = false; - result[0] = obj; - } else { - String event = (String) obj; - String[] item = event.split("@"); - Long current = Long.valueOf(item[1]); - Long last = lastIdMap.get(item[0]); - if (last != null) { - if (current <= last) { - String msg = "Consume disorder of " + item[0] - + ", current" + current + ",last:" + last; - System.err - .println("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!"); - System.err.println(msg); - System.err - .println("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!"); - Assert.fail(msg); - } - } - - lastIdMap.put(item[0], current); - - } - } - }); - - run(producer, PRODUCER_NUM, 1000, consumer, 30000); - Assert.assertEquals( - "We expect to receive first published message first, but received " - + result[0], "1", result[0]); - produceNum.incrementAndGet(); - Assert.assertEquals("produce: " + produceNum.get() + ", consume:" - + consumerNum.get(), produceNum.get(), consumerNum.get()); - System.out.println("!!!!!!!!!!!!!!End testMessageDisorder!!!!!!!!!!!!"); - } - - @Test - public void testPull() { - // @@@ TODO - } - - @Test - public void testTake() { - // @@@ TODO - } - - public void push(DisruptorQueue queue, int num) { - for (int i = 0; i < num; i++) { - String msg = String.valueOf(Thread.currentThread().getId()) + "@" - + i; - try { - queue.publish(msg, false); - } catch (InsufficientCapacityException e) { - e.printStackTrace(); - } - produceNum.incrementAndGet(); - System.out.println(Thread.currentThread().getId() - + " Publish one :" + i); - } - } - - @Test - public void testConsumeBatchWhenAvailable() { - System.out - .println("!!!!!!!!!!!!!!!Begin testConsumeBatchWhenAvailable!!!!!!!!!!!!"); - - resetNum(); - - // Set queue length to bigger enough - DisruptorQueue queue = createQueue("messageOrder", ProducerType.MULTI, - 128); - - push(queue, 128); - - queue.consumeBatchWhenAvailable(handler); - - Assert.assertEquals("produce: " + produceNum.get() + ", consume:" - + consumerNum.get(), produceNum.get(), consumerNum.get()); - System.out - .println("!!!!!! finish testConsumeBatchWhenAvailable test 1"); - resetNum(); - - queue.consumerStarted(); - - push(queue, 128); - - queue.consumeBatchWhenAvailable(handler); - - Assert.assertEquals("produce: " + produceNum.get() + ", consume:" - + consumerNum.get(), produceNum.get(), consumerNum.get()); - System.out - .println("!!!!!! finish testConsumeBatchWhenAvailable test 2"); - - System.out - .println("!!!!!!!!!!!!!!!Finsh testConsumeBatchWhenAvailable for MULTI!!!!!!!!!!!!"); - - resetNum(); - // Set queue length to bigger enough - DisruptorQueue queue2 = createQueue("messageOrder", - ProducerType.SINGLE, 128); - - push(queue2, 128); - - queue2.consumeBatchWhenAvailable(handler); - - Assert.assertEquals("produce: " + produceNum.get() + ", consume:" - + consumerNum.get(), produceNum.get(), consumerNum.get()); - System.out - .println("!!!!!! finish testConsumeBatchWhenAvailable test 3"); - resetNum(); - - queue2.consumerStarted(); - - push(queue2, 128); - - queue2.consumeBatchWhenAvailable(handler); - - Assert.assertEquals("produce: " + produceNum.get() + ", consume:" - + consumerNum.get(), produceNum.get(), consumerNum.get()); - System.out - .println("!!!!!! finish testConsumeBatchWhenAvailable test 4"); - - System.out - .println("!!!!!!!!!!!!!!!Finsh testConsumeBatchWhenAvailable for single !!!!!!!!!!!!"); - System.out - .println("!!!!!!!!!!!!!End testConsumeBatchWhenAvailable!!!!!!!!!!!"); - } - - @Test - public void testTryConsume() { - System.out.println("!!!!!!!!!!!!Begin testTryConsume!!!!!!!!!!!!!!!!"); - - resetNum(); - // Set queue length to bigger enough - DisruptorQueue queue = createQueue("messageOrder", ProducerType.MULTI, - 128); - - push(queue, 128); - - queue.consumeBatch(handler); - - Assert.assertEquals("produce: " + produceNum.get() + ", consume:" - + consumerNum.get(), produceNum.get(), consumerNum.get()); - System.out.println("!!!!!! finish testTryConsume test 1"); - resetNum(); - - queue.consumerStarted(); - - push(queue, 128); - - queue.consumeBatch(handler); - - Assert.assertEquals("produce: " + produceNum.get() + ", consume:" - + consumerNum.get(), produceNum.get(), consumerNum.get()); - System.out.println("!!!!!! finish testTryConsume test 2"); - - resetNum(); - // Set queue length to bigger enough - DisruptorQueue queue2 = createQueue("messageOrder", - ProducerType.SINGLE, 128); - - push(queue2, 128); - - queue2.consumeBatch(handler); - - Assert.assertEquals("produce: " + produceNum.get() + ", consume:" - + consumerNum.get(), produceNum.get(), consumerNum.get()); - System.out.println("!!!!!! finish testTryConsume test 3"); - resetNum(); - - queue2.consumerStarted(); - - push(queue2, 128); - - queue2.consumeBatch(handler); - - Assert.assertEquals("produce: " + produceNum.get() + ", consume:" - + consumerNum.get(), produceNum.get(), consumerNum.get()); - System.out.println("!!!!!! finish testTryConsume test 4"); - - System.out.println("!!!!!!!!!!!!!!!!!End testTryConsume!!!!!!!!!!!!!!"); - } - - private void run(Runnable producer, int producerNum, long produceMs, - Runnable consumer, long waitMs) { - try { - - resetNum(); - - Thread[] producerThreads = new Thread[producerNum]; - for (int i = 0; i < producerNum; i++) { - producerThreads[i] = new Thread(producer); - producerThreads[i].start(); - } - - Thread consumerThread = new Thread(consumer); - consumerThread.start(); - System.out.println("Please wait seconds" + produceMs / 1000); - - Thread.sleep(produceMs); - - for (int i = 0; i < producerNum; i++) { - producerThreads[i].interrupt(); - producerThreads[i].stop(); - producerThreads[i].join(TIMEOUT); - } - - Thread.sleep(waitMs); - System.out.println("Please wait seconds" + waitMs / 1000); - - consumerThread.interrupt(); - consumerThread.stop(); - consumerThread.join(TIMEOUT); - } catch (Throwable e) { - e.printStackTrace(); - } - } - - private class Producer implements Runnable { - private String msg; - private DisruptorQueue queue; - - Producer(DisruptorQueue queue) { - this.queue = queue; - } - - @Override - public void run() { - long count = 0; - try { - while (true) { - - String msg = String.valueOf(Thread.currentThread().getId()) - + "@" + count; - queue.publish(msg, false); - produceNum.incrementAndGet(); - System.out.println(msg); - count++; - } - } catch (InsufficientCapacityException e) { - System.out.println(Thread.currentThread().getId() - + " quit, insufficientCapacityException " + count); - return; - }catch (Exception e) { - System.out.println(Thread.currentThread().getId() - + " quit, Exception " + count); - return; - } - } - } - - private class Consumer implements Runnable { - private EventHandler handler; - private DisruptorQueue queue; - - Consumer(DisruptorQueue queue, EventHandler handler) { - this.handler = handler; - this.queue = queue; - } - - @Override - public void run() { - queue.consumerStarted(); - try { - while (true) { - queue.consumeBatchWhenAvailable(handler); - } - } catch (Exception e) { - // break - } - } - } - - static class EventHandlerTest implements EventHandler { - private Map lastIdMap = new HashMap(); - - public void reset() { - lastIdMap.clear(); - } - - @Override - public void onEvent(Object obj, long sequence, boolean endOfBatch) - throws Exception { - - String event = (String) obj; - String[] item = event.split("@"); - Long current = Long.valueOf(item[1]); - Long last = lastIdMap.get(item[0]); - if (last != null) { - if (current <= last) { - String msg = "Consume disorder of " + item[0] + ", current" - + current + ",last:" + last; - System.err.println("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!"); - System.err.println(msg + "," + event); - System.err.println("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!"); - Assert.fail(msg); - } - } - - lastIdMap.put(item[0], current); - consumerNum.incrementAndGet(); - } - }; - - private static DisruptorQueue createQueue(String name, ProducerType type, - int queueSize) { - - - return DisruptorQueue.mkInstance(name, type, queueSize, - new BlockingWaitStrategy()); - } -} diff --git a/jstorm-core/.classpath b/jstorm-core/.classpath new file mode 100755 index 000000000..f0a60b623 --- /dev/null +++ b/jstorm-core/.classpath @@ -0,0 +1,37 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/jstorm-core/.gitignore b/jstorm-core/.gitignore new file mode 100755 index 000000000..b83d22266 --- /dev/null +++ b/jstorm-core/.gitignore @@ -0,0 +1 @@ +/target/ diff --git a/jstorm-core/.project b/jstorm-core/.project new file mode 100755 index 000000000..617507f0e --- /dev/null +++ b/jstorm-core/.project @@ -0,0 +1,36 @@ + + + jstorm-core + + + + + + org.eclipse.wst.common.project.facet.core.builder + + + + + org.eclipse.jdt.core.javabuilder + + + + + org.eclipse.wst.validation.validationbuilder + + + + + org.eclipse.m2e.core.maven2Builder + + + + + + org.eclipse.jem.workbench.JavaEMFNature + org.eclipse.wst.common.modulecore.ModuleCoreNature + org.eclipse.jdt.core.javanature + org.eclipse.m2e.core.maven2Nature + org.eclipse.wst.common.project.facet.core.nature + + diff --git a/jstorm-core/.settings/org.eclipse.core.resources.prefs b/jstorm-core/.settings/org.eclipse.core.resources.prefs new file mode 100755 index 000000000..04cfa2c1a --- /dev/null +++ b/jstorm-core/.settings/org.eclipse.core.resources.prefs @@ -0,0 +1,6 @@ +eclipse.preferences.version=1 +encoding//src/main/java=UTF-8 +encoding//src/main/resources=UTF-8 +encoding//src/test/java=UTF-8 +encoding//src/test/resources=UTF-8 +encoding/=UTF-8 diff --git a/jstorm-core/.settings/org.eclipse.jdt.core.prefs b/jstorm-core/.settings/org.eclipse.jdt.core.prefs new file mode 100755 index 000000000..c788ee346 --- /dev/null +++ b/jstorm-core/.settings/org.eclipse.jdt.core.prefs @@ -0,0 +1,8 @@ +eclipse.preferences.version=1 +org.eclipse.jdt.core.compiler.codegen.inlineJsrBytecode=enabled +org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.7 +org.eclipse.jdt.core.compiler.compliance=1.7 +org.eclipse.jdt.core.compiler.problem.assertIdentifier=error +org.eclipse.jdt.core.compiler.problem.enumIdentifier=error +org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning +org.eclipse.jdt.core.compiler.source=1.7 diff --git a/jstorm-core/.settings/org.eclipse.m2e.core.prefs b/jstorm-core/.settings/org.eclipse.m2e.core.prefs new file mode 100755 index 000000000..14b697b7b --- /dev/null +++ b/jstorm-core/.settings/org.eclipse.m2e.core.prefs @@ -0,0 +1,4 @@ +activeProfiles= +eclipse.preferences.version=1 +resolveWorkspaceProjects=true +version=1 diff --git a/jstorm-core/.settings/org.eclipse.wst.common.component b/jstorm-core/.settings/org.eclipse.wst.common.component new file mode 100755 index 000000000..aaa3793b9 --- /dev/null +++ b/jstorm-core/.settings/org.eclipse.wst.common.component @@ -0,0 +1,6 @@ + + + + + + diff --git a/jstorm-core/.settings/org.eclipse.wst.common.project.facet.core.xml b/jstorm-core/.settings/org.eclipse.wst.common.project.facet.core.xml new file mode 100755 index 000000000..4f92af543 --- /dev/null +++ b/jstorm-core/.settings/org.eclipse.wst.common.project.facet.core.xml @@ -0,0 +1,5 @@ + + + + + diff --git a/jstorm-core/.settings/org.eclipse.wst.validation.prefs b/jstorm-core/.settings/org.eclipse.wst.validation.prefs new file mode 100644 index 000000000..04cad8cb7 --- /dev/null +++ b/jstorm-core/.settings/org.eclipse.wst.validation.prefs @@ -0,0 +1,2 @@ +disabled=06target +eclipse.preferences.version=1 diff --git a/jstorm-core/pom.xml b/jstorm-core/pom.xml new file mode 100755 index 000000000..2eab1b862 --- /dev/null +++ b/jstorm-core/pom.xml @@ -0,0 +1,254 @@ + + + + + com.alibaba.jstorm + jstorm-all + 2.0.4-SNAPSHOT + .. + + + 4.0.0 + com.alibaba.jstorm + jstorm-core + + jar + ${project.artifactId}-${project.version} + + + + + org.apache.maven.plugins + maven-surefire-plugin + + pertest + -Xms1024m -Xmx3072m + + + + maven-compiler-plugin + 2.3.2 + + 1.7 + 1.7 + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + + jar + + + + + + + + + UTF-8 + 1.4.11 + 3.1.2 + + + + org.clojure + clojure + 1.6.0 + + + commons-io + commons-io + 2.4 + + + org.apache.commons + commons-exec + 1.1 + + + commons-lang + commons-lang + 2.5 + + + commons-cli + commons-cli + 1.2 + + + org.apache.httpcomponents + httpclient + 4.3.3 + + + org.apache.thrift + libthrift + 0.9.2 + compile + + + org.slf4j + slf4j-api + + + javax.servlet + servlet-api + + + + + clj-time + clj-time + 0.8.0 + + + org.apache.curator + curator-framework + 2.5.0 + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + + + com.esotericsoftware.kryo + kryo + 2.21 + + + com.esotericsoftware.minlog + minlog + + + + + + com.googlecode.json-simple + json-simple + 1.1 + + + com.twitter + carbonite + 1.4.0 + + + + org.yaml + snakeyaml + 1.11 + + + com.lmax + disruptor + 3.2.1 + + + io.netty + netty + 3.9.0.Final + + + org.jgrapht + jgrapht-core + 0.9.0 + + + junit + junit + 4.10 + test + + + org.powermock + powermock-module-junit4 + ${powermock.version} + test + + + ch.qos.logback + logback-classic + 1.0.13 + + + org.slf4j + log4j-over-slf4j + 1.6.6 + + + + com.google.code.gson + gson + 2.3.1 + + + org.powermock + powermock-module-junit4 + ${powermock.version} + test + + + io.dropwizard.metrics + metrics-core + ${metrics.version} + + + io.dropwizard.metrics + metrics-healthchecks + ${metrics.version} + + + io.dropwizard.metrics + metrics-jvm + ${metrics.version} + + + + + javax.servlet + servlet-api + 2.5 + + + org.rocksdb + rocksdbjni + 3.10.1 + + + org.assertj + assertj-core + 1.7.1 + test + + + + org.mockito + mockito-all + 1.10.19 + test + + + diff --git a/jstorm-core/src/main/java/backtype/storm/Config.java b/jstorm-core/src/main/java/backtype/storm/Config.java new file mode 100644 index 000000000..4273908fa --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/Config.java @@ -0,0 +1,1579 @@ +/** + * 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 backtype.storm; + +import backtype.storm.ConfigValidation; +import backtype.storm.serialization.IKryoDecorator; +import backtype.storm.serialization.IKryoFactory; + +import com.esotericsoftware.kryo.Serializer; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Topology configs are specified as a plain old map. This class provides a + * convenient way to create a topology config map by providing setter methods for + * all the configs that can be set. It also makes it easier to do things like add + * serializations. + * + *

This class also provides constants for all the configurations possible on + * a Storm cluster and Storm topology. Each constant is paired with a schema + * that defines the validity criterion of the corresponding field. Default + * values for these configs can be found in defaults.yaml.

+ * + *

Note that you may put other configurations in any of the configs. Storm + * will ignore anything it doesn't recognize, but your topologies are free to make + * use of them by reading them in the prepare method of Bolts or the open method of + * Spouts.

+ */ +public class Config extends HashMap { + //DO NOT CHANGE UNLESS WE ADD IN STATE NOT STORED IN THE PARENT CLASS + private static final long serialVersionUID = -1550278723792864455L; + + /** + * This is part of a temporary workaround to a ZK bug, it is the 'scheme:acl' for + * the user Nimbus and Supervisors use to authenticate with ZK. + */ + public static final String STORM_ZOOKEEPER_SUPERACL = "storm.zookeeper.superACL"; + public static final Object STORM_ZOOKEEPER_SUPERACL_SCHEMA = String.class; + + /** + * The transporter for communication among Storm tasks + */ + public static final String STORM_MESSAGING_TRANSPORT = "storm.messaging.transport"; + public static final Object STORM_MESSAGING_TRANSPORT_SCHEMA = String.class; + + /** + * Netty based messaging: The buffer size for send/recv buffer + */ + public static final String STORM_MESSAGING_NETTY_BUFFER_SIZE = "storm.messaging.netty.buffer_size"; + public static final Object STORM_MESSAGING_NETTY_BUFFER_SIZE_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * Netty based messaging: Sets the backlog value to specify when the channel binds to a local address + */ + public static final String STORM_MESSAGING_NETTY_SOCKET_BACKLOG = "storm.messaging.netty.socket.backlog"; + public static final Object STORM_MESSAGING_NETTY_SOCKET_BACKLOG_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * Netty based messaging: The max # of retries that a peer will perform when a remote is not accessible + */ + public static final String STORM_MESSAGING_NETTY_MAX_RETRIES = "storm.messaging.netty.max_retries"; + public static final Object STORM_MESSAGING_NETTY_MAX_RETRIES_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * Netty based messaging: The min # of milliseconds that a peer will wait. + */ + public static final String STORM_MESSAGING_NETTY_MIN_SLEEP_MS = "storm.messaging.netty.min_wait_ms"; + public static final Object STORM_MESSAGING_NETTY_MIN_SLEEP_MS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * Netty based messaging: The max # of milliseconds that a peer will wait. + */ + public static final String STORM_MESSAGING_NETTY_MAX_SLEEP_MS = "storm.messaging.netty.max_wait_ms"; + public static final Object STORM_MESSAGING_NETTY_MAX_SLEEP_MS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * Netty based messaging: The # of worker threads for the server. + */ + public static final String STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS = "storm.messaging.netty.server_worker_threads"; + public static final Object STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * Netty based messaging: The # of worker threads for the client. + */ + public static final String STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS = "storm.messaging.netty.client_worker_threads"; + public static final Object STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * If the Netty messaging layer is busy, the Netty client will try to batch message as more as possible up to the size of STORM_NETTY_MESSAGE_BATCH_SIZE bytes + */ + public static final String STORM_NETTY_MESSAGE_BATCH_SIZE = "storm.messaging.netty.transfer.batch.size"; + public static final Object STORM_NETTY_MESSAGE_BATCH_SIZE_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * We check with this interval that whether the Netty channel is writable and try to write pending messages + */ + public static final String STORM_NETTY_FLUSH_CHECK_INTERVAL_MS = "storm.messaging.netty.flush.check.interval.ms"; + public static final Object STORM_NETTY_FLUSH_CHECK_INTERVAL_MS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * Netty based messaging: Is authentication required for Netty messaging from client worker process to server worker process. + */ + public static final String STORM_MESSAGING_NETTY_AUTHENTICATION = "storm.messaging.netty.authentication"; + public static final Object STORM_MESSAGING_NETTY_AUTHENTICATION_SCHEMA = Boolean.class; + + /** + * The delegate for serializing metadata, should be used for serialized objects stored in zookeeper and on disk. + * This is NOT used for compressing serialized tuples sent between topologies. + */ + public static final String STORM_META_SERIALIZATION_DELEGATE = "storm.meta.serialization.delegate"; + public static final Object STORM_META_SERIALIZATION_DELEGATE_SCHEMA = String.class; + + /** + * A list of hosts of ZooKeeper servers used to manage the cluster. + */ + public static final String STORM_ZOOKEEPER_SERVERS = "storm.zookeeper.servers"; + public static final Object STORM_ZOOKEEPER_SERVERS_SCHEMA = ConfigValidation.StringsValidator; + + /** + * The port Storm will use to connect to each of the ZooKeeper servers. + */ + public static final String STORM_ZOOKEEPER_PORT = "storm.zookeeper.port"; + public static final Object STORM_ZOOKEEPER_PORT_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * A directory on the local filesystem used by Storm for any local + * filesystem usage it needs. The directory must exist and the Storm daemons must + * have permission to read/write from this location. + */ + public static final String STORM_LOCAL_DIR = "storm.local.dir"; + public static final Object STORM_LOCAL_DIR_SCHEMA = String.class; + + /** + * A global task scheduler used to assign topologies's tasks to supervisors' wokers. + * + * If this is not set, a default system scheduler will be used. + */ + public static final String STORM_SCHEDULER = "storm.scheduler"; + public static final Object STORM_SCHEDULER_SCHEMA = String.class; + + /** + * The mode this Storm cluster is running in. Either "distributed" or "local". + */ + public static final String STORM_CLUSTER_MODE = "storm.cluster.mode"; + public static final Object STORM_CLUSTER_MODE_SCHEMA = String.class; + + /** + * The hostname the supervisors/workers should report to nimbus. If unset, Storm will + * get the hostname to report by calling InetAddress.getLocalHost().getCanonicalHostName(). + * + * You should set this config when you dont have a DNS which supervisors/workers + * can utilize to find each other based on hostname got from calls to + * InetAddress.getLocalHost().getCanonicalHostName(). + */ + public static final String STORM_LOCAL_HOSTNAME = "storm.local.hostname"; + public static final Object STORM_LOCAL_HOSTNAME_SCHEMA = String.class; + + /** + * The plugin that will convert a principal to a local user. + */ + public static final String STORM_PRINCIPAL_TO_LOCAL_PLUGIN = "storm.principal.tolocal"; + public static final Object STORM_PRINCIPAL_TO_LOCAL_PLUGIN_SCHEMA = String.class; + + /** + * The plugin that will provide user groups service + */ + public static final String STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN = "storm.group.mapping.service"; + public static final Object STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN_SCHEMA = String.class; + + /** + * Max no.of seconds group mapping service will cache user groups + */ + public static final String STORM_GROUP_MAPPING_SERVICE_CACHE_DURATION_SECS = "storm.group.mapping.service.cache.duration.secs"; + public static final Object STORM_GROUP_MAPPING_SERVICE_CACHE_DURATION_SECS_SCHEMA = Number.class; + + /** + * The default transport plug-in for Thrift client/server communication + */ + public static final String STORM_THRIFT_TRANSPORT_PLUGIN = "storm.thrift.transport"; + public static final Object STORM_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class; + + /** + * The serializer class for ListDelegate (tuple payload). + * The default serializer will be ListDelegateSerializer + */ + public static final String TOPOLOGY_TUPLE_SERIALIZER = "topology.tuple.serializer"; + public static final Object TOPOLOGY_TUPLE_SERIALIZER_SCHEMA = String.class; + + /** + * Try to serialize all tuples, even for local transfers. This should only be used + * for testing, as a sanity check that all of your tuples are setup properly. + */ + public static final String TOPOLOGY_TESTING_ALWAYS_TRY_SERIALIZE = "topology.testing.always.try.serialize"; + public static final Object TOPOLOGY_TESTING_ALWAYS_TRY_SERIALIZE_SCHEMA = Boolean.class; + + /** + * Whether or not to use ZeroMQ for messaging in local mode. If this is set + * to false, then Storm will use a pure-Java messaging system. The purpose + * of this flag is to make it easy to run Storm in local mode by eliminating + * the need for native dependencies, which can be difficult to install. + * + * Defaults to false. + */ + public static final String STORM_LOCAL_MODE_ZMQ = "storm.local.mode.zmq"; + public static final Object STORM_LOCAL_MODE_ZMQ_SCHEMA = Boolean.class; + + /** + * The root location at which Storm stores data in ZooKeeper. + */ + public static final String STORM_ZOOKEEPER_ROOT = "storm.zookeeper.root"; + public static final Object STORM_ZOOKEEPER_ROOT_SCHEMA = String.class; + + /** + * The session timeout for clients to ZooKeeper. + */ + public static final String STORM_ZOOKEEPER_SESSION_TIMEOUT = "storm.zookeeper.session.timeout"; + public static final Object STORM_ZOOKEEPER_SESSION_TIMEOUT_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * The connection timeout for clients to ZooKeeper. + */ + public static final String STORM_ZOOKEEPER_CONNECTION_TIMEOUT = "storm.zookeeper.connection.timeout"; + public static final Object STORM_ZOOKEEPER_CONNECTION_TIMEOUT_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * The number of times to retry a Zookeeper operation. + */ + public static final String STORM_ZOOKEEPER_RETRY_TIMES="storm.zookeeper.retry.times"; + public static final Object STORM_ZOOKEEPER_RETRY_TIMES_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * The interval between retries of a Zookeeper operation. + */ + public static final String STORM_ZOOKEEPER_RETRY_INTERVAL="storm.zookeeper.retry.interval"; + public static final Object STORM_ZOOKEEPER_RETRY_INTERVAL_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * The ceiling of the interval between retries of a Zookeeper operation. + */ + public static final String STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING="storm.zookeeper.retry.intervalceiling.millis"; + public static final Object STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * The cluster Zookeeper authentication scheme to use, e.g. "digest". Defaults to no authentication. + */ + public static final String STORM_ZOOKEEPER_AUTH_SCHEME="storm.zookeeper.auth.scheme"; + public static final Object STORM_ZOOKEEPER_AUTH_SCHEME_SCHEMA = String.class; + + /** + * A string representing the payload for cluster Zookeeper authentication. + * It gets serialized using UTF-8 encoding during authentication. + * Note that if this is set to something with a secret (as when using + * digest authentication) then it should only be set in the + * storm-cluster-auth.yaml file. + * This file storm-cluster-auth.yaml should then be protected with + * appropriate permissions that deny access from workers. + */ + public static final String STORM_ZOOKEEPER_AUTH_PAYLOAD="storm.zookeeper.auth.payload"; + public static final Object STORM_ZOOKEEPER_AUTH_PAYLOAD_SCHEMA = String.class; + + /** + * The topology Zookeeper authentication scheme to use, e.g. "digest". Defaults to no authentication. + */ + public static final String STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME="storm.zookeeper.topology.auth.scheme"; + public static final Object STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME_SCHEMA = String.class; + + /** + * A string representing the payload for topology Zookeeper authentication. It gets serialized using UTF-8 encoding during authentication. + */ + public static final String STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD="storm.zookeeper.topology.auth.payload"; + public static final Object STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD_SCHEMA = String.class; + + /** + * The id assigned to a running topology. The id is the storm name with a unique nonce appended. + */ + public static final String TOPOLOGY_ID = "topology.id"; + public static final Object TOPOLOGY_ID_SCHEMA = String.class; + public static final String STORM_ID = TOPOLOGY_ID; + public static final Object STORM_ID_SCHEMA = String.class; + /** + * The number of times to retry a Nimbus operation. + */ + public static final String STORM_NIMBUS_RETRY_TIMES="storm.nimbus.retry.times"; + public static final Object STORM_NIMBUS_RETRY_TIMES_SCHEMA = Number.class; + + /** + * The starting interval between exponential backoff retries of a Nimbus operation. + */ + public static final String STORM_NIMBUS_RETRY_INTERVAL="storm.nimbus.retry.interval.millis"; + public static final Object STORM_NIMBUS_RETRY_INTERVAL_SCHEMA = Number.class; + + /** + * The ceiling of the interval between retries of a client connect to Nimbus operation. + */ + public static final String STORM_NIMBUS_RETRY_INTERVAL_CEILING="storm.nimbus.retry.intervalceiling.millis"; + public static final Object STORM_NIMBUS_RETRY_INTERVAL_CEILING_SCHEMA = Number.class; + + /** + * The host that the master server is running on. + */ + public static final String NIMBUS_HOST = "nimbus.host"; + public static final Object NIMBUS_HOST_SCHEMA = String.class; + + /** + * The Nimbus transport plug-in for Thrift client/server communication + */ + public static final String NIMBUS_THRIFT_TRANSPORT_PLUGIN = "nimbus.thrift.transport"; + public static final Object NIMBUS_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class; + + /** + * Which port the Thrift interface of Nimbus should run on. Clients should + * connect to this port to upload jars and submit topologies. + */ + public static final String NIMBUS_THRIFT_PORT = "nimbus.thrift.port"; + public static final Object NIMBUS_THRIFT_PORT_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * The number of threads that should be used by the nimbus thrift server. + */ + public static final String NIMBUS_THRIFT_THREADS = "nimbus.thrift.threads"; + public static final Object NIMBUS_THRIFT_THREADS_SCHEMA = Number.class; + + /** + * A list of users that are cluster admins and can run any command. To use this set + * nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer + */ + public static final String NIMBUS_ADMINS = "nimbus.admins"; + public static final Object NIMBUS_ADMINS_SCHEMA = ConfigValidation.StringsValidator; + + /** + * A list of users that are the only ones allowed to run user operation on storm cluster. + * To use this set nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer + */ + public static final String NIMBUS_USERS = "nimbus.users"; + public static final Object NIMBUS_USERS_SCHEMA = ConfigValidation.StringsValidator; + + /** + * A list of groups , users belong to these groups are the only ones allowed to run user operation on storm cluster. + * To use this set nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer + */ + public static final String NIMBUS_GROUPS = "nimbus.groups"; + public static final Object NIMBUS_GROUPS_SCHEMA = ConfigValidation.StringsValidator; + + /** + * A list of users that run the supervisors and should be authorized to interact with + * nimbus as a supervisor would. To use this set + * nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer + */ + public static final String NIMBUS_SUPERVISOR_USERS = "nimbus.supervisor.users"; + public static final Object NIMBUS_SUPERVISOR_USERS_SCHEMA = ConfigValidation.StringsValidator; + + /** + * The maximum buffer size thrift should use when reading messages. + */ + public static final String NIMBUS_THRIFT_MAX_BUFFER_SIZE = "nimbus.thrift.max_buffer_size"; + public static final Object NIMBUS_THRIFT_MAX_BUFFER_SIZE_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * This parameter is used by the storm-deploy project to configure the + * jvm options for the nimbus daemon. + */ + public static final String NIMBUS_CHILDOPTS = "nimbus.childopts"; + public static final Object NIMBUS_CHILDOPTS_SCHEMA = String.class; + + + /** + * How long without heartbeating a task can go before nimbus will consider the + * task dead and reassign it to another location. + */ + public static final String NIMBUS_TASK_TIMEOUT_SECS = "nimbus.task.timeout.secs"; + public static final Object NIMBUS_TASK_TIMEOUT_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + + /** + * How often nimbus should wake up to check heartbeats and do reassignments. Note + * that if a machine ever goes down Nimbus will immediately wake up and take action. + * This parameter is for checking for failures when there's no explicit event like that + * occuring. + */ + public static final String NIMBUS_MONITOR_FREQ_SECS = "nimbus.monitor.freq.secs"; + public static final Object NIMBUS_MONITOR_FREQ_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * How often nimbus should wake the cleanup thread to clean the inbox. + * @see NIMBUS_INBOX_JAR_EXPIRATION_SECS + */ + public static final String NIMBUS_CLEANUP_INBOX_FREQ_SECS = "nimbus.cleanup.inbox.freq.secs"; + public static final Object NIMBUS_CLEANUP_INBOX_FREQ_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * The length of time a jar file lives in the inbox before being deleted by the cleanup thread. + * + * Probably keep this value greater than or equal to NIMBUS_CLEANUP_INBOX_JAR_EXPIRATION_SECS. + * Note that the time it takes to delete an inbox jar file is going to be somewhat more than + * NIMBUS_CLEANUP_INBOX_JAR_EXPIRATION_SECS (depending on how often NIMBUS_CLEANUP_FREQ_SECS + * is set to). + * @see NIMBUS_CLEANUP_FREQ_SECS + */ + public static final String NIMBUS_INBOX_JAR_EXPIRATION_SECS = "nimbus.inbox.jar.expiration.secs"; + public static final Object NIMBUS_INBOX_JAR_EXPIRATION_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * How long before a supervisor can go without heartbeating before nimbus considers it dead + * and stops assigning new work to it. + */ + public static final String NIMBUS_SUPERVISOR_TIMEOUT_SECS = "nimbus.supervisor.timeout.secs"; + public static final Object NIMBUS_SUPERVISOR_TIMEOUT_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * A special timeout used when a task is initially launched. During launch, this is the timeout + * used until the first heartbeat, overriding nimbus.task.timeout.secs. + * + *

A separate timeout exists for launch because there can be quite a bit of overhead + * to launching new JVM's and configuring them.

+ */ + public static final String NIMBUS_TASK_LAUNCH_SECS = "nimbus.task.launch.secs"; + public static final Object NIMBUS_TASK_LAUNCH_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * Whether or not nimbus should reassign tasks if it detects that a task goes down. + * Defaults to true, and it's not recommended to change this value. + */ + public static final String NIMBUS_REASSIGN = "nimbus.reassign"; + public static final Object NIMBUS_REASSIGN_SCHEMA = Boolean.class; + + /** + * During upload/download with the master, how long an upload or download connection is idle + * before nimbus considers it dead and drops the connection. + */ + public static final String NIMBUS_FILE_COPY_EXPIRATION_SECS = "nimbus.file.copy.expiration.secs"; + public static final Object NIMBUS_FILE_COPY_EXPIRATION_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * A custom class that implements ITopologyValidator that is run whenever a + * topology is submitted. Can be used to provide business-specific logic for + * whether topologies are allowed to run or not. + */ + public static final String NIMBUS_TOPOLOGY_VALIDATOR = "nimbus.topology.validator"; + public static final Object NIMBUS_TOPOLOGY_VALIDATOR_SCHEMA = String.class; + + /** + * Class name for authorization plugin for Nimbus + */ + public static final String NIMBUS_AUTHORIZER = "nimbus.authorizer"; + public static final Object NIMBUS_AUTHORIZER_SCHEMA = String.class; + + + /** + * Impersonation user ACL config entries. + */ + public static final String NIMBUS_IMPERSONATION_AUTHORIZER = "nimbus.impersonation.authorizer"; + public static final Object NIMBUS_IMPERSONATION_AUTHORIZER_SCHEMA = String.class; + + + /** + * Impersonation user ACL config entries. + */ + public static final String NIMBUS_IMPERSONATION_ACL = "nimbus.impersonation.acl"; + public static final Object NIMBUS_IMPERSONATION_ACL_SCHEMA = ConfigValidation.MapOfStringToMapValidator; + + /** + * How often nimbus should wake up to renew credentials if needed. + */ + public static final String NIMBUS_CREDENTIAL_RENEW_FREQ_SECS = "nimbus.credential.renewers.freq.secs"; + public static final Object NIMBUS_CREDENTIAL_RENEW_FREQ_SECS_SCHEMA = Number.class; + + /** + * A list of credential renewers that nimbus should load. + */ + public static final String NIMBUS_CREDENTIAL_RENEWERS = "nimbus.credential.renewers.classes"; + public static final Object NIMBUS_CREDENTIAL_RENEWERS_SCHEMA = ConfigValidation.StringsValidator; + + /** + * A list of plugins that nimbus should load during submit topology to populate + * credentials on user's behalf. + */ + public static final String NIMBUS_AUTO_CRED_PLUGINS = "nimbus.autocredential.plugins.classes"; + public static final Object NIMBUS_AUTO_CRED_PLUGINS_SCHEMA = ConfigValidation.StringsValidator; + + /** + * Storm UI binds to this host/interface. + */ + public static final String UI_HOST = "ui.host"; + public static final Object UI_HOST_SCHEMA = String.class; + + /** + * Storm UI binds to this port. + */ + public static final String UI_PORT = "ui.port"; + public static final Object UI_PORT_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * HTTP UI port for log viewer + */ + public static final String LOGVIEWER_PORT = "logviewer.port"; + public static final Object LOGVIEWER_PORT_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * Childopts for log viewer java process. + */ + public static final String LOGVIEWER_CHILDOPTS = "logviewer.childopts"; + public static final Object LOGVIEWER_CHILDOPTS_SCHEMA = String.class; + + /** + * How often to clean up old log files + */ + public static final String LOGVIEWER_CLEANUP_INTERVAL_SECS = "logviewer.cleanup.interval.secs"; + public static final Object LOGVIEWER_CLEANUP_INTERVAL_SECS_SCHEMA = ConfigValidation.PositiveIntegerValidator; + + /** + * How many minutes since a log was last modified for the log to be considered for clean-up + */ + public static final String LOGVIEWER_CLEANUP_AGE_MINS = "logviewer.cleanup.age.mins"; + public static final Object LOGVIEWER_CLEANUP_AGE_MINS_SCHEMA = ConfigValidation.PositiveIntegerValidator; + + /** + * A list of users allowed to view logs via the Log Viewer + */ + public static final String LOGS_USERS = "logs.users"; + public static final Object LOGS_USERS_SCHEMA = ConfigValidation.StringsValidator; + + /** + * A list of groups allowed to view logs via the Log Viewer + */ + public static final String LOGS_GROUPS = "logs.groups"; + public static final Object LOGS_GROUPS_SCHEMA = ConfigValidation.StringsValidator; + + /** + * Appender name used by log viewer to determine log directory. + */ + public static final String LOGVIEWER_APPENDER_NAME = "logviewer.appender.name"; + public static final Object LOGVIEWER_APPENDER_NAME_SCHEMA = String.class; + + /** + * Childopts for Storm UI Java process. + */ + public static final String UI_CHILDOPTS = "ui.childopts"; + public static final Object UI_CHILDOPTS_SCHEMA = String.class; + + /** + * A class implementing javax.servlet.Filter for authenticating/filtering UI requests + */ + public static final String UI_FILTER = "ui.filter"; + public static final Object UI_FILTER_SCHEMA = String.class; + + /** + * Initialization parameters for the javax.servlet.Filter + */ + public static final String UI_FILTER_PARAMS = "ui.filter.params"; + public static final Object UI_FILTER_PARAMS_SCHEMA = Map.class; + + /** + * The size of the header buffer for the UI in bytes + */ + public static final String UI_HEADER_BUFFER_BYTES = "ui.header.buffer.bytes"; + public static final Object UI_HEADER_BUFFER_BYTES_SCHEMA = Number.class; + + /** + * This port is used by Storm DRPC for receiving HTTPS (SSL) DPRC requests from clients. + */ + public static final String UI_HTTPS_PORT = "ui.https.port"; + public static final Object UI_HTTPS_PORT_SCHEMA = Number.class; + + /** + * Path to the keystore used by Storm UI for setting up HTTPS (SSL). + */ + public static final String UI_HTTPS_KEYSTORE_PATH = "ui.https.keystore.path"; + public static final Object UI_HTTPS_KEYSTORE_PATH_SCHEMA = String.class; + + /** + * Password to the keystore used by Storm UI for setting up HTTPS (SSL). + */ + public static final String UI_HTTPS_KEYSTORE_PASSWORD = "ui.https.keystore.password"; + public static final Object UI_HTTPS_KEYSTORE_PASSWORD_SCHEMA = String.class; + + /** + * Type of keystore used by Storm UI for setting up HTTPS (SSL). + * see http://docs.oracle.com/javase/7/docs/api/java/security/KeyStore.html for more details. + */ + public static final String UI_HTTPS_KEYSTORE_TYPE = "ui.https.keystore.type"; + public static final Object UI_HTTPS_KEYSTORE_TYPE_SCHEMA = String.class; + + /** + * Password to the private key in the keystore for settting up HTTPS (SSL). + */ + public static final String UI_HTTPS_KEY_PASSWORD = "ui.https.key.password"; + public static final Object UI_HTTPS_KEY_PASSWORD_SCHEMA = String.class; + + /** + * Path to the truststore used by Storm UI settting up HTTPS (SSL). + */ + public static final String UI_HTTPS_TRUSTSTORE_PATH = "ui.https.truststore.path"; + public static final Object UI_HTTPS_TRUSTSTORE_PATH_SCHEMA = String.class; + + /** + * Password to the truststore used by Storm UI settting up HTTPS (SSL). + */ + public static final String UI_HTTPS_TRUSTSTORE_PASSWORD = "ui.https.truststore.password"; + public static final Object UI_HTTPS_TRUSTSTORE_PASSWORD_SCHEMA = String.class; + + /** + * Type of truststore used by Storm UI for setting up HTTPS (SSL). + * see http://docs.oracle.com/javase/7/docs/api/java/security/KeyStore.html for more details. + */ + public static final String UI_HTTPS_TRUSTSTORE_TYPE = "ui.https.truststore.type"; + public static final Object UI_HTTPS_TRUSTSTORE_TYPE_SCHEMA = String.class; + + /** + * Password to the truststore used by Storm DRPC settting up HTTPS (SSL). + */ + public static final String UI_HTTPS_WANT_CLIENT_AUTH = "ui.https.want.client.auth"; + public static final Object UI_HTTPS_WANT_CLIENT_AUTH_SCHEMA = Boolean.class; + + public static final String UI_HTTPS_NEED_CLIENT_AUTH = "ui.https.need.client.auth"; + public static final Object UI_HTTPS_NEED_CLIENT_AUTH_SCHEMA = Boolean.class; + + + /** + * List of DRPC servers so that the DRPCSpout knows who to talk to. + */ + public static final String DRPC_SERVERS = "drpc.servers"; + public static final Object DRPC_SERVERS_SCHEMA = ConfigValidation.StringsValidator; + + /** + * This port is used by Storm DRPC for receiving HTTP DPRC requests from clients. + */ + public static final String DRPC_HTTP_PORT = "drpc.http.port"; + public static final Object DRPC_HTTP_PORT_SCHEMA = Number.class; + + /** + * This port is used by Storm DRPC for receiving HTTPS (SSL) DPRC requests from clients. + */ + public static final String DRPC_HTTPS_PORT = "drpc.https.port"; + public static final Object DRPC_HTTPS_PORT_SCHEMA = Number.class; + + /** + * Path to the keystore used by Storm DRPC for setting up HTTPS (SSL). + */ + public static final String DRPC_HTTPS_KEYSTORE_PATH = "drpc.https.keystore.path"; + public static final Object DRPC_HTTPS_KEYSTORE_PATH_SCHEMA = String.class; + + /** + * Password to the keystore used by Storm DRPC for setting up HTTPS (SSL). + */ + public static final String DRPC_HTTPS_KEYSTORE_PASSWORD = "drpc.https.keystore.password"; + public static final Object DRPC_HTTPS_KEYSTORE_PASSWORD_SCHEMA = String.class; + + /** + * Type of keystore used by Storm DRPC for setting up HTTPS (SSL). + * see http://docs.oracle.com/javase/7/docs/api/java/security/KeyStore.html for more details. + */ + public static final String DRPC_HTTPS_KEYSTORE_TYPE = "drpc.https.keystore.type"; + public static final Object DRPC_HTTPS_KEYSTORE_TYPE_SCHEMA = String.class; + + /** + * Password to the private key in the keystore for settting up HTTPS (SSL). + */ + public static final String DRPC_HTTPS_KEY_PASSWORD = "drpc.https.key.password"; + public static final Object DRPC_HTTPS_KEY_PASSWORD_SCHEMA = String.class; + + /** + * Path to the truststore used by Storm DRPC settting up HTTPS (SSL). + */ + public static final String DRPC_HTTPS_TRUSTSTORE_PATH = "drpc.https.truststore.path"; + public static final Object DRPC_HTTPS_TRUSTSTORE_PATH_SCHEMA = String.class; + + /** + * Password to the truststore used by Storm DRPC settting up HTTPS (SSL). + */ + public static final String DRPC_HTTPS_TRUSTSTORE_PASSWORD = "drpc.https.truststore.password"; + public static final Object DRPC_HTTPS_TRUSTSTORE_PASSWORD_SCHEMA = String.class; + + /** + * Type of truststore used by Storm DRPC for setting up HTTPS (SSL). + * see http://docs.oracle.com/javase/7/docs/api/java/security/KeyStore.html for more details. + */ + public static final String DRPC_HTTPS_TRUSTSTORE_TYPE = "drpc.https.truststore.type"; + public static final Object DRPC_HTTPS_TRUSTSTORE_TYPE_SCHEMA = String.class; + + /** + * Password to the truststore used by Storm DRPC settting up HTTPS (SSL). + */ + public static final String DRPC_HTTPS_WANT_CLIENT_AUTH = "drpc.https.want.client.auth"; + public static final Object DRPC_HTTPS_WANT_CLIENT_AUTH_SCHEMA = Boolean.class; + + public static final String DRPC_HTTPS_NEED_CLIENT_AUTH = "drpc.https.need.client.auth"; + public static final Object DRPC_HTTPS_NEED_CLIENT_AUTH_SCHEMA = Boolean.class; + + /** + * The DRPC transport plug-in for Thrift client/server communication + */ + public static final String DRPC_THRIFT_TRANSPORT_PLUGIN = "drpc.thrift.transport"; + public static final Object DRPC_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class; + + /** + * This port is used by Storm DRPC for receiving DPRC requests from clients. + */ + public static final String DRPC_PORT = "drpc.port"; + public static final Object DRPC_PORT_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * Class name for authorization plugin for DRPC client + */ + public static final String DRPC_AUTHORIZER = "drpc.authorizer"; + public static final Object DRPC_AUTHORIZER_SCHEMA = String.class; + + /** + * The Access Control List for the DRPC Authorizer. + * @see DRPCSimpleAclAuthorizer + */ + public static final String DRPC_AUTHORIZER_ACL = "drpc.authorizer.acl"; + public static final Object DRPC_AUTHORIZER_ACL_SCHEMA = Map.class; + + /** + * File name of the DRPC Authorizer ACL. + * @see DRPCSimpleAclAuthorizer + */ + public static final String DRPC_AUTHORIZER_ACL_FILENAME = "drpc.authorizer.acl.filename"; + public static final Object DRPC_AUTHORIZER_ACL_FILENAME_SCHEMA = String.class; + + /** + * Whether the DRPCSimpleAclAuthorizer should deny requests for operations + * involving functions that have no explicit ACL entry. When set to false + * (the default) DRPC functions that have no entry in the ACL will be + * permitted, which is appropriate for a development environment. When set + * to true, explicit ACL entries are required for every DRPC function, and + * any request for functions will be denied. + * @see DRPCSimpleAclAuthorizer + */ + public static final String DRPC_AUTHORIZER_ACL_STRICT = "drpc.authorizer.acl.strict"; + public static final Object DRPC_AUTHORIZER_ACL_STRICT_SCHEMA = Boolean.class; + + /** + * DRPC thrift server worker threads + */ + public static final String DRPC_WORKER_THREADS = "drpc.worker.threads"; + public static final Object DRPC_WORKER_THREADS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * The maximum buffer size thrift should use when reading messages for DRPC. + */ + public static final String DRPC_MAX_BUFFER_SIZE = "drpc.max_buffer_size"; + public static final Object DRPC_MAX_BUFFER_SIZE_SCHEMA = Number.class; + + /** + * DRPC thrift server queue size + */ + public static final String DRPC_QUEUE_SIZE = "drpc.queue.size"; + public static final Object DRPC_QUEUE_SIZE_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * The DRPC invocations transport plug-in for Thrift client/server communication + */ + public static final String DRPC_INVOCATIONS_THRIFT_TRANSPORT_PLUGIN = "drpc.invocations.thrift.transport"; + public static final Object DRPC_INVOCATIONS_THRIFT_TRANSPORT_PLUGIN_SCHEMA = String.class; + + /** + * This port on Storm DRPC is used by DRPC topologies to receive function invocations and send results back. + */ + public static final String DRPC_INVOCATIONS_PORT = "drpc.invocations.port"; + public static final Object DRPC_INVOCATIONS_PORT_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * DRPC invocations thrift server worker threads + */ + public static final String DRPC_INVOCATIONS_THREADS = "drpc.invocations.threads"; + public static final Object DRPC_INVOCATIONS_THREADS_SCHEMA = Number.class; + + /** + * The timeout on DRPC requests within the DRPC server. Defaults to 10 minutes. Note that requests can also + * timeout based on the socket timeout on the DRPC client, and separately based on the topology message + * timeout for the topology implementing the DRPC function. + */ + public static final String DRPC_REQUEST_TIMEOUT_SECS = "drpc.request.timeout.secs"; + public static final Object DRPC_REQUEST_TIMEOUT_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * Childopts for Storm DRPC Java process. + */ + public static final String DRPC_CHILDOPTS = "drpc.childopts"; + public static final Object DRPC_CHILDOPTS_SCHEMA = String.class; + + /** + * Class name of the HTTP credentials plugin for the UI. + */ + public static final String UI_HTTP_CREDS_PLUGIN = "ui.http.creds.plugin"; + public static final Object UI_HTTP_CREDS_PLUGIN_SCHEMA = String.class; + + /** + * Class name of the HTTP credentials plugin for DRPC. + */ + public static final String DRPC_HTTP_CREDS_PLUGIN = "drpc.http.creds.plugin"; + public static final Object DRPC_HTTP_CREDS_PLUGIN_SCHEMA = String.class; + + /** + * the metadata configured on the supervisor + */ + public static final String SUPERVISOR_SCHEDULER_META = "supervisor.scheduler.meta"; + public static final Object SUPERVISOR_SCHEDULER_META_SCHEMA = Map.class; + /** + * A list of ports that can run workers on this supervisor. Each worker uses one port, and + * the supervisor will only run one worker per port. Use this configuration to tune + * how many workers run on each machine. + */ + public static final String SUPERVISOR_SLOTS_PORTS = "supervisor.slots.ports"; + public static final Object SUPERVISOR_SLOTS_PORTS_SCHEMA = ConfigValidation.IntegersValidator; + + /** + * A number representing the maximum number of workers any single topology can acquire. + */ + public static final String NIMBUS_SLOTS_PER_TOPOLOGY = "nimbus.slots.perTopology"; + public static final Object NIMBUS_SLOTS_PER_TOPOLOGY_SCHEMA = Number.class; + + /** + * A class implementing javax.servlet.Filter for DRPC HTTP requests + */ + public static final String DRPC_HTTP_FILTER = "drpc.http.filter"; + public static final Object DRPC_HTTP_FILTER_SCHEMA = String.class; + + /** + * Initialization parameters for the javax.servlet.Filter of the DRPC HTTP + * service + */ + public static final String DRPC_HTTP_FILTER_PARAMS = "drpc.http.filter.params"; + public static final Object DRPC_HTTP_FILTER_PARAMS_SCHEMA = Map.class; + + /** + * A number representing the maximum number of executors any single topology can acquire. + */ + public static final String NIMBUS_EXECUTORS_PER_TOPOLOGY = "nimbus.executors.perTopology"; + public static final Object NIMBUS_EXECUTORS_PER_TOPOLOGY_SCHEMA = Number.class; + + /** + * This parameter is used by the storm-deploy project to configure the + * jvm options for the supervisor daemon. + */ + public static final String SUPERVISOR_CHILDOPTS = "supervisor.childopts"; + public static final Object SUPERVISOR_CHILDOPTS_SCHEMA = String.class; + + /** + * How long a worker can go without heartbeating before the supervisor tries to + * restart the worker process. + */ + public static final String SUPERVISOR_WORKER_TIMEOUT_SECS = "supervisor.worker.timeout.secs"; + public static final Object SUPERVISOR_WORKER_TIMEOUT_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * How many seconds to sleep for before shutting down threads on worker + */ + public static final String SUPERVISOR_WORKER_SHUTDOWN_SLEEP_SECS = "supervisor.worker.shutdown.sleep.secs"; + public static final Object SUPERVISOR_WORKER_SHUTDOWN_SLEEP_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * How long a worker can go without heartbeating during the initial launch before + * the supervisor tries to restart the worker process. This value override + * supervisor.worker.timeout.secs during launch because there is additional + * overhead to starting and configuring the JVM on launch. + */ + public static final String SUPERVISOR_WORKER_START_TIMEOUT_SECS = "supervisor.worker.start.timeout.secs"; + public static final Object SUPERVISOR_WORKER_START_TIMEOUT_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * Whether or not the supervisor should launch workers assigned to it. Defaults + * to true -- and you should probably never change this value. This configuration + * is used in the Storm unit tests. + */ + public static final String SUPERVISOR_ENABLE = "supervisor.enable"; + public static final Object SUPERVISOR_ENABLE_SCHEMA = Boolean.class; + + /** + * how often the supervisor sends a heartbeat to the master. + */ + public static final String SUPERVISOR_HEARTBEAT_FREQUENCY_SECS = "supervisor.heartbeat.frequency.secs"; + public static final Object SUPERVISOR_HEARTBEAT_FREQUENCY_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + + /** + * How often the supervisor checks the worker heartbeats to see if any of them + * need to be restarted. + */ + public static final String SUPERVISOR_MONITOR_FREQUENCY_SECS = "supervisor.monitor.frequency.secs"; + public static final Object SUPERVISOR_MONITOR_FREQUENCY_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * Should the supervior try to run the worker as the lauching user or not. Defaults to false. + */ + public static final String SUPERVISOR_RUN_WORKER_AS_USER = "supervisor.run.worker.as.user"; + public static final Object SUPERVISOR_RUN_WORKER_AS_USER_SCHEMA = Boolean.class; + + /** + * Full path to the worker-laucher executable that will be used to lauch workers when + * SUPERVISOR_RUN_WORKER_AS_USER is set to true. + */ + public static final String SUPERVISOR_WORKER_LAUNCHER = "supervisor.worker.launcher"; + public static final Object SUPERVISOR_WORKER_LAUNCHER_SCHEMA = String.class; + + /** + * The jvm opts provided to workers launched by this supervisor. All "%ID%", "%WORKER-ID%", "%TOPOLOGY-ID%" + * and "%WORKER-PORT%" substrings are replaced with: + * %ID% -> port (for backward compatibility), + * %WORKER-ID% -> worker-id, + * %TOPOLOGY-ID% -> topology-id, + * %WORKER-PORT% -> port. + */ + public static final String WORKER_CHILDOPTS = "worker.childopts"; + public static final Object WORKER_CHILDOPTS_SCHEMA = ConfigValidation.StringOrStringListValidator; + + /** + * The jvm opts provided to workers launched by this supervisor for GC. All "%ID%" substrings are replaced + * with an identifier for this worker. Because the JVM complains about multiple GC opts the topology + * can override this default value by setting topology.worker.gc.childopts. + */ + public static final String WORKER_GC_CHILDOPTS = "worker.gc.childopts"; + public static final Object WORKER_GC_CHILDOPTS_SCHEMA = ConfigValidation.StringOrStringListValidator; + + /** + * control how many worker receiver threads we need per worker + */ + public static final String WORKER_RECEIVER_THREAD_COUNT = "topology.worker.receiver.thread.count"; + public static final Object WORKER_RECEIVER_THREAD_COUNT_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * How often this worker should heartbeat to the supervisor. + */ + public static final String WORKER_HEARTBEAT_FREQUENCY_SECS = "worker.heartbeat.frequency.secs"; + public static final Object WORKER_HEARTBEAT_FREQUENCY_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * How often a task should heartbeat its status to the master. + */ + public static final String TASK_HEARTBEAT_FREQUENCY_SECS = "task.heartbeat.frequency.secs"; + public static final Object TASK_HEARTBEAT_FREQUENCY_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + + /** + * How often a task should sync its connections with other tasks (if a task is + * reassigned, the other tasks sending messages to it need to refresh their connections). + * In general though, when a reassignment happens other tasks will be notified + * almost immediately. This configuration is here just in case that notification doesn't + * come through. + */ + public static final String TASK_REFRESH_POLL_SECS = "task.refresh.poll.secs"; + public static final Object TASK_REFRESH_POLL_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + + /** + * How often a task should sync credentials, worst case. + */ + public static final String TASK_CREDENTIALS_POLL_SECS = "task.credentials.poll.secs"; + public static final Object TASK_CREDENTIALS_POLL_SECS_SCHEMA = Number.class; + + + /** + * A list of users that are allowed to interact with the topology. To use this set + * nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer + */ + public static final String TOPOLOGY_USERS = "topology.users"; + public static final Object TOPOLOGY_USERS_SCHEMA = ConfigValidation.StringsValidator; + + /** + * A list of groups that are allowed to interact with the topology. To use this set + * nimbus.authorizer to backtype.storm.security.auth.authorizer.SimpleACLAuthorizer + */ + public static final String TOPOLOGY_GROUPS = "topology.groups"; + public static final Object TOPOLOGY_GROUPS_SCHEMA = ConfigValidation.StringsValidator; + + /** + * True if Storm should timeout messages or not. Defaults to true. This is meant to be used + * in unit tests to prevent tuples from being accidentally timed out during the test. + */ + public static final String TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS = "topology.enable.message.timeouts"; + public static final Object TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS_SCHEMA = Boolean.class; + + /** + * When set to true, Storm will log every message that's emitted. + */ + public static final String TOPOLOGY_DEBUG = "topology.debug"; + public static final Object TOPOLOGY_DEBUG_SCHEMA = Boolean.class; + + /** + * The serializer for communication between shell components and non-JVM + * processes + */ + public static final String TOPOLOGY_MULTILANG_SERIALIZER = "topology.multilang.serializer"; + public static final Object TOPOLOGY_MULTILANG_SERIALIZER_SCHEMA = String.class; + + /** + * How many processes should be spawned around the cluster to execute this + * topology. Each process will execute some number of tasks as threads within + * them. This parameter should be used in conjunction with the parallelism hints + * on each component in the topology to tune the performance of a topology. + */ + public static final String TOPOLOGY_WORKERS = "topology.workers"; + public static final Object TOPOLOGY_WORKERS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * How many instances to create for a spout/bolt. A task runs on a thread with zero or more + * other tasks for the same spout/bolt. The number of tasks for a spout/bolt is always + * the same throughout the lifetime of a topology, but the number of executors (threads) for + * a spout/bolt can change over time. This allows a topology to scale to more or less resources + * without redeploying the topology or violating the constraints of Storm (such as a fields grouping + * guaranteeing that the same value goes to the same task). + */ + public static final String TOPOLOGY_TASKS = "topology.tasks"; + public static final Object TOPOLOGY_TASKS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * How many executors to spawn for ackers. + * + *

If this is set to 0, then Storm will immediately ack tuples as soon + * as they come off the spout, effectively disabling reliability.

+ */ + public static final String TOPOLOGY_ACKER_EXECUTORS = "topology.acker.executors"; + public static final Object TOPOLOGY_ACKER_EXECUTORS_SCHEMA = ConfigValidation.IntegerValidator; + + + /** + * The maximum amount of time given to the topology to fully process a message + * emitted by a spout. If the message is not acked within this time frame, Storm + * will fail the message on the spout. Some spouts implementations will then replay + * the message at a later time. + */ + public static final String TOPOLOGY_MESSAGE_TIMEOUT_SECS = "topology.message.timeout.secs"; + public static final Object TOPOLOGY_MESSAGE_TIMEOUT_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * A list of serialization registrations for Kryo ( http://code.google.com/p/kryo/ ), + * the underlying serialization framework for Storm. A serialization can either + * be the name of a class (in which case Kryo will automatically create a serializer for the class + * that saves all the object's fields), or an implementation of com.esotericsoftware.kryo.Serializer. + * + * See Kryo's documentation for more information about writing custom serializers. + */ + public static final String TOPOLOGY_KRYO_REGISTER = "topology.kryo.register"; + public static final Object TOPOLOGY_KRYO_REGISTER_SCHEMA = ConfigValidation.KryoRegValidator; + + /** + * A list of classes that customize storm's kryo instance during start-up. + * Each listed class name must implement IKryoDecorator. During start-up the + * listed class is instantiated with 0 arguments, then its 'decorate' method + * is called with storm's kryo instance as the only argument. + */ + public static final String TOPOLOGY_KRYO_DECORATORS = "topology.kryo.decorators"; + public static final Object TOPOLOGY_KRYO_DECORATORS_SCHEMA = ConfigValidation.StringsValidator; + + /** + * Class that specifies how to create a Kryo instance for serialization. Storm will then apply + * topology.kryo.register and topology.kryo.decorators on top of this. The default implementation + * implements topology.fall.back.on.java.serialization and turns references off. + */ + public static final String TOPOLOGY_KRYO_FACTORY = "topology.kryo.factory"; + public static final Object TOPOLOGY_KRYO_FACTORY_SCHEMA = String.class; + + + /** + * Whether or not Storm should skip the loading of kryo registrations for which it + * does not know the class or have the serializer implementation. Otherwise, the task will + * fail to load and will throw an error at runtime. The use case of this is if you want to + * declare your serializations on the storm.yaml files on the cluster rather than every single + * time you submit a topology. Different applications may use different serializations and so + * a single application may not have the code for the other serializers used by other apps. + * By setting this config to true, Storm will ignore that it doesn't have those other serializations + * rather than throw an error. + */ + public static final String TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS= "topology.skip.missing.kryo.registrations"; + public static final Object TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS_SCHEMA = Boolean.class; + + /* + * A list of classes implementing IMetricsConsumer (See storm.yaml.example for exact config format). + * Each listed class will be routed all the metrics data generated by the storm metrics API. + * Each listed class maps 1:1 to a system bolt named __metrics_ClassName#N, and it's parallelism is configurable. + */ + public static final String TOPOLOGY_METRICS_CONSUMER_REGISTER = "topology.metrics.consumer.register"; + public static final Object TOPOLOGY_METRICS_CONSUMER_REGISTER_SCHEMA = ConfigValidation.MapsValidator; + + + /** + * The maximum parallelism allowed for a component in this topology. This configuration is + * typically used in testing to limit the number of threads spawned in local mode. + */ + public static final String TOPOLOGY_MAX_TASK_PARALLELISM="topology.max.task.parallelism"; + public static final Object TOPOLOGY_MAX_TASK_PARALLELISM_SCHEMA = ConfigValidation.IntegerValidator; + + + /** + * The maximum number of tuples that can be pending on a spout task at any given time. + * This config applies to individual tasks, not to spouts or topologies as a whole. + * + * A pending tuple is one that has been emitted from a spout but has not been acked or failed yet. + * Note that this config parameter has no effect for unreliable spouts that don't tag + * their tuples with a message id. + */ + public static final String TOPOLOGY_MAX_SPOUT_PENDING="topology.max.spout.pending"; + public static final Object TOPOLOGY_MAX_SPOUT_PENDING_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * A class that implements a strategy for what to do when a spout needs to wait. Waiting is + * triggered in one of two conditions: + * + * 1. nextTuple emits no tuples + * 2. The spout has hit maxSpoutPending and can't emit any more tuples + */ + public static final String TOPOLOGY_SPOUT_WAIT_STRATEGY="topology.spout.wait.strategy"; + public static final Object TOPOLOGY_SPOUT_WAIT_STRATEGY_SCHEMA = String.class; + + /** + * The amount of milliseconds the SleepEmptyEmitStrategy should sleep for. + */ + public static final String TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS="topology.sleep.spout.wait.strategy.time.ms"; + public static final Object TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * The maximum amount of time a component gives a source of state to synchronize before it requests + * synchronization again. + */ + public static final String TOPOLOGY_STATE_SYNCHRONIZATION_TIMEOUT_SECS="topology.state.synchronization.timeout.secs"; + public static final Object TOPOLOGY_STATE_SYNCHRONIZATION_TIMEOUT_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * The percentage of tuples to sample to produce stats for a task. + */ + public static final String TOPOLOGY_STATS_SAMPLE_RATE="topology.stats.sample.rate"; + public static final Object TOPOLOGY_STATS_SAMPLE_RATE_SCHEMA = ConfigValidation.DoubleValidator; + + /** + * The time period that builtin metrics data in bucketed into. + */ + public static final String TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS="topology.builtin.metrics.bucket.size.secs"; + public static final Object TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * Whether or not to use Java serialization in a topology. + */ + public static final String TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION="topology.fall.back.on.java.serialization"; + public static final Object TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION_SCHEMA = Boolean.class; + + /** + * Topology-specific options for the worker child process. This is used in addition to WORKER_CHILDOPTS. + */ + public static final String TOPOLOGY_WORKER_CHILDOPTS="topology.worker.childopts"; + public static final Object TOPOLOGY_WORKER_CHILDOPTS_SCHEMA = ConfigValidation.StringOrStringListValidator; + + /** + * Topology-specific options GC for the worker child process. This overrides WORKER_GC_CHILDOPTS. + */ + public static final String TOPOLOGY_WORKER_GC_CHILDOPTS="topology.worker.gc.childopts"; + public static final Object TOPOLOGY_WORKER_GC_CHILDOPTS_SCHEMA = ConfigValidation.StringOrStringListValidator; + + /** + * Topology-specific classpath for the worker child process. This is combined to the usual classpath. + */ + public static final String TOPOLOGY_CLASSPATH="topology.classpath"; + public static final Object TOPOLOGY_CLASSPATH_SCHEMA = ConfigValidation.StringOrStringListValidator; + + /** + * Topology-specific environment variables for the worker child process. + * This is added to the existing environment (that of the supervisor) + */ + public static final String TOPOLOGY_ENVIRONMENT="topology.environment"; + public static final Object TOPOLOGY_ENVIRONMENT_SCHEMA = Map.class; + + /* + * Topology-specific option to disable/enable bolt's outgoing overflow buffer. + * Enabling this option ensures that the bolt can always clear the incoming messages, + * preventing live-lock for the topology with cyclic flow. + * The overflow buffer can fill degrading the performance gradually, + * eventually running out of memory. + */ + public static final String TOPOLOGY_BOLTS_OUTGOING_OVERFLOW_BUFFER_ENABLE="topology.bolts.outgoing.overflow.buffer.enable"; + public static final Object TOPOLOGY_BOLTS_OUTGOING_OVERFLOW_BUFFER_ENABLE_SCHEMA = Boolean.class; + + /** + * This config is available for TransactionalSpouts, and contains the id ( a String) for + * the transactional topology. This id is used to store the state of the transactional + * topology in Zookeeper. + */ + public static final String TOPOLOGY_TRANSACTIONAL_ID="topology.transactional.id"; + public static final Object TOPOLOGY_TRANSACTIONAL_ID_SCHEMA = String.class; + + /** + * A list of task hooks that are automatically added to every spout and bolt in the topology. An example + * of when you'd do this is to add a hook that integrates with your internal + * monitoring system. These hooks are instantiated using the zero-arg constructor. + */ + public static final String TOPOLOGY_AUTO_TASK_HOOKS="topology.auto.task.hooks"; + public static final Object TOPOLOGY_AUTO_TASK_HOOKS_SCHEMA = ConfigValidation.StringsValidator; + + + /** + * The size of the Disruptor receive queue for each executor. Must be a power of 2. + */ + public static final String TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE="topology.executor.receive.buffer.size"; + public static final Object TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE_SCHEMA = ConfigValidation.PowerOf2Validator; + + /** + * The maximum number of messages to batch from the thread receiving off the network to the + * executor queues. Must be a power of 2. + */ + public static final String TOPOLOGY_RECEIVER_BUFFER_SIZE="topology.receiver.buffer.size"; + public static final Object TOPOLOGY_RECEIVER_BUFFER_SIZE_SCHEMA = ConfigValidation.PowerOf2Validator; + + /** + * The size of the Disruptor send queue for each executor. Must be a power of 2. + */ + public static final String TOPOLOGY_EXECUTOR_SEND_BUFFER_SIZE="topology.executor.send.buffer.size"; + public static final Object TOPOLOGY_EXECUTOR_SEND_BUFFER_SIZE_SCHEMA = ConfigValidation.PowerOf2Validator; + + /** + * The size of the Disruptor transfer queue for each worker. + */ + public static final String TOPOLOGY_TRANSFER_BUFFER_SIZE="topology.transfer.buffer.size"; + public static final Object TOPOLOGY_TRANSFER_BUFFER_SIZE_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * How often a tick tuple from the "__system" component and "__tick" stream should be sent + * to tasks. Meant to be used as a component-specific configuration. + */ + public static final String TOPOLOGY_TICK_TUPLE_FREQ_SECS="topology.tick.tuple.freq.secs"; + public static final Object TOPOLOGY_TICK_TUPLE_FREQ_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + + /** + * Configure the wait strategy used for internal queuing. Can be used to tradeoff latency + * vs. throughput + */ + public static final String TOPOLOGY_DISRUPTOR_WAIT_STRATEGY="topology.disruptor.wait.strategy"; + public static final Object TOPOLOGY_DISRUPTOR_WAIT_STRATEGY_SCHEMA = String.class; + + /** + * Configure the wait timeout used for timeout blocking wait strategy. + */ + public static final String TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT = + "topology.disruptor.wait.timeout"; + public static final Object TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_SCHEMA = + Number.class; + + /* + * The size of the shared thread pool for worker tasks to make use of. The thread pool can be accessed + * via the TopologyContext. + */ + public static final String TOPOLOGY_WORKER_SHARED_THREAD_POOL_SIZE="topology.worker.shared.thread.pool.size"; + public static final Object TOPOLOGY_WORKER_SHARED_THREAD_POOL_SIZE_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * The interval in seconds to use for determining whether to throttle error reported to Zookeeper. For example, + * an interval of 10 seconds with topology.max.error.report.per.interval set to 5 will only allow 5 errors to be + * reported to Zookeeper per task for every 10 second interval of time. + */ + public static final String TOPOLOGY_ERROR_THROTTLE_INTERVAL_SECS="topology.error.throttle.interval.secs"; + public static final Object TOPOLOGY_ERROR_THROTTLE_INTERVAL_SECS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * See doc for TOPOLOGY_ERROR_THROTTLE_INTERVAL_SECS + */ + public static final String TOPOLOGY_MAX_ERROR_REPORT_PER_INTERVAL="topology.max.error.report.per.interval"; + public static final Object TOPOLOGY_MAX_ERROR_REPORT_PER_INTERVAL_SCHEMA = ConfigValidation.IntegerValidator; + + + /** + * How often a batch can be emitted in a Trident topology. + */ + public static final String TOPOLOGY_TRIDENT_BATCH_EMIT_INTERVAL_MILLIS="topology.trident.batch.emit.interval.millis"; + public static final Object TOPOLOGY_TRIDENT_BATCH_EMIT_INTERVAL_MILLIS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * Name of the topology. This config is automatically set by Storm when the topology is submitted. + */ + public final static String TOPOLOGY_NAME="topology.name"; + public static final Object TOPOLOGY_NAME_SCHEMA = String.class; + + /** + * The principal who submitted a topology + */ + public final static String TOPOLOGY_SUBMITTER_PRINCIPAL = "topology.submitter.principal"; + public static final Object TOPOLOGY_SUBMITTER_PRINCIPAL_SCHEMA = String.class; + + /** + * The local user name of the user who submitted a topology. + */ + public static final String TOPOLOGY_SUBMITTER_USER = "topology.submitter.user"; + public static final Object TOPOLOGY_SUBMITTER_USER_SCHEMA = String.class; + + /** + * Array of components that scheduler should try to place on separate hosts. + */ + public static final String TOPOLOGY_SPREAD_COMPONENTS = "topology.spread.components"; + public static final Object TOPOLOGY_SPREAD_COMPONENTS_SCHEMA = ConfigValidation.StringsValidator; + + /** + * A list of IAutoCredentials that the topology should load and use. + */ + public static final String TOPOLOGY_AUTO_CREDENTIALS = "topology.auto-credentials"; + public static final Object TOPOLOGY_AUTO_CREDENTIALS_SCHEMA = ConfigValidation.StringsValidator; + + /** + * Max pending tuples in one ShellBolt + */ + public static final String TOPOLOGY_SHELLBOLT_MAX_PENDING="topology.shellbolt.max.pending"; + public static final Object TOPOLOGY_SHELLBOLT_MAX_PENDING_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * The root directory in ZooKeeper for metadata about TransactionalSpouts. + */ + public static final String TRANSACTIONAL_ZOOKEEPER_ROOT="transactional.zookeeper.root"; + public static final Object TRANSACTIONAL_ZOOKEEPER_ROOT_SCHEMA = String.class; + + /** + * The list of zookeeper servers in which to keep the transactional state. If null (which is default), + * will use storm.zookeeper.servers + */ + public static final String TRANSACTIONAL_ZOOKEEPER_SERVERS="transactional.zookeeper.servers"; + public static final Object TRANSACTIONAL_ZOOKEEPER_SERVERS_SCHEMA = ConfigValidation.StringsValidator; + + /** + * The port to use to connect to the transactional zookeeper servers. If null (which is default), + * will use storm.zookeeper.port + */ + public static final String TRANSACTIONAL_ZOOKEEPER_PORT="transactional.zookeeper.port"; + public static final Object TRANSACTIONAL_ZOOKEEPER_PORT_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * The user as which the nimbus client should be acquired to perform the operation. + */ + public static final String STORM_DO_AS_USER="storm.doAsUser"; + public static final Object STORM_DO_AS_USER_SCHEMA = String.class; + + /** + * The number of threads that should be used by the zeromq context in each worker process. + */ + public static final String ZMQ_THREADS = "zmq.threads"; + public static final Object ZMQ_THREADS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * How long a connection should retry sending messages to a target host when + * the connection is closed. This is an advanced configuration and can almost + * certainly be ignored. + */ + public static final String ZMQ_LINGER_MILLIS = "zmq.linger.millis"; + public static final Object ZMQ_LINGER_MILLIS_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * The high water for the ZeroMQ push sockets used for networking. Use this config to prevent buffer explosion + * on the networking layer. + */ + public static final String ZMQ_HWM = "zmq.hwm"; + public static final Object ZMQ_HWM_SCHEMA = ConfigValidation.IntegerValidator; + + /** + * This value is passed to spawned JVMs (e.g., Nimbus, Supervisor, and Workers) + * for the java.library.path value. java.library.path tells the JVM where + * to look for native libraries. It is necessary to set this config correctly since + * Storm uses the ZeroMQ and JZMQ native libs. + */ + public static final String JAVA_LIBRARY_PATH = "java.library.path"; + public static final Object JAVA_LIBRARY_PATH_SCHEMA = String.class; + + /** + * The path to use as the zookeeper dir when running a zookeeper server via + * "storm dev-zookeeper". This zookeeper instance is only intended for development; + * it is not a production grade zookeeper setup. + */ + public static final String DEV_ZOOKEEPER_PATH = "dev.zookeeper.path"; + public static final Object DEV_ZOOKEEPER_PATH_SCHEMA = String.class; + + /** + * A map from topology name to the number of machines that should be dedicated for that topology. Set storm.scheduler + * to backtype.storm.scheduler.IsolationScheduler to make use of the isolation scheduler. + */ + public static final String ISOLATION_SCHEDULER_MACHINES = "isolation.scheduler.machines"; + public static final Object ISOLATION_SCHEDULER_MACHINES_SCHEMA = ConfigValidation.MapOfStringToNumberValidator; + + /** + * A map from the user name to the number of machines that should that user is allowed to use. Set storm.scheduler + * to backtype.storm.scheduler.multitenant.MultitenantScheduler + */ + public static final String MULTITENANT_SCHEDULER_USER_POOLS = "multitenant.scheduler.user.pools"; + public static final Object MULTITENANT_SCHEDULER_USER_POOLS_SCHEMA = ConfigValidation.MapOfStringToNumberValidator; + + /** + * The number of machines that should be used by this topology to isolate it from all others. Set storm.scheduler + * to backtype.storm.scheduler.multitenant.MultitenantScheduler + */ + public static final String TOPOLOGY_ISOLATED_MACHINES = "topology.isolate.machines"; + public static final Object TOPOLOGY_ISOLATED_MACHINES_SCHEMA = Number.class; + + public static void setClasspath(Map conf, String cp) { + conf.put(Config.TOPOLOGY_CLASSPATH, cp); + } + + public void setClasspath(String cp) { + setClasspath(this, cp); + } + + public static void setEnvironment(Map conf, Map env) { + conf.put(Config.TOPOLOGY_ENVIRONMENT, env); + } + + public void setEnvironment(Map env) { + setEnvironment(this, env); + } + + public static void setDebug(Map conf, boolean isOn) { + conf.put(Config.TOPOLOGY_DEBUG, isOn); + } + + public void setDebug(boolean isOn) { + setDebug(this, isOn); + } + + public static void setNumWorkers(Map conf, int workers) { + conf.put(Config.TOPOLOGY_WORKERS, workers); + } + + public void setNumWorkers(int workers) { + setNumWorkers(this, workers); + } + + public static void setNumAckers(Map conf, int numExecutors) { + conf.put(Config.TOPOLOGY_ACKER_EXECUTORS, numExecutors); + } + + public void setNumAckers(int numExecutors) { + setNumAckers(this, numExecutors); + } + + public static void setMessageTimeoutSecs(Map conf, int secs) { + conf.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, secs); + } + + public void setMessageTimeoutSecs(int secs) { + setMessageTimeoutSecs(this, secs); + } + + public static void registerSerialization(Map conf, Class klass) { + getRegisteredSerializations(conf).add(klass.getName()); + } + + public void registerSerialization(Class klass) { + registerSerialization(this, klass); + } + + public static void registerSerialization(Map conf, Class klass, Class serializerClass) { + Map register = new HashMap(); + register.put(klass.getName(), serializerClass.getName()); + getRegisteredSerializations(conf).add(register); + } + + public void registerSerialization(Class klass, Class serializerClass) { + registerSerialization(this, klass, serializerClass); + } + + public static void registerMetricsConsumer(Map conf, Class klass, Object argument, long parallelismHint) { + HashMap m = new HashMap(); + m.put("class", klass.getCanonicalName()); + m.put("parallelism.hint", parallelismHint); + m.put("argument", argument); + + List l = (List)conf.get(TOPOLOGY_METRICS_CONSUMER_REGISTER); + if (l == null) { l = new ArrayList(); } + l.add(m); + conf.put(TOPOLOGY_METRICS_CONSUMER_REGISTER, l); + } + + public void registerMetricsConsumer(Class klass, Object argument, long parallelismHint) { + registerMetricsConsumer(this, klass, argument, parallelismHint); + } + + public static void registerMetricsConsumer(Map conf, Class klass, long parallelismHint) { + registerMetricsConsumer(conf, klass, null, parallelismHint); + } + + public void registerMetricsConsumer(Class klass, long parallelismHint) { + registerMetricsConsumer(this, klass, parallelismHint); + } + + public static void registerMetricsConsumer(Map conf, Class klass) { + registerMetricsConsumer(conf, klass, null, 1L); + } + + public void registerMetricsConsumer(Class klass) { + registerMetricsConsumer(this, klass); + } + + public static void registerDecorator(Map conf, Class klass) { + getRegisteredDecorators(conf).add(klass.getName()); + } + + public void registerDecorator(Class klass) { + registerDecorator(this, klass); + } + + public static void setKryoFactory(Map conf, Class klass) { + conf.put(Config.TOPOLOGY_KRYO_FACTORY, klass.getName()); + } + + public void setKryoFactory(Class klass) { + setKryoFactory(this, klass); + } + + public static void setSkipMissingKryoRegistrations(Map conf, boolean skip) { + conf.put(Config.TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS, skip); + } + + public void setSkipMissingKryoRegistrations(boolean skip) { + setSkipMissingKryoRegistrations(this, skip); + } + + public static void setMaxTaskParallelism(Map conf, int max) { + conf.put(Config.TOPOLOGY_MAX_TASK_PARALLELISM, max); + } + + public void setMaxTaskParallelism(int max) { + setMaxTaskParallelism(this, max); + } + + public static void setMaxSpoutPending(Map conf, int max) { + conf.put(Config.TOPOLOGY_MAX_SPOUT_PENDING, max); + } + + public void setMaxSpoutPending(int max) { + setMaxSpoutPending(this, max); + } + + public static void setStatsSampleRate(Map conf, double rate) { + conf.put(Config.TOPOLOGY_STATS_SAMPLE_RATE, rate); + } + + public void setStatsSampleRate(double rate) { + setStatsSampleRate(this, rate); + } + + public static void setFallBackOnJavaSerialization(Map conf, boolean fallback) { + conf.put(Config.TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION, fallback); + } + + public void setFallBackOnJavaSerialization(boolean fallback) { + setFallBackOnJavaSerialization(this, fallback); + } + + private static List getRegisteredSerializations(Map conf) { + List ret; + if(!conf.containsKey(Config.TOPOLOGY_KRYO_REGISTER)) { + ret = new ArrayList(); + } else { + ret = new ArrayList((List) conf.get(Config.TOPOLOGY_KRYO_REGISTER)); + } + conf.put(Config.TOPOLOGY_KRYO_REGISTER, ret); + return ret; + } + + private static List getRegisteredDecorators(Map conf) { + List ret; + if(!conf.containsKey(Config.TOPOLOGY_KRYO_DECORATORS)) { + ret = new ArrayList(); + } else { + ret = new ArrayList((List) conf.get(Config.TOPOLOGY_KRYO_DECORATORS)); + } + conf.put(Config.TOPOLOGY_KRYO_DECORATORS, ret); + return ret; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/ConfigValidation.java b/jstorm-core/src/main/java/backtype/storm/ConfigValidation.java new file mode 100755 index 000000000..24991d783 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/ConfigValidation.java @@ -0,0 +1,351 @@ +/** + * 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 backtype.storm; +import java.util.Map; + +import java.util.Map; + +/** + * Provides functionality for validating configuration fields. + */ +public class ConfigValidation { + + /** + * Declares methods for validating configuration values. + */ + public static interface FieldValidator { + /** + * Validates the given field. + * @param name the name of the field. + * @param field The field to be validated. + * @throws IllegalArgumentException if the field fails validation. + */ + public void validateField(String name, Object field) throws IllegalArgumentException; + } + + /** + * Declares a method for validating configuration values that is nestable. + */ + public static abstract class NestableFieldValidator implements FieldValidator { + @Override + public void validateField(String name, Object field) throws IllegalArgumentException { + validateField(null, name, field); + } + + /** + * Validates the given field. + * @param pd describes the parent wrapping this validator. + * @param name the name of the field. + * @param field The field to be validated. + * @throws IllegalArgumentException if the field fails validation. + */ + public abstract void validateField(String pd, String name, Object field) throws IllegalArgumentException; + } + + /** + * Returns a new NestableFieldValidator for a given class. + * @param cls the Class the field should be a type of + * @param nullAllowed whether or not a value of null is valid + * @return a NestableFieldValidator for that class + */ + public static NestableFieldValidator fv(final Class cls, final boolean nullAllowed) { + return new NestableFieldValidator() { + @Override + public void validateField(String pd, String name, Object field) + throws IllegalArgumentException { + if (nullAllowed && field == null) { + return; + } + if (! cls.isInstance(field)) { + throw new IllegalArgumentException( + pd + name + " must be a " + cls.getName() + ". ("+field+")"); + } + } + }; + } + + /** + * Returns a new NestableFieldValidator for a List of the given Class. + * @param cls the Class of elements composing the list + * @param nullAllowed whether or not a value of null is valid + * @return a NestableFieldValidator for a list of the given class + */ + public static NestableFieldValidator listFv(Class cls, boolean nullAllowed) { + return listFv(fv(cls, false), nullAllowed); + } + + /** + * Returns a new NestableFieldValidator for a List where each item is validated by validator. + * @param validator used to validate each item in the list + * @param nullAllowed whether or not a value of null is valid + * @return a NestableFieldValidator for a list with each item validated by a different validator. + */ + public static NestableFieldValidator listFv(final NestableFieldValidator validator, + final boolean nullAllowed) { + return new NestableFieldValidator() { + @Override + public void validateField(String pd, String name, Object field) + throws IllegalArgumentException { + if (nullAllowed && field == null) { + return; + } + if (field instanceof Iterable) { + for (Object e : (Iterable)field) { + validator.validateField(pd + "Each element of the list ", name, e); + } + return; + } + throw new IllegalArgumentException( + "Field " + name + " must be an Iterable but was " + + ((field == null) ? "null" : ("a " + field.getClass()))); + } + }; + } + + /** + * Returns a new NestableFieldValidator for a Map of key to val. + * @param key the Class of keys in the map + * @param val the Class of values in the map + * @param nullAllowed whether or not a value of null is valid + * @return a NestableFieldValidator for a Map of key to val + */ + public static NestableFieldValidator mapFv(Class key, Class val, + boolean nullAllowed) { + return mapFv(fv(key, false), fv(val, false), nullAllowed); + } + + /** + * Returns a new NestableFieldValidator for a Map. + * @param key a validator for the keys in the map + * @param val a validator for the values in the map + * @param nullAllowed whether or not a value of null is valid + * @return a NestableFieldValidator for a Map + */ + public static NestableFieldValidator mapFv(final NestableFieldValidator key, + final NestableFieldValidator val, final boolean nullAllowed) { + return new NestableFieldValidator() { + @SuppressWarnings("unchecked") + @Override + public void validateField(String pd, String name, Object field) + throws IllegalArgumentException { + if (nullAllowed && field == null) { + return; + } + if (field instanceof Map) { + for (Map.Entry entry: ((Map)field).entrySet()) { + key.validateField("Each key of the map ", name, entry.getKey()); + val.validateField("Each value in the map ", name, entry.getValue()); + } + return; + } + throw new IllegalArgumentException( + "Field " + name + " must be a Map"); + } + }; + } + + /** + * Validates a list of Numbers. + */ + public static Object NumbersValidator = listFv(Number.class, true); + + /** + * Validates a list of Strings. + */ + public static Object StringsValidator = listFv(String.class, true); + + /** + * Validates a map of Strings to Numbers. + */ + public static Object MapOfStringToNumberValidator = mapFv(String.class, Number.class, true); + + /** + * Validates a map of Strings to a map of Strings to a list. + * {str -> {str -> [str,str]} + */ + public static Object MapOfStringToMapValidator = mapFv(fv(String.class, false), mapFv(fv(String.class, false), listFv(String.class, false), false), true); + + /** + * Validates is a list of Maps. + */ + public static Object MapsValidator = listFv(Map.class, true); + + /** + * Validates a Integer. + */ + public static Object IntegerValidator = new FieldValidator() { + @Override + public void validateField(String name, Object o) throws IllegalArgumentException { + if (o == null) { + // A null value is acceptable. + return; + } + final long i; + if (o instanceof Number && + (i = ((Number)o).longValue()) == ((Number)o).doubleValue()) { + if (i <= Integer.MAX_VALUE && i >= Integer.MIN_VALUE) { + return; + } + } + + throw new IllegalArgumentException("Field " + name + " must be an Integer within type range."); + } + }; + + /** + * Validates is a list of Integers. + */ + public static Object IntegersValidator = new FieldValidator() { + @Override + public void validateField(String name, Object field) + throws IllegalArgumentException { + if (field == null) { + // A null value is acceptable. + return; + } + if (field instanceof Iterable) { + for (Object o : (Iterable)field) { + final long i; + if (o instanceof Number && + ((i = ((Number)o).longValue()) == ((Number)o).doubleValue()) && + (i <= Integer.MAX_VALUE && i >= Integer.MIN_VALUE)) { + // pass the test + } else { + throw new IllegalArgumentException( + "Each element of the list " + name + " must be an Integer within type range."); + } + } + return; + } + } + }; + + /** + * Validates a Double. + */ + public static Object DoubleValidator = new FieldValidator() { + @Override + public void validateField(String name, Object o) throws IllegalArgumentException { + if (o == null) { + // A null value is acceptable. + return; + } + + // we can provide a lenient way to convert int/long to double with losing some precision + if (o instanceof Number) { + return; + } + + throw new IllegalArgumentException("Field " + name + " must be an Double."); + } + }; + + /** + * Validates a power of 2. + */ + public static Object PowerOf2Validator = new FieldValidator() { + @Override + public void validateField(String name, Object o) throws IllegalArgumentException { + if (o == null) { + // A null value is acceptable. + return; + } + final long i; + if (o instanceof Number && + (i = ((Number)o).longValue()) == ((Number)o).doubleValue()) + { + // Test whether the integer is a power of 2. + if (i > 0 && (i & (i-1)) == 0) { + return; + } + } + throw new IllegalArgumentException("Field " + name + " must be a power of 2."); + } + }; + + /** + * Validates a positive integer. + */ + public static Object PositiveIntegerValidator = new FieldValidator() { + @Override + public void validateField(String name, Object o) throws IllegalArgumentException { + if (o == null) { + // A null value is acceptable. + return; + } + final long i; + if (o instanceof Number && + (i = ((Number)o).longValue()) == ((Number)o).doubleValue()) + { + if (i > 0) { + return; + } + } + throw new IllegalArgumentException("Field " + name + " must be a positive integer."); + } + }; + + /** + * Validates Kryo Registration + */ + public static Object KryoRegValidator = new FieldValidator() { + @Override + public void validateField(String name, Object o) throws IllegalArgumentException { + if (o == null) { + // A null value is acceptable. + return; + } + if (o instanceof Iterable) { + for (Object e : (Iterable)o) { + if (e instanceof Map) { + for (Map.Entry entry: ((Map)e).entrySet()) { + if (!(entry.getKey() instanceof String) || + !(entry.getValue() instanceof String)) { + throw new IllegalArgumentException( + "Each element of the list " + name + " must be a String or a Map of Strings"); + } + } + } else if (!(e instanceof String)) { + throw new IllegalArgumentException( + "Each element of the list " + name + " must be a String or a Map of Strings"); + } + } + return; + } + throw new IllegalArgumentException( + "Field " + name + " must be an Iterable containing only Strings or Maps of Strings"); + } + }; + + /** + * Validates a String or a list of Strings + */ + public static Object StringOrStringListValidator = new FieldValidator() { + + private FieldValidator fv = listFv(String.class, false); + + @Override + public void validateField(String name, Object o) throws IllegalArgumentException { + if (o == null || o instanceof String) { + // A null value or a String value is acceptable + return; + } + this.fv.validateField(name, o); + } + }; +} diff --git a/jstorm-core/src/main/java/backtype/storm/Constants.java b/jstorm-core/src/main/java/backtype/storm/Constants.java new file mode 100755 index 000000000..2797b6995 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/Constants.java @@ -0,0 +1,37 @@ +/** + * 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 backtype.storm; + +import backtype.storm.coordination.CoordinatedBolt; +import clojure.lang.RT; + + +public class Constants { + public static final String COORDINATED_STREAM_ID = CoordinatedBolt.class.getName() + "/coord-stream"; + + public static final long SYSTEM_TASK_ID = -1; + public static final Object SYSTEM_EXECUTOR_ID = RT.readString("[-1 -1]"); + public static final String SYSTEM_COMPONENT_ID = "__system"; + public static final String SYSTEM_TICK_STREAM_ID = "__tick"; + public static final String METRICS_COMPONENT_ID_PREFIX = "__metrics"; + public static final String METRICS_STREAM_ID = "__metrics"; + public static final String METRICS_TICK_STREAM_ID = "__metrics_tick"; + public static final String CREDENTIALS_CHANGED_STREAM_ID = "__credentials"; + + public static final String JSTORM_CONF_DIR = "JSTORM_CONF_DIR"; +} diff --git a/jstorm-core/src/main/java/backtype/storm/GenericOptionsParser.java b/jstorm-core/src/main/java/backtype/storm/GenericOptionsParser.java new file mode 100755 index 000000000..9319ce19f --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/GenericOptionsParser.java @@ -0,0 +1,283 @@ +/** + * 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 backtype.storm; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.InputStreamReader; +import java.io.IOException; +import java.io.PrintStream; +import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.LinkedHashMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.GnuParser; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.OptionBuilder; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.yaml.snakeyaml.Yaml; + +/** + * GenericOptionsParser is a utility to parse command line arguments generic to Storm. + * + * GenericOptionsParser recognizes several standard command line arguments, enabling applications to easily specify additional jar files, + * configuration resources, data files etc. + * + *

Generic Options

+ * + *

+ * The supported generic options are: + *

+ *

+ *

+ * + *
+ * -conf <conf.xml>                           load configurations from
+ *                                            <conf.xml>
+ * -conf <conf.yaml>                          load configurations from
+ *                                            <conf.yaml>
+ * -D <key=value>                             set <key> in configuration to
+ *                                            <value> (preserve value's type)
+ * -libjars <comma separated list of jars>    specify comma separated jars to be
+ *                                            used by the submitted topology
+ * 
+ * + *
+ *

+ * + * Note: The XML configuration file specified by -conf shall be readable by Hadoop's Configuration class. Also note that all configuration + * values of an XML file will be treated as strings, and not as specific types. + * + *

+ * The general command line syntax is: + *

+ *

+ *

+ *     storm jar app.jar [genericOptions] [commandOptions]
+ * 
+ *

+ * + *

+ * Generic command line arguments might modify Config objects, given to constructors. + *

+ * + *

Configuration priority

+ * + * The following list defines the priorities of different configuration sources, in ascending order. Thus, if a configuration appears in more than one of them, + * only the last one will take effect. + * + *
    + *
  • defaults.yaml in classpath. + *
  • storm.yaml in classpath. + *
  • Configurations from files specified with the -conf option, in the order of appearance. + *
  • Configurations defined with the -D option, in order of appearance. + *
+ * + *

+ * The functionality is implemented using Commons CLI. + *

+ * + * @see Tool + * @see ToolRunner + */ + +public class GenericOptionsParser { + static final Logger LOG = LoggerFactory.getLogger(GenericOptionsParser.class); + + static final Charset UTF8 = Charset.forName("UTF-8"); + + public static final String TOPOLOGY_LIB_PATH = "topology.lib.path"; + + public static final String TOPOLOGY_LIB_NAME = "topology.lib.name"; + + Config conf; + + CommandLine commandLine; + + // Order in this map is important for these purposes: + // - configuration priority + static final LinkedHashMap optionProcessors = new LinkedHashMap(); + + public GenericOptionsParser(Config conf, String[] args) throws ParseException { + this(conf, new Options(), args); + } + + public GenericOptionsParser(Config conf, Options options, String[] args) throws ParseException { + this.conf = conf; + parseGeneralOptions(options, conf, args); + } + + public String[] getRemainingArgs() { + return commandLine.getArgs(); + } + + public Config getConfiguration() { + return conf; + } + + static Options buildGeneralOptions(Options opts) { + Options r = new Options(); + + for (Object o : opts.getOptions()) + r.addOption((Option) o); + + Option libjars = OptionBuilder.withArgName("paths").hasArg().withDescription("comma separated jars to be used by the submitted topology").create("libjars"); + r.addOption(libjars); + optionProcessors.put("libjars", new LibjarsProcessor()); + + Option conf = OptionBuilder.withArgName("configuration file").hasArg().withDescription("an application configuration file").create("conf"); + r.addOption(conf); + optionProcessors.put("conf", new ConfFileProcessor()); + + // Must come after `conf': this option is of higher priority + Option extraConfig = OptionBuilder.withArgName("D").hasArg().withDescription("extra configurations (preserving types)").create("D"); + r.addOption(extraConfig); + optionProcessors.put("D", new ExtraConfigProcessor()); + + return r; + } + + void parseGeneralOptions(Options opts, Config conf, String[] args) throws ParseException { + opts = buildGeneralOptions(opts); + CommandLineParser parser = new GnuParser(); + commandLine = parser.parse(opts, args, true); + processGeneralOptions(conf, commandLine); + } + + void processGeneralOptions(Config conf, CommandLine commandLine) throws ParseException { + for (Map.Entry e : optionProcessors.entrySet()) + if (commandLine.hasOption(e.getKey())) + e.getValue().process(conf, commandLine); + } + + static List validateFiles(String pathList) throws IOException { + List l = new ArrayList(); + + for (String s : pathList.split(",")) { + File file = new File(s); + if (!file.exists()) + throw new FileNotFoundException("File `" + file.getAbsolutePath() + "' does not exist"); + + l.add(file); + } + + return l; + } + + public static void printGenericCommandUsage(PrintStream out) { + String[] strs = + new String[] { + "Generic options supported are", + " -conf load configurations from", + " ", + " -conf load configurations from", + " ", + " -D = set in configuration", + " to (preserve value's type)", + " -libjars specify comma separated", + " jars to be used by", + " the submitted topology", }; + for (String s : strs) + out.println(s); + } + + static interface OptionProcessor { + public void process(Config conf, CommandLine commandLine) throws ParseException; + } + + static class LibjarsProcessor implements OptionProcessor { + @Override + public void process(Config conf, CommandLine commandLine) throws ParseException { + try { + List jarFiles = validateFiles(commandLine.getOptionValue("libjars")); + Map jars = new HashMap(jarFiles.size()); + List names = new ArrayList(jarFiles.size()); + for (File f : jarFiles) { + jars.put(f.getName(), f.getAbsolutePath()); + names.add(f.getName()); + } + conf.put(TOPOLOGY_LIB_PATH, jars); + conf.put(TOPOLOGY_LIB_NAME, names); + + } catch (IOException e) { + throw new ParseException(e.getMessage()); + } + } + } + + static class ExtraConfigProcessor implements OptionProcessor { + static final Yaml yaml = new Yaml(); + + @Override + public void process(Config conf, CommandLine commandLine) throws ParseException { + for (String s : commandLine.getOptionValues("D")) { + String[] keyval = s.split("=", 2); + if (keyval.length != 2) + throw new ParseException("Invalid option value `" + s + "'"); + + conf.putAll((Map) yaml.load(keyval[0] + ": " + keyval[1])); + } + } + } + + static class ConfFileProcessor implements OptionProcessor { + static final Yaml yaml = new Yaml(); + + static Map loadYamlConf(String f) throws IOException { + InputStreamReader reader = null; + try { + FileInputStream fis = new FileInputStream(f); + reader = new InputStreamReader(fis, UTF8); + return (Map) yaml.load(reader); + } finally { + if (reader != null) + reader.close(); + } + } + + static Map loadConf(String f) throws IOException { + if (f.endsWith(".yaml")) + return loadYamlConf(f); + throw new IOException("Unknown configuration file type: " + f + " does not end with either .yaml"); + } + + @Override + public void process(Config conf, CommandLine commandLine) throws ParseException { + try { + for (String f : commandLine.getOptionValues("conf")) { + Map m = loadConf(f); + if (m == null) + throw new ParseException("Empty configuration file " + f); + conf.putAll(m); + } + } catch (IOException e) { + throw new ParseException(e.getMessage()); + } + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/ICredentialsListener.java b/jstorm-core/src/main/java/backtype/storm/ICredentialsListener.java new file mode 100755 index 000000000..1a7bc1b84 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/ICredentialsListener.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm; + +import java.util.Map; + +/** + * Allows a bolt or a spout to be informed when the credentials of the topology have changed. + */ +public interface ICredentialsListener { + /** + * Called when the credentials of a topology have changed. + * @param credentials the new credentials, could be null. + */ + public void setCredentials(Map credentials); +} diff --git a/jstorm-core/src/main/java/backtype/storm/ILocalCluster.java b/jstorm-core/src/main/java/backtype/storm/ILocalCluster.java new file mode 100755 index 000000000..7d5aa35b4 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/ILocalCluster.java @@ -0,0 +1,49 @@ +/** + * 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 backtype.storm; + +import backtype.storm.generated.AlreadyAliveException; +import backtype.storm.generated.ClusterSummary; +import backtype.storm.generated.InvalidTopologyException; +import backtype.storm.generated.KillOptions; +import backtype.storm.generated.SubmitOptions; +import backtype.storm.generated.NotAliveException; +import backtype.storm.generated.RebalanceOptions; +import backtype.storm.generated.StormTopology; +import backtype.storm.generated.TopologyInfo; +import backtype.storm.generated.Credentials; + +import java.util.Map; + + +public interface ILocalCluster { + void submitTopology(String topologyName, Map conf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException; + void submitTopologyWithOpts(String topologyName, Map conf, StormTopology topology, SubmitOptions submitOpts) throws AlreadyAliveException, InvalidTopologyException; + void uploadNewCredentials(String topologyName, Credentials creds); + void killTopology(String topologyName) throws NotAliveException; + void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException; + void activate(String topologyName) throws NotAliveException; + void deactivate(String topologyName) throws NotAliveException; + void rebalance(String name, RebalanceOptions options) throws NotAliveException; + void shutdown(); + String getTopologyConf(String id); + StormTopology getTopology(String id); + ClusterSummary getClusterInfo(); + TopologyInfo getTopologyInfo(String id); + Map getState(); +} diff --git a/jstorm-core/src/main/java/backtype/storm/ILocalDRPC.java b/jstorm-core/src/main/java/backtype/storm/ILocalDRPC.java new file mode 100755 index 000000000..e478dcafc --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/ILocalDRPC.java @@ -0,0 +1,27 @@ +/** + * 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 backtype.storm; + +import backtype.storm.daemon.Shutdownable; +import backtype.storm.generated.DistributedRPC; +import backtype.storm.generated.DistributedRPCInvocations; + + +public interface ILocalDRPC extends DistributedRPC.Iface, DistributedRPCInvocations.Iface, Shutdownable { + public String getServiceId(); +} diff --git a/jstorm-core/src/main/java/backtype/storm/LocalCluster.java b/jstorm-core/src/main/java/backtype/storm/LocalCluster.java new file mode 100755 index 000000000..b55bac4f4 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/LocalCluster.java @@ -0,0 +1,275 @@ +/** + * 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 backtype.storm; + +import java.util.Map; + +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.generated.ClusterSummary; +import backtype.storm.generated.Credentials; +import backtype.storm.generated.KillOptions; +import backtype.storm.generated.NotAliveException; +import backtype.storm.generated.RebalanceOptions; +import backtype.storm.generated.StormTopology; +import backtype.storm.generated.SubmitOptions; +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 = LoggerFactory.getLogger(LocalCluster.class); + + private LocalClusterMap state; + + protected void setLogger() { + // the code is for log4j + // 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); + // } + + } + + // this is easy to debug + protected static LocalCluster instance = null; + + public static LocalCluster getInstance() { + return instance; + } + + public LocalCluster() { + synchronized (LocalCluster.class) { + if (instance != null) { + throw new RuntimeException("LocalCluster should be single"); + } + 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"); + + instance = this; + } + } + + @Override + public void submitTopology(String topologyName, Map conf, StormTopology topology) { + submitTopologyWithOpts(topologyName, conf, topology, null); + } + + @Override + public void submitTopologyWithOpts(String topologyName, Map conf, StormTopology topology, SubmitOptions submitOpts) { + // 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, Utils.to_json(conf), topology); + } else { + state.getNimbus().submitTopologyWithOpts(topologyName, null, Utils.to_json(conf), topology, submitOpts); + } + + } catch (Exception e) { + // TODO Auto-generated catch block + LOG.error("Failed to submit topology " + topologyName, e); + throw new RuntimeException(e); + } + } + + @Override + public void killTopology(String topologyName) { + // TODO Auto-generated method stub + try { + // 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); + } + } + + @Override + public void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException { + // TODO Auto-generated method stub + try { + state.getNimbus().killTopologyWithOpts(name, options); + } catch (TException e) { + // TODO Auto-generated catch block + LOG.error("fail to kill Topology " + name, e); + throw new RuntimeException(e); + } + } + + @Override + public void activate(String topologyName) { + // TODO Auto-generated method stub + try { + state.getNimbus().activate(topologyName); + } catch (Exception e) { + // TODO Auto-generated catch block + LOG.error("fail to activate " + topologyName, e); + throw new RuntimeException(e); + } + } + + @Override + public void deactivate(String topologyName) { + // TODO Auto-generated method stub + try { + state.getNimbus().deactivate(topologyName); + } catch (Exception e) { + // TODO Auto-generated catch block + LOG.error("fail to deactivate " + topologyName, e); + throw new RuntimeException(e); + } + } + + @Override + public void rebalance(String name, RebalanceOptions options) { + // TODO Auto-generated method stub + try { + state.getNimbus().rebalance(name, options); + } catch (Exception e) { + // TODO Auto-generated catch block + LOG.error("fail to rebalance " + name, e); + throw new RuntimeException(e); + } + } + + @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(); + } + + @Override + public String getTopologyConf(String id) { + // TODO Auto-generated method stub + try { + return state.getNimbus().getTopologyConf(id); + } catch (Exception e) { + // TODO Auto-generated catch block + LOG.error("fail to get topology Conf of topologId: " + id, e); + } + return null; + } + + @Override + public StormTopology getTopology(String id) { + // TODO Auto-generated method stub + try { + return state.getNimbus().getTopology(id); + } catch (NotAliveException e) { + // TODO Auto-generated catch block + LOG.error("fail to get topology of topologId: " + id, e); + } catch (TException e) { + // TODO Auto-generated catch block + LOG.error("fail to get topology of topologId: " + id, e); + } + return null; + } + + @Override + public ClusterSummary getClusterInfo() { + // TODO Auto-generated method stub + try { + return state.getNimbus().getClusterInfo(); + } catch (TException e) { + // TODO Auto-generated catch block + LOG.error("fail to get cluster info", e); + } + return null; + } + + @Override + public TopologyInfo getTopologyInfo(String id) { + // TODO Auto-generated method stub + try { + return state.getNimbus().getTopologyInfo(id); + } catch (NotAliveException e) { + // TODO Auto-generated catch block + LOG.error("fail to get topology info of topologyId: " + id, e); + } catch (TException e) { + // TODO Auto-generated catch block + LOG.error("fail to get topology info of topologyId: " + id, e); + } + return null; + } + + /*** + * You should use getLocalClusterMap() to instead.This function will always return null + * */ + @Deprecated + @Override + public Map getState() { + // TODO Auto-generated method stub + return null; + } + + public LocalClusterMap getLocalClusterMap() { + return state; + } + + public static void main(String[] args) throws Exception { + LocalCluster localCluster = null; + try { + localCluster = new LocalCluster(); + } finally { + if (localCluster != null) { + localCluster.shutdown(); + } + } + } + + @Override + public void uploadNewCredentials(String topologyName, Credentials creds) { + // TODO Auto-generated method stub + try { + state.getNimbus().uploadNewCredentials(topologyName, creds); + } catch (Exception e) { + // TODO Auto-generated catch block + LOG.error("fail to uploadNewCredentials of topologyId: " + topologyName, e); + } + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/LocalClusterMap.java b/jstorm-core/src/main/java/backtype/storm/LocalClusterMap.java new file mode 100755 index 000000000..bd99c7615 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/LocalClusterMap.java @@ -0,0 +1,126 @@ +/** + * 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 backtype.storm; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.daemon.nimbus.NimbusServer; +import com.alibaba.jstorm.daemon.nimbus.ServiceHandler; +import com.alibaba.jstorm.daemon.supervisor.SupervisorManger; +import com.alibaba.jstorm.utils.PathUtils; +import com.alibaba.jstorm.zk.Factory; + +public class LocalClusterMap { + + public static Logger LOG = LoggerFactory.getLogger(LocalClusterMap.class); + + private NimbusServer nimbusServer; + + private ServiceHandler nimbus; + + private Factory zookeeper; + + private Map conf; + + private List tmpDir; + + private SupervisorManger supervisor; + + public ServiceHandler getNimbus() { + return nimbus; + } + + public void setNimbus(ServiceHandler nimbus) { + this.nimbus = nimbus; + } + + public Factory getZookeeper() { + return zookeeper; + } + + public void setZookeeper(Factory zookeeper) { + this.zookeeper = zookeeper; + } + + public Map getConf() { + return conf; + } + + public void setConf(Map conf) { + this.conf = conf; + } + + public NimbusServer getNimbusServer() { + return nimbusServer; + } + + public void setNimbusServer(NimbusServer nimbusServer) { + this.nimbusServer = nimbusServer; + } + + public SupervisorManger getSupervisor() { + return supervisor; + } + + public void setSupervisor(SupervisorManger supervisor) { + this.supervisor = supervisor; + } + + public List getTmpDir() { + return tmpDir; + } + + public void setTmpDir(List tmpDir) { + this.tmpDir = tmpDir; + } + + public void clean() { + + if (supervisor != null) { + supervisor.ShutdownAllWorkers(); + supervisor.shutdown(); + } + + if (nimbusServer != null) { + nimbusServer.cleanup(); + } + + if (zookeeper != null) + zookeeper.shutdown(); + + // it will hava a problem: + // java.io.IOException: Unable to delete file: + // {TmpPath}\{UUID}\version-2\log.1 + if (tmpDir != null) { + for (String dir : tmpDir) { + try { + PathUtils.rmr(dir); + } catch (IOException e) { + // TODO Auto-generated catch block + LOG.error("Fail to delete " + dir); + } + } + } + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/LocalDRPC.java b/jstorm-core/src/main/java/backtype/storm/LocalDRPC.java new file mode 100755 index 000000000..4113bf4f6 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/LocalDRPC.java @@ -0,0 +1,100 @@ +/** + * 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 backtype.storm; + +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.generated.DRPCRequest; +import backtype.storm.utils.ServiceRegistry; + +import com.alibaba.jstorm.drpc.Drpc; + +public class LocalDRPC implements ILocalDRPC { + private static final Logger LOG = LoggerFactory.getLogger(LocalDRPC.class); + + private Drpc handler = new Drpc(); + private Thread thread; + + private final String serviceId; + + public LocalDRPC() { + + thread = new Thread(new Runnable() { + + @Override + public void run() { + LOG.info("Begin to init local Drpc"); + try { + handler.init(); + } catch (Exception e) { + LOG.info("Failed to start local drpc"); + System.exit(-1); + } + LOG.info("Successfully start local drpc"); + } + }); + thread.start(); + + serviceId = ServiceRegistry.registerService(handler); + } + + @Override + public String execute(String functionName, String funcArgs) { + // TODO Auto-generated method stub + try { + return handler.execute(functionName, funcArgs); + } catch (Exception e) { + LOG.error("", e); + throw new RuntimeException(e); + } + } + + @Override + public void result(String id, String result) throws TException { + // TODO Auto-generated method stub + handler.result(id, result); + } + + @Override + public DRPCRequest fetchRequest(String functionName) throws TException { + // TODO Auto-generated method stub + return handler.fetchRequest(functionName); + } + + @Override + public void failRequest(String id) throws TException { + // TODO Auto-generated method stub + handler.failRequest(id); + } + + @Override + public void shutdown() { + // TODO Auto-generated method stub + ServiceRegistry.unregisterService(this.serviceId); + this.handler.shutdown(); + } + + @Override + public String getServiceId() { + // TODO Auto-generated method stub + return serviceId; + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/LocalUtils.java b/jstorm-core/src/main/java/backtype/storm/LocalUtils.java new file mode 100755 index 000000000..e32c07e31 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/LocalUtils.java @@ -0,0 +1,127 @@ +/** + * 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 backtype.storm; + +import java.io.File; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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; +import com.alibaba.jstorm.message.netty.NettyContext; +import com.alibaba.jstorm.zk.Factory; +import com.alibaba.jstorm.zk.Zookeeper; + +public class LocalUtils { + + public static Logger LOG = LoggerFactory.getLogger(LocalUtils.class); + + public static LocalClusterMap prepareLocalCluster() { + LocalClusterMap state = new LocalClusterMap(); + try { + List tmpDirs = new ArrayList(); + + String zkDir = getTmpDir(); + tmpDirs.add(zkDir); + Factory zookeeper = startLocalZookeeper(zkDir); + Map conf = getLocalConf(zookeeper.getZooKeeperServer().getClientPort()); + + String nimbusDir = getTmpDir(); + tmpDirs.add(nimbusDir); + Map nimbusConf = deepCopyMap(conf); + nimbusConf.put(Config.STORM_LOCAL_DIR, nimbusDir); + NimbusServer instance = new NimbusServer(); + + Map supervisorConf = deepCopyMap(conf); + String supervisorDir = getTmpDir(); + tmpDirs.add(supervisorDir); + supervisorConf.put(Config.STORM_LOCAL_DIR, supervisorDir); + Supervisor supervisor = new Supervisor(); + IContext context = getLocalContext(supervisorConf); + + state.setNimbusServer(instance); + state.setNimbus(instance.launcherLocalServer(nimbusConf, new DefaultInimbus())); + state.setZookeeper(zookeeper); + state.setConf(conf); + state.setTmpDir(tmpDirs); + state.setSupervisor(supervisor.mkSupervisor(supervisorConf, context)); + return state; + } catch (Exception e) { + LOG.error("prepare cluster error!", e); + state.clean(); + + } + return null; + } + + private static Factory startLocalZookeeper(String tmpDir) { + for (int i = 2000; i < 65535; i++) { + try { + return Zookeeper.mkInprocessZookeeper(tmpDir, i); + } catch (Exception e) { + LOG.error("fail to launch zookeeper at port: " + i, e); + } + } + throw new RuntimeException("No port is available to launch an inprocess zookeeper."); + } + + private static String getTmpDir() { + return System.getProperty("java.io.tmpdir") + File.separator + UUID.randomUUID(); + } + + private static Map getLocalConf(int port) { + List zkServers = new ArrayList(1); + zkServers.add("localhost"); + Map conf = Utils.readStormConfig(); + conf.put(Config.STORM_CLUSTER_MODE, "local"); + conf.put(Config.STORM_ZOOKEEPER_SERVERS, zkServers); + conf.put(Config.STORM_ZOOKEEPER_PORT, port); + conf.put(Config.TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS, true); + 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; + } + + private static IContext getLocalContext(Map conf) { + if (!(Boolean) conf.get(Config.STORM_LOCAL_MODE_ZMQ)) { + IContext result = new NettyContext(); + ConfigExtension.setLocalWorkerPort(conf, 6800); + result.prepare(conf); + return result; + } + return null; + } + + private static Map deepCopyMap(Map map) { + return new HashMap(map); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/StormSubmitter.java b/jstorm-core/src/main/java/backtype/storm/StormSubmitter.java new file mode 100644 index 000000000..400875eee --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/StormSubmitter.java @@ -0,0 +1,366 @@ +/** + * 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 backtype.storm; + +import java.io.File; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.commons.lang.StringUtils; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.generated.AlreadyAliveException; +import backtype.storm.generated.InvalidTopologyException; +import backtype.storm.generated.Nimbus; +import backtype.storm.generated.NotAliveException; +import backtype.storm.generated.StormTopology; +import backtype.storm.generated.SubmitOptions; +import backtype.storm.generated.TopologyAssignException; +import backtype.storm.utils.BufferFileInputStream; +import backtype.storm.utils.NimbusClient; +import backtype.storm.utils.Utils; + +/** + * Use this class to submit topologies to run on the Storm cluster. You should + * run your program with the "storm jar" command from the command-line, and then + * use this class to submit your topologies. + */ +public class StormSubmitter { + public static Logger LOG = LoggerFactory.getLogger(StormSubmitter.class); + + private static Nimbus.Iface localNimbus = null; + + public static void setLocalNimbus(Nimbus.Iface localNimbusHandler) { + StormSubmitter.localNimbus = localNimbusHandler; + } + + /** + * Submits a topology to run on the cluster. A topology runs forever or + * until explicitly killed. + * + * + * @param name the name of the storm. + * @param stormConf the topology-specific configuration. See {@link Config}. + * @param topology the processing to execute. + * @throws AlreadyAliveException if a topology with this name is already + * running + * @throws InvalidTopologyException if an invalid topology was submitted + */ + public static void submitTopology(String name, Map stormConf, + StormTopology topology) throws AlreadyAliveException, + InvalidTopologyException { + submitTopology(name, stormConf, topology, null); + } + + public static void submitTopology(String name, Map stormConf, + StormTopology topology, SubmitOptions opts, List jarFiles) + throws AlreadyAliveException, InvalidTopologyException { + if (jarFiles == null) { + jarFiles = new ArrayList(); + } + Map jars = new HashMap(jarFiles.size()); + List names = new ArrayList(jarFiles.size()); + + for (File f : jarFiles) { + if (!f.exists()) { + LOG.info(f.getName() + " is not existed: " + + f.getAbsolutePath()); + continue; + } + jars.put(f.getName(), f.getAbsolutePath()); + names.add(f.getName()); + } + LOG.info("Files: " + names + " will be loaded"); + stormConf.put(GenericOptionsParser.TOPOLOGY_LIB_PATH, jars); + stormConf.put(GenericOptionsParser.TOPOLOGY_LIB_NAME, names); + submitTopology(name, stormConf, topology, opts); + } + + public static void submitTopology(String name, Map stormConf, + StormTopology topology, SubmitOptions opts, + ProgressListener listener) throws AlreadyAliveException, + InvalidTopologyException { + submitTopology(name, stormConf, topology, opts); + } + + /** + * Submits a topology to run on the cluster. A topology runs forever or + * until explicitly killed. + * + * + * @param name the name of the storm. + * @param stormConf the topology-specific configuration. See {@link Config}. + * @param topology the processing to execute. + * @param options to manipulate the starting of the topology + * @throws AlreadyAliveException if a topology with this name is already + * running + * @throws InvalidTopologyException if an invalid topology was submitted + */ + public static void submitTopology(String name, Map stormConf, + StormTopology topology, SubmitOptions opts) + throws AlreadyAliveException, InvalidTopologyException { + if (!Utils.isValidConf(stormConf)) { + throw new IllegalArgumentException( + "Storm conf is not valid. Must be json-serializable"); + } + stormConf = new HashMap(stormConf); + stormConf.putAll(Utils.readCommandLineOpts()); + Map conf = Utils.readStormConfig(); + conf.putAll(stormConf); + putUserInfo(conf, stormConf); + try { + String serConf = Utils.to_json(stormConf); + if (localNimbus != null) { + LOG.info("Submitting topology " + name + " in local mode"); + localNimbus.submitTopology(name, null, serConf, topology); + } else { + NimbusClient client = NimbusClient.getConfiguredClient(conf); + try { + if (topologyNameExists(client, conf, name)) { + throw new RuntimeException("Topology with name `" + name + + "` already exists on cluster"); + } + + submitJar(client, conf); + LOG.info("Submitting topology " + name + + " in distributed mode with conf " + serConf); + if (opts != null) { + client.getClient().submitTopologyWithOpts(name, path, + serConf, topology, opts); + } else { + // this is for backwards compatibility + client.getClient().submitTopology(name, path, serConf, + topology); + } + } finally { + client.close(); + } + } + LOG.info("Finished submitting topology: " + name); + } catch (InvalidTopologyException e) { + LOG.warn("Topology submission exception", e); + throw e; + } catch (AlreadyAliveException e) { + LOG.warn("Topology already alive exception", e); + throw e; + } catch (TopologyAssignException e) { + LOG.warn("Failed to assign " + e.get_msg(), e); + throw new RuntimeException(e); + } catch (TException e) { + LOG.warn("Failed to assign ", e); + throw new RuntimeException(e); + } + } + + /** + * Submits a topology to run on the cluster with a progress bar. A topology + * runs forever or until explicitly killed. + * + * + * @param name the name of the storm. + * @param stormConf the topology-specific configuration. See {@link Config}. + * @param topology the processing to execute. + * @throws AlreadyAliveException if a topology with this name is already + * running + * @throws InvalidTopologyException if an invalid topology was submitted + * @throws TopologyAssignException + */ + + public static void submitTopologyWithProgressBar(String name, + Map stormConf, StormTopology topology) + throws AlreadyAliveException, InvalidTopologyException { + submitTopologyWithProgressBar(name, stormConf, topology, null); + } + + /** + * Submits a topology to run on the cluster with a progress bar. A topology + * runs forever or until explicitly killed. + * + * + * @param name the name of the storm. + * @param stormConf the topology-specific configuration. See {@link Config}. + * @param topology the processing to execute. + * @param opts to manipulate the starting of the topology + * @throws AlreadyAliveException if a topology with this name is already + * running + * @throws InvalidTopologyException if an invalid topology was submitted + * @throws TopologyAssignException + */ + + public static void submitTopologyWithProgressBar(String name, + Map stormConf, StormTopology topology, SubmitOptions opts) + throws AlreadyAliveException, InvalidTopologyException { + + /** + * remove progress bar in jstorm + */ + submitTopology(name, stormConf, topology, opts); + } + + public static boolean topologyNameExists(NimbusClient client, Map conf, String name) { + if (StringUtils.isBlank(name)) { + throw new RuntimeException("TopologyName is empty"); + } + + try { + String topologyId = client.getClient().getTopologyId(name); + if (StringUtils.isBlank(topologyId) == false) { + return true; + } + return false; + + } catch (NotAliveException e) { + return false; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private static String submittedJar = null; + private static String path = null; + + private static void submitJar(NimbusClient client, Map conf) { + if (submittedJar == null) { + try { + LOG.info("Jar not uploaded to master yet. Submitting jar..."); + String localJar = System.getProperty("storm.jar"); + path = client.getClient().beginFileUpload(); + String[] pathCache = path.split("/"); + String uploadLocation = + path + "/stormjar-" + pathCache[pathCache.length - 1] + + ".jar"; + List lib = + (List) conf + .get(GenericOptionsParser.TOPOLOGY_LIB_NAME); + Map libPath = + (Map) conf + .get(GenericOptionsParser.TOPOLOGY_LIB_PATH); + if (lib != null && lib.size() != 0) { + for (String libName : lib) { + String jarPath = path + "/lib/" + libName; + client.getClient().beginLibUpload(jarPath); + submitJar(conf, libPath.get(libName), jarPath, client); + } + + } else { + if (localJar == null) { + // no lib, no client jar + throw new RuntimeException( + "No client app jar, please upload it"); + } + } + + if (localJar != null) { + submittedJar = + submitJar(conf, localJar, uploadLocation, client); + } else { + // no client jar, but with lib jar + client.getClient().finishFileUpload(uploadLocation); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } else { + LOG.info("Jar already uploaded to master. Not submitting jar."); + } + } + + public static String submitJar(Map conf, String localJar, + String uploadLocation, NimbusClient client) { + if (localJar == null) { + throw new RuntimeException( + "Must submit topologies using the 'storm' client script so that StormSubmitter knows which jar to upload."); + } + + try { + + LOG.info("Uploading topology jar " + localJar + + " to assigned location: " + uploadLocation); + int bufferSize = 512 * 1024; + Object maxBufSizeObject = + conf.get(Config.NIMBUS_THRIFT_MAX_BUFFER_SIZE); + if (maxBufSizeObject != null) { + bufferSize = Utils.getInt(maxBufSizeObject) / 2; + } + + BufferFileInputStream is = + new BufferFileInputStream(localJar, bufferSize); + while (true) { + byte[] toSubmit = is.read(); + if (toSubmit.length == 0) + break; + client.getClient().uploadChunk(uploadLocation, + ByteBuffer.wrap(toSubmit)); + } + client.getClient().finishFileUpload(uploadLocation); + LOG.info("Successfully uploaded topology jar to assigned location: " + + uploadLocation); + return uploadLocation; + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + + } + } + + private static void putUserInfo(Map conf, Map stormConf) { + stormConf.put("user.group", conf.get("user.group")); + stormConf.put("user.name", conf.get("user.name")); + stormConf.put("user.password", conf.get("user.password")); + } + + /** + * Interface use to track progress of file upload + */ + public interface ProgressListener { + /** + * called before file is uploaded + * + * @param srcFile - jar file to be uploaded + * @param targetFile - destination file + * @param totalBytes - total number of bytes of the file + */ + public void onStart(String srcFile, String targetFile, long totalBytes); + + /** + * called whenever a chunk of bytes is uploaded + * + * @param srcFile - jar file to be uploaded + * @param targetFile - destination file + * @param bytesUploaded - number of bytes transferred so far + * @param totalBytes - total number of bytes of the file + */ + public void onProgress(String srcFile, String targetFile, + long bytesUploaded, long totalBytes); + + /** + * called when the file is uploaded + * + * @param srcFile - jar file to be uploaded + * @param targetFile - destination file + * @param totalBytes - total number of bytes of the file + */ + public void onCompleted(String srcFile, String targetFile, + long totalBytes); + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/Tool.java b/jstorm-core/src/main/java/backtype/storm/Tool.java new file mode 100755 index 000000000..6722b2405 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/Tool.java @@ -0,0 +1,71 @@ +/** + * 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 backtype.storm; + +/** + * A tool abstract class that supports handling of generic command-line options. + * + *

+ * Here is how a typical Tool is implemented: + *

+ *

+ *

+ * + *
+ *     public class TopologyApp extends Tool {
+ *         {@literal @}Override
+ *         public int run(String[] args) throws Exception {
+ *             // Config processed by ToolRunner
+ *             Config conf = getConf();
+ * 
+ *             // Other setups go here
+ *             String name = "topology";
+ *             StormTopology topology = buildTopology(args);
+ *             StormSubmitter.submitTopology(name, conf, topology);
+ *             return 0;
+ *         }
+ * 
+ *         StormTopology buildTopology(String[] args) { ... }
+ * 
+ *         public static void main(String[] args) throws Exception {
+ *             // Use ToolRunner to handle generic command-line options
+ *             ToolRunner.run(new TopologyApp(), args);
+ *         }
+ *     }
+ * 
+ * + *
+ *

+ * + * @see GenericOptionsParser + * @see ToolRunner + */ + +public abstract class Tool { + Config config; + + public abstract int run(String[] args) throws Exception; + + public Config getConf() { + return config; + } + + public void setConf(Config config) { + this.config = config; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/ToolRunner.java b/jstorm-core/src/main/java/backtype/storm/ToolRunner.java new file mode 100755 index 000000000..33f5034d6 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/ToolRunner.java @@ -0,0 +1,69 @@ +/** + * 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 backtype.storm; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.cli.ParseException; + +import backtype.storm.utils.Utils; + +/** + * A utility to help run {@link Tool}s + * + *

+ * ToolRunner can be used to run classes extending the Tool abstract class. It works in conjunction with {@link GenericOptionsParser} to parse the generic storm command line arguments and modifies the Config of the + * Tool. The application-specific options are passed along without being modified. + * + * @see Tool + * @see GenericOptionsParser + */ + +public class ToolRunner { + static final Logger LOG = LoggerFactory.getLogger(ToolRunner.class); + + public static void run(Tool tool, String[] args) { + run(tool.getConf(), tool, args); + } + + public static void run(Config conf, Tool tool, String[] args) { + try { + if (conf == null) { + conf = new Config(); + conf.putAll(Utils.readStormConfig()); + } + + GenericOptionsParser parser = new GenericOptionsParser(conf, args); + tool.setConf(conf); + + System.exit(tool.run(parser.getRemainingArgs())); + } catch (ParseException e) { + LOG.error("Error parsing generic options: {}", e.getMessage()); + GenericOptionsParser.printGenericCommandUsage(System.err); + System.exit(2); + } catch (Exception e) { + LOG.error("Error running tool", e); + System.exit(1); + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/clojure/ClojureBolt.java b/jstorm-core/src/main/java/backtype/storm/clojure/ClojureBolt.java new file mode 100755 index 000000000..5de9bde98 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/clojure/ClojureBolt.java @@ -0,0 +1,119 @@ +/** + * 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 backtype.storm.clojure; + +import backtype.storm.coordination.CoordinatedBolt.FinishedCallback; +import backtype.storm.generated.StreamInfo; +import backtype.storm.task.IBolt; +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.IRichBolt; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.utils.Utils; +import clojure.lang.IFn; +import clojure.lang.PersistentArrayMap; +import clojure.lang.Keyword; +import clojure.lang.Symbol; +import clojure.lang.RT; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + + +public class ClojureBolt implements IRichBolt, FinishedCallback { + Map _fields; + List _fnSpec; + List _confSpec; + List _params; + + IBolt _bolt; + + public ClojureBolt(List fnSpec, List confSpec, List params, Map fields) { + _fnSpec = fnSpec; + _confSpec = confSpec; + _params = params; + _fields = fields; + } + + @Override + public void prepare(final Map stormConf, final TopologyContext context, final OutputCollector collector) { + IFn hof = Utils.loadClojureFn(_fnSpec.get(0), _fnSpec.get(1)); + try { + IFn preparer = (IFn) hof.applyTo(RT.seq(_params)); + final Map collectorMap = new PersistentArrayMap( new Object[] { + Keyword.intern(Symbol.create("output-collector")), collector, + Keyword.intern(Symbol.create("context")), context}); + List args = new ArrayList() {{ + add(stormConf); + add(context); + add(collectorMap); + }}; + + _bolt = (IBolt) preparer.applyTo(RT.seq(args)); + //this is kind of unnecessary for clojure + try { + _bolt.prepare(stormConf, context, collector); + } catch(AbstractMethodError ame) { + + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public void execute(Tuple input) { + _bolt.execute(input); + } + + @Override + public void cleanup() { + try { + _bolt.cleanup(); + } catch(AbstractMethodError ame) { + + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + for(String stream: _fields.keySet()) { + StreamInfo info = _fields.get(stream); + declarer.declareStream(stream, info.is_direct(), new Fields(info.get_output_fields())); + } + } + + @Override + public void finishedId(Object id) { + if(_bolt instanceof FinishedCallback) { + ((FinishedCallback) _bolt).finishedId(id); + } + } + + @Override + public Map getComponentConfiguration() { + IFn hof = Utils.loadClojureFn(_confSpec.get(0), _confSpec.get(1)); + try { + return (Map) hof.applyTo(RT.seq(_params)); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/clojure/ClojureSpout.java b/jstorm-core/src/main/java/backtype/storm/clojure/ClojureSpout.java new file mode 100755 index 000000000..f6422e34e --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/clojure/ClojureSpout.java @@ -0,0 +1,153 @@ +/** + * 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 backtype.storm.clojure; + +import backtype.storm.generated.StreamInfo; +import backtype.storm.spout.ISpout; +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.IRichSpout; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Fields; +import backtype.storm.utils.Utils; +import clojure.lang.IFn; +import clojure.lang.PersistentArrayMap; +import clojure.lang.Keyword; +import clojure.lang.Symbol; +import clojure.lang.RT; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class ClojureSpout implements IRichSpout { + Map _fields; + List _fnSpec; + List _confSpec; + List _params; + + ISpout _spout; + + public ClojureSpout(List fnSpec, List confSpec, List params, Map fields) { + _fnSpec = fnSpec; + _confSpec = confSpec; + _params = params; + _fields = fields; + } + + + @Override + public void open(final Map conf, final TopologyContext context, final SpoutOutputCollector collector) { + IFn hof = Utils.loadClojureFn(_fnSpec.get(0), _fnSpec.get(1)); + try { + IFn preparer = (IFn) hof.applyTo(RT.seq(_params)); + final Map collectorMap = new PersistentArrayMap( new Object[] { + Keyword.intern(Symbol.create("output-collector")), collector, + Keyword.intern(Symbol.create("context")), context}); + List args = new ArrayList() {{ + add(conf); + add(context); + add(collectorMap); + }}; + + _spout = (ISpout) preparer.applyTo(RT.seq(args)); + //this is kind of unnecessary for clojure + try { + _spout.open(conf, context, collector); + } catch(AbstractMethodError ame) { + + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public void close() { + try { + _spout.close(); + } catch(AbstractMethodError ame) { + + } + } + + @Override + public void nextTuple() { + try { + _spout.nextTuple(); + } catch(AbstractMethodError ame) { + + } + + } + + @Override + public void ack(Object msgId) { + try { + _spout.ack(msgId); + } catch(AbstractMethodError ame) { + + } + + } + + @Override + public void fail(Object msgId) { + try { + _spout.fail(msgId); + } catch(AbstractMethodError ame) { + + } + + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + for(String stream: _fields.keySet()) { + StreamInfo info = _fields.get(stream); + declarer.declareStream(stream, info.is_direct(), new Fields(info.get_output_fields())); + } + } + + @Override + public Map getComponentConfiguration() { + IFn hof = Utils.loadClojureFn(_confSpec.get(0), _confSpec.get(1)); + try { + return (Map) hof.applyTo(RT.seq(_params)); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public void activate() { + try { + _spout.activate(); + } catch(AbstractMethodError ame) { + + } + } + + @Override + public void deactivate() { + try { + _spout.deactivate(); + } catch(AbstractMethodError ame) { + + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/clojure/RichShellBolt.java b/jstorm-core/src/main/java/backtype/storm/clojure/RichShellBolt.java new file mode 100755 index 000000000..a1550084c --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/clojure/RichShellBolt.java @@ -0,0 +1,51 @@ +/** + * 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 backtype.storm.clojure; + +import backtype.storm.generated.StreamInfo; +import backtype.storm.task.ShellBolt; +import backtype.storm.topology.IRichBolt; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Fields; +import java.util.Map; + +public class RichShellBolt extends ShellBolt implements IRichBolt { + private Map _outputs; + + public RichShellBolt(String[] command, Map outputs) { + super(command); + _outputs = outputs; + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + for(String stream: _outputs.keySet()) { + StreamInfo def = _outputs.get(stream); + if(def.is_direct()) { + declarer.declareStream(stream, true, new Fields(def.get_output_fields())); + } else { + declarer.declareStream(stream, new Fields(def.get_output_fields())); + } + } + } + + @Override + public Map getComponentConfiguration() { + return null; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/clojure/RichShellSpout.java b/jstorm-core/src/main/java/backtype/storm/clojure/RichShellSpout.java new file mode 100755 index 000000000..b49fbef9d --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/clojure/RichShellSpout.java @@ -0,0 +1,51 @@ +/** + * 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 backtype.storm.clojure; + +import backtype.storm.generated.StreamInfo; +import backtype.storm.spout.ShellSpout; +import backtype.storm.topology.IRichSpout; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Fields; +import java.util.Map; + +public class RichShellSpout extends ShellSpout implements IRichSpout { + private Map _outputs; + + public RichShellSpout(String[] command, Map outputs) { + super(command); + _outputs = outputs; + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + for(String stream: _outputs.keySet()) { + StreamInfo def = _outputs.get(stream); + if(def.is_direct()) { + declarer.declareStream(stream, true, new Fields(def.get_output_fields())); + } else { + declarer.declareStream(stream, new Fields(def.get_output_fields())); + } + } + } + + @Override + public Map getComponentConfiguration() { + return null; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/command/activate.java b/jstorm-core/src/main/java/backtype/storm/command/activate.java new file mode 100755 index 000000000..ed12e0911 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/command/activate.java @@ -0,0 +1,65 @@ +/** + * 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 backtype.storm.command; + +import java.security.InvalidParameterException; +import java.util.Map; + +import backtype.storm.utils.NimbusClient; +import backtype.storm.utils.Utils; + +/** + * Activate topology + * + * @author longda + * + */ +public class activate { + + /** + * @param args + */ + public static void main(String[] args) { + // TODO Auto-generated method stub + if (args == null || args.length == 0) { + throw new InvalidParameterException("Should input topology name"); + } + + String topologyName = args[0]; + + NimbusClient client = null; + try { + + Map conf = Utils.readStormConfig(); + client = NimbusClient.getConfiguredClient(conf); + + client.getClient().activate(topologyName); + + System.out.println("Successfully submit command activate " + topologyName); + } catch (Exception e) { + System.out.println(e.getMessage()); + e.printStackTrace(); + throw new RuntimeException(e); + } finally { + if (client != null) { + client.close(); + } + } + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/command/config_value.java b/jstorm-core/src/main/java/backtype/storm/command/config_value.java new file mode 100755 index 000000000..868ffdc76 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/command/config_value.java @@ -0,0 +1,50 @@ +/** + * 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 backtype.storm.command; + +import java.security.InvalidParameterException; +import java.util.Map; + +import backtype.storm.utils.NimbusClient; +import backtype.storm.utils.Utils; + +/** + * Get configuration + * + * @author longda + * + */ +public class config_value { + + /** + * @param args + */ + public static void main(String[] args) { + // TODO Auto-generated method stub + if (args == null || args.length == 0) { + throw new InvalidParameterException("Should input key name"); + } + + String key = args[0]; + + Map conf = Utils.readStormConfig(); + + System.out.print("VALUE: " + String.valueOf(conf.get(key))); + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/command/deactivate.java b/jstorm-core/src/main/java/backtype/storm/command/deactivate.java new file mode 100755 index 000000000..22ac20d28 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/command/deactivate.java @@ -0,0 +1,65 @@ +/** + * 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 backtype.storm.command; + +import java.security.InvalidParameterException; +import java.util.Map; + +import backtype.storm.utils.NimbusClient; +import backtype.storm.utils.Utils; + +/** + * Deactivate topology + * + * @author longda + * + */ +public class deactivate { + + /** + * @param args + */ + public static void main(String[] args) { + // TODO Auto-generated method stub + if (args == null || args.length == 0) { + throw new InvalidParameterException("Should input topology name"); + } + + String topologyName = args[0]; + + NimbusClient client = null; + try { + + Map conf = Utils.readStormConfig(); + client = NimbusClient.getConfiguredClient(conf); + + client.getClient().deactivate(topologyName); + + System.out.println("Successfully submit command deactivate " + topologyName); + } catch (Exception e) { + System.out.println(e.getMessage()); + e.printStackTrace(); + throw new RuntimeException(e); + } finally { + if (client != null) { + client.close(); + } + } + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/command/kill_topology.java b/jstorm-core/src/main/java/backtype/storm/command/kill_topology.java new file mode 100755 index 000000000..4ab389396 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/command/kill_topology.java @@ -0,0 +1,77 @@ +/** + * 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 backtype.storm.command; + +import java.security.InvalidParameterException; +import java.util.Map; + +import backtype.storm.generated.KillOptions; +import backtype.storm.utils.NimbusClient; +import backtype.storm.utils.Utils; + +/** + * Kill topology + * + * @author longda + * + */ +public class kill_topology { + + /** + * @param args + */ + public static void main(String[] args) { + // TODO Auto-generated method stub + if (args == null || args.length == 0) { + throw new InvalidParameterException("Should input topology name"); + } + + String topologyName = args[0]; + + NimbusClient client = null; + try { + + Map conf = Utils.readStormConfig(); + client = NimbusClient.getConfiguredClient(conf); + + if (args.length == 1) { + + client.getClient().killTopology(topologyName); + } else { + int delaySeconds = Integer.parseInt(args[1]); + + KillOptions options = new KillOptions(); + options.set_wait_secs(delaySeconds); + + client.getClient().killTopologyWithOpts(topologyName, options); + + } + + System.out.println("Successfully submit command kill " + topologyName); + } catch (Exception e) { + System.out.println(e.getMessage()); + e.printStackTrace(); + throw new RuntimeException(e); + } finally { + if (client != null) { + client.close(); + } + } + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/command/list.java b/jstorm-core/src/main/java/backtype/storm/command/list.java new file mode 100755 index 000000000..3b4efdb6d --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/command/list.java @@ -0,0 +1,70 @@ +/** + * 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 backtype.storm.command; + +import java.util.Map; + +import org.apache.commons.lang.StringUtils; + +import backtype.storm.generated.ClusterSummary; +import backtype.storm.generated.TopologyInfo; +import backtype.storm.utils.NimbusClient; +import backtype.storm.utils.Utils; + +/** + * Activate topology + * + * @author longda + * + */ +public class list { + + /** + * @param args + */ + public static void main(String[] args) { + + NimbusClient client = null; + try { + + Map conf = Utils.readStormConfig(); + client = NimbusClient.getConfiguredClient(conf); + + if (args.length > 0 && StringUtils.isBlank(args[0]) == false) { + String topologyName = args[0]; + TopologyInfo info = client.getClient().getTopologyInfoByName(topologyName); + + System.out.println("Successfully get topology info \n" + Utils.toPrettyJsonString(info)); + } else { + ClusterSummary clusterSummary = client.getClient().getClusterInfo(); + + System.out.println("Successfully get cluster info \n" + Utils.toPrettyJsonString(clusterSummary)); + } + + } catch (Exception e) { + System.out.println(e.getMessage()); + e.printStackTrace(); + throw new RuntimeException(e); + } finally { + if (client != null) { + client.close(); + } + } + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/command/metrics_monitor.java b/jstorm-core/src/main/java/backtype/storm/command/metrics_monitor.java new file mode 100755 index 000000000..660744598 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/command/metrics_monitor.java @@ -0,0 +1,72 @@ +/** + * 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 backtype.storm.command; + +import java.util.Map; +import java.security.InvalidParameterException; + +import backtype.storm.generated.MonitorOptions; +import backtype.storm.utils.NimbusClient; +import backtype.storm.utils.Utils; + +/** + * Monitor topology + * + * @author Basti + * + */ +public class metrics_monitor { + + /** + * @param args + */ + public static void main(String[] args) { + // TODO Auto-generated method stub + if (args == null || args.length <= 1) { + throw new InvalidParameterException("Should input topology name and enable flag"); + } + + String topologyName = args[0]; + + NimbusClient client = null; + try { + + Map conf = Utils.readStormConfig(); + client = NimbusClient.getConfiguredClient(conf); + + boolean isEnable = Boolean.valueOf(args[1]).booleanValue(); + + MonitorOptions options = new MonitorOptions(); + options.set_isEnable(isEnable); + + client.getClient().metricMonitor(topologyName, options); + + String str = (isEnable) ? "enable" : "disable"; + System.out.println("Successfully submit command to " + str + " the monitor of " + topologyName); + } catch (Exception e) { + System.out.println(e.getMessage()); + e.printStackTrace(); + throw new RuntimeException(e); + } finally { + if (client != null) { + client.close(); + } + } + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/command/rebalance.java b/jstorm-core/src/main/java/backtype/storm/command/rebalance.java new file mode 100755 index 000000000..f0cf69ffb --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/command/rebalance.java @@ -0,0 +1,127 @@ +/** + * 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 backtype.storm.command; + +import java.security.InvalidParameterException; +import java.util.Map; + +import backtype.storm.generated.RebalanceOptions; +import backtype.storm.utils.NimbusClient; +import backtype.storm.utils.Utils; + +/** + * Active topology + * + * @author longda + * + */ +public class rebalance { + static final String REASSIGN_FLAG = "-r"; + + /** + * @param args + */ + public static void main(String[] args) { + // TODO Auto-generated method stub + if (args == null || args.length == 0) { + printErrorInfo(); + return; + } + + int argsIndex = 0; + String topologyName = null; + + try { + RebalanceOptions options = new RebalanceOptions(); + options.set_reassign(false); + options.set_conf(null); + + if (args[argsIndex].equalsIgnoreCase(REASSIGN_FLAG)) { + options.set_reassign(true); + argsIndex++; + if (args.length <= argsIndex) { + // Topology name is not set. + printErrorInfo(); + return; + } else { + topologyName = args[argsIndex]; + } + } else { + topologyName = args[argsIndex]; + } + + argsIndex++; + if (args.length > argsIndex) { + for (int i = argsIndex; i < args.length; i++) { + String arg = args[i]; + if (arg.endsWith("yaml") || arg.endsWith("prop")) { + Map userConf = Utils.loadConf(arg); + String jsonConf = Utils.to_json(userConf); + options.set_conf(jsonConf); + } else { + try { + int delaySeconds = Integer.parseInt(args[1]); + options.set_wait_secs(delaySeconds); + } catch (NumberFormatException e) { + System.out.println("Unsupported argument found, arg=" + arg + ". Full args are " + args); + printErrorInfo(); + return; + } + } + } + } + + submitRebalance(topologyName, options); + + System.out.println("Successfully submit command rebalance " + topologyName + ", delaySecs=" + options.get_wait_secs() + ", reassignFlag=" + options.is_reassign() + ", newConfiguration=" + options.get_conf()); + } catch (Exception e) { + System.out.println(e.getMessage()); + e.printStackTrace(); + throw new RuntimeException(e); + } + } + + private static void printErrorInfo() { + System.out.println("Error: Invalid parameters!"); + System.out.println("USAGE: jstorm rebalance [-r] TopologyName [DelayTime] [NewConfig]"); + } + + public static void submitRebalance(String topologyName, RebalanceOptions options) throws Exception { + submitRebalance(topologyName, options, null); + } + + public static void submitRebalance(String topologyName, RebalanceOptions options, Map conf) throws Exception { + Map stormConf = Utils.readStormConfig(); + if (conf != null) { + stormConf.putAll(conf); + } + + NimbusClient client = null; + try { + client = NimbusClient.getConfiguredClient(stormConf); + client.getClient().rebalance(topologyName, options); + } catch (Exception e) { + throw e; + } finally { + if (client != null) { + client.close(); + } + } + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/command/restart.java b/jstorm-core/src/main/java/backtype/storm/command/restart.java new file mode 100755 index 000000000..ecec9a3e4 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/command/restart.java @@ -0,0 +1,80 @@ +/** + * 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 backtype.storm.command; + +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.InputStream; +import java.security.InvalidParameterException; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import org.yaml.snakeyaml.Yaml; + +import backtype.storm.utils.NimbusClient; +import backtype.storm.utils.Utils; + +/** + * Active topology + * + * @author basti + * + */ +public class restart { + /** + * @param args + */ + public static void main(String[] args) { + // TODO Auto-generated method stub + if (args == null || args.length == 0) { + throw new InvalidParameterException("Should input topology name"); + } + + String topologyName = args[0]; + + NimbusClient client = null; + try { + Map conf = Utils.readStormConfig(); + client = NimbusClient.getConfiguredClient(conf); + + System.out.println("It will take 15 ~ 100 seconds to restart, please wait patiently\n"); + + if (args.length == 1) { + client.getClient().restart(topologyName, null); + } else { + Map loadConf = Utils.loadConf(args[1]); + String jsonConf = Utils.to_json(loadConf); + System.out.println("New configuration:\n" + jsonConf); + + client.getClient().restart(topologyName, jsonConf); + } + + System.out.println("Successfully submit command restart " + topologyName); + } catch (Exception e) { + System.out.println(e.getMessage()); + e.printStackTrace(); + throw new RuntimeException(e); + } finally { + if (client != null) { + client.close(); + } + } + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/command/update_config.java b/jstorm-core/src/main/java/backtype/storm/command/update_config.java new file mode 100644 index 000000000..be78f1945 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/command/update_config.java @@ -0,0 +1,69 @@ +/** + * 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 backtype.storm.command; + +import java.security.InvalidParameterException; +import java.util.Map; + +import backtype.storm.utils.NimbusClient; +import backtype.storm.utils.Utils; + +/** + * Update user configuration + * + * @author basti + * + */ +public class update_config { + /** + * @param args + */ + public static void main(String[] args) { + // TODO Auto-generated method stub + if (args == null || args.length < 2) { + throw new InvalidParameterException( + "[USAGE] update_config topologyName config"); + } + + String topologyName = args[0]; + + NimbusClient client = null; + try { + Map conf = Utils.readStormConfig(); + client = NimbusClient.getConfiguredClient(conf); + + Map loadConf = Utils.loadConf(args[1]); + String jsonConf = Utils.to_json(loadConf); + System.out.println("New configuration:\n" + jsonConf); + + client.getClient().updateConf(topologyName, jsonConf); + + System.out.println("Successfully submit command update_conf " + + topologyName); + } catch (Exception e) { + System.out.println(e.getMessage()); + e.printStackTrace(); + throw new RuntimeException(e); + } finally { + if (client != null) { + client.close(); + } + } + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/coordination/BatchBoltExecutor.java b/jstorm-core/src/main/java/backtype/storm/coordination/BatchBoltExecutor.java new file mode 100755 index 000000000..8653010fc --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/coordination/BatchBoltExecutor.java @@ -0,0 +1,108 @@ +/** + * 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 backtype.storm.coordination; + +import backtype.storm.coordination.CoordinatedBolt.FinishedCallback; +import backtype.storm.coordination.CoordinatedBolt.TimeoutCallback; +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.FailedException; +import backtype.storm.topology.IRichBolt; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Tuple; +import backtype.storm.utils.Utils; +import java.util.HashMap; +import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class BatchBoltExecutor implements IRichBolt, FinishedCallback, TimeoutCallback { + public static Logger LOG = LoggerFactory.getLogger(BatchBoltExecutor.class); + + byte[] _boltSer; + Map _openTransactions; + Map _conf; + TopologyContext _context; + BatchOutputCollectorImpl _collector; + + public BatchBoltExecutor(IBatchBolt bolt) { + _boltSer = Utils.javaSerialize(bolt); + } + + @Override + public void prepare(Map conf, TopologyContext context, OutputCollector collector) { + _conf = conf; + _context = context; + _collector = new BatchOutputCollectorImpl(collector); + _openTransactions = new HashMap(); + } + + @Override + public void execute(Tuple input) { + Object id = input.getValue(0); + IBatchBolt bolt = getBatchBolt(id); + try { + bolt.execute(input); + _collector.ack(input); + } catch(FailedException e) { + LOG.error("Failed to process tuple in batch", e); + _collector.fail(input); + } + } + + @Override + public void cleanup() { + } + + @Override + public void finishedId(Object id) { + IBatchBolt bolt = getBatchBolt(id); + _openTransactions.remove(id); + bolt.finishBatch(); + } + + @Override + public void timeoutId(Object attempt) { + _openTransactions.remove(attempt); + } + + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + newTransactionalBolt().declareOutputFields(declarer); + } + + @Override + public Map getComponentConfiguration() { + return newTransactionalBolt().getComponentConfiguration(); + } + + private IBatchBolt getBatchBolt(Object id) { + IBatchBolt bolt = _openTransactions.get(id); + if(bolt==null) { + bolt = newTransactionalBolt(); + bolt.prepare(_conf, _context, _collector, id); + _openTransactions.put(id, bolt); + } + return bolt; + } + + private IBatchBolt newTransactionalBolt() { + return Utils.javaDeserialize(_boltSer, IBatchBolt.class); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/coordination/BatchOutputCollector.java b/jstorm-core/src/main/java/backtype/storm/coordination/BatchOutputCollector.java new file mode 100755 index 000000000..f5f34570c --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/coordination/BatchOutputCollector.java @@ -0,0 +1,46 @@ +/** + * 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 backtype.storm.coordination; + +import backtype.storm.utils.Utils; +import java.util.List; + +public abstract class BatchOutputCollector { + + /** + * Emits a tuple to the default output stream. + */ + public List emit(List tuple) { + return emit(Utils.DEFAULT_STREAM_ID, tuple); + } + + public abstract List emit(String streamId, List tuple); + + /** + * Emits a tuple to the specified task on the default output stream. This output + * stream must have been declared as a direct stream, and the specified task must + * use a direct grouping on this stream to receive the message. + */ + public void emitDirect(int taskId, List tuple) { + emitDirect(taskId, Utils.DEFAULT_STREAM_ID, tuple); + } + + public abstract void emitDirect(int taskId, String streamId, List tuple); + + public abstract void reportError(Throwable error); +} diff --git a/jstorm-core/src/main/java/backtype/storm/coordination/BatchOutputCollectorImpl.java b/jstorm-core/src/main/java/backtype/storm/coordination/BatchOutputCollectorImpl.java new file mode 100755 index 000000000..cae756006 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/coordination/BatchOutputCollectorImpl.java @@ -0,0 +1,53 @@ +/** + * 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 backtype.storm.coordination; + +import backtype.storm.task.OutputCollector; +import backtype.storm.tuple.Tuple; +import java.util.List; + +public class BatchOutputCollectorImpl extends BatchOutputCollector { + OutputCollector _collector; + + public BatchOutputCollectorImpl(OutputCollector collector) { + _collector = collector; + } + + @Override + public List emit(String streamId, List tuple) { + return _collector.emit(streamId, tuple); + } + + @Override + public void emitDirect(int taskId, String streamId, List tuple) { + _collector.emitDirect(taskId, streamId, tuple); + } + + @Override + public void reportError(Throwable error) { + _collector.reportError(error); + } + + public void ack(Tuple tup) { + _collector.ack(tup); + } + + public void fail(Tuple tup) { + _collector.fail(tup); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/coordination/BatchSubtopologyBuilder.java b/jstorm-core/src/main/java/backtype/storm/coordination/BatchSubtopologyBuilder.java new file mode 100755 index 000000000..2a77f3bcd --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/coordination/BatchSubtopologyBuilder.java @@ -0,0 +1,479 @@ +/** + * 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 backtype.storm.coordination; + +import backtype.storm.Constants; +import backtype.storm.coordination.CoordinatedBolt.SourceArgs; +import backtype.storm.generated.GlobalStreamId; +import backtype.storm.generated.Grouping; +import backtype.storm.grouping.CustomStreamGrouping; +import backtype.storm.grouping.PartialKeyGrouping; +import backtype.storm.topology.BaseConfigurationDeclarer; +import backtype.storm.topology.BasicBoltExecutor; +import backtype.storm.topology.BoltDeclarer; +import backtype.storm.topology.IBasicBolt; +import backtype.storm.topology.IRichBolt; +import backtype.storm.topology.InputDeclarer; +import backtype.storm.topology.TopologyBuilder; +import backtype.storm.tuple.Fields; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class BatchSubtopologyBuilder { + Map _bolts = new HashMap(); + Component _masterBolt; + String _masterId; + + public BatchSubtopologyBuilder(String masterBoltId, IBasicBolt masterBolt, Number boltParallelism) { + Integer p = boltParallelism == null ? null : boltParallelism.intValue(); + _masterBolt = new Component(new BasicBoltExecutor(masterBolt), p); + _masterId = masterBoltId; + } + + public BatchSubtopologyBuilder(String masterBoltId, IBasicBolt masterBolt) { + this(masterBoltId, masterBolt, null); + } + + public BoltDeclarer getMasterDeclarer() { + return new BoltDeclarerImpl(_masterBolt); + } + + public BoltDeclarer setBolt(String id, IBatchBolt bolt) { + return setBolt(id, bolt, null); + } + + public BoltDeclarer setBolt(String id, IBatchBolt bolt, Number parallelism) { + return setBolt(id, new BatchBoltExecutor(bolt), parallelism); + } + + public BoltDeclarer setBolt(String id, IBasicBolt bolt) { + return setBolt(id, bolt, null); + } + + public BoltDeclarer setBolt(String id, IBasicBolt bolt, Number parallelism) { + return setBolt(id, new BasicBoltExecutor(bolt), parallelism); + } + + private BoltDeclarer setBolt(String id, IRichBolt bolt, Number parallelism) { + Integer p = null; + if(parallelism!=null) p = parallelism.intValue(); + Component component = new Component(bolt, p); + _bolts.put(id, component); + return new BoltDeclarerImpl(component); + } + + public void extendTopology(TopologyBuilder builder) { + BoltDeclarer declarer = builder.setBolt(_masterId, new CoordinatedBolt(_masterBolt.bolt), _masterBolt.parallelism); + for(InputDeclaration decl: _masterBolt.declarations) { + decl.declare(declarer); + } + for(Map conf: _masterBolt.componentConfs) { + declarer.addConfigurations(conf); + } + for(String id: _bolts.keySet()) { + Component component = _bolts.get(id); + Map coordinatedArgs = new HashMap(); + for(String c: componentBoltSubscriptions(component)) { + SourceArgs source; + if(c.equals(_masterId)) { + source = SourceArgs.single(); + } else { + source = SourceArgs.all(); + } + coordinatedArgs.put(c, source); + } + + + BoltDeclarer input = builder.setBolt(id, + new CoordinatedBolt(component.bolt, + coordinatedArgs, + null), + component.parallelism); + for(Map conf: component.componentConfs) { + input.addConfigurations(conf); + } + for(String c: componentBoltSubscriptions(component)) { + input.directGrouping(c, Constants.COORDINATED_STREAM_ID); + } + for(InputDeclaration d: component.declarations) { + d.declare(input); + } + } + } + + private Set componentBoltSubscriptions(Component component) { + Set ret = new HashSet(); + for(InputDeclaration d: component.declarations) { + ret.add(d.getComponent()); + } + return ret; + } + + private static class Component { + public IRichBolt bolt; + public Integer parallelism; + public List declarations = new ArrayList(); + public List componentConfs = new ArrayList(); + + public Component(IRichBolt bolt, Integer parallelism) { + this.bolt = bolt; + this.parallelism = parallelism; + } + } + + private static interface InputDeclaration { + void declare(InputDeclarer declarer); + String getComponent(); + } + + private class BoltDeclarerImpl extends BaseConfigurationDeclarer implements BoltDeclarer { + Component _component; + + public BoltDeclarerImpl(Component component) { + _component = component; + } + + @Override + public BoltDeclarer fieldsGrouping(final String component, final Fields fields) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.fieldsGrouping(component, fields); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer fieldsGrouping(final String component, final String streamId, final Fields fields) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.fieldsGrouping(component, streamId, fields); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer globalGrouping(final String component) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.globalGrouping(component); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer globalGrouping(final String component, final String streamId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.globalGrouping(component, streamId); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer shuffleGrouping(final String component) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.shuffleGrouping(component); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer shuffleGrouping(final String component, final String streamId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.shuffleGrouping(component, streamId); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer localOrShuffleGrouping(final String component) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.localOrShuffleGrouping(component); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer localOrShuffleGrouping(final String component, final String streamId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.localOrShuffleGrouping(component, streamId); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer localFirstGrouping(final String componentId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.localFirstGrouping(componentId); + } + + @Override + public String getComponent() { + return componentId; + } + }); + return this; + } + + @Override + public BoltDeclarer localFirstGrouping(final String component, final String streamId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.localFirstGrouping(component, streamId); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer noneGrouping(final String component) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.noneGrouping(component); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer noneGrouping(final String component, final String streamId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.noneGrouping(component, streamId); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer allGrouping(final String component) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.allGrouping(component); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer allGrouping(final String component, final String streamId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.allGrouping(component, streamId); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer directGrouping(final String component) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.directGrouping(component); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer directGrouping(final String component, final String streamId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.directGrouping(component, streamId); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer partialKeyGrouping(String componentId, Fields fields) { + return customGrouping(componentId, new PartialKeyGrouping(fields)); + } + + @Override + public BoltDeclarer partialKeyGrouping(String componentId, String streamId, Fields fields) { + return customGrouping(componentId, streamId, new PartialKeyGrouping(fields)); + } + + @Override + public BoltDeclarer customGrouping(final String component, final CustomStreamGrouping grouping) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.customGrouping(component, grouping); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer customGrouping(final String component, final String streamId, final CustomStreamGrouping grouping) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.customGrouping(component, streamId, grouping); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer grouping(final GlobalStreamId stream, final Grouping grouping) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.grouping(stream, grouping); + } + + @Override + public String getComponent() { + return stream.get_componentId(); + } + }); + return this; + } + + private void addDeclaration(InputDeclaration declaration) { + _component.declarations.add(declaration); + } + + @Override + public BoltDeclarer addConfigurations(Map conf) { + _component.componentConfs.add(conf); + return this; + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/coordination/CoordinatedBolt.java b/jstorm-core/src/main/java/backtype/storm/coordination/CoordinatedBolt.java new file mode 100755 index 000000000..6f337a634 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/coordination/CoordinatedBolt.java @@ -0,0 +1,382 @@ +/** + * 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 backtype.storm.coordination; + +import backtype.storm.topology.FailedException; +import java.util.Map.Entry; +import backtype.storm.tuple.Values; +import backtype.storm.generated.GlobalStreamId; +import java.util.Collection; +import backtype.storm.Constants; +import backtype.storm.generated.Grouping; +import backtype.storm.task.IOutputCollector; +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.IRichBolt; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.utils.TimeCacheMap; +import backtype.storm.utils.Utils; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import static backtype.storm.utils.Utils.get; + +/** + * Coordination requires the request ids to be globally unique for awhile. This is so it doesn't get confused + * in the case of retries. + */ +public class CoordinatedBolt implements IRichBolt { + public static Logger LOG = LoggerFactory.getLogger(CoordinatedBolt.class); + + public static interface FinishedCallback { + void finishedId(Object id); + } + + public static interface TimeoutCallback { + void timeoutId(Object id); + } + + + public static class SourceArgs implements Serializable { + public boolean singleCount; + + protected SourceArgs(boolean singleCount) { + this.singleCount = singleCount; + } + + public static SourceArgs single() { + return new SourceArgs(true); + } + + public static SourceArgs all() { + return new SourceArgs(false); + } + + @Override + public String toString() { + return ""; + } + } + + public class CoordinatedOutputCollector implements IOutputCollector { + IOutputCollector _delegate; + + public CoordinatedOutputCollector(IOutputCollector delegate) { + _delegate = delegate; + } + + public List emit(String stream, Collection anchors, List tuple) { + List tasks = _delegate.emit(stream, anchors, tuple); + updateTaskCounts(tuple.get(0), tasks); + return tasks; + } + + public void emitDirect(int task, String stream, Collection anchors, List tuple) { + updateTaskCounts(tuple.get(0), Arrays.asList(task)); + _delegate.emitDirect(task, stream, anchors, tuple); + } + + public void ack(Tuple tuple) { + Object id = tuple.getValue(0); + synchronized(_tracked) { + TrackingInfo track = _tracked.get(id); + if (track != null) + track.receivedTuples++; + } + boolean failed = checkFinishId(tuple, TupleType.REGULAR); + if(failed) { + _delegate.fail(tuple); + } else { + _delegate.ack(tuple); + } + } + + public void fail(Tuple tuple) { + Object id = tuple.getValue(0); + synchronized(_tracked) { + TrackingInfo track = _tracked.get(id); + if (track != null) + track.failed = true; + } + checkFinishId(tuple, TupleType.REGULAR); + _delegate.fail(tuple); + } + + public void reportError(Throwable error) { + _delegate.reportError(error); + } + + + private void updateTaskCounts(Object id, List tasks) { + synchronized(_tracked) { + TrackingInfo track = _tracked.get(id); + if (track != null) { + Map taskEmittedTuples = track.taskEmittedTuples; + for(Integer task: tasks) { + int newCount = get(taskEmittedTuples, task, 0) + 1; + taskEmittedTuples.put(task, newCount); + } + } + } + } + } + + private Map _sourceArgs; + private IdStreamSpec _idStreamSpec; + private IRichBolt _delegate; + private Integer _numSourceReports; + private List _countOutTasks = new ArrayList();; + private OutputCollector _collector; + private TimeCacheMap _tracked; + + public static class TrackingInfo { + int reportCount = 0; + int expectedTupleCount = 0; + int receivedTuples = 0; + boolean failed = false; + Map taskEmittedTuples = new HashMap(); + boolean receivedId = false; + boolean finished = false; + List ackTuples = new ArrayList(); + + @Override + public String toString() { + return "reportCount: " + reportCount + "\n" + + "expectedTupleCount: " + expectedTupleCount + "\n" + + "receivedTuples: " + receivedTuples + "\n" + + "failed: " + failed + "\n" + + taskEmittedTuples.toString(); + } + } + + + public static class IdStreamSpec implements Serializable { + GlobalStreamId _id; + + public GlobalStreamId getGlobalStreamId() { + return _id; + } + + public static IdStreamSpec makeDetectSpec(String component, String stream) { + return new IdStreamSpec(component, stream); + } + + protected IdStreamSpec(String component, String stream) { + _id = new GlobalStreamId(component, stream); + } + } + + public CoordinatedBolt(IRichBolt delegate) { + this(delegate, null, null); + } + + public CoordinatedBolt(IRichBolt delegate, String sourceComponent, SourceArgs sourceArgs, IdStreamSpec idStreamSpec) { + this(delegate, singleSourceArgs(sourceComponent, sourceArgs), idStreamSpec); + } + + public CoordinatedBolt(IRichBolt delegate, Map sourceArgs, IdStreamSpec idStreamSpec) { + _sourceArgs = sourceArgs; + if(_sourceArgs==null) _sourceArgs = new HashMap(); + _delegate = delegate; + _idStreamSpec = idStreamSpec; + } + + public void prepare(Map config, TopologyContext context, OutputCollector collector) { + TimeCacheMap.ExpiredCallback callback = null; + if(_delegate instanceof TimeoutCallback) { + callback = new TimeoutItems(); + } + _tracked = new TimeCacheMap(context.maxTopologyMessageTimeout(), callback); + _collector = collector; + _delegate.prepare(config, context, new OutputCollector(new CoordinatedOutputCollector(collector))); + for(String component: Utils.get(context.getThisTargets(), + Constants.COORDINATED_STREAM_ID, + new HashMap()) + .keySet()) { + for(Integer task: context.getComponentTasks(component)) { + _countOutTasks.add(task); + } + } + if(!_sourceArgs.isEmpty()) { + _numSourceReports = 0; + for(Entry entry: _sourceArgs.entrySet()) { + if(entry.getValue().singleCount) { + _numSourceReports+=1; + } else { + _numSourceReports+=context.getComponentTasks(entry.getKey()).size(); + } + } + } + } + + private boolean checkFinishId(Tuple tup, TupleType type) { + Object id = tup.getValue(0); + boolean failed = false; + + synchronized(_tracked) { + TrackingInfo track = _tracked.get(id); + try { + if(track!=null) { + boolean delayed = false; + if(_idStreamSpec==null && type == TupleType.COORD || _idStreamSpec!=null && type==TupleType.ID) { + track.ackTuples.add(tup); + delayed = true; + } + if(track.failed) { + failed = true; + for(Tuple t: track.ackTuples) { + _collector.fail(t); + } + _tracked.remove(id); + } else if(track.receivedId + && (_sourceArgs.isEmpty() || + track.reportCount==_numSourceReports && + track.expectedTupleCount == track.receivedTuples)){ + if(_delegate instanceof FinishedCallback) { + ((FinishedCallback)_delegate).finishedId(id); + } + if(!(_sourceArgs.isEmpty() || type!=TupleType.REGULAR)) { + throw new IllegalStateException("Coordination condition met on a non-coordinating tuple. Should be impossible"); + } + Iterator outTasks = _countOutTasks.iterator(); + while(outTasks.hasNext()) { + int task = outTasks.next(); + int numTuples = get(track.taskEmittedTuples, task, 0); + _collector.emitDirect(task, Constants.COORDINATED_STREAM_ID, tup, new Values(id, numTuples)); + } + for(Tuple t: track.ackTuples) { + _collector.ack(t); + } + track.finished = true; + _tracked.remove(id); + } + if(!delayed && type!=TupleType.REGULAR) { + if(track.failed) { + _collector.fail(tup); + } else { + _collector.ack(tup); + } + } + } else { + if(type!=TupleType.REGULAR) _collector.fail(tup); + } + } catch(FailedException e) { + LOG.error("Failed to finish batch", e); + for(Tuple t: track.ackTuples) { + _collector.fail(t); + } + _tracked.remove(id); + failed = true; + } + } + return failed; + } + + public void execute(Tuple tuple) { + Object id = tuple.getValue(0); + TrackingInfo track; + TupleType type = getTupleType(tuple); + synchronized(_tracked) { + track = _tracked.get(id); + if(track==null) { + track = new TrackingInfo(); + if(_idStreamSpec==null) track.receivedId = true; + _tracked.put(id, track); + } + } + + if(type==TupleType.ID) { + synchronized(_tracked) { + track.receivedId = true; + } + checkFinishId(tuple, type); + } else if(type==TupleType.COORD) { + int count = (Integer) tuple.getValue(1); + synchronized(_tracked) { + track.reportCount++; + track.expectedTupleCount+=count; + } + checkFinishId(tuple, type); + } else { + synchronized(_tracked) { + _delegate.execute(tuple); + } + } + } + + public void cleanup() { + _delegate.cleanup(); + _tracked.cleanup(); + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + _delegate.declareOutputFields(declarer); + declarer.declareStream(Constants.COORDINATED_STREAM_ID, true, new Fields("id", "count")); + } + + @Override + public Map getComponentConfiguration() { + return _delegate.getComponentConfiguration(); + } + + private static Map singleSourceArgs(String sourceComponent, SourceArgs sourceArgs) { + Map ret = new HashMap(); + ret.put(sourceComponent, sourceArgs); + return ret; + } + + private class TimeoutItems implements TimeCacheMap.ExpiredCallback { + @Override + public void expire(Object id, TrackingInfo val) { + synchronized(_tracked) { + // the combination of the lock and the finished flag ensure that + // an id is never timed out if it has been finished + val.failed = true; + if(!val.finished) { + ((TimeoutCallback) _delegate).timeoutId(id); + } + } + } + } + + private TupleType getTupleType(Tuple tuple) { + if(_idStreamSpec!=null + && tuple.getSourceGlobalStreamid().equals(_idStreamSpec._id)) { + return TupleType.ID; + } else if(!_sourceArgs.isEmpty() + && tuple.getSourceStreamId().equals(Constants.COORDINATED_STREAM_ID)) { + return TupleType.COORD; + } else { + return TupleType.REGULAR; + } + } + + static enum TupleType { + REGULAR, + ID, + COORD + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/coordination/IBatchBolt.java b/jstorm-core/src/main/java/backtype/storm/coordination/IBatchBolt.java new file mode 100755 index 000000000..ee5d9bd62 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/coordination/IBatchBolt.java @@ -0,0 +1,30 @@ +/** + * 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 backtype.storm.coordination; + +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.IComponent; +import backtype.storm.tuple.Tuple; +import java.io.Serializable; +import java.util.Map; + +public interface IBatchBolt extends Serializable, IComponent { + void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, T id); + void execute(Tuple tuple); + void finishBatch(); +} diff --git a/jstorm-core/src/main/java/backtype/storm/daemon/Shutdownable.java b/jstorm-core/src/main/java/backtype/storm/daemon/Shutdownable.java new file mode 100755 index 000000000..b1d8ddfd3 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/daemon/Shutdownable.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.daemon; + +public interface Shutdownable { + public void shutdown(); +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/drpc/DRPCInvocationsClient.java b/jstorm-core/src/main/java/backtype/storm/drpc/DRPCInvocationsClient.java new file mode 100755 index 000000000..624db3e53 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/drpc/DRPCInvocationsClient.java @@ -0,0 +1,114 @@ +/** + * 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 backtype.storm.drpc; + +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; + +import backtype.storm.generated.DRPCRequest; +import backtype.storm.generated.DistributedRPCInvocations; +import backtype.storm.generated.AuthorizationException; +import backtype.storm.security.auth.ThriftClient; +import backtype.storm.security.auth.ThriftConnectionType; +import org.apache.thrift.transport.TTransportException; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DRPCInvocationsClient extends ThriftClient implements DistributedRPCInvocations.Iface { + public static Logger LOG = LoggerFactory.getLogger(DRPCInvocationsClient.class); + private final AtomicReference client = + new AtomicReference(); + private String host; + private int port; + + public DRPCInvocationsClient(Map conf, String host, int port) throws TTransportException { + super(conf, ThriftConnectionType.DRPC_INVOCATIONS, host, port, null); + this.host = host; + this.port = port; + client.set(new DistributedRPCInvocations.Client(_protocol)); + } + + public String getHost() { + return host; + } + + public int getPort() { + return port; + } + + public void reconnectClient() throws TException { + if (client.get() == null) { + reconnect(); + client.set(new DistributedRPCInvocations.Client(_protocol)); + } + } + + public boolean isConnected() { + return client.get() != null; + } + + public void result(String id, String result) throws TException, AuthorizationException { + DistributedRPCInvocations.Client c = client.get(); + try { + if (c == null) { + throw new TException("Client is not connected..."); + } + c.result(id, result); + } catch(AuthorizationException aze) { + throw aze; + } catch(TException e) { + client.compareAndSet(c, null); + throw e; + } + } + + public DRPCRequest fetchRequest(String func) throws TException, AuthorizationException { + DistributedRPCInvocations.Client c = client.get(); + try { + if (c == null) { + throw new TException("Client is not connected..."); + } + return c.fetchRequest(func); + } catch(AuthorizationException aze) { + throw aze; + } catch(TException e) { + client.compareAndSet(c, null); + throw e; + } + } + + public void failRequest(String id) throws TException, AuthorizationException { + DistributedRPCInvocations.Client c = client.get(); + try { + if (c == null) { + throw new TException("Client is not connected..."); + } + c.failRequest(id); + } catch(AuthorizationException aze) { + throw aze; + } catch(TException e) { + client.compareAndSet(c, null); + throw e; + } + } + + public DistributedRPCInvocations.Client getClient() { + return client.get(); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/drpc/DRPCSpout.java b/jstorm-core/src/main/java/backtype/storm/drpc/DRPCSpout.java new file mode 100644 index 000000000..4ed24d42b --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/drpc/DRPCSpout.java @@ -0,0 +1,263 @@ +/** + * 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 backtype.storm.drpc; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.TimeUnit; + +import org.apache.thrift.TException; +import org.json.simple.JSONValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.utils.NetWorkUtils; + +import backtype.storm.Config; +import backtype.storm.ILocalDRPC; +import backtype.storm.generated.AuthorizationException; +import backtype.storm.generated.DRPCRequest; +import backtype.storm.generated.DistributedRPCInvocations; +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseRichSpout; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; +import backtype.storm.utils.ExtendedThreadPoolExecutor; +import backtype.storm.utils.ServiceRegistry; +import backtype.storm.utils.Utils; + +public class DRPCSpout extends BaseRichSpout { + //ANY CHANGE TO THIS CODE MUST BE SERIALIZABLE COMPATIBLE OR THERE WILL BE PROBLEMS + static final long serialVersionUID = 2387848310969237877L; + + public static Logger LOG = LoggerFactory.getLogger(DRPCSpout.class); + + SpoutOutputCollector _collector; + List _clients = new ArrayList(); + transient LinkedList> _futures = null; + transient ExecutorService _backround = null; + String _function; + String _local_drpc_id = null; + + private static class DRPCMessageId { + String id; + int index; + + public DRPCMessageId(String id, int index) { + this.id = id; + this.index = index; + } + } + + + public DRPCSpout(String function) { + _function = function; + } + + public DRPCSpout(String function, ILocalDRPC drpc) { + _function = function; + _local_drpc_id = drpc.getServiceId(); + } + + private class Adder implements Callable { + private String server; + private int port; + private Map conf; + + public Adder(String server, int port, Map conf) { + this.server = server; + this.port = port; + this.conf = conf; + } + + @Override + public Void call() throws Exception { + DRPCInvocationsClient c = new DRPCInvocationsClient(conf, server, port); + synchronized (_clients) { + _clients.add(c); + } + return null; + } + } + + private void reconnect(final DRPCInvocationsClient c) { + _futures.add(_backround.submit(new Callable() { + @Override + public Void call() throws Exception { + c.reconnectClient(); + return null; + } + })); + } + + private void checkFutures() { + Iterator> i = _futures.iterator(); + while (i.hasNext()) { + Future f = i.next(); + if (f.isDone()) { + i.remove(); + } + try { + f.get(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + + + @Override + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + _collector = collector; + if(_local_drpc_id==null) { + _backround = new ExtendedThreadPoolExecutor(0, Integer.MAX_VALUE, + 60L, TimeUnit.SECONDS, + new SynchronousQueue()); + _futures = new LinkedList>(); + + int numTasks = context.getComponentTasks(context.getThisComponentId()).size(); + int index = context.getThisTaskIndex(); + + int port = Utils.getInt(conf.get(Config.DRPC_INVOCATIONS_PORT)); + List servers = NetWorkUtils.host2Ip((List) conf.get(Config.DRPC_SERVERS)); + + if(servers == null || servers.isEmpty()) { + throw new RuntimeException("No DRPC servers configured for topology"); + } + + if (numTasks < servers.size()) { + for (String s: servers) { + _futures.add(_backround.submit(new Adder(s, port, conf))); + } + } else { + int i = index % servers.size(); + _futures.add(_backround.submit(new Adder(servers.get(i), port, conf))); + } + } + + } + + @Override + public void close() { + for(DRPCInvocationsClient client: _clients) { + client.close(); + } + } + + @Override + public void nextTuple() { + boolean gotRequest = false; + if(_local_drpc_id==null) { + int size = 0; + synchronized (_clients) { + size = _clients.size(); //This will only ever grow, so no need to worry about falling off the end + } + for(int i=0; i 0) { + Map returnInfo = new HashMap(); + returnInfo.put("id", req.get_request_id()); + returnInfo.put("host", client.getHost()); + returnInfo.put("port", client.getPort()); + gotRequest = true; + _collector.emit(new Values(req.get_func_args(), JSONValue.toJSONString(returnInfo)), new DRPCMessageId(req.get_request_id(), i)); + break; + } + } catch (AuthorizationException aze) { + reconnect(client); + LOG.error("Not authorized to fetch DRPC result from DRPC server", aze); + } catch (TException e) { + reconnect(client); + LOG.error("Failed to fetch DRPC result from DRPC server", e); + } catch (Exception e) { + LOG.error("Failed to fetch DRPC result from DRPC server", e); + } + } + checkFutures(); + } else { + DistributedRPCInvocations.Iface drpc = (DistributedRPCInvocations.Iface) ServiceRegistry.getService(_local_drpc_id); + if(drpc!=null) { // can happen during shutdown of drpc while topology is still up + try { + DRPCRequest req = drpc.fetchRequest(_function); + if(req.get_request_id().length() > 0) { + Map returnInfo = new HashMap(); + returnInfo.put("id", req.get_request_id()); + returnInfo.put("host", _local_drpc_id); + returnInfo.put("port", 0); + gotRequest = true; + _collector.emit(new Values(req.get_func_args(), JSONValue.toJSONString(returnInfo)), new DRPCMessageId(req.get_request_id(), 0)); + } + } catch (AuthorizationException aze) { + throw new RuntimeException(aze); + } catch (TException e) { + throw new RuntimeException(e); + } + } + } + if(!gotRequest) { + Utils.sleep(1); + } + } + + @Override + public void ack(Object msgId) { + } + + @Override + public void fail(Object msgId) { + DRPCMessageId did = (DRPCMessageId) msgId; + DistributedRPCInvocations.Iface client; + + if(_local_drpc_id == null) { + client = _clients.get(did.index); + } else { + client = (DistributedRPCInvocations.Iface) ServiceRegistry.getService(_local_drpc_id); + } + try { + client.failRequest(did.id); + } catch (AuthorizationException aze) { + LOG.error("Not authorized to failREquest from DRPC server", aze); + } catch (TException e) { + LOG.error("Failed to fail request", e); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("args", "return-info")); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/drpc/JoinResult.java b/jstorm-core/src/main/java/backtype/storm/drpc/JoinResult.java new file mode 100755 index 000000000..b74b97ed6 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/drpc/JoinResult.java @@ -0,0 +1,75 @@ +/** + * 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 backtype.storm.drpc; + +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.tuple.Values; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class JoinResult extends BaseRichBolt { + public static Logger LOG = LoggerFactory.getLogger(JoinResult.class); + + String returnComponent; + Map returns = new HashMap(); + Map results = new HashMap(); + OutputCollector _collector; + + public JoinResult(String returnComponent) { + this.returnComponent = returnComponent; + } + + public void prepare(Map map, TopologyContext context, OutputCollector collector) { + _collector = collector; + } + + public void execute(Tuple tuple) { + Object requestId = tuple.getValue(0); + if(tuple.getSourceComponent().equals(returnComponent)) { + returns.put(requestId, tuple); + } else { + results.put(requestId, tuple); + } + + if(returns.containsKey(requestId) && results.containsKey(requestId)) { + Tuple result = results.remove(requestId); + Tuple returner = returns.remove(requestId); + LOG.debug(result.getValue(1).toString()); + List anchors = new ArrayList(); + anchors.add(result); + anchors.add(returner); + _collector.emit(anchors, new Values(""+result.getValue(1), returner.getValue(1))); + _collector.ack(result); + _collector.ack(returner); + } + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("result", "return-info")); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/drpc/KeyedFairBolt.java b/jstorm-core/src/main/java/backtype/storm/drpc/KeyedFairBolt.java new file mode 100755 index 000000000..113163dd3 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/drpc/KeyedFairBolt.java @@ -0,0 +1,93 @@ +/** + * 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 backtype.storm.drpc; + +import backtype.storm.coordination.CoordinatedBolt.FinishedCallback; +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.BasicBoltExecutor; +import backtype.storm.topology.IBasicBolt; +import backtype.storm.topology.IRichBolt; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Tuple; +import backtype.storm.utils.KeyedRoundRobinQueue; +import java.util.HashMap; +import java.util.Map; + + +public class KeyedFairBolt implements IRichBolt, FinishedCallback { + IRichBolt _delegate; + KeyedRoundRobinQueue _rrQueue; + Thread _executor; + FinishedCallback _callback; + + public KeyedFairBolt(IRichBolt delegate) { + _delegate = delegate; + } + + public KeyedFairBolt(IBasicBolt delegate) { + this(new BasicBoltExecutor(delegate)); + } + + + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + if(_delegate instanceof FinishedCallback) { + _callback = (FinishedCallback) _delegate; + } + _delegate.prepare(stormConf, context, collector); + _rrQueue = new KeyedRoundRobinQueue(); + _executor = new Thread(new Runnable() { + public void run() { + try { + while(true) { + _delegate.execute(_rrQueue.take()); + } + } catch (InterruptedException e) { + + } + } + }); + _executor.setDaemon(true); + _executor.start(); + } + + public void execute(Tuple input) { + Object key = input.getValue(0); + _rrQueue.add(key, input); + } + + public void cleanup() { + _executor.interrupt(); + _delegate.cleanup(); + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + _delegate.declareOutputFields(declarer); + } + + public void finishedId(Object id) { + if(_callback!=null) { + _callback.finishedId(id); + } + } + + @Override + public Map getComponentConfiguration() { + return new HashMap(); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/drpc/LinearDRPCInputDeclarer.java b/jstorm-core/src/main/java/backtype/storm/drpc/LinearDRPCInputDeclarer.java new file mode 100755 index 000000000..d03075e6b --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/drpc/LinearDRPCInputDeclarer.java @@ -0,0 +1,52 @@ +/** + * 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 backtype.storm.drpc; + +import backtype.storm.grouping.CustomStreamGrouping; +import backtype.storm.topology.ComponentConfigurationDeclarer; +import backtype.storm.tuple.Fields; + +public interface LinearDRPCInputDeclarer extends ComponentConfigurationDeclarer { + public LinearDRPCInputDeclarer fieldsGrouping(Fields fields); + public LinearDRPCInputDeclarer fieldsGrouping(String streamId, Fields fields); + + public LinearDRPCInputDeclarer globalGrouping(); + public LinearDRPCInputDeclarer globalGrouping(String streamId); + + public LinearDRPCInputDeclarer shuffleGrouping(); + public LinearDRPCInputDeclarer shuffleGrouping(String streamId); + + public LinearDRPCInputDeclarer localOrShuffleGrouping(); + public LinearDRPCInputDeclarer localOrShuffleGrouping(String streamId); + + public LinearDRPCInputDeclarer noneGrouping(); + public LinearDRPCInputDeclarer noneGrouping(String streamId); + + public LinearDRPCInputDeclarer allGrouping(); + public LinearDRPCInputDeclarer allGrouping(String streamId); + + public LinearDRPCInputDeclarer directGrouping(); + public LinearDRPCInputDeclarer directGrouping(String streamId); + + public LinearDRPCInputDeclarer partialKeyGrouping(Fields fields); + public LinearDRPCInputDeclarer partialKeyGrouping(String streamId, Fields fields); + + public LinearDRPCInputDeclarer customGrouping(CustomStreamGrouping grouping); + public LinearDRPCInputDeclarer customGrouping(String streamId, CustomStreamGrouping grouping); + +} diff --git a/jstorm-core/src/main/java/backtype/storm/drpc/LinearDRPCTopologyBuilder.java b/jstorm-core/src/main/java/backtype/storm/drpc/LinearDRPCTopologyBuilder.java new file mode 100755 index 000000000..ebbbc6d55 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/drpc/LinearDRPCTopologyBuilder.java @@ -0,0 +1,394 @@ +/** + * 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 backtype.storm.drpc; + +import backtype.storm.Constants; +import backtype.storm.ILocalDRPC; +import backtype.storm.coordination.BatchBoltExecutor; +import backtype.storm.coordination.CoordinatedBolt; +import backtype.storm.coordination.CoordinatedBolt.FinishedCallback; +import backtype.storm.coordination.CoordinatedBolt.IdStreamSpec; +import backtype.storm.coordination.CoordinatedBolt.SourceArgs; +import backtype.storm.coordination.IBatchBolt; +import backtype.storm.generated.StormTopology; +import backtype.storm.generated.StreamInfo; +import backtype.storm.grouping.CustomStreamGrouping; +import backtype.storm.grouping.PartialKeyGrouping; +import backtype.storm.topology.BaseConfigurationDeclarer; +import backtype.storm.topology.BasicBoltExecutor; +import backtype.storm.topology.BoltDeclarer; +import backtype.storm.topology.IBasicBolt; +import backtype.storm.topology.IRichBolt; +import backtype.storm.topology.InputDeclarer; +import backtype.storm.topology.OutputFieldsGetter; +import backtype.storm.topology.TopologyBuilder; +import backtype.storm.tuple.Fields; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + + +// Trident subsumes the functionality provided by this class, so it's deprecated +@Deprecated +public class LinearDRPCTopologyBuilder { + String _function; + List _components = new ArrayList(); + + + public LinearDRPCTopologyBuilder(String function) { + _function = function; + } + + public LinearDRPCInputDeclarer addBolt(IBatchBolt bolt, Number parallelism) { + return addBolt(new BatchBoltExecutor(bolt), parallelism); + } + + public LinearDRPCInputDeclarer addBolt(IBatchBolt bolt) { + return addBolt(bolt, 1); + } + + @Deprecated + public LinearDRPCInputDeclarer addBolt(IRichBolt bolt, Number parallelism) { + if(parallelism==null) parallelism = 1; + Component component = new Component(bolt, parallelism.intValue()); + _components.add(component); + return new InputDeclarerImpl(component); + } + + @Deprecated + public LinearDRPCInputDeclarer addBolt(IRichBolt bolt) { + return addBolt(bolt, null); + } + + public LinearDRPCInputDeclarer addBolt(IBasicBolt bolt, Number parallelism) { + return addBolt(new BasicBoltExecutor(bolt), parallelism); + } + + public LinearDRPCInputDeclarer addBolt(IBasicBolt bolt) { + return addBolt(bolt, null); + } + + public StormTopology createLocalTopology(ILocalDRPC drpc) { + return createTopology(new DRPCSpout(_function, drpc)); + } + + public StormTopology createRemoteTopology() { + return createTopology(new DRPCSpout(_function)); + } + + + private StormTopology createTopology(DRPCSpout spout) { + final String SPOUT_ID = "spout"; + final String PREPARE_ID = "prepare-request"; + + TopologyBuilder builder = new TopologyBuilder(); + builder.setSpout(SPOUT_ID, spout); + builder.setBolt(PREPARE_ID, new PrepareRequest()) + .noneGrouping(SPOUT_ID); + int i=0; + for(; i<_components.size();i++) { + Component component = _components.get(i); + + Map source = new HashMap(); + if (i==1) { + source.put(boltId(i-1), SourceArgs.single()); + } else if (i>=2) { + source.put(boltId(i-1), SourceArgs.all()); + } + IdStreamSpec idSpec = null; + if(i==_components.size()-1 && component.bolt instanceof FinishedCallback) { + idSpec = IdStreamSpec.makeDetectSpec(PREPARE_ID, PrepareRequest.ID_STREAM); + } + BoltDeclarer declarer = builder.setBolt( + boltId(i), + new CoordinatedBolt(component.bolt, source, idSpec), + component.parallelism); + + for(Map conf: component.componentConfs) { + declarer.addConfigurations(conf); + } + + if(idSpec!=null) { + declarer.fieldsGrouping(idSpec.getGlobalStreamId().get_componentId(), PrepareRequest.ID_STREAM, new Fields("request")); + } + if(i==0 && component.declarations.isEmpty()) { + declarer.noneGrouping(PREPARE_ID, PrepareRequest.ARGS_STREAM); + } else { + String prevId; + if(i==0) { + prevId = PREPARE_ID; + } else { + prevId = boltId(i-1); + } + for(InputDeclaration declaration: component.declarations) { + declaration.declare(prevId, declarer); + } + } + if(i>0) { + declarer.directGrouping(boltId(i-1), Constants.COORDINATED_STREAM_ID); + } + } + + IRichBolt lastBolt = _components.get(_components.size()-1).bolt; + OutputFieldsGetter getter = new OutputFieldsGetter(); + lastBolt.declareOutputFields(getter); + Map streams = getter.getFieldsDeclaration(); + if(streams.size()!=1) { + throw new RuntimeException("Must declare exactly one stream from last bolt in LinearDRPCTopology"); + } + String outputStream = streams.keySet().iterator().next(); + List fields = streams.get(outputStream).get_output_fields(); + if(fields.size()!=2) { + throw new RuntimeException("Output stream of last component in LinearDRPCTopology must contain exactly two fields. The first should be the request id, and the second should be the result."); + } + + builder.setBolt("JoinResult", new JoinResult(PREPARE_ID)) + .fieldsGrouping(boltId(i-1), outputStream, new Fields(fields.get(0))) + .fieldsGrouping(PREPARE_ID, PrepareRequest.RETURN_STREAM, new Fields("request")); + i++; + builder.setBolt("ReturnResults", new ReturnResults()) + .noneGrouping("JoinResult"); + return builder.createTopology(); + } + + private static String boltId(int index) { + return "bolt" + index; + } + + private static class Component { + public IRichBolt bolt; + public int parallelism; + public List componentConfs; + public List declarations = new ArrayList(); + + public Component(IRichBolt bolt, int parallelism) { + this.bolt = bolt; + this.parallelism = parallelism; + this.componentConfs = new ArrayList(); + } + } + + private static interface InputDeclaration { + public void declare(String prevComponent, InputDeclarer declarer); + } + + private class InputDeclarerImpl extends BaseConfigurationDeclarer implements LinearDRPCInputDeclarer { + Component _component; + + public InputDeclarerImpl(Component component) { + _component = component; + } + + @Override + public LinearDRPCInputDeclarer fieldsGrouping(final Fields fields) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(String prevComponent, InputDeclarer declarer) { + declarer.fieldsGrouping(prevComponent, fields); + } + }); + return this; + } + + @Override + public LinearDRPCInputDeclarer fieldsGrouping(final String streamId, final Fields fields) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(String prevComponent, InputDeclarer declarer) { + declarer.fieldsGrouping(prevComponent, streamId, fields); + } + }); + return this; + } + + @Override + public LinearDRPCInputDeclarer globalGrouping() { + addDeclaration(new InputDeclaration() { + @Override + public void declare(String prevComponent, InputDeclarer declarer) { + declarer.globalGrouping(prevComponent); + } + }); + return this; + } + + @Override + public LinearDRPCInputDeclarer globalGrouping(final String streamId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(String prevComponent, InputDeclarer declarer) { + declarer.globalGrouping(prevComponent, streamId); + } + }); + return this; + } + + @Override + public LinearDRPCInputDeclarer shuffleGrouping() { + addDeclaration(new InputDeclaration() { + @Override + public void declare(String prevComponent, InputDeclarer declarer) { + declarer.shuffleGrouping(prevComponent); + } + }); + return this; + } + + @Override + public LinearDRPCInputDeclarer shuffleGrouping(final String streamId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(String prevComponent, InputDeclarer declarer) { + declarer.shuffleGrouping(prevComponent, streamId); + } + }); + return this; + } + + @Override + public LinearDRPCInputDeclarer localOrShuffleGrouping() { + addDeclaration(new InputDeclaration() { + @Override + public void declare(String prevComponent, InputDeclarer declarer) { + declarer.localOrShuffleGrouping(prevComponent); + } + }); + return this; + } + + @Override + public LinearDRPCInputDeclarer localOrShuffleGrouping(final String streamId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(String prevComponent, InputDeclarer declarer) { + declarer.localOrShuffleGrouping(prevComponent, streamId); + } + }); + return this; + } + + @Override + public LinearDRPCInputDeclarer noneGrouping() { + addDeclaration(new InputDeclaration() { + @Override + public void declare(String prevComponent, InputDeclarer declarer) { + declarer.noneGrouping(prevComponent); + } + }); + return this; + } + + @Override + public LinearDRPCInputDeclarer noneGrouping(final String streamId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(String prevComponent, InputDeclarer declarer) { + declarer.noneGrouping(prevComponent, streamId); + } + }); + return this; + } + + @Override + public LinearDRPCInputDeclarer allGrouping() { + addDeclaration(new InputDeclaration() { + @Override + public void declare(String prevComponent, InputDeclarer declarer) { + declarer.allGrouping(prevComponent); + } + }); + return this; + } + + @Override + public LinearDRPCInputDeclarer allGrouping(final String streamId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(String prevComponent, InputDeclarer declarer) { + declarer.allGrouping(prevComponent, streamId); + } + }); + return this; + } + + @Override + public LinearDRPCInputDeclarer directGrouping() { + addDeclaration(new InputDeclaration() { + @Override + public void declare(String prevComponent, InputDeclarer declarer) { + declarer.directGrouping(prevComponent); + } + }); + return this; + } + + @Override + public LinearDRPCInputDeclarer directGrouping(final String streamId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(String prevComponent, InputDeclarer declarer) { + declarer.directGrouping(prevComponent, streamId); + } + }); + return this; + } + + @Override + public LinearDRPCInputDeclarer partialKeyGrouping(Fields fields) { + return customGrouping(new PartialKeyGrouping(fields)); + } + + @Override + public LinearDRPCInputDeclarer partialKeyGrouping(String streamId, Fields fields) { + return customGrouping(streamId, new PartialKeyGrouping(fields)); + } + + @Override + public LinearDRPCInputDeclarer customGrouping(final CustomStreamGrouping grouping) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(String prevComponent, InputDeclarer declarer) { + declarer.customGrouping(prevComponent, grouping); + } + }); + return this; + } + + @Override + public LinearDRPCInputDeclarer customGrouping(final String streamId, final CustomStreamGrouping grouping) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(String prevComponent, InputDeclarer declarer) { + declarer.customGrouping(prevComponent, streamId, grouping); + } + }); + return this; + } + + private void addDeclaration(InputDeclaration declaration) { + _component.declarations.add(declaration); + } + + @Override + public LinearDRPCInputDeclarer addConfigurations(Map conf) { + _component.componentConfs.add(conf); + return this; + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/drpc/PrepareRequest.java b/jstorm-core/src/main/java/backtype/storm/drpc/PrepareRequest.java new file mode 100755 index 000000000..bd3216900 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/drpc/PrepareRequest.java @@ -0,0 +1,59 @@ +/** + * 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 backtype.storm.drpc; + +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseBasicBolt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; +import java.util.Map; +import java.util.Random; +import backtype.storm.utils.Utils; + + +public class PrepareRequest extends BaseBasicBolt { + public static final String ARGS_STREAM = Utils.DEFAULT_STREAM_ID; + public static final String RETURN_STREAM = "ret"; + public static final String ID_STREAM = "id"; + + Random rand; + + @Override + public void prepare(Map map, TopologyContext context) { + rand = new Random(); + } + + @Override + public void execute(Tuple tuple, BasicOutputCollector collector) { + String args = tuple.getString(0); + String returnInfo = tuple.getString(1); + long requestId = rand.nextLong(); + collector.emit(ARGS_STREAM, new Values(requestId, args)); + collector.emit(RETURN_STREAM, new Values(requestId, returnInfo)); + collector.emit(ID_STREAM, new Values(requestId)); + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declareStream(ARGS_STREAM, new Fields("request", "args")); + declarer.declareStream(RETURN_STREAM, new Fields("request", "return")); + declarer.declareStream(ID_STREAM, new Fields("request")); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/drpc/ReturnResults.java b/jstorm-core/src/main/java/backtype/storm/drpc/ReturnResults.java new file mode 100644 index 000000000..2ca517e3f --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/drpc/ReturnResults.java @@ -0,0 +1,125 @@ +/** + * 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 backtype.storm.drpc; + +import backtype.storm.Config; +import backtype.storm.generated.DistributedRPCInvocations; +import backtype.storm.generated.AuthorizationException; +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.Tuple; +import backtype.storm.utils.ServiceRegistry; +import backtype.storm.utils.Utils; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.thrift.TException; +import org.apache.thrift.transport.TTransportException; +import org.json.simple.JSONValue; + + +public class ReturnResults extends BaseRichBolt { + //ANY CHANGE TO THIS CODE MUST BE SERIALIZABLE COMPATIBLE OR THERE WILL BE PROBLEMS + static final long serialVersionUID = -774882142710631591L; + + public static final Logger LOG = LoggerFactory.getLogger(ReturnResults.class); + OutputCollector _collector; + boolean local; + Map _conf; + Map _clients = new HashMap(); + + @Override + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + _conf = stormConf; + _collector = collector; + local = stormConf.get(Config.STORM_CLUSTER_MODE).equals("local"); + } + + @Override + public void execute(Tuple input) { + String result = (String) input.getValue(0); + String returnInfo = (String) input.getValue(1); + //LOG.info("Receive one message, resultInfo:{}, result:{}", returnInfo, result); + if(returnInfo!=null) { + Map retMap = (Map) JSONValue.parse(returnInfo); + final String host = (String) retMap.get("host"); + final int port = Utils.getInt(retMap.get("port")); + String id = (String) retMap.get("id"); + DistributedRPCInvocations.Iface client; + if(local) { + client = (DistributedRPCInvocations.Iface) ServiceRegistry.getService(host); + } else { + List server = new ArrayList() {{ + add(host); + add(port); + }}; + + if(!_clients.containsKey(server)) { + try { + _clients.put(server, new DRPCInvocationsClient(_conf, host, port)); + } catch (TTransportException ex) { + throw new RuntimeException(ex); + } + } + client = _clients.get(server); + } + + try { + client.result(id, result); + _collector.ack(input); + } catch (AuthorizationException aze) { + LOG.error("Not authorized to return results to DRPC server", aze); + _collector.fail(input); + if (client instanceof DRPCInvocationsClient) { + try { + LOG.info("reconnecting... "); + ((DRPCInvocationsClient)client).reconnectClient(); //Blocking call + } catch (TException e2) { + throw new RuntimeException(e2); + } + } + } catch(TException e) { + LOG.error("Failed to return results to DRPC server", e); + _collector.fail(input); + if (client instanceof DRPCInvocationsClient) { + try { + LOG.info("reconnecting... "); + ((DRPCInvocationsClient)client).reconnectClient(); //Blocking call + } catch (TException e2) { + throw new RuntimeException(e2); + } + } + } + } + } + + @Override + public void cleanup() { + for(DRPCInvocationsClient c: _clients.values()) { + c.close(); + } + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/generated/AlreadyAliveException.java b/jstorm-core/src/main/java/backtype/storm/generated/AlreadyAliveException.java new file mode 100644 index 000000000..06eadafb9 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/AlreadyAliveException.java @@ -0,0 +1,389 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class AlreadyAliveException extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AlreadyAliveException"); + + private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new AlreadyAliveExceptionStandardSchemeFactory()); + schemes.put(TupleScheme.class, new AlreadyAliveExceptionTupleSchemeFactory()); + } + + private String msg; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + MSG((short)1, "msg"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // MSG + return MSG; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AlreadyAliveException.class, metaDataMap); + } + + public AlreadyAliveException() { + } + + public AlreadyAliveException( + String msg) + { + this(); + this.msg = msg; + } + + /** + * Performs a deep copy on other. + */ + public AlreadyAliveException(AlreadyAliveException other) { + if (other.is_set_msg()) { + this.msg = other.msg; + } + } + + public AlreadyAliveException deepCopy() { + return new AlreadyAliveException(this); + } + + @Override + public void clear() { + this.msg = null; + } + + public String get_msg() { + return this.msg; + } + + public void set_msg(String msg) { + this.msg = msg; + } + + public void unset_msg() { + this.msg = null; + } + + /** Returns true if field msg is set (has been assigned a value) and false otherwise */ + public boolean is_set_msg() { + return this.msg != null; + } + + public void set_msg_isSet(boolean value) { + if (!value) { + this.msg = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case MSG: + if (value == null) { + unset_msg(); + } else { + set_msg((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case MSG: + return get_msg(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case MSG: + return is_set_msg(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof AlreadyAliveException) + return this.equals((AlreadyAliveException)that); + return false; + } + + public boolean equals(AlreadyAliveException that) { + if (that == null) + return false; + + boolean this_present_msg = true && this.is_set_msg(); + boolean that_present_msg = true && that.is_set_msg(); + if (this_present_msg || that_present_msg) { + if (!(this_present_msg && that_present_msg)) + return false; + if (!this.msg.equals(that.msg)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_msg = true && (is_set_msg()); + list.add(present_msg); + if (present_msg) + list.add(msg); + + return list.hashCode(); + } + + @Override + public int compareTo(AlreadyAliveException other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_msg()).compareTo(other.is_set_msg()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_msg()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, other.msg); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("AlreadyAliveException("); + boolean first = true; + + sb.append("msg:"); + if (this.msg == null) { + sb.append("null"); + } else { + sb.append(this.msg); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_msg()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class AlreadyAliveExceptionStandardSchemeFactory implements SchemeFactory { + public AlreadyAliveExceptionStandardScheme getScheme() { + return new AlreadyAliveExceptionStandardScheme(); + } + } + + private static class AlreadyAliveExceptionStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, AlreadyAliveException struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // MSG + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.msg = iprot.readString(); + struct.set_msg_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, AlreadyAliveException struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.msg != null) { + oprot.writeFieldBegin(MSG_FIELD_DESC); + oprot.writeString(struct.msg); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class AlreadyAliveExceptionTupleSchemeFactory implements SchemeFactory { + public AlreadyAliveExceptionTupleScheme getScheme() { + return new AlreadyAliveExceptionTupleScheme(); + } + } + + private static class AlreadyAliveExceptionTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, AlreadyAliveException struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.msg); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, AlreadyAliveException struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.msg = iprot.readString(); + struct.set_msg_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/AuthorizationException.java b/jstorm-core/src/main/java/backtype/storm/generated/AuthorizationException.java new file mode 100644 index 000000000..02f72f0ea --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/AuthorizationException.java @@ -0,0 +1,389 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class AuthorizationException extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AuthorizationException"); + + private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new AuthorizationExceptionStandardSchemeFactory()); + schemes.put(TupleScheme.class, new AuthorizationExceptionTupleSchemeFactory()); + } + + private String msg; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + MSG((short)1, "msg"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // MSG + return MSG; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AuthorizationException.class, metaDataMap); + } + + public AuthorizationException() { + } + + public AuthorizationException( + String msg) + { + this(); + this.msg = msg; + } + + /** + * Performs a deep copy on other. + */ + public AuthorizationException(AuthorizationException other) { + if (other.is_set_msg()) { + this.msg = other.msg; + } + } + + public AuthorizationException deepCopy() { + return new AuthorizationException(this); + } + + @Override + public void clear() { + this.msg = null; + } + + public String get_msg() { + return this.msg; + } + + public void set_msg(String msg) { + this.msg = msg; + } + + public void unset_msg() { + this.msg = null; + } + + /** Returns true if field msg is set (has been assigned a value) and false otherwise */ + public boolean is_set_msg() { + return this.msg != null; + } + + public void set_msg_isSet(boolean value) { + if (!value) { + this.msg = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case MSG: + if (value == null) { + unset_msg(); + } else { + set_msg((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case MSG: + return get_msg(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case MSG: + return is_set_msg(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof AuthorizationException) + return this.equals((AuthorizationException)that); + return false; + } + + public boolean equals(AuthorizationException that) { + if (that == null) + return false; + + boolean this_present_msg = true && this.is_set_msg(); + boolean that_present_msg = true && that.is_set_msg(); + if (this_present_msg || that_present_msg) { + if (!(this_present_msg && that_present_msg)) + return false; + if (!this.msg.equals(that.msg)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_msg = true && (is_set_msg()); + list.add(present_msg); + if (present_msg) + list.add(msg); + + return list.hashCode(); + } + + @Override + public int compareTo(AuthorizationException other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_msg()).compareTo(other.is_set_msg()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_msg()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, other.msg); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("AuthorizationException("); + boolean first = true; + + sb.append("msg:"); + if (this.msg == null) { + sb.append("null"); + } else { + sb.append(this.msg); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_msg()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class AuthorizationExceptionStandardSchemeFactory implements SchemeFactory { + public AuthorizationExceptionStandardScheme getScheme() { + return new AuthorizationExceptionStandardScheme(); + } + } + + private static class AuthorizationExceptionStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, AuthorizationException struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // MSG + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.msg = iprot.readString(); + struct.set_msg_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, AuthorizationException struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.msg != null) { + oprot.writeFieldBegin(MSG_FIELD_DESC); + oprot.writeString(struct.msg); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class AuthorizationExceptionTupleSchemeFactory implements SchemeFactory { + public AuthorizationExceptionTupleScheme getScheme() { + return new AuthorizationExceptionTupleScheme(); + } + } + + private static class AuthorizationExceptionTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, AuthorizationException struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.msg); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, AuthorizationException struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.msg = iprot.readString(); + struct.set_msg_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/Bolt.java b/jstorm-core/src/main/java/backtype/storm/generated/Bolt.java new file mode 100644 index 000000000..e3d0a0783 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/Bolt.java @@ -0,0 +1,497 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class Bolt implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Bolt"); + + private static final org.apache.thrift.protocol.TField BOLT_OBJECT_FIELD_DESC = new org.apache.thrift.protocol.TField("bolt_object", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField COMMON_FIELD_DESC = new org.apache.thrift.protocol.TField("common", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new BoltStandardSchemeFactory()); + schemes.put(TupleScheme.class, new BoltTupleSchemeFactory()); + } + + private ComponentObject bolt_object; // required + private ComponentCommon common; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + BOLT_OBJECT((short)1, "bolt_object"), + COMMON((short)2, "common"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // BOLT_OBJECT + return BOLT_OBJECT; + case 2: // COMMON + return COMMON; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.BOLT_OBJECT, new org.apache.thrift.meta_data.FieldMetaData("bolt_object", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentObject.class))); + tmpMap.put(_Fields.COMMON, new org.apache.thrift.meta_data.FieldMetaData("common", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentCommon.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Bolt.class, metaDataMap); + } + + public Bolt() { + } + + public Bolt( + ComponentObject bolt_object, + ComponentCommon common) + { + this(); + this.bolt_object = bolt_object; + this.common = common; + } + + /** + * Performs a deep copy on other. + */ + public Bolt(Bolt other) { + if (other.is_set_bolt_object()) { + this.bolt_object = new ComponentObject(other.bolt_object); + } + if (other.is_set_common()) { + this.common = new ComponentCommon(other.common); + } + } + + public Bolt deepCopy() { + return new Bolt(this); + } + + @Override + public void clear() { + this.bolt_object = null; + this.common = null; + } + + public ComponentObject get_bolt_object() { + return this.bolt_object; + } + + public void set_bolt_object(ComponentObject bolt_object) { + this.bolt_object = bolt_object; + } + + public void unset_bolt_object() { + this.bolt_object = null; + } + + /** Returns true if field bolt_object is set (has been assigned a value) and false otherwise */ + public boolean is_set_bolt_object() { + return this.bolt_object != null; + } + + public void set_bolt_object_isSet(boolean value) { + if (!value) { + this.bolt_object = null; + } + } + + public ComponentCommon get_common() { + return this.common; + } + + public void set_common(ComponentCommon common) { + this.common = common; + } + + public void unset_common() { + this.common = null; + } + + /** Returns true if field common is set (has been assigned a value) and false otherwise */ + public boolean is_set_common() { + return this.common != null; + } + + public void set_common_isSet(boolean value) { + if (!value) { + this.common = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case BOLT_OBJECT: + if (value == null) { + unset_bolt_object(); + } else { + set_bolt_object((ComponentObject)value); + } + break; + + case COMMON: + if (value == null) { + unset_common(); + } else { + set_common((ComponentCommon)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case BOLT_OBJECT: + return get_bolt_object(); + + case COMMON: + return get_common(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case BOLT_OBJECT: + return is_set_bolt_object(); + case COMMON: + return is_set_common(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof Bolt) + return this.equals((Bolt)that); + return false; + } + + public boolean equals(Bolt that) { + if (that == null) + return false; + + boolean this_present_bolt_object = true && this.is_set_bolt_object(); + boolean that_present_bolt_object = true && that.is_set_bolt_object(); + if (this_present_bolt_object || that_present_bolt_object) { + if (!(this_present_bolt_object && that_present_bolt_object)) + return false; + if (!this.bolt_object.equals(that.bolt_object)) + return false; + } + + boolean this_present_common = true && this.is_set_common(); + boolean that_present_common = true && that.is_set_common(); + if (this_present_common || that_present_common) { + if (!(this_present_common && that_present_common)) + return false; + if (!this.common.equals(that.common)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_bolt_object = true && (is_set_bolt_object()); + list.add(present_bolt_object); + if (present_bolt_object) + list.add(bolt_object); + + boolean present_common = true && (is_set_common()); + list.add(present_common); + if (present_common) + list.add(common); + + return list.hashCode(); + } + + @Override + public int compareTo(Bolt other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_bolt_object()).compareTo(other.is_set_bolt_object()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_bolt_object()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.bolt_object, other.bolt_object); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_common()).compareTo(other.is_set_common()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_common()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.common, other.common); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("Bolt("); + boolean first = true; + + sb.append("bolt_object:"); + if (this.bolt_object == null) { + sb.append("null"); + } else { + sb.append(this.bolt_object); + } + first = false; + if (!first) sb.append(", "); + sb.append("common:"); + if (this.common == null) { + sb.append("null"); + } else { + sb.append(this.common); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_bolt_object()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'bolt_object' is unset! Struct:" + toString()); + } + + if (!is_set_common()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'common' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (common != null) { + common.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class BoltStandardSchemeFactory implements SchemeFactory { + public BoltStandardScheme getScheme() { + return new BoltStandardScheme(); + } + } + + private static class BoltStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, Bolt struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // BOLT_OBJECT + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.bolt_object = new ComponentObject(); + struct.bolt_object.read(iprot); + struct.set_bolt_object_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // COMMON + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.common = new ComponentCommon(); + struct.common.read(iprot); + struct.set_common_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, Bolt struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.bolt_object != null) { + oprot.writeFieldBegin(BOLT_OBJECT_FIELD_DESC); + struct.bolt_object.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.common != null) { + oprot.writeFieldBegin(COMMON_FIELD_DESC); + struct.common.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class BoltTupleSchemeFactory implements SchemeFactory { + public BoltTupleScheme getScheme() { + return new BoltTupleScheme(); + } + } + + private static class BoltTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, Bolt struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.bolt_object.write(oprot); + struct.common.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, Bolt struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.bolt_object = new ComponentObject(); + struct.bolt_object.read(iprot); + struct.set_bolt_object_isSet(true); + struct.common = new ComponentCommon(); + struct.common.read(iprot); + struct.set_common_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/ClusterSummary.java b/jstorm-core/src/main/java/backtype/storm/generated/ClusterSummary.java new file mode 100644 index 000000000..1735b8acf --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/ClusterSummary.java @@ -0,0 +1,704 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class ClusterSummary implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ClusterSummary"); + + private static final org.apache.thrift.protocol.TField NIMBUS_FIELD_DESC = new org.apache.thrift.protocol.TField("nimbus", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField SUPERVISORS_FIELD_DESC = new org.apache.thrift.protocol.TField("supervisors", org.apache.thrift.protocol.TType.LIST, (short)2); + private static final org.apache.thrift.protocol.TField TOPOLOGIES_FIELD_DESC = new org.apache.thrift.protocol.TField("topologies", org.apache.thrift.protocol.TType.LIST, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ClusterSummaryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ClusterSummaryTupleSchemeFactory()); + } + + private NimbusSummary nimbus; // required + private List supervisors; // required + private List topologies; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NIMBUS((short)1, "nimbus"), + SUPERVISORS((short)2, "supervisors"), + TOPOLOGIES((short)3, "topologies"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NIMBUS + return NIMBUS; + case 2: // SUPERVISORS + return SUPERVISORS; + case 3: // TOPOLOGIES + return TOPOLOGIES; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NIMBUS, new org.apache.thrift.meta_data.FieldMetaData("nimbus", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NimbusSummary.class))); + tmpMap.put(_Fields.SUPERVISORS, new org.apache.thrift.meta_data.FieldMetaData("supervisors", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SupervisorSummary.class)))); + tmpMap.put(_Fields.TOPOLOGIES, new org.apache.thrift.meta_data.FieldMetaData("topologies", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologySummary.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ClusterSummary.class, metaDataMap); + } + + public ClusterSummary() { + } + + public ClusterSummary( + NimbusSummary nimbus, + List supervisors, + List topologies) + { + this(); + this.nimbus = nimbus; + this.supervisors = supervisors; + this.topologies = topologies; + } + + /** + * Performs a deep copy on other. + */ + public ClusterSummary(ClusterSummary other) { + if (other.is_set_nimbus()) { + this.nimbus = new NimbusSummary(other.nimbus); + } + if (other.is_set_supervisors()) { + List __this__supervisors = new ArrayList(other.supervisors.size()); + for (SupervisorSummary other_element : other.supervisors) { + __this__supervisors.add(new SupervisorSummary(other_element)); + } + this.supervisors = __this__supervisors; + } + if (other.is_set_topologies()) { + List __this__topologies = new ArrayList(other.topologies.size()); + for (TopologySummary other_element : other.topologies) { + __this__topologies.add(new TopologySummary(other_element)); + } + this.topologies = __this__topologies; + } + } + + public ClusterSummary deepCopy() { + return new ClusterSummary(this); + } + + @Override + public void clear() { + this.nimbus = null; + this.supervisors = null; + this.topologies = null; + } + + public NimbusSummary get_nimbus() { + return this.nimbus; + } + + public void set_nimbus(NimbusSummary nimbus) { + this.nimbus = nimbus; + } + + public void unset_nimbus() { + this.nimbus = null; + } + + /** Returns true if field nimbus is set (has been assigned a value) and false otherwise */ + public boolean is_set_nimbus() { + return this.nimbus != null; + } + + public void set_nimbus_isSet(boolean value) { + if (!value) { + this.nimbus = null; + } + } + + public int get_supervisors_size() { + return (this.supervisors == null) ? 0 : this.supervisors.size(); + } + + public java.util.Iterator get_supervisors_iterator() { + return (this.supervisors == null) ? null : this.supervisors.iterator(); + } + + public void add_to_supervisors(SupervisorSummary elem) { + if (this.supervisors == null) { + this.supervisors = new ArrayList(); + } + this.supervisors.add(elem); + } + + public List get_supervisors() { + return this.supervisors; + } + + public void set_supervisors(List supervisors) { + this.supervisors = supervisors; + } + + public void unset_supervisors() { + this.supervisors = null; + } + + /** Returns true if field supervisors is set (has been assigned a value) and false otherwise */ + public boolean is_set_supervisors() { + return this.supervisors != null; + } + + public void set_supervisors_isSet(boolean value) { + if (!value) { + this.supervisors = null; + } + } + + public int get_topologies_size() { + return (this.topologies == null) ? 0 : this.topologies.size(); + } + + public java.util.Iterator get_topologies_iterator() { + return (this.topologies == null) ? null : this.topologies.iterator(); + } + + public void add_to_topologies(TopologySummary elem) { + if (this.topologies == null) { + this.topologies = new ArrayList(); + } + this.topologies.add(elem); + } + + public List get_topologies() { + return this.topologies; + } + + public void set_topologies(List topologies) { + this.topologies = topologies; + } + + public void unset_topologies() { + this.topologies = null; + } + + /** Returns true if field topologies is set (has been assigned a value) and false otherwise */ + public boolean is_set_topologies() { + return this.topologies != null; + } + + public void set_topologies_isSet(boolean value) { + if (!value) { + this.topologies = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NIMBUS: + if (value == null) { + unset_nimbus(); + } else { + set_nimbus((NimbusSummary)value); + } + break; + + case SUPERVISORS: + if (value == null) { + unset_supervisors(); + } else { + set_supervisors((List)value); + } + break; + + case TOPOLOGIES: + if (value == null) { + unset_topologies(); + } else { + set_topologies((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NIMBUS: + return get_nimbus(); + + case SUPERVISORS: + return get_supervisors(); + + case TOPOLOGIES: + return get_topologies(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NIMBUS: + return is_set_nimbus(); + case SUPERVISORS: + return is_set_supervisors(); + case TOPOLOGIES: + return is_set_topologies(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ClusterSummary) + return this.equals((ClusterSummary)that); + return false; + } + + public boolean equals(ClusterSummary that) { + if (that == null) + return false; + + boolean this_present_nimbus = true && this.is_set_nimbus(); + boolean that_present_nimbus = true && that.is_set_nimbus(); + if (this_present_nimbus || that_present_nimbus) { + if (!(this_present_nimbus && that_present_nimbus)) + return false; + if (!this.nimbus.equals(that.nimbus)) + return false; + } + + boolean this_present_supervisors = true && this.is_set_supervisors(); + boolean that_present_supervisors = true && that.is_set_supervisors(); + if (this_present_supervisors || that_present_supervisors) { + if (!(this_present_supervisors && that_present_supervisors)) + return false; + if (!this.supervisors.equals(that.supervisors)) + return false; + } + + boolean this_present_topologies = true && this.is_set_topologies(); + boolean that_present_topologies = true && that.is_set_topologies(); + if (this_present_topologies || that_present_topologies) { + if (!(this_present_topologies && that_present_topologies)) + return false; + if (!this.topologies.equals(that.topologies)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_nimbus = true && (is_set_nimbus()); + list.add(present_nimbus); + if (present_nimbus) + list.add(nimbus); + + boolean present_supervisors = true && (is_set_supervisors()); + list.add(present_supervisors); + if (present_supervisors) + list.add(supervisors); + + boolean present_topologies = true && (is_set_topologies()); + list.add(present_topologies); + if (present_topologies) + list.add(topologies); + + return list.hashCode(); + } + + @Override + public int compareTo(ClusterSummary other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_nimbus()).compareTo(other.is_set_nimbus()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_nimbus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nimbus, other.nimbus); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_supervisors()).compareTo(other.is_set_supervisors()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_supervisors()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.supervisors, other.supervisors); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_topologies()).compareTo(other.is_set_topologies()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_topologies()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topologies, other.topologies); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ClusterSummary("); + boolean first = true; + + sb.append("nimbus:"); + if (this.nimbus == null) { + sb.append("null"); + } else { + sb.append(this.nimbus); + } + first = false; + if (!first) sb.append(", "); + sb.append("supervisors:"); + if (this.supervisors == null) { + sb.append("null"); + } else { + sb.append(this.supervisors); + } + first = false; + if (!first) sb.append(", "); + sb.append("topologies:"); + if (this.topologies == null) { + sb.append("null"); + } else { + sb.append(this.topologies); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_nimbus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nimbus' is unset! Struct:" + toString()); + } + + if (!is_set_supervisors()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'supervisors' is unset! Struct:" + toString()); + } + + if (!is_set_topologies()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'topologies' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (nimbus != null) { + nimbus.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ClusterSummaryStandardSchemeFactory implements SchemeFactory { + public ClusterSummaryStandardScheme getScheme() { + return new ClusterSummaryStandardScheme(); + } + } + + private static class ClusterSummaryStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ClusterSummary struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NIMBUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.nimbus = new NimbusSummary(); + struct.nimbus.read(iprot); + struct.set_nimbus_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // SUPERVISORS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list82 = iprot.readListBegin(); + struct.supervisors = new ArrayList(_list82.size); + SupervisorSummary _elem83; + for (int _i84 = 0; _i84 < _list82.size; ++_i84) + { + _elem83 = new SupervisorSummary(); + _elem83.read(iprot); + struct.supervisors.add(_elem83); + } + iprot.readListEnd(); + } + struct.set_supervisors_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // TOPOLOGIES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list85 = iprot.readListBegin(); + struct.topologies = new ArrayList(_list85.size); + TopologySummary _elem86; + for (int _i87 = 0; _i87 < _list85.size; ++_i87) + { + _elem86 = new TopologySummary(); + _elem86.read(iprot); + struct.topologies.add(_elem86); + } + iprot.readListEnd(); + } + struct.set_topologies_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ClusterSummary struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.nimbus != null) { + oprot.writeFieldBegin(NIMBUS_FIELD_DESC); + struct.nimbus.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.supervisors != null) { + oprot.writeFieldBegin(SUPERVISORS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.supervisors.size())); + for (SupervisorSummary _iter88 : struct.supervisors) + { + _iter88.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.topologies != null) { + oprot.writeFieldBegin(TOPOLOGIES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.topologies.size())); + for (TopologySummary _iter89 : struct.topologies) + { + _iter89.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ClusterSummaryTupleSchemeFactory implements SchemeFactory { + public ClusterSummaryTupleScheme getScheme() { + return new ClusterSummaryTupleScheme(); + } + } + + private static class ClusterSummaryTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ClusterSummary struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.nimbus.write(oprot); + { + oprot.writeI32(struct.supervisors.size()); + for (SupervisorSummary _iter90 : struct.supervisors) + { + _iter90.write(oprot); + } + } + { + oprot.writeI32(struct.topologies.size()); + for (TopologySummary _iter91 : struct.topologies) + { + _iter91.write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ClusterSummary struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.nimbus = new NimbusSummary(); + struct.nimbus.read(iprot); + struct.set_nimbus_isSet(true); + { + org.apache.thrift.protocol.TList _list92 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.supervisors = new ArrayList(_list92.size); + SupervisorSummary _elem93; + for (int _i94 = 0; _i94 < _list92.size; ++_i94) + { + _elem93 = new SupervisorSummary(); + _elem93.read(iprot); + struct.supervisors.add(_elem93); + } + } + struct.set_supervisors_isSet(true); + { + org.apache.thrift.protocol.TList _list95 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.topologies = new ArrayList(_list95.size); + TopologySummary _elem96; + for (int _i97 = 0; _i97 < _list95.size; ++_i97) + { + _elem96 = new TopologySummary(); + _elem96.read(iprot); + struct.topologies.add(_elem96); + } + } + struct.set_topologies_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/ComponentCommon.java b/jstorm-core/src/main/java/backtype/storm/generated/ComponentCommon.java new file mode 100644 index 000000000..0a98a62ed --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/ComponentCommon.java @@ -0,0 +1,835 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class ComponentCommon implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ComponentCommon"); + + private static final org.apache.thrift.protocol.TField INPUTS_FIELD_DESC = new org.apache.thrift.protocol.TField("inputs", org.apache.thrift.protocol.TType.MAP, (short)1); + private static final org.apache.thrift.protocol.TField STREAMS_FIELD_DESC = new org.apache.thrift.protocol.TField("streams", org.apache.thrift.protocol.TType.MAP, (short)2); + private static final org.apache.thrift.protocol.TField PARALLELISM_HINT_FIELD_DESC = new org.apache.thrift.protocol.TField("parallelism_hint", org.apache.thrift.protocol.TType.I32, (short)3); + private static final org.apache.thrift.protocol.TField JSON_CONF_FIELD_DESC = new org.apache.thrift.protocol.TField("json_conf", org.apache.thrift.protocol.TType.STRING, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ComponentCommonStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ComponentCommonTupleSchemeFactory()); + } + + private Map inputs; // required + private Map streams; // required + private int parallelism_hint; // optional + private String json_conf; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + INPUTS((short)1, "inputs"), + STREAMS((short)2, "streams"), + PARALLELISM_HINT((short)3, "parallelism_hint"), + JSON_CONF((short)4, "json_conf"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // INPUTS + return INPUTS; + case 2: // STREAMS + return STREAMS; + case 3: // PARALLELISM_HINT + return PARALLELISM_HINT; + case 4: // JSON_CONF + return JSON_CONF; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __PARALLELISM_HINT_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.PARALLELISM_HINT,_Fields.JSON_CONF}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.INPUTS, new org.apache.thrift.meta_data.FieldMetaData("inputs", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GlobalStreamId.class), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Grouping.class)))); + tmpMap.put(_Fields.STREAMS, new org.apache.thrift.meta_data.FieldMetaData("streams", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, StreamInfo.class)))); + tmpMap.put(_Fields.PARALLELISM_HINT, new org.apache.thrift.meta_data.FieldMetaData("parallelism_hint", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.JSON_CONF, new org.apache.thrift.meta_data.FieldMetaData("json_conf", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ComponentCommon.class, metaDataMap); + } + + public ComponentCommon() { + } + + public ComponentCommon( + Map inputs, + Map streams) + { + this(); + this.inputs = inputs; + this.streams = streams; + } + + /** + * Performs a deep copy on other. + */ + public ComponentCommon(ComponentCommon other) { + __isset_bitfield = other.__isset_bitfield; + if (other.is_set_inputs()) { + Map __this__inputs = new HashMap(other.inputs.size()); + for (Map.Entry other_element : other.inputs.entrySet()) { + + GlobalStreamId other_element_key = other_element.getKey(); + Grouping other_element_value = other_element.getValue(); + + GlobalStreamId __this__inputs_copy_key = new GlobalStreamId(other_element_key); + + Grouping __this__inputs_copy_value = new Grouping(other_element_value); + + __this__inputs.put(__this__inputs_copy_key, __this__inputs_copy_value); + } + this.inputs = __this__inputs; + } + if (other.is_set_streams()) { + Map __this__streams = new HashMap(other.streams.size()); + for (Map.Entry other_element : other.streams.entrySet()) { + + String other_element_key = other_element.getKey(); + StreamInfo other_element_value = other_element.getValue(); + + String __this__streams_copy_key = other_element_key; + + StreamInfo __this__streams_copy_value = new StreamInfo(other_element_value); + + __this__streams.put(__this__streams_copy_key, __this__streams_copy_value); + } + this.streams = __this__streams; + } + this.parallelism_hint = other.parallelism_hint; + if (other.is_set_json_conf()) { + this.json_conf = other.json_conf; + } + } + + public ComponentCommon deepCopy() { + return new ComponentCommon(this); + } + + @Override + public void clear() { + this.inputs = null; + this.streams = null; + set_parallelism_hint_isSet(false); + this.parallelism_hint = 0; + this.json_conf = null; + } + + public int get_inputs_size() { + return (this.inputs == null) ? 0 : this.inputs.size(); + } + + public void put_to_inputs(GlobalStreamId key, Grouping val) { + if (this.inputs == null) { + this.inputs = new HashMap(); + } + this.inputs.put(key, val); + } + + public Map get_inputs() { + return this.inputs; + } + + public void set_inputs(Map inputs) { + this.inputs = inputs; + } + + public void unset_inputs() { + this.inputs = null; + } + + /** Returns true if field inputs is set (has been assigned a value) and false otherwise */ + public boolean is_set_inputs() { + return this.inputs != null; + } + + public void set_inputs_isSet(boolean value) { + if (!value) { + this.inputs = null; + } + } + + public int get_streams_size() { + return (this.streams == null) ? 0 : this.streams.size(); + } + + public void put_to_streams(String key, StreamInfo val) { + if (this.streams == null) { + this.streams = new HashMap(); + } + this.streams.put(key, val); + } + + public Map get_streams() { + return this.streams; + } + + public void set_streams(Map streams) { + this.streams = streams; + } + + public void unset_streams() { + this.streams = null; + } + + /** Returns true if field streams is set (has been assigned a value) and false otherwise */ + public boolean is_set_streams() { + return this.streams != null; + } + + public void set_streams_isSet(boolean value) { + if (!value) { + this.streams = null; + } + } + + public int get_parallelism_hint() { + return this.parallelism_hint; + } + + public void set_parallelism_hint(int parallelism_hint) { + this.parallelism_hint = parallelism_hint; + set_parallelism_hint_isSet(true); + } + + public void unset_parallelism_hint() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PARALLELISM_HINT_ISSET_ID); + } + + /** Returns true if field parallelism_hint is set (has been assigned a value) and false otherwise */ + public boolean is_set_parallelism_hint() { + return EncodingUtils.testBit(__isset_bitfield, __PARALLELISM_HINT_ISSET_ID); + } + + public void set_parallelism_hint_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PARALLELISM_HINT_ISSET_ID, value); + } + + public String get_json_conf() { + return this.json_conf; + } + + public void set_json_conf(String json_conf) { + this.json_conf = json_conf; + } + + public void unset_json_conf() { + this.json_conf = null; + } + + /** Returns true if field json_conf is set (has been assigned a value) and false otherwise */ + public boolean is_set_json_conf() { + return this.json_conf != null; + } + + public void set_json_conf_isSet(boolean value) { + if (!value) { + this.json_conf = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case INPUTS: + if (value == null) { + unset_inputs(); + } else { + set_inputs((Map)value); + } + break; + + case STREAMS: + if (value == null) { + unset_streams(); + } else { + set_streams((Map)value); + } + break; + + case PARALLELISM_HINT: + if (value == null) { + unset_parallelism_hint(); + } else { + set_parallelism_hint((Integer)value); + } + break; + + case JSON_CONF: + if (value == null) { + unset_json_conf(); + } else { + set_json_conf((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case INPUTS: + return get_inputs(); + + case STREAMS: + return get_streams(); + + case PARALLELISM_HINT: + return Integer.valueOf(get_parallelism_hint()); + + case JSON_CONF: + return get_json_conf(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case INPUTS: + return is_set_inputs(); + case STREAMS: + return is_set_streams(); + case PARALLELISM_HINT: + return is_set_parallelism_hint(); + case JSON_CONF: + return is_set_json_conf(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ComponentCommon) + return this.equals((ComponentCommon)that); + return false; + } + + public boolean equals(ComponentCommon that) { + if (that == null) + return false; + + boolean this_present_inputs = true && this.is_set_inputs(); + boolean that_present_inputs = true && that.is_set_inputs(); + if (this_present_inputs || that_present_inputs) { + if (!(this_present_inputs && that_present_inputs)) + return false; + if (!this.inputs.equals(that.inputs)) + return false; + } + + boolean this_present_streams = true && this.is_set_streams(); + boolean that_present_streams = true && that.is_set_streams(); + if (this_present_streams || that_present_streams) { + if (!(this_present_streams && that_present_streams)) + return false; + if (!this.streams.equals(that.streams)) + return false; + } + + boolean this_present_parallelism_hint = true && this.is_set_parallelism_hint(); + boolean that_present_parallelism_hint = true && that.is_set_parallelism_hint(); + if (this_present_parallelism_hint || that_present_parallelism_hint) { + if (!(this_present_parallelism_hint && that_present_parallelism_hint)) + return false; + if (this.parallelism_hint != that.parallelism_hint) + return false; + } + + boolean this_present_json_conf = true && this.is_set_json_conf(); + boolean that_present_json_conf = true && that.is_set_json_conf(); + if (this_present_json_conf || that_present_json_conf) { + if (!(this_present_json_conf && that_present_json_conf)) + return false; + if (!this.json_conf.equals(that.json_conf)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_inputs = true && (is_set_inputs()); + list.add(present_inputs); + if (present_inputs) + list.add(inputs); + + boolean present_streams = true && (is_set_streams()); + list.add(present_streams); + if (present_streams) + list.add(streams); + + boolean present_parallelism_hint = true && (is_set_parallelism_hint()); + list.add(present_parallelism_hint); + if (present_parallelism_hint) + list.add(parallelism_hint); + + boolean present_json_conf = true && (is_set_json_conf()); + list.add(present_json_conf); + if (present_json_conf) + list.add(json_conf); + + return list.hashCode(); + } + + @Override + public int compareTo(ComponentCommon other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_inputs()).compareTo(other.is_set_inputs()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_inputs()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.inputs, other.inputs); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_streams()).compareTo(other.is_set_streams()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_streams()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.streams, other.streams); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_parallelism_hint()).compareTo(other.is_set_parallelism_hint()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_parallelism_hint()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.parallelism_hint, other.parallelism_hint); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_json_conf()).compareTo(other.is_set_json_conf()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_json_conf()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.json_conf, other.json_conf); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ComponentCommon("); + boolean first = true; + + sb.append("inputs:"); + if (this.inputs == null) { + sb.append("null"); + } else { + sb.append(this.inputs); + } + first = false; + if (!first) sb.append(", "); + sb.append("streams:"); + if (this.streams == null) { + sb.append("null"); + } else { + sb.append(this.streams); + } + first = false; + if (is_set_parallelism_hint()) { + if (!first) sb.append(", "); + sb.append("parallelism_hint:"); + sb.append(this.parallelism_hint); + first = false; + } + if (is_set_json_conf()) { + if (!first) sb.append(", "); + sb.append("json_conf:"); + if (this.json_conf == null) { + sb.append("null"); + } else { + sb.append(this.json_conf); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_inputs()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'inputs' is unset! Struct:" + toString()); + } + + if (!is_set_streams()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'streams' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ComponentCommonStandardSchemeFactory implements SchemeFactory { + public ComponentCommonStandardScheme getScheme() { + return new ComponentCommonStandardScheme(); + } + } + + private static class ComponentCommonStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ComponentCommon struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // INPUTS + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map24 = iprot.readMapBegin(); + struct.inputs = new HashMap(2*_map24.size); + GlobalStreamId _key25; + Grouping _val26; + for (int _i27 = 0; _i27 < _map24.size; ++_i27) + { + _key25 = new GlobalStreamId(); + _key25.read(iprot); + _val26 = new Grouping(); + _val26.read(iprot); + struct.inputs.put(_key25, _val26); + } + iprot.readMapEnd(); + } + struct.set_inputs_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // STREAMS + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map28 = iprot.readMapBegin(); + struct.streams = new HashMap(2*_map28.size); + String _key29; + StreamInfo _val30; + for (int _i31 = 0; _i31 < _map28.size; ++_i31) + { + _key29 = iprot.readString(); + _val30 = new StreamInfo(); + _val30.read(iprot); + struct.streams.put(_key29, _val30); + } + iprot.readMapEnd(); + } + struct.set_streams_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // PARALLELISM_HINT + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.parallelism_hint = iprot.readI32(); + struct.set_parallelism_hint_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // JSON_CONF + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.json_conf = iprot.readString(); + struct.set_json_conf_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ComponentCommon struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.inputs != null) { + oprot.writeFieldBegin(INPUTS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, struct.inputs.size())); + for (Map.Entry _iter32 : struct.inputs.entrySet()) + { + _iter32.getKey().write(oprot); + _iter32.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.streams != null) { + oprot.writeFieldBegin(STREAMS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.streams.size())); + for (Map.Entry _iter33 : struct.streams.entrySet()) + { + oprot.writeString(_iter33.getKey()); + _iter33.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.is_set_parallelism_hint()) { + oprot.writeFieldBegin(PARALLELISM_HINT_FIELD_DESC); + oprot.writeI32(struct.parallelism_hint); + oprot.writeFieldEnd(); + } + if (struct.json_conf != null) { + if (struct.is_set_json_conf()) { + oprot.writeFieldBegin(JSON_CONF_FIELD_DESC); + oprot.writeString(struct.json_conf); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ComponentCommonTupleSchemeFactory implements SchemeFactory { + public ComponentCommonTupleScheme getScheme() { + return new ComponentCommonTupleScheme(); + } + } + + private static class ComponentCommonTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ComponentCommon struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.inputs.size()); + for (Map.Entry _iter34 : struct.inputs.entrySet()) + { + _iter34.getKey().write(oprot); + _iter34.getValue().write(oprot); + } + } + { + oprot.writeI32(struct.streams.size()); + for (Map.Entry _iter35 : struct.streams.entrySet()) + { + oprot.writeString(_iter35.getKey()); + _iter35.getValue().write(oprot); + } + } + BitSet optionals = new BitSet(); + if (struct.is_set_parallelism_hint()) { + optionals.set(0); + } + if (struct.is_set_json_conf()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_parallelism_hint()) { + oprot.writeI32(struct.parallelism_hint); + } + if (struct.is_set_json_conf()) { + oprot.writeString(struct.json_conf); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ComponentCommon struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TMap _map36 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRUCT, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.inputs = new HashMap(2*_map36.size); + GlobalStreamId _key37; + Grouping _val38; + for (int _i39 = 0; _i39 < _map36.size; ++_i39) + { + _key37 = new GlobalStreamId(); + _key37.read(iprot); + _val38 = new Grouping(); + _val38.read(iprot); + struct.inputs.put(_key37, _val38); + } + } + struct.set_inputs_isSet(true); + { + org.apache.thrift.protocol.TMap _map40 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.streams = new HashMap(2*_map40.size); + String _key41; + StreamInfo _val42; + for (int _i43 = 0; _i43 < _map40.size; ++_i43) + { + _key41 = iprot.readString(); + _val42 = new StreamInfo(); + _val42.read(iprot); + struct.streams.put(_key41, _val42); + } + } + struct.set_streams_isSet(true); + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.parallelism_hint = iprot.readI32(); + struct.set_parallelism_hint_isSet(true); + } + if (incoming.get(1)) { + struct.json_conf = iprot.readString(); + struct.set_json_conf_isSet(true); + } + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/ComponentObject.java b/jstorm-core/src/main/java/backtype/storm/generated/ComponentObject.java new file mode 100755 index 000000000..ab3222555 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/ComponentObject.java @@ -0,0 +1,445 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +public class ComponentObject extends org.apache.thrift.TUnion { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ComponentObject"); + private static final org.apache.thrift.protocol.TField SERIALIZED_JAVA_FIELD_DESC = new org.apache.thrift.protocol.TField("serialized_java", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField SHELL_FIELD_DESC = new org.apache.thrift.protocol.TField("shell", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField JAVA_OBJECT_FIELD_DESC = new org.apache.thrift.protocol.TField("java_object", org.apache.thrift.protocol.TType.STRUCT, (short)3); + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SERIALIZED_JAVA((short)1, "serialized_java"), + SHELL((short)2, "shell"), + JAVA_OBJECT((short)3, "java_object"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SERIALIZED_JAVA + return SERIALIZED_JAVA; + case 2: // SHELL + return SHELL; + case 3: // JAVA_OBJECT + return JAVA_OBJECT; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SERIALIZED_JAVA, new org.apache.thrift.meta_data.FieldMetaData("serialized_java", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + tmpMap.put(_Fields.SHELL, new org.apache.thrift.meta_data.FieldMetaData("shell", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ShellComponent.class))); + tmpMap.put(_Fields.JAVA_OBJECT, new org.apache.thrift.meta_data.FieldMetaData("java_object", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, JavaObject.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ComponentObject.class, metaDataMap); + } + + public ComponentObject() { + super(); + } + + public ComponentObject(_Fields setField, Object value) { + super(setField, value); + } + + public ComponentObject(ComponentObject other) { + super(other); + } + public ComponentObject deepCopy() { + return new ComponentObject(this); + } + + public static ComponentObject serialized_java(ByteBuffer value) { + ComponentObject x = new ComponentObject(); + x.set_serialized_java(value); + return x; + } + + public static ComponentObject serialized_java(byte[] value) { + ComponentObject x = new ComponentObject(); + x.set_serialized_java(ByteBuffer.wrap(Arrays.copyOf(value, value.length))); + return x; + } + + public static ComponentObject shell(ShellComponent value) { + ComponentObject x = new ComponentObject(); + x.set_shell(value); + return x; + } + + public static ComponentObject java_object(JavaObject value) { + ComponentObject x = new ComponentObject(); + x.set_java_object(value); + return x; + } + + + @Override + protected void checkType(_Fields setField, Object value) throws ClassCastException { + switch (setField) { + case SERIALIZED_JAVA: + if (value instanceof ByteBuffer) { + break; + } + throw new ClassCastException("Was expecting value of type ByteBuffer for field 'serialized_java', but got " + value.getClass().getSimpleName()); + case SHELL: + if (value instanceof ShellComponent) { + break; + } + throw new ClassCastException("Was expecting value of type ShellComponent for field 'shell', but got " + value.getClass().getSimpleName()); + case JAVA_OBJECT: + if (value instanceof JavaObject) { + break; + } + throw new ClassCastException("Was expecting value of type JavaObject for field 'java_object', but got " + value.getClass().getSimpleName()); + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(field.id); + if (setField != null) { + switch (setField) { + case SERIALIZED_JAVA: + if (field.type == SERIALIZED_JAVA_FIELD_DESC.type) { + ByteBuffer serialized_java; + serialized_java = iprot.readBinary(); + return serialized_java; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case SHELL: + if (field.type == SHELL_FIELD_DESC.type) { + ShellComponent shell; + shell = new ShellComponent(); + shell.read(iprot); + return shell; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case JAVA_OBJECT: + if (field.type == JAVA_OBJECT_FIELD_DESC.type) { + JavaObject java_object; + java_object = new JavaObject(); + java_object.read(iprot); + return java_object; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + } + + @Override + protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case SERIALIZED_JAVA: + ByteBuffer serialized_java = (ByteBuffer)value_; + oprot.writeBinary(serialized_java); + return; + case SHELL: + ShellComponent shell = (ShellComponent)value_; + shell.write(oprot); + return; + case JAVA_OBJECT: + JavaObject java_object = (JavaObject)value_; + java_object.write(oprot); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(fieldID); + if (setField != null) { + switch (setField) { + case SERIALIZED_JAVA: + ByteBuffer serialized_java; + serialized_java = iprot.readBinary(); + return serialized_java; + case SHELL: + ShellComponent shell; + shell = new ShellComponent(); + shell.read(iprot); + return shell; + case JAVA_OBJECT: + JavaObject java_object; + java_object = new JavaObject(); + java_object.read(iprot); + return java_object; + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + throw new TProtocolException("Couldn't find a field with field id " + fieldID); + } + } + + @Override + protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case SERIALIZED_JAVA: + ByteBuffer serialized_java = (ByteBuffer)value_; + oprot.writeBinary(serialized_java); + return; + case SHELL: + ShellComponent shell = (ShellComponent)value_; + shell.write(oprot); + return; + case JAVA_OBJECT: + JavaObject java_object = (JavaObject)value_; + java_object.write(oprot); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { + switch (setField) { + case SERIALIZED_JAVA: + return SERIALIZED_JAVA_FIELD_DESC; + case SHELL: + return SHELL_FIELD_DESC; + case JAVA_OBJECT: + return JAVA_OBJECT_FIELD_DESC; + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected org.apache.thrift.protocol.TStruct getStructDesc() { + return STRUCT_DESC; + } + + @Override + protected _Fields enumForId(short id) { + return _Fields.findByThriftIdOrThrow(id); + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + + public byte[] get_serialized_java() { + set_serialized_java(org.apache.thrift.TBaseHelper.rightSize(buffer_for_serialized_java())); + ByteBuffer b = buffer_for_serialized_java(); + return b == null ? null : b.array(); + } + + public ByteBuffer buffer_for_serialized_java() { + if (getSetField() == _Fields.SERIALIZED_JAVA) { + return org.apache.thrift.TBaseHelper.copyBinary((ByteBuffer)getFieldValue()); + } else { + throw new RuntimeException("Cannot get field 'serialized_java' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void set_serialized_java(byte[] value) { + set_serialized_java(ByteBuffer.wrap(Arrays.copyOf(value, value.length))); + } + + public void set_serialized_java(ByteBuffer value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.SERIALIZED_JAVA; + value_ = value; + } + + public ShellComponent get_shell() { + if (getSetField() == _Fields.SHELL) { + return (ShellComponent)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'shell' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void set_shell(ShellComponent value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.SHELL; + value_ = value; + } + + public JavaObject get_java_object() { + if (getSetField() == _Fields.JAVA_OBJECT) { + return (JavaObject)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'java_object' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void set_java_object(JavaObject value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.JAVA_OBJECT; + value_ = value; + } + + public boolean is_set_serialized_java() { + return setField_ == _Fields.SERIALIZED_JAVA; + } + + + public boolean is_set_shell() { + return setField_ == _Fields.SHELL; + } + + + public boolean is_set_java_object() { + return setField_ == _Fields.JAVA_OBJECT; + } + + + public boolean equals(Object other) { + if (other instanceof ComponentObject) { + return equals((ComponentObject)other); + } else { + return false; + } + } + + public boolean equals(ComponentObject other) { + return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); + } + + @Override + public int compareTo(ComponentObject other) { + int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); + if (lastComparison == 0) { + return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); + } + return lastComparison; + } + + + @Override + public int hashCode() { + List list = new ArrayList(); + list.add(this.getClass().getName()); + org.apache.thrift.TFieldIdEnum setField = getSetField(); + if (setField != null) { + list.add(setField.getThriftFieldId()); + Object value = getFieldValue(); + if (value instanceof org.apache.thrift.TEnum) { + list.add(((org.apache.thrift.TEnum)getFieldValue()).getValue()); + } else { + list.add(value); + } + } + return list.hashCode(); + } + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + +} diff --git a/jstorm-core/src/main/java/backtype/storm/generated/ComponentSummary.java b/jstorm-core/src/main/java/backtype/storm/generated/ComponentSummary.java new file mode 100644 index 000000000..8161f72e4 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/ComponentSummary.java @@ -0,0 +1,903 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class ComponentSummary implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ComponentSummary"); + + private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField PARALLEL_FIELD_DESC = new org.apache.thrift.protocol.TField("parallel", org.apache.thrift.protocol.TType.I32, (short)2); + private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField TASK_IDS_FIELD_DESC = new org.apache.thrift.protocol.TField("task_ids", org.apache.thrift.protocol.TType.LIST, (short)4); + private static final org.apache.thrift.protocol.TField ERRORS_FIELD_DESC = new org.apache.thrift.protocol.TField("errors", org.apache.thrift.protocol.TType.LIST, (short)5); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ComponentSummaryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ComponentSummaryTupleSchemeFactory()); + } + + private String name; // required + private int parallel; // required + private String type; // required + private List task_ids; // required + private List errors; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NAME((short)1, "name"), + PARALLEL((short)2, "parallel"), + TYPE((short)3, "type"), + TASK_IDS((short)4, "task_ids"), + ERRORS((short)5, "errors"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NAME + return NAME; + case 2: // PARALLEL + return PARALLEL; + case 3: // TYPE + return TYPE; + case 4: // TASK_IDS + return TASK_IDS; + case 5: // ERRORS + return ERRORS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __PARALLEL_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.ERRORS}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.PARALLEL, new org.apache.thrift.meta_data.FieldMetaData("parallel", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TASK_IDS, new org.apache.thrift.meta_data.FieldMetaData("task_ids", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)))); + tmpMap.put(_Fields.ERRORS, new org.apache.thrift.meta_data.FieldMetaData("errors", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ErrorInfo.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ComponentSummary.class, metaDataMap); + } + + public ComponentSummary() { + } + + public ComponentSummary( + String name, + int parallel, + String type, + List task_ids) + { + this(); + this.name = name; + this.parallel = parallel; + set_parallel_isSet(true); + this.type = type; + this.task_ids = task_ids; + } + + /** + * Performs a deep copy on other. + */ + public ComponentSummary(ComponentSummary other) { + __isset_bitfield = other.__isset_bitfield; + if (other.is_set_name()) { + this.name = other.name; + } + this.parallel = other.parallel; + if (other.is_set_type()) { + this.type = other.type; + } + if (other.is_set_task_ids()) { + List __this__task_ids = new ArrayList(other.task_ids); + this.task_ids = __this__task_ids; + } + if (other.is_set_errors()) { + List __this__errors = new ArrayList(other.errors.size()); + for (ErrorInfo other_element : other.errors) { + __this__errors.add(new ErrorInfo(other_element)); + } + this.errors = __this__errors; + } + } + + public ComponentSummary deepCopy() { + return new ComponentSummary(this); + } + + @Override + public void clear() { + this.name = null; + set_parallel_isSet(false); + this.parallel = 0; + this.type = null; + this.task_ids = null; + this.errors = null; + } + + public String get_name() { + return this.name; + } + + public void set_name(String name) { + this.name = name; + } + + public void unset_name() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean is_set_name() { + return this.name != null; + } + + public void set_name_isSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public int get_parallel() { + return this.parallel; + } + + public void set_parallel(int parallel) { + this.parallel = parallel; + set_parallel_isSet(true); + } + + public void unset_parallel() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PARALLEL_ISSET_ID); + } + + /** Returns true if field parallel is set (has been assigned a value) and false otherwise */ + public boolean is_set_parallel() { + return EncodingUtils.testBit(__isset_bitfield, __PARALLEL_ISSET_ID); + } + + public void set_parallel_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PARALLEL_ISSET_ID, value); + } + + public String get_type() { + return this.type; + } + + public void set_type(String type) { + this.type = type; + } + + public void unset_type() { + this.type = null; + } + + /** Returns true if field type is set (has been assigned a value) and false otherwise */ + public boolean is_set_type() { + return this.type != null; + } + + public void set_type_isSet(boolean value) { + if (!value) { + this.type = null; + } + } + + public int get_task_ids_size() { + return (this.task_ids == null) ? 0 : this.task_ids.size(); + } + + public java.util.Iterator get_task_ids_iterator() { + return (this.task_ids == null) ? null : this.task_ids.iterator(); + } + + public void add_to_task_ids(int elem) { + if (this.task_ids == null) { + this.task_ids = new ArrayList(); + } + this.task_ids.add(elem); + } + + public List get_task_ids() { + return this.task_ids; + } + + public void set_task_ids(List task_ids) { + this.task_ids = task_ids; + } + + public void unset_task_ids() { + this.task_ids = null; + } + + /** Returns true if field task_ids is set (has been assigned a value) and false otherwise */ + public boolean is_set_task_ids() { + return this.task_ids != null; + } + + public void set_task_ids_isSet(boolean value) { + if (!value) { + this.task_ids = null; + } + } + + public int get_errors_size() { + return (this.errors == null) ? 0 : this.errors.size(); + } + + public java.util.Iterator get_errors_iterator() { + return (this.errors == null) ? null : this.errors.iterator(); + } + + public void add_to_errors(ErrorInfo elem) { + if (this.errors == null) { + this.errors = new ArrayList(); + } + this.errors.add(elem); + } + + public List get_errors() { + return this.errors; + } + + public void set_errors(List errors) { + this.errors = errors; + } + + public void unset_errors() { + this.errors = null; + } + + /** Returns true if field errors is set (has been assigned a value) and false otherwise */ + public boolean is_set_errors() { + return this.errors != null; + } + + public void set_errors_isSet(boolean value) { + if (!value) { + this.errors = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME: + if (value == null) { + unset_name(); + } else { + set_name((String)value); + } + break; + + case PARALLEL: + if (value == null) { + unset_parallel(); + } else { + set_parallel((Integer)value); + } + break; + + case TYPE: + if (value == null) { + unset_type(); + } else { + set_type((String)value); + } + break; + + case TASK_IDS: + if (value == null) { + unset_task_ids(); + } else { + set_task_ids((List)value); + } + break; + + case ERRORS: + if (value == null) { + unset_errors(); + } else { + set_errors((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME: + return get_name(); + + case PARALLEL: + return Integer.valueOf(get_parallel()); + + case TYPE: + return get_type(); + + case TASK_IDS: + return get_task_ids(); + + case ERRORS: + return get_errors(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NAME: + return is_set_name(); + case PARALLEL: + return is_set_parallel(); + case TYPE: + return is_set_type(); + case TASK_IDS: + return is_set_task_ids(); + case ERRORS: + return is_set_errors(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ComponentSummary) + return this.equals((ComponentSummary)that); + return false; + } + + public boolean equals(ComponentSummary that) { + if (that == null) + return false; + + boolean this_present_name = true && this.is_set_name(); + boolean that_present_name = true && that.is_set_name(); + if (this_present_name || that_present_name) { + if (!(this_present_name && that_present_name)) + return false; + if (!this.name.equals(that.name)) + return false; + } + + boolean this_present_parallel = true; + boolean that_present_parallel = true; + if (this_present_parallel || that_present_parallel) { + if (!(this_present_parallel && that_present_parallel)) + return false; + if (this.parallel != that.parallel) + return false; + } + + boolean this_present_type = true && this.is_set_type(); + boolean that_present_type = true && that.is_set_type(); + if (this_present_type || that_present_type) { + if (!(this_present_type && that_present_type)) + return false; + if (!this.type.equals(that.type)) + return false; + } + + boolean this_present_task_ids = true && this.is_set_task_ids(); + boolean that_present_task_ids = true && that.is_set_task_ids(); + if (this_present_task_ids || that_present_task_ids) { + if (!(this_present_task_ids && that_present_task_ids)) + return false; + if (!this.task_ids.equals(that.task_ids)) + return false; + } + + boolean this_present_errors = true && this.is_set_errors(); + boolean that_present_errors = true && that.is_set_errors(); + if (this_present_errors || that_present_errors) { + if (!(this_present_errors && that_present_errors)) + return false; + if (!this.errors.equals(that.errors)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_name = true && (is_set_name()); + list.add(present_name); + if (present_name) + list.add(name); + + boolean present_parallel = true; + list.add(present_parallel); + if (present_parallel) + list.add(parallel); + + boolean present_type = true && (is_set_type()); + list.add(present_type); + if (present_type) + list.add(type); + + boolean present_task_ids = true && (is_set_task_ids()); + list.add(present_task_ids); + if (present_task_ids) + list.add(task_ids); + + boolean present_errors = true && (is_set_errors()); + list.add(present_errors); + if (present_errors) + list.add(errors); + + return list.hashCode(); + } + + @Override + public int compareTo(ComponentSummary other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_parallel()).compareTo(other.is_set_parallel()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_parallel()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.parallel, other.parallel); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_type()).compareTo(other.is_set_type()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_type()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_task_ids()).compareTo(other.is_set_task_ids()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_task_ids()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.task_ids, other.task_ids); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_errors()).compareTo(other.is_set_errors()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_errors()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errors, other.errors); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ComponentSummary("); + boolean first = true; + + sb.append("name:"); + if (this.name == null) { + sb.append("null"); + } else { + sb.append(this.name); + } + first = false; + if (!first) sb.append(", "); + sb.append("parallel:"); + sb.append(this.parallel); + first = false; + if (!first) sb.append(", "); + sb.append("type:"); + if (this.type == null) { + sb.append("null"); + } else { + sb.append(this.type); + } + first = false; + if (!first) sb.append(", "); + sb.append("task_ids:"); + if (this.task_ids == null) { + sb.append("null"); + } else { + sb.append(this.task_ids); + } + first = false; + if (is_set_errors()) { + if (!first) sb.append(", "); + sb.append("errors:"); + if (this.errors == null) { + sb.append("null"); + } else { + sb.append(this.errors); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_name()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'name' is unset! Struct:" + toString()); + } + + if (!is_set_parallel()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'parallel' is unset! Struct:" + toString()); + } + + if (!is_set_type()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'type' is unset! Struct:" + toString()); + } + + if (!is_set_task_ids()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'task_ids' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ComponentSummaryStandardSchemeFactory implements SchemeFactory { + public ComponentSummaryStandardScheme getScheme() { + return new ComponentSummaryStandardScheme(); + } + } + + private static class ComponentSummaryStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ComponentSummary struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // PARALLEL + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.parallel = iprot.readI32(); + struct.set_parallel_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.type = iprot.readString(); + struct.set_type_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // TASK_IDS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list214 = iprot.readListBegin(); + struct.task_ids = new ArrayList(_list214.size); + int _elem215; + for (int _i216 = 0; _i216 < _list214.size; ++_i216) + { + _elem215 = iprot.readI32(); + struct.task_ids.add(_elem215); + } + iprot.readListEnd(); + } + struct.set_task_ids_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // ERRORS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list217 = iprot.readListBegin(); + struct.errors = new ArrayList(_list217.size); + ErrorInfo _elem218; + for (int _i219 = 0; _i219 < _list217.size; ++_i219) + { + _elem218 = new ErrorInfo(); + _elem218.read(iprot); + struct.errors.add(_elem218); + } + iprot.readListEnd(); + } + struct.set_errors_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ComponentSummary struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.name != null) { + oprot.writeFieldBegin(NAME_FIELD_DESC); + oprot.writeString(struct.name); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(PARALLEL_FIELD_DESC); + oprot.writeI32(struct.parallel); + oprot.writeFieldEnd(); + if (struct.type != null) { + oprot.writeFieldBegin(TYPE_FIELD_DESC); + oprot.writeString(struct.type); + oprot.writeFieldEnd(); + } + if (struct.task_ids != null) { + oprot.writeFieldBegin(TASK_IDS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.task_ids.size())); + for (int _iter220 : struct.task_ids) + { + oprot.writeI32(_iter220); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.errors != null) { + if (struct.is_set_errors()) { + oprot.writeFieldBegin(ERRORS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.errors.size())); + for (ErrorInfo _iter221 : struct.errors) + { + _iter221.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ComponentSummaryTupleSchemeFactory implements SchemeFactory { + public ComponentSummaryTupleScheme getScheme() { + return new ComponentSummaryTupleScheme(); + } + } + + private static class ComponentSummaryTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ComponentSummary struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.name); + oprot.writeI32(struct.parallel); + oprot.writeString(struct.type); + { + oprot.writeI32(struct.task_ids.size()); + for (int _iter222 : struct.task_ids) + { + oprot.writeI32(_iter222); + } + } + BitSet optionals = new BitSet(); + if (struct.is_set_errors()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_errors()) { + { + oprot.writeI32(struct.errors.size()); + for (ErrorInfo _iter223 : struct.errors) + { + _iter223.write(oprot); + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ComponentSummary struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.name = iprot.readString(); + struct.set_name_isSet(true); + struct.parallel = iprot.readI32(); + struct.set_parallel_isSet(true); + struct.type = iprot.readString(); + struct.set_type_isSet(true); + { + org.apache.thrift.protocol.TList _list224 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.task_ids = new ArrayList(_list224.size); + int _elem225; + for (int _i226 = 0; _i226 < _list224.size; ++_i226) + { + _elem225 = iprot.readI32(); + struct.task_ids.add(_elem225); + } + } + struct.set_task_ids_isSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + { + org.apache.thrift.protocol.TList _list227 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.errors = new ArrayList(_list227.size); + ErrorInfo _elem228; + for (int _i229 = 0; _i229 < _list227.size; ++_i229) + { + _elem228 = new ErrorInfo(); + _elem228.read(iprot); + struct.errors.add(_elem228); + } + } + struct.set_errors_isSet(true); + } + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/Credentials.java b/jstorm-core/src/main/java/backtype/storm/generated/Credentials.java new file mode 100644 index 000000000..e2ca92d4f --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/Credentials.java @@ -0,0 +1,441 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class Credentials implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Credentials"); + + private static final org.apache.thrift.protocol.TField CREDS_FIELD_DESC = new org.apache.thrift.protocol.TField("creds", org.apache.thrift.protocol.TType.MAP, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CredentialsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CredentialsTupleSchemeFactory()); + } + + private Map creds; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + CREDS((short)1, "creds"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // CREDS + return CREDS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.CREDS, new org.apache.thrift.meta_data.FieldMetaData("creds", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Credentials.class, metaDataMap); + } + + public Credentials() { + } + + public Credentials( + Map creds) + { + this(); + this.creds = creds; + } + + /** + * Performs a deep copy on other. + */ + public Credentials(Credentials other) { + if (other.is_set_creds()) { + Map __this__creds = new HashMap(other.creds); + this.creds = __this__creds; + } + } + + public Credentials deepCopy() { + return new Credentials(this); + } + + @Override + public void clear() { + this.creds = null; + } + + public int get_creds_size() { + return (this.creds == null) ? 0 : this.creds.size(); + } + + public void put_to_creds(String key, String val) { + if (this.creds == null) { + this.creds = new HashMap(); + } + this.creds.put(key, val); + } + + public Map get_creds() { + return this.creds; + } + + public void set_creds(Map creds) { + this.creds = creds; + } + + public void unset_creds() { + this.creds = null; + } + + /** Returns true if field creds is set (has been assigned a value) and false otherwise */ + public boolean is_set_creds() { + return this.creds != null; + } + + public void set_creds_isSet(boolean value) { + if (!value) { + this.creds = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case CREDS: + if (value == null) { + unset_creds(); + } else { + set_creds((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case CREDS: + return get_creds(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case CREDS: + return is_set_creds(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof Credentials) + return this.equals((Credentials)that); + return false; + } + + public boolean equals(Credentials that) { + if (that == null) + return false; + + boolean this_present_creds = true && this.is_set_creds(); + boolean that_present_creds = true && that.is_set_creds(); + if (this_present_creds || that_present_creds) { + if (!(this_present_creds && that_present_creds)) + return false; + if (!this.creds.equals(that.creds)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_creds = true && (is_set_creds()); + list.add(present_creds); + if (present_creds) + list.add(creds); + + return list.hashCode(); + } + + @Override + public int compareTo(Credentials other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_creds()).compareTo(other.is_set_creds()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_creds()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.creds, other.creds); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("Credentials("); + boolean first = true; + + sb.append("creds:"); + if (this.creds == null) { + sb.append("null"); + } else { + sb.append(this.creds); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_creds()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'creds' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CredentialsStandardSchemeFactory implements SchemeFactory { + public CredentialsStandardScheme getScheme() { + return new CredentialsStandardScheme(); + } + } + + private static class CredentialsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, Credentials struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // CREDS + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map254 = iprot.readMapBegin(); + struct.creds = new HashMap(2*_map254.size); + String _key255; + String _val256; + for (int _i257 = 0; _i257 < _map254.size; ++_i257) + { + _key255 = iprot.readString(); + _val256 = iprot.readString(); + struct.creds.put(_key255, _val256); + } + iprot.readMapEnd(); + } + struct.set_creds_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, Credentials struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.creds != null) { + oprot.writeFieldBegin(CREDS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.creds.size())); + for (Map.Entry _iter258 : struct.creds.entrySet()) + { + oprot.writeString(_iter258.getKey()); + oprot.writeString(_iter258.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CredentialsTupleSchemeFactory implements SchemeFactory { + public CredentialsTupleScheme getScheme() { + return new CredentialsTupleScheme(); + } + } + + private static class CredentialsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, Credentials struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.creds.size()); + for (Map.Entry _iter259 : struct.creds.entrySet()) + { + oprot.writeString(_iter259.getKey()); + oprot.writeString(_iter259.getValue()); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, Credentials struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TMap _map260 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.creds = new HashMap(2*_map260.size); + String _key261; + String _val262; + for (int _i263 = 0; _i263 < _map260.size; ++_i263) + { + _key261 = iprot.readString(); + _val262 = iprot.readString(); + struct.creds.put(_key261, _val262); + } + } + struct.set_creds_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/DRPCExecutionException.java b/jstorm-core/src/main/java/backtype/storm/generated/DRPCExecutionException.java new file mode 100644 index 000000000..3d8502f58 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/DRPCExecutionException.java @@ -0,0 +1,389 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class DRPCExecutionException extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DRPCExecutionException"); + + private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new DRPCExecutionExceptionStandardSchemeFactory()); + schemes.put(TupleScheme.class, new DRPCExecutionExceptionTupleSchemeFactory()); + } + + private String msg; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + MSG((short)1, "msg"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // MSG + return MSG; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(DRPCExecutionException.class, metaDataMap); + } + + public DRPCExecutionException() { + } + + public DRPCExecutionException( + String msg) + { + this(); + this.msg = msg; + } + + /** + * Performs a deep copy on other. + */ + public DRPCExecutionException(DRPCExecutionException other) { + if (other.is_set_msg()) { + this.msg = other.msg; + } + } + + public DRPCExecutionException deepCopy() { + return new DRPCExecutionException(this); + } + + @Override + public void clear() { + this.msg = null; + } + + public String get_msg() { + return this.msg; + } + + public void set_msg(String msg) { + this.msg = msg; + } + + public void unset_msg() { + this.msg = null; + } + + /** Returns true if field msg is set (has been assigned a value) and false otherwise */ + public boolean is_set_msg() { + return this.msg != null; + } + + public void set_msg_isSet(boolean value) { + if (!value) { + this.msg = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case MSG: + if (value == null) { + unset_msg(); + } else { + set_msg((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case MSG: + return get_msg(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case MSG: + return is_set_msg(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof DRPCExecutionException) + return this.equals((DRPCExecutionException)that); + return false; + } + + public boolean equals(DRPCExecutionException that) { + if (that == null) + return false; + + boolean this_present_msg = true && this.is_set_msg(); + boolean that_present_msg = true && that.is_set_msg(); + if (this_present_msg || that_present_msg) { + if (!(this_present_msg && that_present_msg)) + return false; + if (!this.msg.equals(that.msg)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_msg = true && (is_set_msg()); + list.add(present_msg); + if (present_msg) + list.add(msg); + + return list.hashCode(); + } + + @Override + public int compareTo(DRPCExecutionException other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_msg()).compareTo(other.is_set_msg()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_msg()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, other.msg); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("DRPCExecutionException("); + boolean first = true; + + sb.append("msg:"); + if (this.msg == null) { + sb.append("null"); + } else { + sb.append(this.msg); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_msg()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class DRPCExecutionExceptionStandardSchemeFactory implements SchemeFactory { + public DRPCExecutionExceptionStandardScheme getScheme() { + return new DRPCExecutionExceptionStandardScheme(); + } + } + + private static class DRPCExecutionExceptionStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, DRPCExecutionException struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // MSG + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.msg = iprot.readString(); + struct.set_msg_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, DRPCExecutionException struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.msg != null) { + oprot.writeFieldBegin(MSG_FIELD_DESC); + oprot.writeString(struct.msg); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class DRPCExecutionExceptionTupleSchemeFactory implements SchemeFactory { + public DRPCExecutionExceptionTupleScheme getScheme() { + return new DRPCExecutionExceptionTupleScheme(); + } + } + + private static class DRPCExecutionExceptionTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, DRPCExecutionException struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.msg); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, DRPCExecutionException struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.msg = iprot.readString(); + struct.set_msg_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/DRPCRequest.java b/jstorm-core/src/main/java/backtype/storm/generated/DRPCRequest.java new file mode 100644 index 000000000..00448f510 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/DRPCRequest.java @@ -0,0 +1,490 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class DRPCRequest implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("DRPCRequest"); + + private static final org.apache.thrift.protocol.TField FUNC_ARGS_FIELD_DESC = new org.apache.thrift.protocol.TField("func_args", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField REQUEST_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("request_id", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new DRPCRequestStandardSchemeFactory()); + schemes.put(TupleScheme.class, new DRPCRequestTupleSchemeFactory()); + } + + private String func_args; // required + private String request_id; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + FUNC_ARGS((short)1, "func_args"), + REQUEST_ID((short)2, "request_id"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // FUNC_ARGS + return FUNC_ARGS; + case 2: // REQUEST_ID + return REQUEST_ID; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.FUNC_ARGS, new org.apache.thrift.meta_data.FieldMetaData("func_args", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.REQUEST_ID, new org.apache.thrift.meta_data.FieldMetaData("request_id", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(DRPCRequest.class, metaDataMap); + } + + public DRPCRequest() { + } + + public DRPCRequest( + String func_args, + String request_id) + { + this(); + this.func_args = func_args; + this.request_id = request_id; + } + + /** + * Performs a deep copy on other. + */ + public DRPCRequest(DRPCRequest other) { + if (other.is_set_func_args()) { + this.func_args = other.func_args; + } + if (other.is_set_request_id()) { + this.request_id = other.request_id; + } + } + + public DRPCRequest deepCopy() { + return new DRPCRequest(this); + } + + @Override + public void clear() { + this.func_args = null; + this.request_id = null; + } + + public String get_func_args() { + return this.func_args; + } + + public void set_func_args(String func_args) { + this.func_args = func_args; + } + + public void unset_func_args() { + this.func_args = null; + } + + /** Returns true if field func_args is set (has been assigned a value) and false otherwise */ + public boolean is_set_func_args() { + return this.func_args != null; + } + + public void set_func_args_isSet(boolean value) { + if (!value) { + this.func_args = null; + } + } + + public String get_request_id() { + return this.request_id; + } + + public void set_request_id(String request_id) { + this.request_id = request_id; + } + + public void unset_request_id() { + this.request_id = null; + } + + /** Returns true if field request_id is set (has been assigned a value) and false otherwise */ + public boolean is_set_request_id() { + return this.request_id != null; + } + + public void set_request_id_isSet(boolean value) { + if (!value) { + this.request_id = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case FUNC_ARGS: + if (value == null) { + unset_func_args(); + } else { + set_func_args((String)value); + } + break; + + case REQUEST_ID: + if (value == null) { + unset_request_id(); + } else { + set_request_id((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case FUNC_ARGS: + return get_func_args(); + + case REQUEST_ID: + return get_request_id(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case FUNC_ARGS: + return is_set_func_args(); + case REQUEST_ID: + return is_set_request_id(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof DRPCRequest) + return this.equals((DRPCRequest)that); + return false; + } + + public boolean equals(DRPCRequest that) { + if (that == null) + return false; + + boolean this_present_func_args = true && this.is_set_func_args(); + boolean that_present_func_args = true && that.is_set_func_args(); + if (this_present_func_args || that_present_func_args) { + if (!(this_present_func_args && that_present_func_args)) + return false; + if (!this.func_args.equals(that.func_args)) + return false; + } + + boolean this_present_request_id = true && this.is_set_request_id(); + boolean that_present_request_id = true && that.is_set_request_id(); + if (this_present_request_id || that_present_request_id) { + if (!(this_present_request_id && that_present_request_id)) + return false; + if (!this.request_id.equals(that.request_id)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_func_args = true && (is_set_func_args()); + list.add(present_func_args); + if (present_func_args) + list.add(func_args); + + boolean present_request_id = true && (is_set_request_id()); + list.add(present_request_id); + if (present_request_id) + list.add(request_id); + + return list.hashCode(); + } + + @Override + public int compareTo(DRPCRequest other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_func_args()).compareTo(other.is_set_func_args()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_func_args()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.func_args, other.func_args); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_request_id()).compareTo(other.is_set_request_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_request_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.request_id, other.request_id); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("DRPCRequest("); + boolean first = true; + + sb.append("func_args:"); + if (this.func_args == null) { + sb.append("null"); + } else { + sb.append(this.func_args); + } + first = false; + if (!first) sb.append(", "); + sb.append("request_id:"); + if (this.request_id == null) { + sb.append("null"); + } else { + sb.append(this.request_id); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_func_args()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'func_args' is unset! Struct:" + toString()); + } + + if (!is_set_request_id()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'request_id' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class DRPCRequestStandardSchemeFactory implements SchemeFactory { + public DRPCRequestStandardScheme getScheme() { + return new DRPCRequestStandardScheme(); + } + } + + private static class DRPCRequestStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, DRPCRequest struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // FUNC_ARGS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.func_args = iprot.readString(); + struct.set_func_args_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // REQUEST_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.request_id = iprot.readString(); + struct.set_request_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, DRPCRequest struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.func_args != null) { + oprot.writeFieldBegin(FUNC_ARGS_FIELD_DESC); + oprot.writeString(struct.func_args); + oprot.writeFieldEnd(); + } + if (struct.request_id != null) { + oprot.writeFieldBegin(REQUEST_ID_FIELD_DESC); + oprot.writeString(struct.request_id); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class DRPCRequestTupleSchemeFactory implements SchemeFactory { + public DRPCRequestTupleScheme getScheme() { + return new DRPCRequestTupleScheme(); + } + } + + private static class DRPCRequestTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, DRPCRequest struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.func_args); + oprot.writeString(struct.request_id); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, DRPCRequest struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.func_args = iprot.readString(); + struct.set_func_args_isSet(true); + struct.request_id = iprot.readString(); + struct.set_request_id_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/DistributedRPC.java b/jstorm-core/src/main/java/backtype/storm/generated/DistributedRPC.java new file mode 100644 index 000000000..ff3c11264 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/DistributedRPC.java @@ -0,0 +1,1195 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class DistributedRPC { + + public interface Iface { + + public String execute(String functionName, String funcArgs) throws DRPCExecutionException, org.apache.thrift.TException; + + } + + public interface AsyncIface { + + public void execute(String functionName, String funcArgs, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + } + + public static class Client extends org.apache.thrift.TServiceClient implements Iface { + public static class Factory implements org.apache.thrift.TServiceClientFactory { + public Factory() {} + public Client getClient(org.apache.thrift.protocol.TProtocol prot) { + return new Client(prot); + } + public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) { + return new Client(iprot, oprot); + } + } + + public Client(org.apache.thrift.protocol.TProtocol prot) + { + super(prot, prot); + } + + public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) { + super(iprot, oprot); + } + + public String execute(String functionName, String funcArgs) throws DRPCExecutionException, org.apache.thrift.TException + { + send_execute(functionName, funcArgs); + return recv_execute(); + } + + public void send_execute(String functionName, String funcArgs) throws org.apache.thrift.TException + { + execute_args args = new execute_args(); + args.set_functionName(functionName); + args.set_funcArgs(funcArgs); + sendBase("execute", args); + } + + public String recv_execute() throws DRPCExecutionException, org.apache.thrift.TException + { + execute_result result = new execute_result(); + receiveBase(result, "execute"); + if (result.is_set_success()) { + return result.success; + } + if (result.e != null) { + throw result.e; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "execute failed: unknown result"); + } + + } + public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface { + public static class Factory implements org.apache.thrift.async.TAsyncClientFactory { + private org.apache.thrift.async.TAsyncClientManager clientManager; + private org.apache.thrift.protocol.TProtocolFactory protocolFactory; + public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) { + this.clientManager = clientManager; + this.protocolFactory = protocolFactory; + } + public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) { + return new AsyncClient(protocolFactory, clientManager, transport); + } + } + + public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) { + super(protocolFactory, clientManager, transport); + } + + public void execute(String functionName, String funcArgs, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + execute_call method_call = new execute_call(functionName, funcArgs, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class execute_call extends org.apache.thrift.async.TAsyncMethodCall { + private String functionName; + private String funcArgs; + public execute_call(String functionName, String funcArgs, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.functionName = functionName; + this.funcArgs = funcArgs; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("execute", org.apache.thrift.protocol.TMessageType.CALL, 0)); + execute_args args = new execute_args(); + args.set_functionName(functionName); + args.set_funcArgs(funcArgs); + args.write(prot); + prot.writeMessageEnd(); + } + + public String getResult() throws DRPCExecutionException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_execute(); + } + } + + } + + public static class Processor extends org.apache.thrift.TBaseProcessor implements org.apache.thrift.TProcessor { + private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName()); + public Processor(I iface) { + super(iface, getProcessMap(new HashMap>())); + } + + protected Processor(I iface, Map> processMap) { + super(iface, getProcessMap(processMap)); + } + + private static Map> getProcessMap(Map> processMap) { + processMap.put("execute", new execute()); + return processMap; + } + + public static class execute extends org.apache.thrift.ProcessFunction { + public execute() { + super("execute"); + } + + public execute_args getEmptyArgsInstance() { + return new execute_args(); + } + + protected boolean isOneway() { + return false; + } + + public execute_result getResult(I iface, execute_args args) throws org.apache.thrift.TException { + execute_result result = new execute_result(); + try { + result.success = iface.execute(args.functionName, args.funcArgs); + } catch (DRPCExecutionException e) { + result.e = e; + } + return result; + } + } + + } + + public static class AsyncProcessor extends org.apache.thrift.TBaseAsyncProcessor { + private static final Logger LOGGER = LoggerFactory.getLogger(AsyncProcessor.class.getName()); + public AsyncProcessor(I iface) { + super(iface, getProcessMap(new HashMap>())); + } + + protected AsyncProcessor(I iface, Map> processMap) { + super(iface, getProcessMap(processMap)); + } + + private static Map> getProcessMap(Map> processMap) { + processMap.put("execute", new execute()); + return processMap; + } + + public static class execute extends org.apache.thrift.AsyncProcessFunction { + public execute() { + super("execute"); + } + + public execute_args getEmptyArgsInstance() { + return new execute_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(String o) { + execute_result result = new execute_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + execute_result result = new execute_result(); + if (e instanceof DRPCExecutionException) { + result.e = (DRPCExecutionException) e; + result.set_e_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, execute_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.execute(args.functionName, args.funcArgs,resultHandler); + } + } + + } + + public static class execute_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("execute_args"); + + private static final org.apache.thrift.protocol.TField FUNCTION_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("functionName", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField FUNC_ARGS_FIELD_DESC = new org.apache.thrift.protocol.TField("funcArgs", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new execute_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new execute_argsTupleSchemeFactory()); + } + + private String functionName; // required + private String funcArgs; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + FUNCTION_NAME((short)1, "functionName"), + FUNC_ARGS((short)2, "funcArgs"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // FUNCTION_NAME + return FUNCTION_NAME; + case 2: // FUNC_ARGS + return FUNC_ARGS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.FUNCTION_NAME, new org.apache.thrift.meta_data.FieldMetaData("functionName", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.FUNC_ARGS, new org.apache.thrift.meta_data.FieldMetaData("funcArgs", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(execute_args.class, metaDataMap); + } + + public execute_args() { + } + + public execute_args( + String functionName, + String funcArgs) + { + this(); + this.functionName = functionName; + this.funcArgs = funcArgs; + } + + /** + * Performs a deep copy on other. + */ + public execute_args(execute_args other) { + if (other.is_set_functionName()) { + this.functionName = other.functionName; + } + if (other.is_set_funcArgs()) { + this.funcArgs = other.funcArgs; + } + } + + public execute_args deepCopy() { + return new execute_args(this); + } + + @Override + public void clear() { + this.functionName = null; + this.funcArgs = null; + } + + public String get_functionName() { + return this.functionName; + } + + public void set_functionName(String functionName) { + this.functionName = functionName; + } + + public void unset_functionName() { + this.functionName = null; + } + + /** Returns true if field functionName is set (has been assigned a value) and false otherwise */ + public boolean is_set_functionName() { + return this.functionName != null; + } + + public void set_functionName_isSet(boolean value) { + if (!value) { + this.functionName = null; + } + } + + public String get_funcArgs() { + return this.funcArgs; + } + + public void set_funcArgs(String funcArgs) { + this.funcArgs = funcArgs; + } + + public void unset_funcArgs() { + this.funcArgs = null; + } + + /** Returns true if field funcArgs is set (has been assigned a value) and false otherwise */ + public boolean is_set_funcArgs() { + return this.funcArgs != null; + } + + public void set_funcArgs_isSet(boolean value) { + if (!value) { + this.funcArgs = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case FUNCTION_NAME: + if (value == null) { + unset_functionName(); + } else { + set_functionName((String)value); + } + break; + + case FUNC_ARGS: + if (value == null) { + unset_funcArgs(); + } else { + set_funcArgs((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case FUNCTION_NAME: + return get_functionName(); + + case FUNC_ARGS: + return get_funcArgs(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case FUNCTION_NAME: + return is_set_functionName(); + case FUNC_ARGS: + return is_set_funcArgs(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof execute_args) + return this.equals((execute_args)that); + return false; + } + + public boolean equals(execute_args that) { + if (that == null) + return false; + + boolean this_present_functionName = true && this.is_set_functionName(); + boolean that_present_functionName = true && that.is_set_functionName(); + if (this_present_functionName || that_present_functionName) { + if (!(this_present_functionName && that_present_functionName)) + return false; + if (!this.functionName.equals(that.functionName)) + return false; + } + + boolean this_present_funcArgs = true && this.is_set_funcArgs(); + boolean that_present_funcArgs = true && that.is_set_funcArgs(); + if (this_present_funcArgs || that_present_funcArgs) { + if (!(this_present_funcArgs && that_present_funcArgs)) + return false; + if (!this.funcArgs.equals(that.funcArgs)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_functionName = true && (is_set_functionName()); + list.add(present_functionName); + if (present_functionName) + list.add(functionName); + + boolean present_funcArgs = true && (is_set_funcArgs()); + list.add(present_funcArgs); + if (present_funcArgs) + list.add(funcArgs); + + return list.hashCode(); + } + + @Override + public int compareTo(execute_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_functionName()).compareTo(other.is_set_functionName()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_functionName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.functionName, other.functionName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_funcArgs()).compareTo(other.is_set_funcArgs()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_funcArgs()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.funcArgs, other.funcArgs); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("execute_args("); + boolean first = true; + + sb.append("functionName:"); + if (this.functionName == null) { + sb.append("null"); + } else { + sb.append(this.functionName); + } + first = false; + if (!first) sb.append(", "); + sb.append("funcArgs:"); + if (this.funcArgs == null) { + sb.append("null"); + } else { + sb.append(this.funcArgs); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class execute_argsStandardSchemeFactory implements SchemeFactory { + public execute_argsStandardScheme getScheme() { + return new execute_argsStandardScheme(); + } + } + + private static class execute_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, execute_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // FUNCTION_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.functionName = iprot.readString(); + struct.set_functionName_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // FUNC_ARGS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.funcArgs = iprot.readString(); + struct.set_funcArgs_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, execute_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.functionName != null) { + oprot.writeFieldBegin(FUNCTION_NAME_FIELD_DESC); + oprot.writeString(struct.functionName); + oprot.writeFieldEnd(); + } + if (struct.funcArgs != null) { + oprot.writeFieldBegin(FUNC_ARGS_FIELD_DESC); + oprot.writeString(struct.funcArgs); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class execute_argsTupleSchemeFactory implements SchemeFactory { + public execute_argsTupleScheme getScheme() { + return new execute_argsTupleScheme(); + } + } + + private static class execute_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, execute_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_functionName()) { + optionals.set(0); + } + if (struct.is_set_funcArgs()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_functionName()) { + oprot.writeString(struct.functionName); + } + if (struct.is_set_funcArgs()) { + oprot.writeString(struct.funcArgs); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, execute_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.functionName = iprot.readString(); + struct.set_functionName_isSet(true); + } + if (incoming.get(1)) { + struct.funcArgs = iprot.readString(); + struct.set_funcArgs_isSet(true); + } + } + } + + } + + public static class execute_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("execute_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0); + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new execute_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new execute_resultTupleSchemeFactory()); + } + + private String success; // required + private DRPCExecutionException e; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"), + E((short)1, "e"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // E + return E; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(execute_result.class, metaDataMap); + } + + public execute_result() { + } + + public execute_result( + String success, + DRPCExecutionException e) + { + this(); + this.success = success; + this.e = e; + } + + /** + * Performs a deep copy on other. + */ + public execute_result(execute_result other) { + if (other.is_set_success()) { + this.success = other.success; + } + if (other.is_set_e()) { + this.e = new DRPCExecutionException(other.e); + } + } + + public execute_result deepCopy() { + return new execute_result(this); + } + + @Override + public void clear() { + this.success = null; + this.e = null; + } + + public String get_success() { + return this.success; + } + + public void set_success(String success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public DRPCExecutionException get_e() { + return this.e; + } + + public void set_e(DRPCExecutionException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((String)value); + } + break; + + case E: + if (value == null) { + unset_e(); + } else { + set_e((DRPCExecutionException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + case E: + return get_e(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + case E: + return is_set_e(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof execute_result) + return this.equals((execute_result)that); + return false; + } + + public boolean equals(execute_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + return list.hashCode(); + } + + @Override + public int compareTo(execute_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("execute_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + if (!first) sb.append(", "); + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class execute_resultStandardSchemeFactory implements SchemeFactory { + public execute_resultStandardScheme getScheme() { + return new execute_resultStandardScheme(); + } + } + + private static class execute_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, execute_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.success = iprot.readString(); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new DRPCExecutionException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, execute_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeString(struct.success); + oprot.writeFieldEnd(); + } + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class execute_resultTupleSchemeFactory implements SchemeFactory { + public execute_resultTupleScheme getScheme() { + return new execute_resultTupleScheme(); + } + } + + private static class execute_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, execute_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + if (struct.is_set_e()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_success()) { + oprot.writeString(struct.success); + } + if (struct.is_set_e()) { + struct.e.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, execute_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.success = iprot.readString(); + struct.set_success_isSet(true); + } + if (incoming.get(1)) { + struct.e = new DRPCExecutionException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + } + } + + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/generated/DistributedRPCInvocations.java b/jstorm-core/src/main/java/backtype/storm/generated/DistributedRPCInvocations.java new file mode 100644 index 000000000..08aab63da --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/DistributedRPCInvocations.java @@ -0,0 +1,2545 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class DistributedRPCInvocations { + + public interface Iface { + + public void result(String id, String result) throws org.apache.thrift.TException; + + public DRPCRequest fetchRequest(String functionName) throws org.apache.thrift.TException; + + public void failRequest(String id) throws org.apache.thrift.TException; + + } + + public interface AsyncIface { + + public void result(String id, String result, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void fetchRequest(String functionName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void failRequest(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + } + + public static class Client extends org.apache.thrift.TServiceClient implements Iface { + public static class Factory implements org.apache.thrift.TServiceClientFactory { + public Factory() {} + public Client getClient(org.apache.thrift.protocol.TProtocol prot) { + return new Client(prot); + } + public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) { + return new Client(iprot, oprot); + } + } + + public Client(org.apache.thrift.protocol.TProtocol prot) + { + super(prot, prot); + } + + public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) { + super(iprot, oprot); + } + + public void result(String id, String result) throws org.apache.thrift.TException + { + send_result(id, result); + recv_result(); + } + + public void send_result(String id, String result) throws org.apache.thrift.TException + { + result_args args = new result_args(); + args.set_id(id); + args.set_result(result); + sendBase("result", args); + } + + public void recv_result() throws org.apache.thrift.TException + { + result_result result = new result_result(); + receiveBase(result, "result"); + return; + } + + public DRPCRequest fetchRequest(String functionName) throws org.apache.thrift.TException + { + send_fetchRequest(functionName); + return recv_fetchRequest(); + } + + public void send_fetchRequest(String functionName) throws org.apache.thrift.TException + { + fetchRequest_args args = new fetchRequest_args(); + args.set_functionName(functionName); + sendBase("fetchRequest", args); + } + + public DRPCRequest recv_fetchRequest() throws org.apache.thrift.TException + { + fetchRequest_result result = new fetchRequest_result(); + receiveBase(result, "fetchRequest"); + if (result.is_set_success()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "fetchRequest failed: unknown result"); + } + + public void failRequest(String id) throws org.apache.thrift.TException + { + send_failRequest(id); + recv_failRequest(); + } + + public void send_failRequest(String id) throws org.apache.thrift.TException + { + failRequest_args args = new failRequest_args(); + args.set_id(id); + sendBase("failRequest", args); + } + + public void recv_failRequest() throws org.apache.thrift.TException + { + failRequest_result result = new failRequest_result(); + receiveBase(result, "failRequest"); + return; + } + + } + public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface { + public static class Factory implements org.apache.thrift.async.TAsyncClientFactory { + private org.apache.thrift.async.TAsyncClientManager clientManager; + private org.apache.thrift.protocol.TProtocolFactory protocolFactory; + public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) { + this.clientManager = clientManager; + this.protocolFactory = protocolFactory; + } + public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) { + return new AsyncClient(protocolFactory, clientManager, transport); + } + } + + public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) { + super(protocolFactory, clientManager, transport); + } + + public void result(String id, String result, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + result_call method_call = new result_call(id, result, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class result_call extends org.apache.thrift.async.TAsyncMethodCall { + private String id; + private String result; + public result_call(String id, String result, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.id = id; + this.result = result; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("result", org.apache.thrift.protocol.TMessageType.CALL, 0)); + result_args args = new result_args(); + args.set_id(id); + args.set_result(result); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_result(); + } + } + + public void fetchRequest(String functionName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + fetchRequest_call method_call = new fetchRequest_call(functionName, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class fetchRequest_call extends org.apache.thrift.async.TAsyncMethodCall { + private String functionName; + public fetchRequest_call(String functionName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.functionName = functionName; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("fetchRequest", org.apache.thrift.protocol.TMessageType.CALL, 0)); + fetchRequest_args args = new fetchRequest_args(); + args.set_functionName(functionName); + args.write(prot); + prot.writeMessageEnd(); + } + + public DRPCRequest getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_fetchRequest(); + } + } + + public void failRequest(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + failRequest_call method_call = new failRequest_call(id, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class failRequest_call extends org.apache.thrift.async.TAsyncMethodCall { + private String id; + public failRequest_call(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.id = id; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("failRequest", org.apache.thrift.protocol.TMessageType.CALL, 0)); + failRequest_args args = new failRequest_args(); + args.set_id(id); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_failRequest(); + } + } + + } + + public static class Processor extends org.apache.thrift.TBaseProcessor implements org.apache.thrift.TProcessor { + private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName()); + public Processor(I iface) { + super(iface, getProcessMap(new HashMap>())); + } + + protected Processor(I iface, Map> processMap) { + super(iface, getProcessMap(processMap)); + } + + private static Map> getProcessMap(Map> processMap) { + processMap.put("result", new result()); + processMap.put("fetchRequest", new fetchRequest()); + processMap.put("failRequest", new failRequest()); + return processMap; + } + + public static class result extends org.apache.thrift.ProcessFunction { + public result() { + super("result"); + } + + public result_args getEmptyArgsInstance() { + return new result_args(); + } + + protected boolean isOneway() { + return false; + } + + public result_result getResult(I iface, result_args args) throws org.apache.thrift.TException { + result_result result = new result_result(); + iface.result(args.id, args.result); + return result; + } + } + + public static class fetchRequest extends org.apache.thrift.ProcessFunction { + public fetchRequest() { + super("fetchRequest"); + } + + public fetchRequest_args getEmptyArgsInstance() { + return new fetchRequest_args(); + } + + protected boolean isOneway() { + return false; + } + + public fetchRequest_result getResult(I iface, fetchRequest_args args) throws org.apache.thrift.TException { + fetchRequest_result result = new fetchRequest_result(); + result.success = iface.fetchRequest(args.functionName); + return result; + } + } + + public static class failRequest extends org.apache.thrift.ProcessFunction { + public failRequest() { + super("failRequest"); + } + + public failRequest_args getEmptyArgsInstance() { + return new failRequest_args(); + } + + protected boolean isOneway() { + return false; + } + + public failRequest_result getResult(I iface, failRequest_args args) throws org.apache.thrift.TException { + failRequest_result result = new failRequest_result(); + iface.failRequest(args.id); + return result; + } + } + + } + + public static class AsyncProcessor extends org.apache.thrift.TBaseAsyncProcessor { + private static final Logger LOGGER = LoggerFactory.getLogger(AsyncProcessor.class.getName()); + public AsyncProcessor(I iface) { + super(iface, getProcessMap(new HashMap>())); + } + + protected AsyncProcessor(I iface, Map> processMap) { + super(iface, getProcessMap(processMap)); + } + + private static Map> getProcessMap(Map> processMap) { + processMap.put("result", new result()); + processMap.put("fetchRequest", new fetchRequest()); + processMap.put("failRequest", new failRequest()); + return processMap; + } + + public static class result extends org.apache.thrift.AsyncProcessFunction { + public result() { + super("result"); + } + + public result_args getEmptyArgsInstance() { + return new result_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + result_result result = new result_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + result_result result = new result_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, result_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.result(args.id, args.result,resultHandler); + } + } + + public static class fetchRequest extends org.apache.thrift.AsyncProcessFunction { + public fetchRequest() { + super("fetchRequest"); + } + + public fetchRequest_args getEmptyArgsInstance() { + return new fetchRequest_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(DRPCRequest o) { + fetchRequest_result result = new fetchRequest_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + fetchRequest_result result = new fetchRequest_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, fetchRequest_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.fetchRequest(args.functionName,resultHandler); + } + } + + public static class failRequest extends org.apache.thrift.AsyncProcessFunction { + public failRequest() { + super("failRequest"); + } + + public failRequest_args getEmptyArgsInstance() { + return new failRequest_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + failRequest_result result = new failRequest_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + failRequest_result result = new failRequest_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, failRequest_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.failRequest(args.id,resultHandler); + } + } + + } + + public static class result_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("result_args"); + + private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField RESULT_FIELD_DESC = new org.apache.thrift.protocol.TField("result", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new result_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new result_argsTupleSchemeFactory()); + } + + private String id; // required + private String result; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + ID((short)1, "id"), + RESULT((short)2, "result"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // ID + return ID; + case 2: // RESULT + return RESULT; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.RESULT, new org.apache.thrift.meta_data.FieldMetaData("result", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(result_args.class, metaDataMap); + } + + public result_args() { + } + + public result_args( + String id, + String result) + { + this(); + this.id = id; + this.result = result; + } + + /** + * Performs a deep copy on other. + */ + public result_args(result_args other) { + if (other.is_set_id()) { + this.id = other.id; + } + if (other.is_set_result()) { + this.result = other.result; + } + } + + public result_args deepCopy() { + return new result_args(this); + } + + @Override + public void clear() { + this.id = null; + this.result = null; + } + + public String get_id() { + return this.id; + } + + public void set_id(String id) { + this.id = id; + } + + public void unset_id() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean is_set_id() { + return this.id != null; + } + + public void set_id_isSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public String get_result() { + return this.result; + } + + public void set_result(String result) { + this.result = result; + } + + public void unset_result() { + this.result = null; + } + + /** Returns true if field result is set (has been assigned a value) and false otherwise */ + public boolean is_set_result() { + return this.result != null; + } + + public void set_result_isSet(boolean value) { + if (!value) { + this.result = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unset_id(); + } else { + set_id((String)value); + } + break; + + case RESULT: + if (value == null) { + unset_result(); + } else { + set_result((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return get_id(); + + case RESULT: + return get_result(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case ID: + return is_set_id(); + case RESULT: + return is_set_result(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof result_args) + return this.equals((result_args)that); + return false; + } + + public boolean equals(result_args that) { + if (that == null) + return false; + + boolean this_present_id = true && this.is_set_id(); + boolean that_present_id = true && that.is_set_id(); + if (this_present_id || that_present_id) { + if (!(this_present_id && that_present_id)) + return false; + if (!this.id.equals(that.id)) + return false; + } + + boolean this_present_result = true && this.is_set_result(); + boolean that_present_result = true && that.is_set_result(); + if (this_present_result || that_present_result) { + if (!(this_present_result && that_present_result)) + return false; + if (!this.result.equals(that.result)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_id = true && (is_set_id()); + list.add(present_id); + if (present_id) + list.add(id); + + boolean present_result = true && (is_set_result()); + list.add(present_result); + if (present_result) + list.add(result); + + return list.hashCode(); + } + + @Override + public int compareTo(result_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_result()).compareTo(other.is_set_result()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_result()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.result, other.result); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("result_args("); + boolean first = true; + + sb.append("id:"); + if (this.id == null) { + sb.append("null"); + } else { + sb.append(this.id); + } + first = false; + if (!first) sb.append(", "); + sb.append("result:"); + if (this.result == null) { + sb.append("null"); + } else { + sb.append(this.result); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class result_argsStandardSchemeFactory implements SchemeFactory { + public result_argsStandardScheme getScheme() { + return new result_argsStandardScheme(); + } + } + + private static class result_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, result_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // RESULT + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.result = iprot.readString(); + struct.set_result_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, result_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.id != null) { + oprot.writeFieldBegin(ID_FIELD_DESC); + oprot.writeString(struct.id); + oprot.writeFieldEnd(); + } + if (struct.result != null) { + oprot.writeFieldBegin(RESULT_FIELD_DESC); + oprot.writeString(struct.result); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class result_argsTupleSchemeFactory implements SchemeFactory { + public result_argsTupleScheme getScheme() { + return new result_argsTupleScheme(); + } + } + + private static class result_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, result_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_id()) { + optionals.set(0); + } + if (struct.is_set_result()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_id()) { + oprot.writeString(struct.id); + } + if (struct.is_set_result()) { + oprot.writeString(struct.result); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, result_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } + if (incoming.get(1)) { + struct.result = iprot.readString(); + struct.set_result_isSet(true); + } + } + } + + } + + public static class result_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("result_result"); + + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new result_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new result_resultTupleSchemeFactory()); + } + + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { +; + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(result_result.class, metaDataMap); + } + + public result_result() { + } + + /** + * Performs a deep copy on other. + */ + public result_result(result_result other) { + } + + public result_result deepCopy() { + return new result_result(this); + } + + @Override + public void clear() { + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof result_result) + return this.equals((result_result)that); + return false; + } + + public boolean equals(result_result that) { + if (that == null) + return false; + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + return list.hashCode(); + } + + @Override + public int compareTo(result_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("result_result("); + boolean first = true; + + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class result_resultStandardSchemeFactory implements SchemeFactory { + public result_resultStandardScheme getScheme() { + return new result_resultStandardScheme(); + } + } + + private static class result_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, result_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, result_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class result_resultTupleSchemeFactory implements SchemeFactory { + public result_resultTupleScheme getScheme() { + return new result_resultTupleScheme(); + } + } + + private static class result_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, result_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, result_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + } + } + + } + + public static class fetchRequest_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("fetchRequest_args"); + + private static final org.apache.thrift.protocol.TField FUNCTION_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("functionName", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new fetchRequest_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new fetchRequest_argsTupleSchemeFactory()); + } + + private String functionName; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + FUNCTION_NAME((short)1, "functionName"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // FUNCTION_NAME + return FUNCTION_NAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.FUNCTION_NAME, new org.apache.thrift.meta_data.FieldMetaData("functionName", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(fetchRequest_args.class, metaDataMap); + } + + public fetchRequest_args() { + } + + public fetchRequest_args( + String functionName) + { + this(); + this.functionName = functionName; + } + + /** + * Performs a deep copy on other. + */ + public fetchRequest_args(fetchRequest_args other) { + if (other.is_set_functionName()) { + this.functionName = other.functionName; + } + } + + public fetchRequest_args deepCopy() { + return new fetchRequest_args(this); + } + + @Override + public void clear() { + this.functionName = null; + } + + public String get_functionName() { + return this.functionName; + } + + public void set_functionName(String functionName) { + this.functionName = functionName; + } + + public void unset_functionName() { + this.functionName = null; + } + + /** Returns true if field functionName is set (has been assigned a value) and false otherwise */ + public boolean is_set_functionName() { + return this.functionName != null; + } + + public void set_functionName_isSet(boolean value) { + if (!value) { + this.functionName = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case FUNCTION_NAME: + if (value == null) { + unset_functionName(); + } else { + set_functionName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case FUNCTION_NAME: + return get_functionName(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case FUNCTION_NAME: + return is_set_functionName(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof fetchRequest_args) + return this.equals((fetchRequest_args)that); + return false; + } + + public boolean equals(fetchRequest_args that) { + if (that == null) + return false; + + boolean this_present_functionName = true && this.is_set_functionName(); + boolean that_present_functionName = true && that.is_set_functionName(); + if (this_present_functionName || that_present_functionName) { + if (!(this_present_functionName && that_present_functionName)) + return false; + if (!this.functionName.equals(that.functionName)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_functionName = true && (is_set_functionName()); + list.add(present_functionName); + if (present_functionName) + list.add(functionName); + + return list.hashCode(); + } + + @Override + public int compareTo(fetchRequest_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_functionName()).compareTo(other.is_set_functionName()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_functionName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.functionName, other.functionName); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("fetchRequest_args("); + boolean first = true; + + sb.append("functionName:"); + if (this.functionName == null) { + sb.append("null"); + } else { + sb.append(this.functionName); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class fetchRequest_argsStandardSchemeFactory implements SchemeFactory { + public fetchRequest_argsStandardScheme getScheme() { + return new fetchRequest_argsStandardScheme(); + } + } + + private static class fetchRequest_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, fetchRequest_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // FUNCTION_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.functionName = iprot.readString(); + struct.set_functionName_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, fetchRequest_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.functionName != null) { + oprot.writeFieldBegin(FUNCTION_NAME_FIELD_DESC); + oprot.writeString(struct.functionName); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class fetchRequest_argsTupleSchemeFactory implements SchemeFactory { + public fetchRequest_argsTupleScheme getScheme() { + return new fetchRequest_argsTupleScheme(); + } + } + + private static class fetchRequest_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, fetchRequest_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_functionName()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_functionName()) { + oprot.writeString(struct.functionName); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, fetchRequest_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.functionName = iprot.readString(); + struct.set_functionName_isSet(true); + } + } + } + + } + + public static class fetchRequest_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("fetchRequest_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new fetchRequest_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new fetchRequest_resultTupleSchemeFactory()); + } + + private DRPCRequest success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, DRPCRequest.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(fetchRequest_result.class, metaDataMap); + } + + public fetchRequest_result() { + } + + public fetchRequest_result( + DRPCRequest success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public fetchRequest_result(fetchRequest_result other) { + if (other.is_set_success()) { + this.success = new DRPCRequest(other.success); + } + } + + public fetchRequest_result deepCopy() { + return new fetchRequest_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public DRPCRequest get_success() { + return this.success; + } + + public void set_success(DRPCRequest success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((DRPCRequest)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof fetchRequest_result) + return this.equals((fetchRequest_result)that); + return false; + } + + public boolean equals(fetchRequest_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(fetchRequest_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("fetchRequest_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class fetchRequest_resultStandardSchemeFactory implements SchemeFactory { + public fetchRequest_resultStandardScheme getScheme() { + return new fetchRequest_resultStandardScheme(); + } + } + + private static class fetchRequest_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, fetchRequest_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new DRPCRequest(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, fetchRequest_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class fetchRequest_resultTupleSchemeFactory implements SchemeFactory { + public fetchRequest_resultTupleScheme getScheme() { + return new fetchRequest_resultTupleScheme(); + } + } + + private static class fetchRequest_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, fetchRequest_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_success()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, fetchRequest_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new DRPCRequest(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } + } + } + + } + + public static class failRequest_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("failRequest_args"); + + private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new failRequest_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new failRequest_argsTupleSchemeFactory()); + } + + private String id; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + ID((short)1, "id"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // ID + return ID; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(failRequest_args.class, metaDataMap); + } + + public failRequest_args() { + } + + public failRequest_args( + String id) + { + this(); + this.id = id; + } + + /** + * Performs a deep copy on other. + */ + public failRequest_args(failRequest_args other) { + if (other.is_set_id()) { + this.id = other.id; + } + } + + public failRequest_args deepCopy() { + return new failRequest_args(this); + } + + @Override + public void clear() { + this.id = null; + } + + public String get_id() { + return this.id; + } + + public void set_id(String id) { + this.id = id; + } + + public void unset_id() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean is_set_id() { + return this.id != null; + } + + public void set_id_isSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unset_id(); + } else { + set_id((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return get_id(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case ID: + return is_set_id(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof failRequest_args) + return this.equals((failRequest_args)that); + return false; + } + + public boolean equals(failRequest_args that) { + if (that == null) + return false; + + boolean this_present_id = true && this.is_set_id(); + boolean that_present_id = true && that.is_set_id(); + if (this_present_id || that_present_id) { + if (!(this_present_id && that_present_id)) + return false; + if (!this.id.equals(that.id)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_id = true && (is_set_id()); + list.add(present_id); + if (present_id) + list.add(id); + + return list.hashCode(); + } + + @Override + public int compareTo(failRequest_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("failRequest_args("); + boolean first = true; + + sb.append("id:"); + if (this.id == null) { + sb.append("null"); + } else { + sb.append(this.id); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class failRequest_argsStandardSchemeFactory implements SchemeFactory { + public failRequest_argsStandardScheme getScheme() { + return new failRequest_argsStandardScheme(); + } + } + + private static class failRequest_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, failRequest_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, failRequest_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.id != null) { + oprot.writeFieldBegin(ID_FIELD_DESC); + oprot.writeString(struct.id); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class failRequest_argsTupleSchemeFactory implements SchemeFactory { + public failRequest_argsTupleScheme getScheme() { + return new failRequest_argsTupleScheme(); + } + } + + private static class failRequest_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, failRequest_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_id()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_id()) { + oprot.writeString(struct.id); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, failRequest_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } + } + } + + } + + public static class failRequest_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("failRequest_result"); + + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new failRequest_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new failRequest_resultTupleSchemeFactory()); + } + + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { +; + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(failRequest_result.class, metaDataMap); + } + + public failRequest_result() { + } + + /** + * Performs a deep copy on other. + */ + public failRequest_result(failRequest_result other) { + } + + public failRequest_result deepCopy() { + return new failRequest_result(this); + } + + @Override + public void clear() { + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof failRequest_result) + return this.equals((failRequest_result)that); + return false; + } + + public boolean equals(failRequest_result that) { + if (that == null) + return false; + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + return list.hashCode(); + } + + @Override + public int compareTo(failRequest_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("failRequest_result("); + boolean first = true; + + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class failRequest_resultStandardSchemeFactory implements SchemeFactory { + public failRequest_resultStandardScheme getScheme() { + return new failRequest_resultStandardScheme(); + } + } + + private static class failRequest_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, failRequest_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, failRequest_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class failRequest_resultTupleSchemeFactory implements SchemeFactory { + public failRequest_resultTupleScheme getScheme() { + return new failRequest_resultTupleScheme(); + } + } + + private static class failRequest_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, failRequest_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, failRequest_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + } + } + + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/generated/ErrorInfo.java b/jstorm-core/src/main/java/backtype/storm/generated/ErrorInfo.java new file mode 100644 index 000000000..f52e5266d --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/ErrorInfo.java @@ -0,0 +1,488 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class ErrorInfo implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ErrorInfo"); + + private static final org.apache.thrift.protocol.TField ERROR_FIELD_DESC = new org.apache.thrift.protocol.TField("error", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField ERROR_TIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("error_time_secs", org.apache.thrift.protocol.TType.I32, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ErrorInfoStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ErrorInfoTupleSchemeFactory()); + } + + private String error; // required + private int error_time_secs; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + ERROR((short)1, "error"), + ERROR_TIME_SECS((short)2, "error_time_secs"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // ERROR + return ERROR; + case 2: // ERROR_TIME_SECS + return ERROR_TIME_SECS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __ERROR_TIME_SECS_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.ERROR, new org.apache.thrift.meta_data.FieldMetaData("error", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.ERROR_TIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("error_time_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ErrorInfo.class, metaDataMap); + } + + public ErrorInfo() { + } + + public ErrorInfo( + String error, + int error_time_secs) + { + this(); + this.error = error; + this.error_time_secs = error_time_secs; + set_error_time_secs_isSet(true); + } + + /** + * Performs a deep copy on other. + */ + public ErrorInfo(ErrorInfo other) { + __isset_bitfield = other.__isset_bitfield; + if (other.is_set_error()) { + this.error = other.error; + } + this.error_time_secs = other.error_time_secs; + } + + public ErrorInfo deepCopy() { + return new ErrorInfo(this); + } + + @Override + public void clear() { + this.error = null; + set_error_time_secs_isSet(false); + this.error_time_secs = 0; + } + + public String get_error() { + return this.error; + } + + public void set_error(String error) { + this.error = error; + } + + public void unset_error() { + this.error = null; + } + + /** Returns true if field error is set (has been assigned a value) and false otherwise */ + public boolean is_set_error() { + return this.error != null; + } + + public void set_error_isSet(boolean value) { + if (!value) { + this.error = null; + } + } + + public int get_error_time_secs() { + return this.error_time_secs; + } + + public void set_error_time_secs(int error_time_secs) { + this.error_time_secs = error_time_secs; + set_error_time_secs_isSet(true); + } + + public void unset_error_time_secs() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ERROR_TIME_SECS_ISSET_ID); + } + + /** Returns true if field error_time_secs is set (has been assigned a value) and false otherwise */ + public boolean is_set_error_time_secs() { + return EncodingUtils.testBit(__isset_bitfield, __ERROR_TIME_SECS_ISSET_ID); + } + + public void set_error_time_secs_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ERROR_TIME_SECS_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ERROR: + if (value == null) { + unset_error(); + } else { + set_error((String)value); + } + break; + + case ERROR_TIME_SECS: + if (value == null) { + unset_error_time_secs(); + } else { + set_error_time_secs((Integer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ERROR: + return get_error(); + + case ERROR_TIME_SECS: + return Integer.valueOf(get_error_time_secs()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case ERROR: + return is_set_error(); + case ERROR_TIME_SECS: + return is_set_error_time_secs(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ErrorInfo) + return this.equals((ErrorInfo)that); + return false; + } + + public boolean equals(ErrorInfo that) { + if (that == null) + return false; + + boolean this_present_error = true && this.is_set_error(); + boolean that_present_error = true && that.is_set_error(); + if (this_present_error || that_present_error) { + if (!(this_present_error && that_present_error)) + return false; + if (!this.error.equals(that.error)) + return false; + } + + boolean this_present_error_time_secs = true; + boolean that_present_error_time_secs = true; + if (this_present_error_time_secs || that_present_error_time_secs) { + if (!(this_present_error_time_secs && that_present_error_time_secs)) + return false; + if (this.error_time_secs != that.error_time_secs) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_error = true && (is_set_error()); + list.add(present_error); + if (present_error) + list.add(error); + + boolean present_error_time_secs = true; + list.add(present_error_time_secs); + if (present_error_time_secs) + list.add(error_time_secs); + + return list.hashCode(); + } + + @Override + public int compareTo(ErrorInfo other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_error()).compareTo(other.is_set_error()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_error()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.error, other.error); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_error_time_secs()).compareTo(other.is_set_error_time_secs()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_error_time_secs()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.error_time_secs, other.error_time_secs); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ErrorInfo("); + boolean first = true; + + sb.append("error:"); + if (this.error == null) { + sb.append("null"); + } else { + sb.append(this.error); + } + first = false; + if (!first) sb.append(", "); + sb.append("error_time_secs:"); + sb.append(this.error_time_secs); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_error()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'error' is unset! Struct:" + toString()); + } + + if (!is_set_error_time_secs()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'error_time_secs' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ErrorInfoStandardSchemeFactory implements SchemeFactory { + public ErrorInfoStandardScheme getScheme() { + return new ErrorInfoStandardScheme(); + } + } + + private static class ErrorInfoStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ErrorInfo struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // ERROR + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.error = iprot.readString(); + struct.set_error_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // ERROR_TIME_SECS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.error_time_secs = iprot.readI32(); + struct.set_error_time_secs_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ErrorInfo struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.error != null) { + oprot.writeFieldBegin(ERROR_FIELD_DESC); + oprot.writeString(struct.error); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(ERROR_TIME_SECS_FIELD_DESC); + oprot.writeI32(struct.error_time_secs); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ErrorInfoTupleSchemeFactory implements SchemeFactory { + public ErrorInfoTupleScheme getScheme() { + return new ErrorInfoTupleScheme(); + } + } + + private static class ErrorInfoTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ErrorInfo struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.error); + oprot.writeI32(struct.error_time_secs); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ErrorInfo struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.error = iprot.readString(); + struct.set_error_isSet(true); + struct.error_time_secs = iprot.readI32(); + struct.set_error_time_secs_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/GlobalStreamId.java b/jstorm-core/src/main/java/backtype/storm/generated/GlobalStreamId.java new file mode 100644 index 000000000..490a81d35 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/GlobalStreamId.java @@ -0,0 +1,490 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class GlobalStreamId implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GlobalStreamId"); + + private static final org.apache.thrift.protocol.TField COMPONENT_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("componentId", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField STREAM_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("streamId", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GlobalStreamIdStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GlobalStreamIdTupleSchemeFactory()); + } + + private String componentId; // required + private String streamId; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + COMPONENT_ID((short)1, "componentId"), + STREAM_ID((short)2, "streamId"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // COMPONENT_ID + return COMPONENT_ID; + case 2: // STREAM_ID + return STREAM_ID; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.COMPONENT_ID, new org.apache.thrift.meta_data.FieldMetaData("componentId", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.STREAM_ID, new org.apache.thrift.meta_data.FieldMetaData("streamId", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GlobalStreamId.class, metaDataMap); + } + + public GlobalStreamId() { + } + + public GlobalStreamId( + String componentId, + String streamId) + { + this(); + this.componentId = componentId; + this.streamId = streamId; + } + + /** + * Performs a deep copy on other. + */ + public GlobalStreamId(GlobalStreamId other) { + if (other.is_set_componentId()) { + this.componentId = other.componentId; + } + if (other.is_set_streamId()) { + this.streamId = other.streamId; + } + } + + public GlobalStreamId deepCopy() { + return new GlobalStreamId(this); + } + + @Override + public void clear() { + this.componentId = null; + this.streamId = null; + } + + public String get_componentId() { + return this.componentId; + } + + public void set_componentId(String componentId) { + this.componentId = componentId; + } + + public void unset_componentId() { + this.componentId = null; + } + + /** Returns true if field componentId is set (has been assigned a value) and false otherwise */ + public boolean is_set_componentId() { + return this.componentId != null; + } + + public void set_componentId_isSet(boolean value) { + if (!value) { + this.componentId = null; + } + } + + public String get_streamId() { + return this.streamId; + } + + public void set_streamId(String streamId) { + this.streamId = streamId; + } + + public void unset_streamId() { + this.streamId = null; + } + + /** Returns true if field streamId is set (has been assigned a value) and false otherwise */ + public boolean is_set_streamId() { + return this.streamId != null; + } + + public void set_streamId_isSet(boolean value) { + if (!value) { + this.streamId = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case COMPONENT_ID: + if (value == null) { + unset_componentId(); + } else { + set_componentId((String)value); + } + break; + + case STREAM_ID: + if (value == null) { + unset_streamId(); + } else { + set_streamId((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case COMPONENT_ID: + return get_componentId(); + + case STREAM_ID: + return get_streamId(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case COMPONENT_ID: + return is_set_componentId(); + case STREAM_ID: + return is_set_streamId(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GlobalStreamId) + return this.equals((GlobalStreamId)that); + return false; + } + + public boolean equals(GlobalStreamId that) { + if (that == null) + return false; + + boolean this_present_componentId = true && this.is_set_componentId(); + boolean that_present_componentId = true && that.is_set_componentId(); + if (this_present_componentId || that_present_componentId) { + if (!(this_present_componentId && that_present_componentId)) + return false; + if (!this.componentId.equals(that.componentId)) + return false; + } + + boolean this_present_streamId = true && this.is_set_streamId(); + boolean that_present_streamId = true && that.is_set_streamId(); + if (this_present_streamId || that_present_streamId) { + if (!(this_present_streamId && that_present_streamId)) + return false; + if (!this.streamId.equals(that.streamId)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_componentId = true && (is_set_componentId()); + list.add(present_componentId); + if (present_componentId) + list.add(componentId); + + boolean present_streamId = true && (is_set_streamId()); + list.add(present_streamId); + if (present_streamId) + list.add(streamId); + + return list.hashCode(); + } + + @Override + public int compareTo(GlobalStreamId other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_componentId()).compareTo(other.is_set_componentId()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_componentId()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.componentId, other.componentId); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_streamId()).compareTo(other.is_set_streamId()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_streamId()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.streamId, other.streamId); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GlobalStreamId("); + boolean first = true; + + sb.append("componentId:"); + if (this.componentId == null) { + sb.append("null"); + } else { + sb.append(this.componentId); + } + first = false; + if (!first) sb.append(", "); + sb.append("streamId:"); + if (this.streamId == null) { + sb.append("null"); + } else { + sb.append(this.streamId); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_componentId()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'componentId' is unset! Struct:" + toString()); + } + + if (!is_set_streamId()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'streamId' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GlobalStreamIdStandardSchemeFactory implements SchemeFactory { + public GlobalStreamIdStandardScheme getScheme() { + return new GlobalStreamIdStandardScheme(); + } + } + + private static class GlobalStreamIdStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GlobalStreamId struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // COMPONENT_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.componentId = iprot.readString(); + struct.set_componentId_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // STREAM_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.streamId = iprot.readString(); + struct.set_streamId_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GlobalStreamId struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.componentId != null) { + oprot.writeFieldBegin(COMPONENT_ID_FIELD_DESC); + oprot.writeString(struct.componentId); + oprot.writeFieldEnd(); + } + if (struct.streamId != null) { + oprot.writeFieldBegin(STREAM_ID_FIELD_DESC); + oprot.writeString(struct.streamId); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GlobalStreamIdTupleSchemeFactory implements SchemeFactory { + public GlobalStreamIdTupleScheme getScheme() { + return new GlobalStreamIdTupleScheme(); + } + } + + private static class GlobalStreamIdTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GlobalStreamId struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.componentId); + oprot.writeString(struct.streamId); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GlobalStreamId struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.componentId = iprot.readString(); + struct.set_componentId_isSet(true); + struct.streamId = iprot.readString(); + struct.set_streamId_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/Grouping.java b/jstorm-core/src/main/java/backtype/storm/generated/Grouping.java new file mode 100755 index 000000000..bc60a06fd --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/Grouping.java @@ -0,0 +1,844 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +public class Grouping extends org.apache.thrift.TUnion { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Grouping"); + private static final org.apache.thrift.protocol.TField FIELDS_FIELD_DESC = new org.apache.thrift.protocol.TField("fields", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField SHUFFLE_FIELD_DESC = new org.apache.thrift.protocol.TField("shuffle", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField ALL_FIELD_DESC = new org.apache.thrift.protocol.TField("all", org.apache.thrift.protocol.TType.STRUCT, (short)3); + private static final org.apache.thrift.protocol.TField NONE_FIELD_DESC = new org.apache.thrift.protocol.TField("none", org.apache.thrift.protocol.TType.STRUCT, (short)4); + private static final org.apache.thrift.protocol.TField DIRECT_FIELD_DESC = new org.apache.thrift.protocol.TField("direct", org.apache.thrift.protocol.TType.STRUCT, (short)5); + private static final org.apache.thrift.protocol.TField CUSTOM_OBJECT_FIELD_DESC = new org.apache.thrift.protocol.TField("custom_object", org.apache.thrift.protocol.TType.STRUCT, (short)6); + private static final org.apache.thrift.protocol.TField CUSTOM_SERIALIZED_FIELD_DESC = new org.apache.thrift.protocol.TField("custom_serialized", org.apache.thrift.protocol.TType.STRING, (short)7); + private static final org.apache.thrift.protocol.TField LOCAL_OR_SHUFFLE_FIELD_DESC = new org.apache.thrift.protocol.TField("local_or_shuffle", org.apache.thrift.protocol.TType.STRUCT, (short)8); + private static final org.apache.thrift.protocol.TField LOCAL_FIRST_FIELD_DESC = new org.apache.thrift.protocol.TField("localFirst", org.apache.thrift.protocol.TType.STRUCT, (short)9); + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + FIELDS((short)1, "fields"), + SHUFFLE((short)2, "shuffle"), + ALL((short)3, "all"), + NONE((short)4, "none"), + DIRECT((short)5, "direct"), + CUSTOM_OBJECT((short)6, "custom_object"), + CUSTOM_SERIALIZED((short)7, "custom_serialized"), + LOCAL_OR_SHUFFLE((short)8, "local_or_shuffle"), + LOCAL_FIRST((short)9, "localFirst"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // FIELDS + return FIELDS; + case 2: // SHUFFLE + return SHUFFLE; + case 3: // ALL + return ALL; + case 4: // NONE + return NONE; + case 5: // DIRECT + return DIRECT; + case 6: // CUSTOM_OBJECT + return CUSTOM_OBJECT; + case 7: // CUSTOM_SERIALIZED + return CUSTOM_SERIALIZED; + case 8: // LOCAL_OR_SHUFFLE + return LOCAL_OR_SHUFFLE; + case 9: // LOCAL_FIRST + return LOCAL_FIRST; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.FIELDS, new org.apache.thrift.meta_data.FieldMetaData("fields", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + tmpMap.put(_Fields.SHUFFLE, new org.apache.thrift.meta_data.FieldMetaData("shuffle", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NullStruct.class))); + tmpMap.put(_Fields.ALL, new org.apache.thrift.meta_data.FieldMetaData("all", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NullStruct.class))); + tmpMap.put(_Fields.NONE, new org.apache.thrift.meta_data.FieldMetaData("none", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NullStruct.class))); + tmpMap.put(_Fields.DIRECT, new org.apache.thrift.meta_data.FieldMetaData("direct", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NullStruct.class))); + tmpMap.put(_Fields.CUSTOM_OBJECT, new org.apache.thrift.meta_data.FieldMetaData("custom_object", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, JavaObject.class))); + tmpMap.put(_Fields.CUSTOM_SERIALIZED, new org.apache.thrift.meta_data.FieldMetaData("custom_serialized", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + tmpMap.put(_Fields.LOCAL_OR_SHUFFLE, new org.apache.thrift.meta_data.FieldMetaData("local_or_shuffle", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NullStruct.class))); + tmpMap.put(_Fields.LOCAL_FIRST, new org.apache.thrift.meta_data.FieldMetaData("localFirst", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NullStruct.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Grouping.class, metaDataMap); + } + + public Grouping() { + super(); + } + + public Grouping(_Fields setField, Object value) { + super(setField, value); + } + + public Grouping(Grouping other) { + super(other); + } + public Grouping deepCopy() { + return new Grouping(this); + } + + public static Grouping fields(List value) { + Grouping x = new Grouping(); + x.set_fields(value); + return x; + } + + public static Grouping shuffle(NullStruct value) { + Grouping x = new Grouping(); + x.set_shuffle(value); + return x; + } + + public static Grouping all(NullStruct value) { + Grouping x = new Grouping(); + x.set_all(value); + return x; + } + + public static Grouping none(NullStruct value) { + Grouping x = new Grouping(); + x.set_none(value); + return x; + } + + public static Grouping direct(NullStruct value) { + Grouping x = new Grouping(); + x.set_direct(value); + return x; + } + + public static Grouping custom_object(JavaObject value) { + Grouping x = new Grouping(); + x.set_custom_object(value); + return x; + } + + public static Grouping custom_serialized(ByteBuffer value) { + Grouping x = new Grouping(); + x.set_custom_serialized(value); + return x; + } + + public static Grouping custom_serialized(byte[] value) { + Grouping x = new Grouping(); + x.set_custom_serialized(ByteBuffer.wrap(Arrays.copyOf(value, value.length))); + return x; + } + + public static Grouping local_or_shuffle(NullStruct value) { + Grouping x = new Grouping(); + x.set_local_or_shuffle(value); + return x; + } + + public static Grouping localFirst(NullStruct value) { + Grouping x = new Grouping(); + x.set_localFirst(value); + return x; + } + + + @Override + protected void checkType(_Fields setField, Object value) throws ClassCastException { + switch (setField) { + case FIELDS: + if (value instanceof List) { + break; + } + throw new ClassCastException("Was expecting value of type List for field 'fields', but got " + value.getClass().getSimpleName()); + case SHUFFLE: + if (value instanceof NullStruct) { + break; + } + throw new ClassCastException("Was expecting value of type NullStruct for field 'shuffle', but got " + value.getClass().getSimpleName()); + case ALL: + if (value instanceof NullStruct) { + break; + } + throw new ClassCastException("Was expecting value of type NullStruct for field 'all', but got " + value.getClass().getSimpleName()); + case NONE: + if (value instanceof NullStruct) { + break; + } + throw new ClassCastException("Was expecting value of type NullStruct for field 'none', but got " + value.getClass().getSimpleName()); + case DIRECT: + if (value instanceof NullStruct) { + break; + } + throw new ClassCastException("Was expecting value of type NullStruct for field 'direct', but got " + value.getClass().getSimpleName()); + case CUSTOM_OBJECT: + if (value instanceof JavaObject) { + break; + } + throw new ClassCastException("Was expecting value of type JavaObject for field 'custom_object', but got " + value.getClass().getSimpleName()); + case CUSTOM_SERIALIZED: + if (value instanceof ByteBuffer) { + break; + } + throw new ClassCastException("Was expecting value of type ByteBuffer for field 'custom_serialized', but got " + value.getClass().getSimpleName()); + case LOCAL_OR_SHUFFLE: + if (value instanceof NullStruct) { + break; + } + throw new ClassCastException("Was expecting value of type NullStruct for field 'local_or_shuffle', but got " + value.getClass().getSimpleName()); + case LOCAL_FIRST: + if (value instanceof NullStruct) { + break; + } + throw new ClassCastException("Was expecting value of type NullStruct for field 'localFirst', but got " + value.getClass().getSimpleName()); + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(field.id); + if (setField != null) { + switch (setField) { + case FIELDS: + if (field.type == FIELDS_FIELD_DESC.type) { + List fields; + { + org.apache.thrift.protocol.TList _list8 = iprot.readListBegin(); + fields = new ArrayList(_list8.size); + String _elem9; + for (int _i10 = 0; _i10 < _list8.size; ++_i10) + { + _elem9 = iprot.readString(); + fields.add(_elem9); + } + iprot.readListEnd(); + } + return fields; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case SHUFFLE: + if (field.type == SHUFFLE_FIELD_DESC.type) { + NullStruct shuffle; + shuffle = new NullStruct(); + shuffle.read(iprot); + return shuffle; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case ALL: + if (field.type == ALL_FIELD_DESC.type) { + NullStruct all; + all = new NullStruct(); + all.read(iprot); + return all; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case NONE: + if (field.type == NONE_FIELD_DESC.type) { + NullStruct none; + none = new NullStruct(); + none.read(iprot); + return none; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case DIRECT: + if (field.type == DIRECT_FIELD_DESC.type) { + NullStruct direct; + direct = new NullStruct(); + direct.read(iprot); + return direct; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case CUSTOM_OBJECT: + if (field.type == CUSTOM_OBJECT_FIELD_DESC.type) { + JavaObject custom_object; + custom_object = new JavaObject(); + custom_object.read(iprot); + return custom_object; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case CUSTOM_SERIALIZED: + if (field.type == CUSTOM_SERIALIZED_FIELD_DESC.type) { + ByteBuffer custom_serialized; + custom_serialized = iprot.readBinary(); + return custom_serialized; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case LOCAL_OR_SHUFFLE: + if (field.type == LOCAL_OR_SHUFFLE_FIELD_DESC.type) { + NullStruct local_or_shuffle; + local_or_shuffle = new NullStruct(); + local_or_shuffle.read(iprot); + return local_or_shuffle; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case LOCAL_FIRST: + if (field.type == LOCAL_FIRST_FIELD_DESC.type) { + NullStruct localFirst; + localFirst = new NullStruct(); + localFirst.read(iprot); + return localFirst; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + } + + @Override + protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case FIELDS: + List fields = (List)value_; + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, fields.size())); + for (String _iter11 : fields) + { + oprot.writeString(_iter11); + } + oprot.writeListEnd(); + } + return; + case SHUFFLE: + NullStruct shuffle = (NullStruct)value_; + shuffle.write(oprot); + return; + case ALL: + NullStruct all = (NullStruct)value_; + all.write(oprot); + return; + case NONE: + NullStruct none = (NullStruct)value_; + none.write(oprot); + return; + case DIRECT: + NullStruct direct = (NullStruct)value_; + direct.write(oprot); + return; + case CUSTOM_OBJECT: + JavaObject custom_object = (JavaObject)value_; + custom_object.write(oprot); + return; + case CUSTOM_SERIALIZED: + ByteBuffer custom_serialized = (ByteBuffer)value_; + oprot.writeBinary(custom_serialized); + return; + case LOCAL_OR_SHUFFLE: + NullStruct local_or_shuffle = (NullStruct)value_; + local_or_shuffle.write(oprot); + return; + case LOCAL_FIRST: + NullStruct localFirst = (NullStruct)value_; + localFirst.write(oprot); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(fieldID); + if (setField != null) { + switch (setField) { + case FIELDS: + List fields; + { + org.apache.thrift.protocol.TList _list12 = iprot.readListBegin(); + fields = new ArrayList(_list12.size); + String _elem13; + for (int _i14 = 0; _i14 < _list12.size; ++_i14) + { + _elem13 = iprot.readString(); + fields.add(_elem13); + } + iprot.readListEnd(); + } + return fields; + case SHUFFLE: + NullStruct shuffle; + shuffle = new NullStruct(); + shuffle.read(iprot); + return shuffle; + case ALL: + NullStruct all; + all = new NullStruct(); + all.read(iprot); + return all; + case NONE: + NullStruct none; + none = new NullStruct(); + none.read(iprot); + return none; + case DIRECT: + NullStruct direct; + direct = new NullStruct(); + direct.read(iprot); + return direct; + case CUSTOM_OBJECT: + JavaObject custom_object; + custom_object = new JavaObject(); + custom_object.read(iprot); + return custom_object; + case CUSTOM_SERIALIZED: + ByteBuffer custom_serialized; + custom_serialized = iprot.readBinary(); + return custom_serialized; + case LOCAL_OR_SHUFFLE: + NullStruct local_or_shuffle; + local_or_shuffle = new NullStruct(); + local_or_shuffle.read(iprot); + return local_or_shuffle; + case LOCAL_FIRST: + NullStruct localFirst; + localFirst = new NullStruct(); + localFirst.read(iprot); + return localFirst; + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + throw new TProtocolException("Couldn't find a field with field id " + fieldID); + } + } + + @Override + protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case FIELDS: + List fields = (List)value_; + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, fields.size())); + for (String _iter15 : fields) + { + oprot.writeString(_iter15); + } + oprot.writeListEnd(); + } + return; + case SHUFFLE: + NullStruct shuffle = (NullStruct)value_; + shuffle.write(oprot); + return; + case ALL: + NullStruct all = (NullStruct)value_; + all.write(oprot); + return; + case NONE: + NullStruct none = (NullStruct)value_; + none.write(oprot); + return; + case DIRECT: + NullStruct direct = (NullStruct)value_; + direct.write(oprot); + return; + case CUSTOM_OBJECT: + JavaObject custom_object = (JavaObject)value_; + custom_object.write(oprot); + return; + case CUSTOM_SERIALIZED: + ByteBuffer custom_serialized = (ByteBuffer)value_; + oprot.writeBinary(custom_serialized); + return; + case LOCAL_OR_SHUFFLE: + NullStruct local_or_shuffle = (NullStruct)value_; + local_or_shuffle.write(oprot); + return; + case LOCAL_FIRST: + NullStruct localFirst = (NullStruct)value_; + localFirst.write(oprot); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { + switch (setField) { + case FIELDS: + return FIELDS_FIELD_DESC; + case SHUFFLE: + return SHUFFLE_FIELD_DESC; + case ALL: + return ALL_FIELD_DESC; + case NONE: + return NONE_FIELD_DESC; + case DIRECT: + return DIRECT_FIELD_DESC; + case CUSTOM_OBJECT: + return CUSTOM_OBJECT_FIELD_DESC; + case CUSTOM_SERIALIZED: + return CUSTOM_SERIALIZED_FIELD_DESC; + case LOCAL_OR_SHUFFLE: + return LOCAL_OR_SHUFFLE_FIELD_DESC; + case LOCAL_FIRST: + return LOCAL_FIRST_FIELD_DESC; + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected org.apache.thrift.protocol.TStruct getStructDesc() { + return STRUCT_DESC; + } + + @Override + protected _Fields enumForId(short id) { + return _Fields.findByThriftIdOrThrow(id); + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + + public List get_fields() { + if (getSetField() == _Fields.FIELDS) { + return (List)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'fields' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void set_fields(List value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.FIELDS; + value_ = value; + } + + public NullStruct get_shuffle() { + if (getSetField() == _Fields.SHUFFLE) { + return (NullStruct)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'shuffle' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void set_shuffle(NullStruct value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.SHUFFLE; + value_ = value; + } + + public NullStruct get_all() { + if (getSetField() == _Fields.ALL) { + return (NullStruct)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'all' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void set_all(NullStruct value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.ALL; + value_ = value; + } + + public NullStruct get_none() { + if (getSetField() == _Fields.NONE) { + return (NullStruct)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'none' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void set_none(NullStruct value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.NONE; + value_ = value; + } + + public NullStruct get_direct() { + if (getSetField() == _Fields.DIRECT) { + return (NullStruct)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'direct' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void set_direct(NullStruct value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.DIRECT; + value_ = value; + } + + public JavaObject get_custom_object() { + if (getSetField() == _Fields.CUSTOM_OBJECT) { + return (JavaObject)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'custom_object' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void set_custom_object(JavaObject value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.CUSTOM_OBJECT; + value_ = value; + } + + public byte[] get_custom_serialized() { + set_custom_serialized(org.apache.thrift.TBaseHelper.rightSize(buffer_for_custom_serialized())); + ByteBuffer b = buffer_for_custom_serialized(); + return b == null ? null : b.array(); + } + + public ByteBuffer buffer_for_custom_serialized() { + if (getSetField() == _Fields.CUSTOM_SERIALIZED) { + return org.apache.thrift.TBaseHelper.copyBinary((ByteBuffer)getFieldValue()); + } else { + throw new RuntimeException("Cannot get field 'custom_serialized' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void set_custom_serialized(byte[] value) { + set_custom_serialized(ByteBuffer.wrap(Arrays.copyOf(value, value.length))); + } + + public void set_custom_serialized(ByteBuffer value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.CUSTOM_SERIALIZED; + value_ = value; + } + + public NullStruct get_local_or_shuffle() { + if (getSetField() == _Fields.LOCAL_OR_SHUFFLE) { + return (NullStruct)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'local_or_shuffle' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void set_local_or_shuffle(NullStruct value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.LOCAL_OR_SHUFFLE; + value_ = value; + } + + public NullStruct get_localFirst() { + if (getSetField() == _Fields.LOCAL_FIRST) { + return (NullStruct)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'localFirst' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void set_localFirst(NullStruct value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.LOCAL_FIRST; + value_ = value; + } + + public boolean is_set_fields() { + return setField_ == _Fields.FIELDS; + } + + + public boolean is_set_shuffle() { + return setField_ == _Fields.SHUFFLE; + } + + + public boolean is_set_all() { + return setField_ == _Fields.ALL; + } + + + public boolean is_set_none() { + return setField_ == _Fields.NONE; + } + + + public boolean is_set_direct() { + return setField_ == _Fields.DIRECT; + } + + + public boolean is_set_custom_object() { + return setField_ == _Fields.CUSTOM_OBJECT; + } + + + public boolean is_set_custom_serialized() { + return setField_ == _Fields.CUSTOM_SERIALIZED; + } + + + public boolean is_set_local_or_shuffle() { + return setField_ == _Fields.LOCAL_OR_SHUFFLE; + } + + + public boolean is_set_localFirst() { + return setField_ == _Fields.LOCAL_FIRST; + } + + + public boolean equals(Object other) { + if (other instanceof Grouping) { + return equals((Grouping)other); + } else { + return false; + } + } + + public boolean equals(Grouping other) { + return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); + } + + @Override + public int compareTo(Grouping other) { + int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); + if (lastComparison == 0) { + return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); + } + return lastComparison; + } + + + @Override + public int hashCode() { + List list = new ArrayList(); + list.add(this.getClass().getName()); + org.apache.thrift.TFieldIdEnum setField = getSetField(); + if (setField != null) { + list.add(setField.getThriftFieldId()); + Object value = getFieldValue(); + if (value instanceof org.apache.thrift.TEnum) { + list.add(((org.apache.thrift.TEnum)getFieldValue()).getValue()); + } else { + list.add(value); + } + } + return list.hashCode(); + } + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + +} diff --git a/jstorm-core/src/main/java/backtype/storm/generated/InvalidTopologyException.java b/jstorm-core/src/main/java/backtype/storm/generated/InvalidTopologyException.java new file mode 100644 index 000000000..3d5424a2c --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/InvalidTopologyException.java @@ -0,0 +1,389 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class InvalidTopologyException extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("InvalidTopologyException"); + + private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new InvalidTopologyExceptionStandardSchemeFactory()); + schemes.put(TupleScheme.class, new InvalidTopologyExceptionTupleSchemeFactory()); + } + + private String msg; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + MSG((short)1, "msg"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // MSG + return MSG; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(InvalidTopologyException.class, metaDataMap); + } + + public InvalidTopologyException() { + } + + public InvalidTopologyException( + String msg) + { + this(); + this.msg = msg; + } + + /** + * Performs a deep copy on other. + */ + public InvalidTopologyException(InvalidTopologyException other) { + if (other.is_set_msg()) { + this.msg = other.msg; + } + } + + public InvalidTopologyException deepCopy() { + return new InvalidTopologyException(this); + } + + @Override + public void clear() { + this.msg = null; + } + + public String get_msg() { + return this.msg; + } + + public void set_msg(String msg) { + this.msg = msg; + } + + public void unset_msg() { + this.msg = null; + } + + /** Returns true if field msg is set (has been assigned a value) and false otherwise */ + public boolean is_set_msg() { + return this.msg != null; + } + + public void set_msg_isSet(boolean value) { + if (!value) { + this.msg = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case MSG: + if (value == null) { + unset_msg(); + } else { + set_msg((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case MSG: + return get_msg(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case MSG: + return is_set_msg(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof InvalidTopologyException) + return this.equals((InvalidTopologyException)that); + return false; + } + + public boolean equals(InvalidTopologyException that) { + if (that == null) + return false; + + boolean this_present_msg = true && this.is_set_msg(); + boolean that_present_msg = true && that.is_set_msg(); + if (this_present_msg || that_present_msg) { + if (!(this_present_msg && that_present_msg)) + return false; + if (!this.msg.equals(that.msg)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_msg = true && (is_set_msg()); + list.add(present_msg); + if (present_msg) + list.add(msg); + + return list.hashCode(); + } + + @Override + public int compareTo(InvalidTopologyException other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_msg()).compareTo(other.is_set_msg()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_msg()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, other.msg); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("InvalidTopologyException("); + boolean first = true; + + sb.append("msg:"); + if (this.msg == null) { + sb.append("null"); + } else { + sb.append(this.msg); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_msg()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class InvalidTopologyExceptionStandardSchemeFactory implements SchemeFactory { + public InvalidTopologyExceptionStandardScheme getScheme() { + return new InvalidTopologyExceptionStandardScheme(); + } + } + + private static class InvalidTopologyExceptionStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, InvalidTopologyException struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // MSG + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.msg = iprot.readString(); + struct.set_msg_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, InvalidTopologyException struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.msg != null) { + oprot.writeFieldBegin(MSG_FIELD_DESC); + oprot.writeString(struct.msg); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class InvalidTopologyExceptionTupleSchemeFactory implements SchemeFactory { + public InvalidTopologyExceptionTupleScheme getScheme() { + return new InvalidTopologyExceptionTupleScheme(); + } + } + + private static class InvalidTopologyExceptionTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, InvalidTopologyException struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.msg); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, InvalidTopologyException struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.msg = iprot.readString(); + struct.set_msg_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/JavaObject.java b/jstorm-core/src/main/java/backtype/storm/generated/JavaObject.java new file mode 100644 index 000000000..5998993a5 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/JavaObject.java @@ -0,0 +1,544 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class JavaObject implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("JavaObject"); + + private static final org.apache.thrift.protocol.TField FULL_CLASS_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("full_class_name", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField ARGS_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("args_list", org.apache.thrift.protocol.TType.LIST, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new JavaObjectStandardSchemeFactory()); + schemes.put(TupleScheme.class, new JavaObjectTupleSchemeFactory()); + } + + private String full_class_name; // required + private List args_list; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + FULL_CLASS_NAME((short)1, "full_class_name"), + ARGS_LIST((short)2, "args_list"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // FULL_CLASS_NAME + return FULL_CLASS_NAME; + case 2: // ARGS_LIST + return ARGS_LIST; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.FULL_CLASS_NAME, new org.apache.thrift.meta_data.FieldMetaData("full_class_name", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.ARGS_LIST, new org.apache.thrift.meta_data.FieldMetaData("args_list", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, JavaObjectArg.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(JavaObject.class, metaDataMap); + } + + public JavaObject() { + } + + public JavaObject( + String full_class_name, + List args_list) + { + this(); + this.full_class_name = full_class_name; + this.args_list = args_list; + } + + /** + * Performs a deep copy on other. + */ + public JavaObject(JavaObject other) { + if (other.is_set_full_class_name()) { + this.full_class_name = other.full_class_name; + } + if (other.is_set_args_list()) { + List __this__args_list = new ArrayList(other.args_list.size()); + for (JavaObjectArg other_element : other.args_list) { + __this__args_list.add(new JavaObjectArg(other_element)); + } + this.args_list = __this__args_list; + } + } + + public JavaObject deepCopy() { + return new JavaObject(this); + } + + @Override + public void clear() { + this.full_class_name = null; + this.args_list = null; + } + + public String get_full_class_name() { + return this.full_class_name; + } + + public void set_full_class_name(String full_class_name) { + this.full_class_name = full_class_name; + } + + public void unset_full_class_name() { + this.full_class_name = null; + } + + /** Returns true if field full_class_name is set (has been assigned a value) and false otherwise */ + public boolean is_set_full_class_name() { + return this.full_class_name != null; + } + + public void set_full_class_name_isSet(boolean value) { + if (!value) { + this.full_class_name = null; + } + } + + public int get_args_list_size() { + return (this.args_list == null) ? 0 : this.args_list.size(); + } + + public java.util.Iterator get_args_list_iterator() { + return (this.args_list == null) ? null : this.args_list.iterator(); + } + + public void add_to_args_list(JavaObjectArg elem) { + if (this.args_list == null) { + this.args_list = new ArrayList(); + } + this.args_list.add(elem); + } + + public List get_args_list() { + return this.args_list; + } + + public void set_args_list(List args_list) { + this.args_list = args_list; + } + + public void unset_args_list() { + this.args_list = null; + } + + /** Returns true if field args_list is set (has been assigned a value) and false otherwise */ + public boolean is_set_args_list() { + return this.args_list != null; + } + + public void set_args_list_isSet(boolean value) { + if (!value) { + this.args_list = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case FULL_CLASS_NAME: + if (value == null) { + unset_full_class_name(); + } else { + set_full_class_name((String)value); + } + break; + + case ARGS_LIST: + if (value == null) { + unset_args_list(); + } else { + set_args_list((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case FULL_CLASS_NAME: + return get_full_class_name(); + + case ARGS_LIST: + return get_args_list(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case FULL_CLASS_NAME: + return is_set_full_class_name(); + case ARGS_LIST: + return is_set_args_list(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof JavaObject) + return this.equals((JavaObject)that); + return false; + } + + public boolean equals(JavaObject that) { + if (that == null) + return false; + + boolean this_present_full_class_name = true && this.is_set_full_class_name(); + boolean that_present_full_class_name = true && that.is_set_full_class_name(); + if (this_present_full_class_name || that_present_full_class_name) { + if (!(this_present_full_class_name && that_present_full_class_name)) + return false; + if (!this.full_class_name.equals(that.full_class_name)) + return false; + } + + boolean this_present_args_list = true && this.is_set_args_list(); + boolean that_present_args_list = true && that.is_set_args_list(); + if (this_present_args_list || that_present_args_list) { + if (!(this_present_args_list && that_present_args_list)) + return false; + if (!this.args_list.equals(that.args_list)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_full_class_name = true && (is_set_full_class_name()); + list.add(present_full_class_name); + if (present_full_class_name) + list.add(full_class_name); + + boolean present_args_list = true && (is_set_args_list()); + list.add(present_args_list); + if (present_args_list) + list.add(args_list); + + return list.hashCode(); + } + + @Override + public int compareTo(JavaObject other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_full_class_name()).compareTo(other.is_set_full_class_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_full_class_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.full_class_name, other.full_class_name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_args_list()).compareTo(other.is_set_args_list()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_args_list()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.args_list, other.args_list); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("JavaObject("); + boolean first = true; + + sb.append("full_class_name:"); + if (this.full_class_name == null) { + sb.append("null"); + } else { + sb.append(this.full_class_name); + } + first = false; + if (!first) sb.append(", "); + sb.append("args_list:"); + if (this.args_list == null) { + sb.append("null"); + } else { + sb.append(this.args_list); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_full_class_name()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'full_class_name' is unset! Struct:" + toString()); + } + + if (!is_set_args_list()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'args_list' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class JavaObjectStandardSchemeFactory implements SchemeFactory { + public JavaObjectStandardScheme getScheme() { + return new JavaObjectStandardScheme(); + } + } + + private static class JavaObjectStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, JavaObject struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // FULL_CLASS_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.full_class_name = iprot.readString(); + struct.set_full_class_name_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // ARGS_LIST + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list0 = iprot.readListBegin(); + struct.args_list = new ArrayList(_list0.size); + JavaObjectArg _elem1; + for (int _i2 = 0; _i2 < _list0.size; ++_i2) + { + _elem1 = new JavaObjectArg(); + _elem1.read(iprot); + struct.args_list.add(_elem1); + } + iprot.readListEnd(); + } + struct.set_args_list_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, JavaObject struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.full_class_name != null) { + oprot.writeFieldBegin(FULL_CLASS_NAME_FIELD_DESC); + oprot.writeString(struct.full_class_name); + oprot.writeFieldEnd(); + } + if (struct.args_list != null) { + oprot.writeFieldBegin(ARGS_LIST_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.args_list.size())); + for (JavaObjectArg _iter3 : struct.args_list) + { + _iter3.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class JavaObjectTupleSchemeFactory implements SchemeFactory { + public JavaObjectTupleScheme getScheme() { + return new JavaObjectTupleScheme(); + } + } + + private static class JavaObjectTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, JavaObject struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.full_class_name); + { + oprot.writeI32(struct.args_list.size()); + for (JavaObjectArg _iter4 : struct.args_list) + { + _iter4.write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, JavaObject struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.full_class_name = iprot.readString(); + struct.set_full_class_name_isSet(true); + { + org.apache.thrift.protocol.TList _list5 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.args_list = new ArrayList(_list5.size); + JavaObjectArg _elem6; + for (int _i7 = 0; _i7 < _list5.size; ++_i7) + { + _elem6 = new JavaObjectArg(); + _elem6.read(iprot); + struct.args_list.add(_elem6); + } + } + struct.set_args_list_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/JavaObjectArg.java b/jstorm-core/src/main/java/backtype/storm/generated/JavaObjectArg.java new file mode 100755 index 000000000..446930688 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/JavaObjectArg.java @@ -0,0 +1,614 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +public class JavaObjectArg extends org.apache.thrift.TUnion { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("JavaObjectArg"); + private static final org.apache.thrift.protocol.TField INT_ARG_FIELD_DESC = new org.apache.thrift.protocol.TField("int_arg", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField LONG_ARG_FIELD_DESC = new org.apache.thrift.protocol.TField("long_arg", org.apache.thrift.protocol.TType.I64, (short)2); + private static final org.apache.thrift.protocol.TField STRING_ARG_FIELD_DESC = new org.apache.thrift.protocol.TField("string_arg", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField BOOL_ARG_FIELD_DESC = new org.apache.thrift.protocol.TField("bool_arg", org.apache.thrift.protocol.TType.BOOL, (short)4); + private static final org.apache.thrift.protocol.TField BINARY_ARG_FIELD_DESC = new org.apache.thrift.protocol.TField("binary_arg", org.apache.thrift.protocol.TType.STRING, (short)5); + private static final org.apache.thrift.protocol.TField DOUBLE_ARG_FIELD_DESC = new org.apache.thrift.protocol.TField("double_arg", org.apache.thrift.protocol.TType.DOUBLE, (short)6); + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + INT_ARG((short)1, "int_arg"), + LONG_ARG((short)2, "long_arg"), + STRING_ARG((short)3, "string_arg"), + BOOL_ARG((short)4, "bool_arg"), + BINARY_ARG((short)5, "binary_arg"), + DOUBLE_ARG((short)6, "double_arg"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // INT_ARG + return INT_ARG; + case 2: // LONG_ARG + return LONG_ARG; + case 3: // STRING_ARG + return STRING_ARG; + case 4: // BOOL_ARG + return BOOL_ARG; + case 5: // BINARY_ARG + return BINARY_ARG; + case 6: // DOUBLE_ARG + return DOUBLE_ARG; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.INT_ARG, new org.apache.thrift.meta_data.FieldMetaData("int_arg", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.LONG_ARG, new org.apache.thrift.meta_data.FieldMetaData("long_arg", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.STRING_ARG, new org.apache.thrift.meta_data.FieldMetaData("string_arg", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.BOOL_ARG, new org.apache.thrift.meta_data.FieldMetaData("bool_arg", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + tmpMap.put(_Fields.BINARY_ARG, new org.apache.thrift.meta_data.FieldMetaData("binary_arg", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + tmpMap.put(_Fields.DOUBLE_ARG, new org.apache.thrift.meta_data.FieldMetaData("double_arg", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(JavaObjectArg.class, metaDataMap); + } + + public JavaObjectArg() { + super(); + } + + public JavaObjectArg(_Fields setField, Object value) { + super(setField, value); + } + + public JavaObjectArg(JavaObjectArg other) { + super(other); + } + public JavaObjectArg deepCopy() { + return new JavaObjectArg(this); + } + + public static JavaObjectArg int_arg(int value) { + JavaObjectArg x = new JavaObjectArg(); + x.set_int_arg(value); + return x; + } + + public static JavaObjectArg long_arg(long value) { + JavaObjectArg x = new JavaObjectArg(); + x.set_long_arg(value); + return x; + } + + public static JavaObjectArg string_arg(String value) { + JavaObjectArg x = new JavaObjectArg(); + x.set_string_arg(value); + return x; + } + + public static JavaObjectArg bool_arg(boolean value) { + JavaObjectArg x = new JavaObjectArg(); + x.set_bool_arg(value); + return x; + } + + public static JavaObjectArg binary_arg(ByteBuffer value) { + JavaObjectArg x = new JavaObjectArg(); + x.set_binary_arg(value); + return x; + } + + public static JavaObjectArg binary_arg(byte[] value) { + JavaObjectArg x = new JavaObjectArg(); + x.set_binary_arg(ByteBuffer.wrap(Arrays.copyOf(value, value.length))); + return x; + } + + public static JavaObjectArg double_arg(double value) { + JavaObjectArg x = new JavaObjectArg(); + x.set_double_arg(value); + return x; + } + + + @Override + protected void checkType(_Fields setField, Object value) throws ClassCastException { + switch (setField) { + case INT_ARG: + if (value instanceof Integer) { + break; + } + throw new ClassCastException("Was expecting value of type Integer for field 'int_arg', but got " + value.getClass().getSimpleName()); + case LONG_ARG: + if (value instanceof Long) { + break; + } + throw new ClassCastException("Was expecting value of type Long for field 'long_arg', but got " + value.getClass().getSimpleName()); + case STRING_ARG: + if (value instanceof String) { + break; + } + throw new ClassCastException("Was expecting value of type String for field 'string_arg', but got " + value.getClass().getSimpleName()); + case BOOL_ARG: + if (value instanceof Boolean) { + break; + } + throw new ClassCastException("Was expecting value of type Boolean for field 'bool_arg', but got " + value.getClass().getSimpleName()); + case BINARY_ARG: + if (value instanceof ByteBuffer) { + break; + } + throw new ClassCastException("Was expecting value of type ByteBuffer for field 'binary_arg', but got " + value.getClass().getSimpleName()); + case DOUBLE_ARG: + if (value instanceof Double) { + break; + } + throw new ClassCastException("Was expecting value of type Double for field 'double_arg', but got " + value.getClass().getSimpleName()); + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(field.id); + if (setField != null) { + switch (setField) { + case INT_ARG: + if (field.type == INT_ARG_FIELD_DESC.type) { + Integer int_arg; + int_arg = iprot.readI32(); + return int_arg; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case LONG_ARG: + if (field.type == LONG_ARG_FIELD_DESC.type) { + Long long_arg; + long_arg = iprot.readI64(); + return long_arg; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case STRING_ARG: + if (field.type == STRING_ARG_FIELD_DESC.type) { + String string_arg; + string_arg = iprot.readString(); + return string_arg; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case BOOL_ARG: + if (field.type == BOOL_ARG_FIELD_DESC.type) { + Boolean bool_arg; + bool_arg = iprot.readBool(); + return bool_arg; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case BINARY_ARG: + if (field.type == BINARY_ARG_FIELD_DESC.type) { + ByteBuffer binary_arg; + binary_arg = iprot.readBinary(); + return binary_arg; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case DOUBLE_ARG: + if (field.type == DOUBLE_ARG_FIELD_DESC.type) { + Double double_arg; + double_arg = iprot.readDouble(); + return double_arg; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + } + + @Override + protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case INT_ARG: + Integer int_arg = (Integer)value_; + oprot.writeI32(int_arg); + return; + case LONG_ARG: + Long long_arg = (Long)value_; + oprot.writeI64(long_arg); + return; + case STRING_ARG: + String string_arg = (String)value_; + oprot.writeString(string_arg); + return; + case BOOL_ARG: + Boolean bool_arg = (Boolean)value_; + oprot.writeBool(bool_arg); + return; + case BINARY_ARG: + ByteBuffer binary_arg = (ByteBuffer)value_; + oprot.writeBinary(binary_arg); + return; + case DOUBLE_ARG: + Double double_arg = (Double)value_; + oprot.writeDouble(double_arg); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(fieldID); + if (setField != null) { + switch (setField) { + case INT_ARG: + Integer int_arg; + int_arg = iprot.readI32(); + return int_arg; + case LONG_ARG: + Long long_arg; + long_arg = iprot.readI64(); + return long_arg; + case STRING_ARG: + String string_arg; + string_arg = iprot.readString(); + return string_arg; + case BOOL_ARG: + Boolean bool_arg; + bool_arg = iprot.readBool(); + return bool_arg; + case BINARY_ARG: + ByteBuffer binary_arg; + binary_arg = iprot.readBinary(); + return binary_arg; + case DOUBLE_ARG: + Double double_arg; + double_arg = iprot.readDouble(); + return double_arg; + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + throw new TProtocolException("Couldn't find a field with field id " + fieldID); + } + } + + @Override + protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case INT_ARG: + Integer int_arg = (Integer)value_; + oprot.writeI32(int_arg); + return; + case LONG_ARG: + Long long_arg = (Long)value_; + oprot.writeI64(long_arg); + return; + case STRING_ARG: + String string_arg = (String)value_; + oprot.writeString(string_arg); + return; + case BOOL_ARG: + Boolean bool_arg = (Boolean)value_; + oprot.writeBool(bool_arg); + return; + case BINARY_ARG: + ByteBuffer binary_arg = (ByteBuffer)value_; + oprot.writeBinary(binary_arg); + return; + case DOUBLE_ARG: + Double double_arg = (Double)value_; + oprot.writeDouble(double_arg); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { + switch (setField) { + case INT_ARG: + return INT_ARG_FIELD_DESC; + case LONG_ARG: + return LONG_ARG_FIELD_DESC; + case STRING_ARG: + return STRING_ARG_FIELD_DESC; + case BOOL_ARG: + return BOOL_ARG_FIELD_DESC; + case BINARY_ARG: + return BINARY_ARG_FIELD_DESC; + case DOUBLE_ARG: + return DOUBLE_ARG_FIELD_DESC; + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected org.apache.thrift.protocol.TStruct getStructDesc() { + return STRUCT_DESC; + } + + @Override + protected _Fields enumForId(short id) { + return _Fields.findByThriftIdOrThrow(id); + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + + public int get_int_arg() { + if (getSetField() == _Fields.INT_ARG) { + return (Integer)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'int_arg' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void set_int_arg(int value) { + setField_ = _Fields.INT_ARG; + value_ = value; + } + + public long get_long_arg() { + if (getSetField() == _Fields.LONG_ARG) { + return (Long)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'long_arg' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void set_long_arg(long value) { + setField_ = _Fields.LONG_ARG; + value_ = value; + } + + public String get_string_arg() { + if (getSetField() == _Fields.STRING_ARG) { + return (String)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'string_arg' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void set_string_arg(String value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.STRING_ARG; + value_ = value; + } + + public boolean get_bool_arg() { + if (getSetField() == _Fields.BOOL_ARG) { + return (Boolean)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'bool_arg' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void set_bool_arg(boolean value) { + setField_ = _Fields.BOOL_ARG; + value_ = value; + } + + public byte[] get_binary_arg() { + set_binary_arg(org.apache.thrift.TBaseHelper.rightSize(buffer_for_binary_arg())); + ByteBuffer b = buffer_for_binary_arg(); + return b == null ? null : b.array(); + } + + public ByteBuffer buffer_for_binary_arg() { + if (getSetField() == _Fields.BINARY_ARG) { + return org.apache.thrift.TBaseHelper.copyBinary((ByteBuffer)getFieldValue()); + } else { + throw new RuntimeException("Cannot get field 'binary_arg' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void set_binary_arg(byte[] value) { + set_binary_arg(ByteBuffer.wrap(Arrays.copyOf(value, value.length))); + } + + public void set_binary_arg(ByteBuffer value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.BINARY_ARG; + value_ = value; + } + + public double get_double_arg() { + if (getSetField() == _Fields.DOUBLE_ARG) { + return (Double)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'double_arg' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void set_double_arg(double value) { + setField_ = _Fields.DOUBLE_ARG; + value_ = value; + } + + public boolean is_set_int_arg() { + return setField_ == _Fields.INT_ARG; + } + + + public boolean is_set_long_arg() { + return setField_ == _Fields.LONG_ARG; + } + + + public boolean is_set_string_arg() { + return setField_ == _Fields.STRING_ARG; + } + + + public boolean is_set_bool_arg() { + return setField_ == _Fields.BOOL_ARG; + } + + + public boolean is_set_binary_arg() { + return setField_ == _Fields.BINARY_ARG; + } + + + public boolean is_set_double_arg() { + return setField_ == _Fields.DOUBLE_ARG; + } + + + public boolean equals(Object other) { + if (other instanceof JavaObjectArg) { + return equals((JavaObjectArg)other); + } else { + return false; + } + } + + public boolean equals(JavaObjectArg other) { + return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); + } + + @Override + public int compareTo(JavaObjectArg other) { + int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); + if (lastComparison == 0) { + return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); + } + return lastComparison; + } + + + @Override + public int hashCode() { + List list = new ArrayList(); + list.add(this.getClass().getName()); + org.apache.thrift.TFieldIdEnum setField = getSetField(); + if (setField != null) { + list.add(setField.getThriftFieldId()); + Object value = getFieldValue(); + if (value instanceof org.apache.thrift.TEnum) { + list.add(((org.apache.thrift.TEnum)getFieldValue()).getValue()); + } else { + list.add(value); + } + } + return list.hashCode(); + } + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + +} diff --git a/jstorm-core/src/main/java/backtype/storm/generated/KillOptions.java b/jstorm-core/src/main/java/backtype/storm/generated/KillOptions.java new file mode 100644 index 000000000..7abb762fd --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/KillOptions.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class KillOptions implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("KillOptions"); + + private static final org.apache.thrift.protocol.TField WAIT_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("wait_secs", org.apache.thrift.protocol.TType.I32, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new KillOptionsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new KillOptionsTupleSchemeFactory()); + } + + private int wait_secs; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + WAIT_SECS((short)1, "wait_secs"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // WAIT_SECS + return WAIT_SECS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __WAIT_SECS_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.WAIT_SECS}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.WAIT_SECS, new org.apache.thrift.meta_data.FieldMetaData("wait_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(KillOptions.class, metaDataMap); + } + + public KillOptions() { + } + + /** + * Performs a deep copy on other. + */ + public KillOptions(KillOptions other) { + __isset_bitfield = other.__isset_bitfield; + this.wait_secs = other.wait_secs; + } + + public KillOptions deepCopy() { + return new KillOptions(this); + } + + @Override + public void clear() { + set_wait_secs_isSet(false); + this.wait_secs = 0; + } + + public int get_wait_secs() { + return this.wait_secs; + } + + public void set_wait_secs(int wait_secs) { + this.wait_secs = wait_secs; + set_wait_secs_isSet(true); + } + + public void unset_wait_secs() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __WAIT_SECS_ISSET_ID); + } + + /** Returns true if field wait_secs is set (has been assigned a value) and false otherwise */ + public boolean is_set_wait_secs() { + return EncodingUtils.testBit(__isset_bitfield, __WAIT_SECS_ISSET_ID); + } + + public void set_wait_secs_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __WAIT_SECS_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case WAIT_SECS: + if (value == null) { + unset_wait_secs(); + } else { + set_wait_secs((Integer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case WAIT_SECS: + return Integer.valueOf(get_wait_secs()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case WAIT_SECS: + return is_set_wait_secs(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof KillOptions) + return this.equals((KillOptions)that); + return false; + } + + public boolean equals(KillOptions that) { + if (that == null) + return false; + + boolean this_present_wait_secs = true && this.is_set_wait_secs(); + boolean that_present_wait_secs = true && that.is_set_wait_secs(); + if (this_present_wait_secs || that_present_wait_secs) { + if (!(this_present_wait_secs && that_present_wait_secs)) + return false; + if (this.wait_secs != that.wait_secs) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_wait_secs = true && (is_set_wait_secs()); + list.add(present_wait_secs); + if (present_wait_secs) + list.add(wait_secs); + + return list.hashCode(); + } + + @Override + public int compareTo(KillOptions other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_wait_secs()).compareTo(other.is_set_wait_secs()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_wait_secs()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.wait_secs, other.wait_secs); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("KillOptions("); + boolean first = true; + + if (is_set_wait_secs()) { + sb.append("wait_secs:"); + sb.append(this.wait_secs); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class KillOptionsStandardSchemeFactory implements SchemeFactory { + public KillOptionsStandardScheme getScheme() { + return new KillOptionsStandardScheme(); + } + } + + private static class KillOptionsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, KillOptions struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // WAIT_SECS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.wait_secs = iprot.readI32(); + struct.set_wait_secs_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, KillOptions struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.is_set_wait_secs()) { + oprot.writeFieldBegin(WAIT_SECS_FIELD_DESC); + oprot.writeI32(struct.wait_secs); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class KillOptionsTupleSchemeFactory implements SchemeFactory { + public KillOptionsTupleScheme getScheme() { + return new KillOptionsTupleScheme(); + } + } + + private static class KillOptionsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, KillOptions struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_wait_secs()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_wait_secs()) { + oprot.writeI32(struct.wait_secs); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, KillOptions struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.wait_secs = iprot.readI32(); + struct.set_wait_secs_isSet(true); + } + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/LocalStateData.java b/jstorm-core/src/main/java/backtype/storm/generated/LocalStateData.java new file mode 100644 index 000000000..2fd49b48b --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/LocalStateData.java @@ -0,0 +1,454 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class LocalStateData implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("LocalStateData"); + + private static final org.apache.thrift.protocol.TField SERIALIZED_PARTS_FIELD_DESC = new org.apache.thrift.protocol.TField("serialized_parts", org.apache.thrift.protocol.TType.MAP, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new LocalStateDataStandardSchemeFactory()); + schemes.put(TupleScheme.class, new LocalStateDataTupleSchemeFactory()); + } + + private Map serialized_parts; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SERIALIZED_PARTS((short)1, "serialized_parts"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SERIALIZED_PARTS + return SERIALIZED_PARTS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SERIALIZED_PARTS, new org.apache.thrift.meta_data.FieldMetaData("serialized_parts", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ThriftSerializedObject.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LocalStateData.class, metaDataMap); + } + + public LocalStateData() { + } + + public LocalStateData( + Map serialized_parts) + { + this(); + this.serialized_parts = serialized_parts; + } + + /** + * Performs a deep copy on other. + */ + public LocalStateData(LocalStateData other) { + if (other.is_set_serialized_parts()) { + Map __this__serialized_parts = new HashMap(other.serialized_parts.size()); + for (Map.Entry other_element : other.serialized_parts.entrySet()) { + + String other_element_key = other_element.getKey(); + ThriftSerializedObject other_element_value = other_element.getValue(); + + String __this__serialized_parts_copy_key = other_element_key; + + ThriftSerializedObject __this__serialized_parts_copy_value = new ThriftSerializedObject(other_element_value); + + __this__serialized_parts.put(__this__serialized_parts_copy_key, __this__serialized_parts_copy_value); + } + this.serialized_parts = __this__serialized_parts; + } + } + + public LocalStateData deepCopy() { + return new LocalStateData(this); + } + + @Override + public void clear() { + this.serialized_parts = null; + } + + public int get_serialized_parts_size() { + return (this.serialized_parts == null) ? 0 : this.serialized_parts.size(); + } + + public void put_to_serialized_parts(String key, ThriftSerializedObject val) { + if (this.serialized_parts == null) { + this.serialized_parts = new HashMap(); + } + this.serialized_parts.put(key, val); + } + + public Map get_serialized_parts() { + return this.serialized_parts; + } + + public void set_serialized_parts(Map serialized_parts) { + this.serialized_parts = serialized_parts; + } + + public void unset_serialized_parts() { + this.serialized_parts = null; + } + + /** Returns true if field serialized_parts is set (has been assigned a value) and false otherwise */ + public boolean is_set_serialized_parts() { + return this.serialized_parts != null; + } + + public void set_serialized_parts_isSet(boolean value) { + if (!value) { + this.serialized_parts = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SERIALIZED_PARTS: + if (value == null) { + unset_serialized_parts(); + } else { + set_serialized_parts((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SERIALIZED_PARTS: + return get_serialized_parts(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SERIALIZED_PARTS: + return is_set_serialized_parts(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof LocalStateData) + return this.equals((LocalStateData)that); + return false; + } + + public boolean equals(LocalStateData that) { + if (that == null) + return false; + + boolean this_present_serialized_parts = true && this.is_set_serialized_parts(); + boolean that_present_serialized_parts = true && that.is_set_serialized_parts(); + if (this_present_serialized_parts || that_present_serialized_parts) { + if (!(this_present_serialized_parts && that_present_serialized_parts)) + return false; + if (!this.serialized_parts.equals(that.serialized_parts)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_serialized_parts = true && (is_set_serialized_parts()); + list.add(present_serialized_parts); + if (present_serialized_parts) + list.add(serialized_parts); + + return list.hashCode(); + } + + @Override + public int compareTo(LocalStateData other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_serialized_parts()).compareTo(other.is_set_serialized_parts()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_serialized_parts()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.serialized_parts, other.serialized_parts); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("LocalStateData("); + boolean first = true; + + sb.append("serialized_parts:"); + if (this.serialized_parts == null) { + sb.append("null"); + } else { + sb.append(this.serialized_parts); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_serialized_parts()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'serialized_parts' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class LocalStateDataStandardSchemeFactory implements SchemeFactory { + public LocalStateDataStandardScheme getScheme() { + return new LocalStateDataStandardScheme(); + } + } + + private static class LocalStateDataStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, LocalStateData struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SERIALIZED_PARTS + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map264 = iprot.readMapBegin(); + struct.serialized_parts = new HashMap(2*_map264.size); + String _key265; + ThriftSerializedObject _val266; + for (int _i267 = 0; _i267 < _map264.size; ++_i267) + { + _key265 = iprot.readString(); + _val266 = new ThriftSerializedObject(); + _val266.read(iprot); + struct.serialized_parts.put(_key265, _val266); + } + iprot.readMapEnd(); + } + struct.set_serialized_parts_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, LocalStateData struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.serialized_parts != null) { + oprot.writeFieldBegin(SERIALIZED_PARTS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.serialized_parts.size())); + for (Map.Entry _iter268 : struct.serialized_parts.entrySet()) + { + oprot.writeString(_iter268.getKey()); + _iter268.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class LocalStateDataTupleSchemeFactory implements SchemeFactory { + public LocalStateDataTupleScheme getScheme() { + return new LocalStateDataTupleScheme(); + } + } + + private static class LocalStateDataTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, LocalStateData struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.serialized_parts.size()); + for (Map.Entry _iter269 : struct.serialized_parts.entrySet()) + { + oprot.writeString(_iter269.getKey()); + _iter269.getValue().write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, LocalStateData struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TMap _map270 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.serialized_parts = new HashMap(2*_map270.size); + String _key271; + ThriftSerializedObject _val272; + for (int _i273 = 0; _i273 < _map270.size; ++_i273) + { + _key271 = iprot.readString(); + _val272 = new ThriftSerializedObject(); + _val272.read(iprot); + struct.serialized_parts.put(_key271, _val272); + } + } + struct.set_serialized_parts_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/MetricInfo.java b/jstorm-core/src/main/java/backtype/storm/generated/MetricInfo.java new file mode 100644 index 000000000..2703777ad --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/MetricInfo.java @@ -0,0 +1,902 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class MetricInfo implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("MetricInfo"); + + private static final org.apache.thrift.protocol.TField BASE_METRIC_FIELD_DESC = new org.apache.thrift.protocol.TField("baseMetric", org.apache.thrift.protocol.TType.MAP, (short)1); + private static final org.apache.thrift.protocol.TField INPUT_METRIC_FIELD_DESC = new org.apache.thrift.protocol.TField("inputMetric", org.apache.thrift.protocol.TType.MAP, (short)2); + private static final org.apache.thrift.protocol.TField OUTPUT_METRIC_FIELD_DESC = new org.apache.thrift.protocol.TField("outputMetric", org.apache.thrift.protocol.TType.MAP, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new MetricInfoStandardSchemeFactory()); + schemes.put(TupleScheme.class, new MetricInfoTupleSchemeFactory()); + } + + private Map baseMetric; // required + private Map> inputMetric; // optional + private Map> outputMetric; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + BASE_METRIC((short)1, "baseMetric"), + INPUT_METRIC((short)2, "inputMetric"), + OUTPUT_METRIC((short)3, "outputMetric"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // BASE_METRIC + return BASE_METRIC; + case 2: // INPUT_METRIC + return INPUT_METRIC; + case 3: // OUTPUT_METRIC + return OUTPUT_METRIC; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.INPUT_METRIC,_Fields.OUTPUT_METRIC}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.BASE_METRIC, new org.apache.thrift.meta_data.FieldMetaData("baseMetric", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, MetricWindow.class)))); + tmpMap.put(_Fields.INPUT_METRIC, new org.apache.thrift.meta_data.FieldMetaData("inputMetric", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, MetricWindow.class))))); + tmpMap.put(_Fields.OUTPUT_METRIC, new org.apache.thrift.meta_data.FieldMetaData("outputMetric", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, MetricWindow.class))))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(MetricInfo.class, metaDataMap); + } + + public MetricInfo() { + } + + public MetricInfo( + Map baseMetric) + { + this(); + this.baseMetric = baseMetric; + } + + /** + * Performs a deep copy on other. + */ + public MetricInfo(MetricInfo other) { + if (other.is_set_baseMetric()) { + Map __this__baseMetric = new HashMap(other.baseMetric.size()); + for (Map.Entry other_element : other.baseMetric.entrySet()) { + + String other_element_key = other_element.getKey(); + MetricWindow other_element_value = other_element.getValue(); + + String __this__baseMetric_copy_key = other_element_key; + + MetricWindow __this__baseMetric_copy_value = new MetricWindow(other_element_value); + + __this__baseMetric.put(__this__baseMetric_copy_key, __this__baseMetric_copy_value); + } + this.baseMetric = __this__baseMetric; + } + if (other.is_set_inputMetric()) { + Map> __this__inputMetric = new HashMap>(other.inputMetric.size()); + for (Map.Entry> other_element : other.inputMetric.entrySet()) { + + String other_element_key = other_element.getKey(); + Map other_element_value = other_element.getValue(); + + String __this__inputMetric_copy_key = other_element_key; + + Map __this__inputMetric_copy_value = new HashMap(other_element_value.size()); + for (Map.Entry other_element_value_element : other_element_value.entrySet()) { + + String other_element_value_element_key = other_element_value_element.getKey(); + MetricWindow other_element_value_element_value = other_element_value_element.getValue(); + + String __this__inputMetric_copy_value_copy_key = other_element_value_element_key; + + MetricWindow __this__inputMetric_copy_value_copy_value = new MetricWindow(other_element_value_element_value); + + __this__inputMetric_copy_value.put(__this__inputMetric_copy_value_copy_key, __this__inputMetric_copy_value_copy_value); + } + + __this__inputMetric.put(__this__inputMetric_copy_key, __this__inputMetric_copy_value); + } + this.inputMetric = __this__inputMetric; + } + if (other.is_set_outputMetric()) { + Map> __this__outputMetric = new HashMap>(other.outputMetric.size()); + for (Map.Entry> other_element : other.outputMetric.entrySet()) { + + String other_element_key = other_element.getKey(); + Map other_element_value = other_element.getValue(); + + String __this__outputMetric_copy_key = other_element_key; + + Map __this__outputMetric_copy_value = new HashMap(other_element_value.size()); + for (Map.Entry other_element_value_element : other_element_value.entrySet()) { + + String other_element_value_element_key = other_element_value_element.getKey(); + MetricWindow other_element_value_element_value = other_element_value_element.getValue(); + + String __this__outputMetric_copy_value_copy_key = other_element_value_element_key; + + MetricWindow __this__outputMetric_copy_value_copy_value = new MetricWindow(other_element_value_element_value); + + __this__outputMetric_copy_value.put(__this__outputMetric_copy_value_copy_key, __this__outputMetric_copy_value_copy_value); + } + + __this__outputMetric.put(__this__outputMetric_copy_key, __this__outputMetric_copy_value); + } + this.outputMetric = __this__outputMetric; + } + } + + public MetricInfo deepCopy() { + return new MetricInfo(this); + } + + @Override + public void clear() { + this.baseMetric = null; + this.inputMetric = null; + this.outputMetric = null; + } + + public int get_baseMetric_size() { + return (this.baseMetric == null) ? 0 : this.baseMetric.size(); + } + + public void put_to_baseMetric(String key, MetricWindow val) { + if (this.baseMetric == null) { + this.baseMetric = new HashMap(); + } + this.baseMetric.put(key, val); + } + + public Map get_baseMetric() { + return this.baseMetric; + } + + public void set_baseMetric(Map baseMetric) { + this.baseMetric = baseMetric; + } + + public void unset_baseMetric() { + this.baseMetric = null; + } + + /** Returns true if field baseMetric is set (has been assigned a value) and false otherwise */ + public boolean is_set_baseMetric() { + return this.baseMetric != null; + } + + public void set_baseMetric_isSet(boolean value) { + if (!value) { + this.baseMetric = null; + } + } + + public int get_inputMetric_size() { + return (this.inputMetric == null) ? 0 : this.inputMetric.size(); + } + + public void put_to_inputMetric(String key, Map val) { + if (this.inputMetric == null) { + this.inputMetric = new HashMap>(); + } + this.inputMetric.put(key, val); + } + + public Map> get_inputMetric() { + return this.inputMetric; + } + + public void set_inputMetric(Map> inputMetric) { + this.inputMetric = inputMetric; + } + + public void unset_inputMetric() { + this.inputMetric = null; + } + + /** Returns true if field inputMetric is set (has been assigned a value) and false otherwise */ + public boolean is_set_inputMetric() { + return this.inputMetric != null; + } + + public void set_inputMetric_isSet(boolean value) { + if (!value) { + this.inputMetric = null; + } + } + + public int get_outputMetric_size() { + return (this.outputMetric == null) ? 0 : this.outputMetric.size(); + } + + public void put_to_outputMetric(String key, Map val) { + if (this.outputMetric == null) { + this.outputMetric = new HashMap>(); + } + this.outputMetric.put(key, val); + } + + public Map> get_outputMetric() { + return this.outputMetric; + } + + public void set_outputMetric(Map> outputMetric) { + this.outputMetric = outputMetric; + } + + public void unset_outputMetric() { + this.outputMetric = null; + } + + /** Returns true if field outputMetric is set (has been assigned a value) and false otherwise */ + public boolean is_set_outputMetric() { + return this.outputMetric != null; + } + + public void set_outputMetric_isSet(boolean value) { + if (!value) { + this.outputMetric = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case BASE_METRIC: + if (value == null) { + unset_baseMetric(); + } else { + set_baseMetric((Map)value); + } + break; + + case INPUT_METRIC: + if (value == null) { + unset_inputMetric(); + } else { + set_inputMetric((Map>)value); + } + break; + + case OUTPUT_METRIC: + if (value == null) { + unset_outputMetric(); + } else { + set_outputMetric((Map>)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case BASE_METRIC: + return get_baseMetric(); + + case INPUT_METRIC: + return get_inputMetric(); + + case OUTPUT_METRIC: + return get_outputMetric(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case BASE_METRIC: + return is_set_baseMetric(); + case INPUT_METRIC: + return is_set_inputMetric(); + case OUTPUT_METRIC: + return is_set_outputMetric(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof MetricInfo) + return this.equals((MetricInfo)that); + return false; + } + + public boolean equals(MetricInfo that) { + if (that == null) + return false; + + boolean this_present_baseMetric = true && this.is_set_baseMetric(); + boolean that_present_baseMetric = true && that.is_set_baseMetric(); + if (this_present_baseMetric || that_present_baseMetric) { + if (!(this_present_baseMetric && that_present_baseMetric)) + return false; + if (!this.baseMetric.equals(that.baseMetric)) + return false; + } + + boolean this_present_inputMetric = true && this.is_set_inputMetric(); + boolean that_present_inputMetric = true && that.is_set_inputMetric(); + if (this_present_inputMetric || that_present_inputMetric) { + if (!(this_present_inputMetric && that_present_inputMetric)) + return false; + if (!this.inputMetric.equals(that.inputMetric)) + return false; + } + + boolean this_present_outputMetric = true && this.is_set_outputMetric(); + boolean that_present_outputMetric = true && that.is_set_outputMetric(); + if (this_present_outputMetric || that_present_outputMetric) { + if (!(this_present_outputMetric && that_present_outputMetric)) + return false; + if (!this.outputMetric.equals(that.outputMetric)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_baseMetric = true && (is_set_baseMetric()); + list.add(present_baseMetric); + if (present_baseMetric) + list.add(baseMetric); + + boolean present_inputMetric = true && (is_set_inputMetric()); + list.add(present_inputMetric); + if (present_inputMetric) + list.add(inputMetric); + + boolean present_outputMetric = true && (is_set_outputMetric()); + list.add(present_outputMetric); + if (present_outputMetric) + list.add(outputMetric); + + return list.hashCode(); + } + + @Override + public int compareTo(MetricInfo other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_baseMetric()).compareTo(other.is_set_baseMetric()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_baseMetric()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.baseMetric, other.baseMetric); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_inputMetric()).compareTo(other.is_set_inputMetric()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_inputMetric()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.inputMetric, other.inputMetric); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_outputMetric()).compareTo(other.is_set_outputMetric()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_outputMetric()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.outputMetric, other.outputMetric); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("MetricInfo("); + boolean first = true; + + sb.append("baseMetric:"); + if (this.baseMetric == null) { + sb.append("null"); + } else { + sb.append(this.baseMetric); + } + first = false; + if (is_set_inputMetric()) { + if (!first) sb.append(", "); + sb.append("inputMetric:"); + if (this.inputMetric == null) { + sb.append("null"); + } else { + sb.append(this.inputMetric); + } + first = false; + } + if (is_set_outputMetric()) { + if (!first) sb.append(", "); + sb.append("outputMetric:"); + if (this.outputMetric == null) { + sb.append("null"); + } else { + sb.append(this.outputMetric); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_baseMetric()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'baseMetric' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class MetricInfoStandardSchemeFactory implements SchemeFactory { + public MetricInfoStandardScheme getScheme() { + return new MetricInfoStandardScheme(); + } + } + + private static class MetricInfoStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, MetricInfo struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // BASE_METRIC + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map116 = iprot.readMapBegin(); + struct.baseMetric = new HashMap(2*_map116.size); + String _key117; + MetricWindow _val118; + for (int _i119 = 0; _i119 < _map116.size; ++_i119) + { + _key117 = iprot.readString(); + _val118 = new MetricWindow(); + _val118.read(iprot); + struct.baseMetric.put(_key117, _val118); + } + iprot.readMapEnd(); + } + struct.set_baseMetric_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // INPUT_METRIC + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map120 = iprot.readMapBegin(); + struct.inputMetric = new HashMap>(2*_map120.size); + String _key121; + Map _val122; + for (int _i123 = 0; _i123 < _map120.size; ++_i123) + { + _key121 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map124 = iprot.readMapBegin(); + _val122 = new HashMap(2*_map124.size); + String _key125; + MetricWindow _val126; + for (int _i127 = 0; _i127 < _map124.size; ++_i127) + { + _key125 = iprot.readString(); + _val126 = new MetricWindow(); + _val126.read(iprot); + _val122.put(_key125, _val126); + } + iprot.readMapEnd(); + } + struct.inputMetric.put(_key121, _val122); + } + iprot.readMapEnd(); + } + struct.set_inputMetric_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // OUTPUT_METRIC + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map128 = iprot.readMapBegin(); + struct.outputMetric = new HashMap>(2*_map128.size); + String _key129; + Map _val130; + for (int _i131 = 0; _i131 < _map128.size; ++_i131) + { + _key129 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map132 = iprot.readMapBegin(); + _val130 = new HashMap(2*_map132.size); + String _key133; + MetricWindow _val134; + for (int _i135 = 0; _i135 < _map132.size; ++_i135) + { + _key133 = iprot.readString(); + _val134 = new MetricWindow(); + _val134.read(iprot); + _val130.put(_key133, _val134); + } + iprot.readMapEnd(); + } + struct.outputMetric.put(_key129, _val130); + } + iprot.readMapEnd(); + } + struct.set_outputMetric_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, MetricInfo struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.baseMetric != null) { + oprot.writeFieldBegin(BASE_METRIC_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.baseMetric.size())); + for (Map.Entry _iter136 : struct.baseMetric.entrySet()) + { + oprot.writeString(_iter136.getKey()); + _iter136.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.inputMetric != null) { + if (struct.is_set_inputMetric()) { + oprot.writeFieldBegin(INPUT_METRIC_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.inputMetric.size())); + for (Map.Entry> _iter137 : struct.inputMetric.entrySet()) + { + oprot.writeString(_iter137.getKey()); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, _iter137.getValue().size())); + for (Map.Entry _iter138 : _iter137.getValue().entrySet()) + { + oprot.writeString(_iter138.getKey()); + _iter138.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + } + if (struct.outputMetric != null) { + if (struct.is_set_outputMetric()) { + oprot.writeFieldBegin(OUTPUT_METRIC_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, struct.outputMetric.size())); + for (Map.Entry> _iter139 : struct.outputMetric.entrySet()) + { + oprot.writeString(_iter139.getKey()); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, _iter139.getValue().size())); + for (Map.Entry _iter140 : _iter139.getValue().entrySet()) + { + oprot.writeString(_iter140.getKey()); + _iter140.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class MetricInfoTupleSchemeFactory implements SchemeFactory { + public MetricInfoTupleScheme getScheme() { + return new MetricInfoTupleScheme(); + } + } + + private static class MetricInfoTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, MetricInfo struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.baseMetric.size()); + for (Map.Entry _iter141 : struct.baseMetric.entrySet()) + { + oprot.writeString(_iter141.getKey()); + _iter141.getValue().write(oprot); + } + } + BitSet optionals = new BitSet(); + if (struct.is_set_inputMetric()) { + optionals.set(0); + } + if (struct.is_set_outputMetric()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_inputMetric()) { + { + oprot.writeI32(struct.inputMetric.size()); + for (Map.Entry> _iter142 : struct.inputMetric.entrySet()) + { + oprot.writeString(_iter142.getKey()); + { + oprot.writeI32(_iter142.getValue().size()); + for (Map.Entry _iter143 : _iter142.getValue().entrySet()) + { + oprot.writeString(_iter143.getKey()); + _iter143.getValue().write(oprot); + } + } + } + } + } + if (struct.is_set_outputMetric()) { + { + oprot.writeI32(struct.outputMetric.size()); + for (Map.Entry> _iter144 : struct.outputMetric.entrySet()) + { + oprot.writeString(_iter144.getKey()); + { + oprot.writeI32(_iter144.getValue().size()); + for (Map.Entry _iter145 : _iter144.getValue().entrySet()) + { + oprot.writeString(_iter145.getKey()); + _iter145.getValue().write(oprot); + } + } + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, MetricInfo struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TMap _map146 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.baseMetric = new HashMap(2*_map146.size); + String _key147; + MetricWindow _val148; + for (int _i149 = 0; _i149 < _map146.size; ++_i149) + { + _key147 = iprot.readString(); + _val148 = new MetricWindow(); + _val148.read(iprot); + struct.baseMetric.put(_key147, _val148); + } + } + struct.set_baseMetric_isSet(true); + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + { + org.apache.thrift.protocol.TMap _map150 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32()); + struct.inputMetric = new HashMap>(2*_map150.size); + String _key151; + Map _val152; + for (int _i153 = 0; _i153 < _map150.size; ++_i153) + { + _key151 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map154 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + _val152 = new HashMap(2*_map154.size); + String _key155; + MetricWindow _val156; + for (int _i157 = 0; _i157 < _map154.size; ++_i157) + { + _key155 = iprot.readString(); + _val156 = new MetricWindow(); + _val156.read(iprot); + _val152.put(_key155, _val156); + } + } + struct.inputMetric.put(_key151, _val152); + } + } + struct.set_inputMetric_isSet(true); + } + if (incoming.get(1)) { + { + org.apache.thrift.protocol.TMap _map158 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.MAP, iprot.readI32()); + struct.outputMetric = new HashMap>(2*_map158.size); + String _key159; + Map _val160; + for (int _i161 = 0; _i161 < _map158.size; ++_i161) + { + _key159 = iprot.readString(); + { + org.apache.thrift.protocol.TMap _map162 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + _val160 = new HashMap(2*_map162.size); + String _key163; + MetricWindow _val164; + for (int _i165 = 0; _i165 < _map162.size; ++_i165) + { + _key163 = iprot.readString(); + _val164 = new MetricWindow(); + _val164.read(iprot); + _val160.put(_key163, _val164); + } + } + struct.outputMetric.put(_key159, _val160); + } + } + struct.set_outputMetric_isSet(true); + } + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/MetricWindow.java b/jstorm-core/src/main/java/backtype/storm/generated/MetricWindow.java new file mode 100644 index 000000000..73faddd2a --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/MetricWindow.java @@ -0,0 +1,441 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class MetricWindow implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("MetricWindow"); + + private static final org.apache.thrift.protocol.TField METRIC_WINDOW_FIELD_DESC = new org.apache.thrift.protocol.TField("metricWindow", org.apache.thrift.protocol.TType.MAP, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new MetricWindowStandardSchemeFactory()); + schemes.put(TupleScheme.class, new MetricWindowTupleSchemeFactory()); + } + + private Map metricWindow; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + METRIC_WINDOW((short)1, "metricWindow"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // METRIC_WINDOW + return METRIC_WINDOW; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.METRIC_WINDOW, new org.apache.thrift.meta_data.FieldMetaData("metricWindow", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(MetricWindow.class, metaDataMap); + } + + public MetricWindow() { + } + + public MetricWindow( + Map metricWindow) + { + this(); + this.metricWindow = metricWindow; + } + + /** + * Performs a deep copy on other. + */ + public MetricWindow(MetricWindow other) { + if (other.is_set_metricWindow()) { + Map __this__metricWindow = new HashMap(other.metricWindow); + this.metricWindow = __this__metricWindow; + } + } + + public MetricWindow deepCopy() { + return new MetricWindow(this); + } + + @Override + public void clear() { + this.metricWindow = null; + } + + public int get_metricWindow_size() { + return (this.metricWindow == null) ? 0 : this.metricWindow.size(); + } + + public void put_to_metricWindow(int key, double val) { + if (this.metricWindow == null) { + this.metricWindow = new HashMap(); + } + this.metricWindow.put(key, val); + } + + public Map get_metricWindow() { + return this.metricWindow; + } + + public void set_metricWindow(Map metricWindow) { + this.metricWindow = metricWindow; + } + + public void unset_metricWindow() { + this.metricWindow = null; + } + + /** Returns true if field metricWindow is set (has been assigned a value) and false otherwise */ + public boolean is_set_metricWindow() { + return this.metricWindow != null; + } + + public void set_metricWindow_isSet(boolean value) { + if (!value) { + this.metricWindow = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case METRIC_WINDOW: + if (value == null) { + unset_metricWindow(); + } else { + set_metricWindow((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case METRIC_WINDOW: + return get_metricWindow(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case METRIC_WINDOW: + return is_set_metricWindow(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof MetricWindow) + return this.equals((MetricWindow)that); + return false; + } + + public boolean equals(MetricWindow that) { + if (that == null) + return false; + + boolean this_present_metricWindow = true && this.is_set_metricWindow(); + boolean that_present_metricWindow = true && that.is_set_metricWindow(); + if (this_present_metricWindow || that_present_metricWindow) { + if (!(this_present_metricWindow && that_present_metricWindow)) + return false; + if (!this.metricWindow.equals(that.metricWindow)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_metricWindow = true && (is_set_metricWindow()); + list.add(present_metricWindow); + if (present_metricWindow) + list.add(metricWindow); + + return list.hashCode(); + } + + @Override + public int compareTo(MetricWindow other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_metricWindow()).compareTo(other.is_set_metricWindow()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_metricWindow()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.metricWindow, other.metricWindow); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("MetricWindow("); + boolean first = true; + + sb.append("metricWindow:"); + if (this.metricWindow == null) { + sb.append("null"); + } else { + sb.append(this.metricWindow); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_metricWindow()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'metricWindow' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class MetricWindowStandardSchemeFactory implements SchemeFactory { + public MetricWindowStandardScheme getScheme() { + return new MetricWindowStandardScheme(); + } + } + + private static class MetricWindowStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, MetricWindow struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // METRIC_WINDOW + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map106 = iprot.readMapBegin(); + struct.metricWindow = new HashMap(2*_map106.size); + int _key107; + double _val108; + for (int _i109 = 0; _i109 < _map106.size; ++_i109) + { + _key107 = iprot.readI32(); + _val108 = iprot.readDouble(); + struct.metricWindow.put(_key107, _val108); + } + iprot.readMapEnd(); + } + struct.set_metricWindow_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, MetricWindow struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.metricWindow != null) { + oprot.writeFieldBegin(METRIC_WINDOW_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.DOUBLE, struct.metricWindow.size())); + for (Map.Entry _iter110 : struct.metricWindow.entrySet()) + { + oprot.writeI32(_iter110.getKey()); + oprot.writeDouble(_iter110.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class MetricWindowTupleSchemeFactory implements SchemeFactory { + public MetricWindowTupleScheme getScheme() { + return new MetricWindowTupleScheme(); + } + } + + private static class MetricWindowTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, MetricWindow struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.metricWindow.size()); + for (Map.Entry _iter111 : struct.metricWindow.entrySet()) + { + oprot.writeI32(_iter111.getKey()); + oprot.writeDouble(_iter111.getValue()); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, MetricWindow struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TMap _map112 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32()); + struct.metricWindow = new HashMap(2*_map112.size); + int _key113; + double _val114; + for (int _i115 = 0; _i115 < _map112.size; ++_i115) + { + _key113 = iprot.readI32(); + _val114 = iprot.readDouble(); + struct.metricWindow.put(_key113, _val114); + } + } + struct.set_metricWindow_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/MonitorOptions.java b/jstorm-core/src/main/java/backtype/storm/generated/MonitorOptions.java new file mode 100644 index 000000000..9b80b78f2 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/MonitorOptions.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class MonitorOptions implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("MonitorOptions"); + + private static final org.apache.thrift.protocol.TField IS_ENABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("isEnable", org.apache.thrift.protocol.TType.BOOL, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new MonitorOptionsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new MonitorOptionsTupleSchemeFactory()); + } + + private boolean isEnable; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + IS_ENABLE((short)1, "isEnable"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // IS_ENABLE + return IS_ENABLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __ISENABLE_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.IS_ENABLE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.IS_ENABLE, new org.apache.thrift.meta_data.FieldMetaData("isEnable", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(MonitorOptions.class, metaDataMap); + } + + public MonitorOptions() { + } + + /** + * Performs a deep copy on other. + */ + public MonitorOptions(MonitorOptions other) { + __isset_bitfield = other.__isset_bitfield; + this.isEnable = other.isEnable; + } + + public MonitorOptions deepCopy() { + return new MonitorOptions(this); + } + + @Override + public void clear() { + set_isEnable_isSet(false); + this.isEnable = false; + } + + public boolean is_isEnable() { + return this.isEnable; + } + + public void set_isEnable(boolean isEnable) { + this.isEnable = isEnable; + set_isEnable_isSet(true); + } + + public void unset_isEnable() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ISENABLE_ISSET_ID); + } + + /** Returns true if field isEnable is set (has been assigned a value) and false otherwise */ + public boolean is_set_isEnable() { + return EncodingUtils.testBit(__isset_bitfield, __ISENABLE_ISSET_ID); + } + + public void set_isEnable_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ISENABLE_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case IS_ENABLE: + if (value == null) { + unset_isEnable(); + } else { + set_isEnable((Boolean)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case IS_ENABLE: + return Boolean.valueOf(is_isEnable()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case IS_ENABLE: + return is_set_isEnable(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof MonitorOptions) + return this.equals((MonitorOptions)that); + return false; + } + + public boolean equals(MonitorOptions that) { + if (that == null) + return false; + + boolean this_present_isEnable = true && this.is_set_isEnable(); + boolean that_present_isEnable = true && that.is_set_isEnable(); + if (this_present_isEnable || that_present_isEnable) { + if (!(this_present_isEnable && that_present_isEnable)) + return false; + if (this.isEnable != that.isEnable) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_isEnable = true && (is_set_isEnable()); + list.add(present_isEnable); + if (present_isEnable) + list.add(isEnable); + + return list.hashCode(); + } + + @Override + public int compareTo(MonitorOptions other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_isEnable()).compareTo(other.is_set_isEnable()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_isEnable()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.isEnable, other.isEnable); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("MonitorOptions("); + boolean first = true; + + if (is_set_isEnable()) { + sb.append("isEnable:"); + sb.append(this.isEnable); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class MonitorOptionsStandardSchemeFactory implements SchemeFactory { + public MonitorOptionsStandardScheme getScheme() { + return new MonitorOptionsStandardScheme(); + } + } + + private static class MonitorOptionsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, MonitorOptions struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // IS_ENABLE + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.isEnable = iprot.readBool(); + struct.set_isEnable_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, MonitorOptions struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.is_set_isEnable()) { + oprot.writeFieldBegin(IS_ENABLE_FIELD_DESC); + oprot.writeBool(struct.isEnable); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class MonitorOptionsTupleSchemeFactory implements SchemeFactory { + public MonitorOptionsTupleScheme getScheme() { + return new MonitorOptionsTupleScheme(); + } + } + + private static class MonitorOptionsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, MonitorOptions struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_isEnable()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_isEnable()) { + oprot.writeBool(struct.isEnable); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, MonitorOptions struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.isEnable = iprot.readBool(); + struct.set_isEnable_isSet(true); + } + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/NettyMetric.java b/jstorm-core/src/main/java/backtype/storm/generated/NettyMetric.java new file mode 100644 index 000000000..b6a9bc9c9 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/NettyMetric.java @@ -0,0 +1,553 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class NettyMetric implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NettyMetric"); + + private static final org.apache.thrift.protocol.TField CONNECTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("connections", org.apache.thrift.protocol.TType.MAP, (short)1); + private static final org.apache.thrift.protocol.TField CONNECTION_NUM_FIELD_DESC = new org.apache.thrift.protocol.TField("connectionNum", org.apache.thrift.protocol.TType.I32, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new NettyMetricStandardSchemeFactory()); + schemes.put(TupleScheme.class, new NettyMetricTupleSchemeFactory()); + } + + private Map connections; // required + private int connectionNum; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + CONNECTIONS((short)1, "connections"), + CONNECTION_NUM((short)2, "connectionNum"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // CONNECTIONS + return CONNECTIONS; + case 2: // CONNECTION_NUM + return CONNECTION_NUM; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __CONNECTIONNUM_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.CONNECTIONS, new org.apache.thrift.meta_data.FieldMetaData("connections", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, MetricInfo.class)))); + tmpMap.put(_Fields.CONNECTION_NUM, new org.apache.thrift.meta_data.FieldMetaData("connectionNum", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(NettyMetric.class, metaDataMap); + } + + public NettyMetric() { + } + + public NettyMetric( + Map connections, + int connectionNum) + { + this(); + this.connections = connections; + this.connectionNum = connectionNum; + set_connectionNum_isSet(true); + } + + /** + * Performs a deep copy on other. + */ + public NettyMetric(NettyMetric other) { + __isset_bitfield = other.__isset_bitfield; + if (other.is_set_connections()) { + Map __this__connections = new HashMap(other.connections.size()); + for (Map.Entry other_element : other.connections.entrySet()) { + + String other_element_key = other_element.getKey(); + MetricInfo other_element_value = other_element.getValue(); + + String __this__connections_copy_key = other_element_key; + + MetricInfo __this__connections_copy_value = new MetricInfo(other_element_value); + + __this__connections.put(__this__connections_copy_key, __this__connections_copy_value); + } + this.connections = __this__connections; + } + this.connectionNum = other.connectionNum; + } + + public NettyMetric deepCopy() { + return new NettyMetric(this); + } + + @Override + public void clear() { + this.connections = null; + set_connectionNum_isSet(false); + this.connectionNum = 0; + } + + public int get_connections_size() { + return (this.connections == null) ? 0 : this.connections.size(); + } + + public void put_to_connections(String key, MetricInfo val) { + if (this.connections == null) { + this.connections = new HashMap(); + } + this.connections.put(key, val); + } + + public Map get_connections() { + return this.connections; + } + + public void set_connections(Map connections) { + this.connections = connections; + } + + public void unset_connections() { + this.connections = null; + } + + /** Returns true if field connections is set (has been assigned a value) and false otherwise */ + public boolean is_set_connections() { + return this.connections != null; + } + + public void set_connections_isSet(boolean value) { + if (!value) { + this.connections = null; + } + } + + public int get_connectionNum() { + return this.connectionNum; + } + + public void set_connectionNum(int connectionNum) { + this.connectionNum = connectionNum; + set_connectionNum_isSet(true); + } + + public void unset_connectionNum() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __CONNECTIONNUM_ISSET_ID); + } + + /** Returns true if field connectionNum is set (has been assigned a value) and false otherwise */ + public boolean is_set_connectionNum() { + return EncodingUtils.testBit(__isset_bitfield, __CONNECTIONNUM_ISSET_ID); + } + + public void set_connectionNum_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __CONNECTIONNUM_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case CONNECTIONS: + if (value == null) { + unset_connections(); + } else { + set_connections((Map)value); + } + break; + + case CONNECTION_NUM: + if (value == null) { + unset_connectionNum(); + } else { + set_connectionNum((Integer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case CONNECTIONS: + return get_connections(); + + case CONNECTION_NUM: + return Integer.valueOf(get_connectionNum()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case CONNECTIONS: + return is_set_connections(); + case CONNECTION_NUM: + return is_set_connectionNum(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof NettyMetric) + return this.equals((NettyMetric)that); + return false; + } + + public boolean equals(NettyMetric that) { + if (that == null) + return false; + + boolean this_present_connections = true && this.is_set_connections(); + boolean that_present_connections = true && that.is_set_connections(); + if (this_present_connections || that_present_connections) { + if (!(this_present_connections && that_present_connections)) + return false; + if (!this.connections.equals(that.connections)) + return false; + } + + boolean this_present_connectionNum = true; + boolean that_present_connectionNum = true; + if (this_present_connectionNum || that_present_connectionNum) { + if (!(this_present_connectionNum && that_present_connectionNum)) + return false; + if (this.connectionNum != that.connectionNum) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_connections = true && (is_set_connections()); + list.add(present_connections); + if (present_connections) + list.add(connections); + + boolean present_connectionNum = true; + list.add(present_connectionNum); + if (present_connectionNum) + list.add(connectionNum); + + return list.hashCode(); + } + + @Override + public int compareTo(NettyMetric other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_connections()).compareTo(other.is_set_connections()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_connections()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.connections, other.connections); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_connectionNum()).compareTo(other.is_set_connectionNum()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_connectionNum()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.connectionNum, other.connectionNum); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("NettyMetric("); + boolean first = true; + + sb.append("connections:"); + if (this.connections == null) { + sb.append("null"); + } else { + sb.append(this.connections); + } + first = false; + if (!first) sb.append(", "); + sb.append("connectionNum:"); + sb.append(this.connectionNum); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_connections()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'connections' is unset! Struct:" + toString()); + } + + if (!is_set_connectionNum()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'connectionNum' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class NettyMetricStandardSchemeFactory implements SchemeFactory { + public NettyMetricStandardScheme getScheme() { + return new NettyMetricStandardScheme(); + } + } + + private static class NettyMetricStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, NettyMetric struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // CONNECTIONS + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map274 = iprot.readMapBegin(); + struct.connections = new HashMap(2*_map274.size); + String _key275; + MetricInfo _val276; + for (int _i277 = 0; _i277 < _map274.size; ++_i277) + { + _key275 = iprot.readString(); + _val276 = new MetricInfo(); + _val276.read(iprot); + struct.connections.put(_key275, _val276); + } + iprot.readMapEnd(); + } + struct.set_connections_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // CONNECTION_NUM + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.connectionNum = iprot.readI32(); + struct.set_connectionNum_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, NettyMetric struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.connections != null) { + oprot.writeFieldBegin(CONNECTIONS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.connections.size())); + for (Map.Entry _iter278 : struct.connections.entrySet()) + { + oprot.writeString(_iter278.getKey()); + _iter278.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(CONNECTION_NUM_FIELD_DESC); + oprot.writeI32(struct.connectionNum); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class NettyMetricTupleSchemeFactory implements SchemeFactory { + public NettyMetricTupleScheme getScheme() { + return new NettyMetricTupleScheme(); + } + } + + private static class NettyMetricTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, NettyMetric struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.connections.size()); + for (Map.Entry _iter279 : struct.connections.entrySet()) + { + oprot.writeString(_iter279.getKey()); + _iter279.getValue().write(oprot); + } + } + oprot.writeI32(struct.connectionNum); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, NettyMetric struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TMap _map280 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.connections = new HashMap(2*_map280.size); + String _key281; + MetricInfo _val282; + for (int _i283 = 0; _i283 < _map280.size; ++_i283) + { + _key281 = iprot.readString(); + _val282 = new MetricInfo(); + _val282.read(iprot); + struct.connections.put(_key281, _val282); + } + } + struct.set_connections_isSet(true); + struct.connectionNum = iprot.readI32(); + struct.set_connectionNum_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/Nimbus.java b/jstorm-core/src/main/java/backtype/storm/generated/Nimbus.java new file mode 100644 index 000000000..c25d643a6 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/Nimbus.java @@ -0,0 +1,28951 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class Nimbus { + + public interface Iface { + + public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift.TException; + + public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws AlreadyAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift.TException; + + public void killTopology(String name) throws NotAliveException, org.apache.thrift.TException; + + public void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException, org.apache.thrift.TException; + + public void activate(String name) throws NotAliveException, org.apache.thrift.TException; + + public void deactivate(String name) throws NotAliveException, org.apache.thrift.TException; + + public void rebalance(String name, RebalanceOptions options) throws NotAliveException, InvalidTopologyException, org.apache.thrift.TException; + + public void metricMonitor(String name, MonitorOptions options) throws NotAliveException, org.apache.thrift.TException; + + public void restart(String name, String jsonConf) throws NotAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift.TException; + + public void beginLibUpload(String libName) throws org.apache.thrift.TException; + + public String beginFileUpload() throws org.apache.thrift.TException; + + public void uploadChunk(String location, ByteBuffer chunk) throws org.apache.thrift.TException; + + public void finishFileUpload(String location) throws org.apache.thrift.TException; + + public String beginFileDownload(String file) throws org.apache.thrift.TException; + + public ByteBuffer downloadChunk(String id) throws org.apache.thrift.TException; + + public void finishFileDownload(String id) throws org.apache.thrift.TException; + + public String getNimbusConf() throws org.apache.thrift.TException; + + public String getTopologyConf(String id) throws NotAliveException, org.apache.thrift.TException; + + public String getTopologyId(String topologyName) throws NotAliveException, org.apache.thrift.TException; + + public ClusterSummary getClusterInfo() throws org.apache.thrift.TException; + + public SupervisorWorkers getSupervisorWorkers(String host) throws NotAliveException, org.apache.thrift.TException; + + public TopologyInfo getTopologyInfo(String id) throws NotAliveException, org.apache.thrift.TException; + + public TopologyInfo getTopologyInfoByName(String topologyName) throws NotAliveException, org.apache.thrift.TException; + + public StormTopology getTopology(String id) throws NotAliveException, org.apache.thrift.TException; + + public StormTopology getUserTopology(String id) throws NotAliveException, org.apache.thrift.TException; + + public void workerUploadMetric(WorkerUploadMetrics uploadMetrics) throws org.apache.thrift.TException; + + public TopologyMetric getTopologyMetric(String topologyName) throws org.apache.thrift.TException; + + public NettyMetric getNettyMetric(String topologyName, int pos) throws org.apache.thrift.TException; + + public NettyMetric getServerNettyMetric(String topologyName, String serverName) throws org.apache.thrift.TException; + + public String getVersion() throws org.apache.thrift.TException; + + public void updateConf(String name, String conf) throws NotAliveException, InvalidTopologyException, org.apache.thrift.TException; + + } + + public interface AsyncIface { + + public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void killTopology(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void killTopologyWithOpts(String name, KillOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void activate(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void deactivate(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void rebalance(String name, RebalanceOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void metricMonitor(String name, MonitorOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void restart(String name, String jsonConf, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void beginLibUpload(String libName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void beginFileUpload(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void uploadChunk(String location, ByteBuffer chunk, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void finishFileUpload(String location, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void beginFileDownload(String file, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void downloadChunk(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void finishFileDownload(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void getNimbusConf(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void getTopologyConf(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void getTopologyId(String topologyName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void getClusterInfo(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void getSupervisorWorkers(String host, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void getTopologyInfo(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void getTopologyInfoByName(String topologyName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void getTopology(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void getUserTopology(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void workerUploadMetric(WorkerUploadMetrics uploadMetrics, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void getTopologyMetric(String topologyName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void getNettyMetric(String topologyName, int pos, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void getServerNettyMetric(String topologyName, String serverName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void getVersion(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void updateConf(String name, String conf, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + } + + public static class Client extends org.apache.thrift.TServiceClient implements Iface { + public static class Factory implements org.apache.thrift.TServiceClientFactory { + public Factory() {} + public Client getClient(org.apache.thrift.protocol.TProtocol prot) { + return new Client(prot); + } + public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) { + return new Client(iprot, oprot); + } + } + + public Client(org.apache.thrift.protocol.TProtocol prot) + { + super(prot, prot); + } + + public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) { + super(iprot, oprot); + } + + public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift.TException + { + send_submitTopology(name, uploadedJarLocation, jsonConf, topology); + recv_submitTopology(); + } + + public void send_submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws org.apache.thrift.TException + { + submitTopology_args args = new submitTopology_args(); + args.set_name(name); + args.set_uploadedJarLocation(uploadedJarLocation); + args.set_jsonConf(jsonConf); + args.set_topology(topology); + sendBase("submitTopology", args); + } + + public void recv_submitTopology() throws AlreadyAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift.TException + { + submitTopology_result result = new submitTopology_result(); + receiveBase(result, "submitTopology"); + if (result.e != null) { + throw result.e; + } + if (result.ite != null) { + throw result.ite; + } + if (result.tae != null) { + throw result.tae; + } + return; + } + + public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws AlreadyAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift.TException + { + send_submitTopologyWithOpts(name, uploadedJarLocation, jsonConf, topology, options); + recv_submitTopologyWithOpts(); + } + + public void send_submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options) throws org.apache.thrift.TException + { + submitTopologyWithOpts_args args = new submitTopologyWithOpts_args(); + args.set_name(name); + args.set_uploadedJarLocation(uploadedJarLocation); + args.set_jsonConf(jsonConf); + args.set_topology(topology); + args.set_options(options); + sendBase("submitTopologyWithOpts", args); + } + + public void recv_submitTopologyWithOpts() throws AlreadyAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift.TException + { + submitTopologyWithOpts_result result = new submitTopologyWithOpts_result(); + receiveBase(result, "submitTopologyWithOpts"); + if (result.e != null) { + throw result.e; + } + if (result.ite != null) { + throw result.ite; + } + if (result.tae != null) { + throw result.tae; + } + return; + } + + public void killTopology(String name) throws NotAliveException, org.apache.thrift.TException + { + send_killTopology(name); + recv_killTopology(); + } + + public void send_killTopology(String name) throws org.apache.thrift.TException + { + killTopology_args args = new killTopology_args(); + args.set_name(name); + sendBase("killTopology", args); + } + + public void recv_killTopology() throws NotAliveException, org.apache.thrift.TException + { + killTopology_result result = new killTopology_result(); + receiveBase(result, "killTopology"); + if (result.e != null) { + throw result.e; + } + return; + } + + public void killTopologyWithOpts(String name, KillOptions options) throws NotAliveException, org.apache.thrift.TException + { + send_killTopologyWithOpts(name, options); + recv_killTopologyWithOpts(); + } + + public void send_killTopologyWithOpts(String name, KillOptions options) throws org.apache.thrift.TException + { + killTopologyWithOpts_args args = new killTopologyWithOpts_args(); + args.set_name(name); + args.set_options(options); + sendBase("killTopologyWithOpts", args); + } + + public void recv_killTopologyWithOpts() throws NotAliveException, org.apache.thrift.TException + { + killTopologyWithOpts_result result = new killTopologyWithOpts_result(); + receiveBase(result, "killTopologyWithOpts"); + if (result.e != null) { + throw result.e; + } + return; + } + + public void activate(String name) throws NotAliveException, org.apache.thrift.TException + { + send_activate(name); + recv_activate(); + } + + public void send_activate(String name) throws org.apache.thrift.TException + { + activate_args args = new activate_args(); + args.set_name(name); + sendBase("activate", args); + } + + public void recv_activate() throws NotAliveException, org.apache.thrift.TException + { + activate_result result = new activate_result(); + receiveBase(result, "activate"); + if (result.e != null) { + throw result.e; + } + return; + } + + public void deactivate(String name) throws NotAliveException, org.apache.thrift.TException + { + send_deactivate(name); + recv_deactivate(); + } + + public void send_deactivate(String name) throws org.apache.thrift.TException + { + deactivate_args args = new deactivate_args(); + args.set_name(name); + sendBase("deactivate", args); + } + + public void recv_deactivate() throws NotAliveException, org.apache.thrift.TException + { + deactivate_result result = new deactivate_result(); + receiveBase(result, "deactivate"); + if (result.e != null) { + throw result.e; + } + return; + } + + public void rebalance(String name, RebalanceOptions options) throws NotAliveException, InvalidTopologyException, org.apache.thrift.TException + { + send_rebalance(name, options); + recv_rebalance(); + } + + public void send_rebalance(String name, RebalanceOptions options) throws org.apache.thrift.TException + { + rebalance_args args = new rebalance_args(); + args.set_name(name); + args.set_options(options); + sendBase("rebalance", args); + } + + public void recv_rebalance() throws NotAliveException, InvalidTopologyException, org.apache.thrift.TException + { + rebalance_result result = new rebalance_result(); + receiveBase(result, "rebalance"); + if (result.e != null) { + throw result.e; + } + if (result.ite != null) { + throw result.ite; + } + return; + } + + public void metricMonitor(String name, MonitorOptions options) throws NotAliveException, org.apache.thrift.TException + { + send_metricMonitor(name, options); + recv_metricMonitor(); + } + + public void send_metricMonitor(String name, MonitorOptions options) throws org.apache.thrift.TException + { + metricMonitor_args args = new metricMonitor_args(); + args.set_name(name); + args.set_options(options); + sendBase("metricMonitor", args); + } + + public void recv_metricMonitor() throws NotAliveException, org.apache.thrift.TException + { + metricMonitor_result result = new metricMonitor_result(); + receiveBase(result, "metricMonitor"); + if (result.e != null) { + throw result.e; + } + return; + } + + public void restart(String name, String jsonConf) throws NotAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift.TException + { + send_restart(name, jsonConf); + recv_restart(); + } + + public void send_restart(String name, String jsonConf) throws org.apache.thrift.TException + { + restart_args args = new restart_args(); + args.set_name(name); + args.set_jsonConf(jsonConf); + sendBase("restart", args); + } + + public void recv_restart() throws NotAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift.TException + { + restart_result result = new restart_result(); + receiveBase(result, "restart"); + if (result.e != null) { + throw result.e; + } + if (result.ite != null) { + throw result.ite; + } + if (result.tae != null) { + throw result.tae; + } + return; + } + + public void beginLibUpload(String libName) throws org.apache.thrift.TException + { + send_beginLibUpload(libName); + recv_beginLibUpload(); + } + + public void send_beginLibUpload(String libName) throws org.apache.thrift.TException + { + beginLibUpload_args args = new beginLibUpload_args(); + args.set_libName(libName); + sendBase("beginLibUpload", args); + } + + public void recv_beginLibUpload() throws org.apache.thrift.TException + { + beginLibUpload_result result = new beginLibUpload_result(); + receiveBase(result, "beginLibUpload"); + return; + } + + public String beginFileUpload() throws org.apache.thrift.TException + { + send_beginFileUpload(); + return recv_beginFileUpload(); + } + + public void send_beginFileUpload() throws org.apache.thrift.TException + { + beginFileUpload_args args = new beginFileUpload_args(); + sendBase("beginFileUpload", args); + } + + public String recv_beginFileUpload() throws org.apache.thrift.TException + { + beginFileUpload_result result = new beginFileUpload_result(); + receiveBase(result, "beginFileUpload"); + if (result.is_set_success()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "beginFileUpload failed: unknown result"); + } + + public void uploadChunk(String location, ByteBuffer chunk) throws org.apache.thrift.TException + { + send_uploadChunk(location, chunk); + recv_uploadChunk(); + } + + public void send_uploadChunk(String location, ByteBuffer chunk) throws org.apache.thrift.TException + { + uploadChunk_args args = new uploadChunk_args(); + args.set_location(location); + args.set_chunk(chunk); + sendBase("uploadChunk", args); + } + + public void recv_uploadChunk() throws org.apache.thrift.TException + { + uploadChunk_result result = new uploadChunk_result(); + receiveBase(result, "uploadChunk"); + return; + } + + public void finishFileUpload(String location) throws org.apache.thrift.TException + { + send_finishFileUpload(location); + recv_finishFileUpload(); + } + + public void send_finishFileUpload(String location) throws org.apache.thrift.TException + { + finishFileUpload_args args = new finishFileUpload_args(); + args.set_location(location); + sendBase("finishFileUpload", args); + } + + public void recv_finishFileUpload() throws org.apache.thrift.TException + { + finishFileUpload_result result = new finishFileUpload_result(); + receiveBase(result, "finishFileUpload"); + return; + } + + public String beginFileDownload(String file) throws org.apache.thrift.TException + { + send_beginFileDownload(file); + return recv_beginFileDownload(); + } + + public void send_beginFileDownload(String file) throws org.apache.thrift.TException + { + beginFileDownload_args args = new beginFileDownload_args(); + args.set_file(file); + sendBase("beginFileDownload", args); + } + + public String recv_beginFileDownload() throws org.apache.thrift.TException + { + beginFileDownload_result result = new beginFileDownload_result(); + receiveBase(result, "beginFileDownload"); + if (result.is_set_success()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "beginFileDownload failed: unknown result"); + } + + public ByteBuffer downloadChunk(String id) throws org.apache.thrift.TException + { + send_downloadChunk(id); + return recv_downloadChunk(); + } + + public void send_downloadChunk(String id) throws org.apache.thrift.TException + { + downloadChunk_args args = new downloadChunk_args(); + args.set_id(id); + sendBase("downloadChunk", args); + } + + public ByteBuffer recv_downloadChunk() throws org.apache.thrift.TException + { + downloadChunk_result result = new downloadChunk_result(); + receiveBase(result, "downloadChunk"); + if (result.is_set_success()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "downloadChunk failed: unknown result"); + } + + public void finishFileDownload(String id) throws org.apache.thrift.TException + { + send_finishFileDownload(id); + recv_finishFileDownload(); + } + + public void send_finishFileDownload(String id) throws org.apache.thrift.TException + { + finishFileDownload_args args = new finishFileDownload_args(); + args.set_id(id); + sendBase("finishFileDownload", args); + } + + public void recv_finishFileDownload() throws org.apache.thrift.TException + { + finishFileDownload_result result = new finishFileDownload_result(); + receiveBase(result, "finishFileDownload"); + return; + } + + public String getNimbusConf() throws org.apache.thrift.TException + { + send_getNimbusConf(); + return recv_getNimbusConf(); + } + + public void send_getNimbusConf() throws org.apache.thrift.TException + { + getNimbusConf_args args = new getNimbusConf_args(); + sendBase("getNimbusConf", args); + } + + public String recv_getNimbusConf() throws org.apache.thrift.TException + { + getNimbusConf_result result = new getNimbusConf_result(); + receiveBase(result, "getNimbusConf"); + if (result.is_set_success()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getNimbusConf failed: unknown result"); + } + + public String getTopologyConf(String id) throws NotAliveException, org.apache.thrift.TException + { + send_getTopologyConf(id); + return recv_getTopologyConf(); + } + + public void send_getTopologyConf(String id) throws org.apache.thrift.TException + { + getTopologyConf_args args = new getTopologyConf_args(); + args.set_id(id); + sendBase("getTopologyConf", args); + } + + public String recv_getTopologyConf() throws NotAliveException, org.apache.thrift.TException + { + getTopologyConf_result result = new getTopologyConf_result(); + receiveBase(result, "getTopologyConf"); + if (result.is_set_success()) { + return result.success; + } + if (result.e != null) { + throw result.e; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTopologyConf failed: unknown result"); + } + + public String getTopologyId(String topologyName) throws NotAliveException, org.apache.thrift.TException + { + send_getTopologyId(topologyName); + return recv_getTopologyId(); + } + + public void send_getTopologyId(String topologyName) throws org.apache.thrift.TException + { + getTopologyId_args args = new getTopologyId_args(); + args.set_topologyName(topologyName); + sendBase("getTopologyId", args); + } + + public String recv_getTopologyId() throws NotAliveException, org.apache.thrift.TException + { + getTopologyId_result result = new getTopologyId_result(); + receiveBase(result, "getTopologyId"); + if (result.is_set_success()) { + return result.success; + } + if (result.e != null) { + throw result.e; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTopologyId failed: unknown result"); + } + + public ClusterSummary getClusterInfo() throws org.apache.thrift.TException + { + send_getClusterInfo(); + return recv_getClusterInfo(); + } + + public void send_getClusterInfo() throws org.apache.thrift.TException + { + getClusterInfo_args args = new getClusterInfo_args(); + sendBase("getClusterInfo", args); + } + + public ClusterSummary recv_getClusterInfo() throws org.apache.thrift.TException + { + getClusterInfo_result result = new getClusterInfo_result(); + receiveBase(result, "getClusterInfo"); + if (result.is_set_success()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getClusterInfo failed: unknown result"); + } + + public SupervisorWorkers getSupervisorWorkers(String host) throws NotAliveException, org.apache.thrift.TException + { + send_getSupervisorWorkers(host); + return recv_getSupervisorWorkers(); + } + + public void send_getSupervisorWorkers(String host) throws org.apache.thrift.TException + { + getSupervisorWorkers_args args = new getSupervisorWorkers_args(); + args.set_host(host); + sendBase("getSupervisorWorkers", args); + } + + public SupervisorWorkers recv_getSupervisorWorkers() throws NotAliveException, org.apache.thrift.TException + { + getSupervisorWorkers_result result = new getSupervisorWorkers_result(); + receiveBase(result, "getSupervisorWorkers"); + if (result.is_set_success()) { + return result.success; + } + if (result.e != null) { + throw result.e; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getSupervisorWorkers failed: unknown result"); + } + + public TopologyInfo getTopologyInfo(String id) throws NotAliveException, org.apache.thrift.TException + { + send_getTopologyInfo(id); + return recv_getTopologyInfo(); + } + + public void send_getTopologyInfo(String id) throws org.apache.thrift.TException + { + getTopologyInfo_args args = new getTopologyInfo_args(); + args.set_id(id); + sendBase("getTopologyInfo", args); + } + + public TopologyInfo recv_getTopologyInfo() throws NotAliveException, org.apache.thrift.TException + { + getTopologyInfo_result result = new getTopologyInfo_result(); + receiveBase(result, "getTopologyInfo"); + if (result.is_set_success()) { + return result.success; + } + if (result.e != null) { + throw result.e; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTopologyInfo failed: unknown result"); + } + + public TopologyInfo getTopologyInfoByName(String topologyName) throws NotAliveException, org.apache.thrift.TException + { + send_getTopologyInfoByName(topologyName); + return recv_getTopologyInfoByName(); + } + + public void send_getTopologyInfoByName(String topologyName) throws org.apache.thrift.TException + { + getTopologyInfoByName_args args = new getTopologyInfoByName_args(); + args.set_topologyName(topologyName); + sendBase("getTopologyInfoByName", args); + } + + public TopologyInfo recv_getTopologyInfoByName() throws NotAliveException, org.apache.thrift.TException + { + getTopologyInfoByName_result result = new getTopologyInfoByName_result(); + receiveBase(result, "getTopologyInfoByName"); + if (result.is_set_success()) { + return result.success; + } + if (result.e != null) { + throw result.e; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTopologyInfoByName failed: unknown result"); + } + + public StormTopology getTopology(String id) throws NotAliveException, org.apache.thrift.TException + { + send_getTopology(id); + return recv_getTopology(); + } + + public void send_getTopology(String id) throws org.apache.thrift.TException + { + getTopology_args args = new getTopology_args(); + args.set_id(id); + sendBase("getTopology", args); + } + + public StormTopology recv_getTopology() throws NotAliveException, org.apache.thrift.TException + { + getTopology_result result = new getTopology_result(); + receiveBase(result, "getTopology"); + if (result.is_set_success()) { + return result.success; + } + if (result.e != null) { + throw result.e; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTopology failed: unknown result"); + } + + public StormTopology getUserTopology(String id) throws NotAliveException, org.apache.thrift.TException + { + send_getUserTopology(id); + return recv_getUserTopology(); + } + + public void send_getUserTopology(String id) throws org.apache.thrift.TException + { + getUserTopology_args args = new getUserTopology_args(); + args.set_id(id); + sendBase("getUserTopology", args); + } + + public StormTopology recv_getUserTopology() throws NotAliveException, org.apache.thrift.TException + { + getUserTopology_result result = new getUserTopology_result(); + receiveBase(result, "getUserTopology"); + if (result.is_set_success()) { + return result.success; + } + if (result.e != null) { + throw result.e; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getUserTopology failed: unknown result"); + } + + public void workerUploadMetric(WorkerUploadMetrics uploadMetrics) throws org.apache.thrift.TException + { + send_workerUploadMetric(uploadMetrics); + recv_workerUploadMetric(); + } + + public void send_workerUploadMetric(WorkerUploadMetrics uploadMetrics) throws org.apache.thrift.TException + { + workerUploadMetric_args args = new workerUploadMetric_args(); + args.set_uploadMetrics(uploadMetrics); + sendBase("workerUploadMetric", args); + } + + public void recv_workerUploadMetric() throws org.apache.thrift.TException + { + workerUploadMetric_result result = new workerUploadMetric_result(); + receiveBase(result, "workerUploadMetric"); + return; + } + + public TopologyMetric getTopologyMetric(String topologyName) throws org.apache.thrift.TException + { + send_getTopologyMetric(topologyName); + return recv_getTopologyMetric(); + } + + public void send_getTopologyMetric(String topologyName) throws org.apache.thrift.TException + { + getTopologyMetric_args args = new getTopologyMetric_args(); + args.set_topologyName(topologyName); + sendBase("getTopologyMetric", args); + } + + public TopologyMetric recv_getTopologyMetric() throws org.apache.thrift.TException + { + getTopologyMetric_result result = new getTopologyMetric_result(); + receiveBase(result, "getTopologyMetric"); + if (result.is_set_success()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getTopologyMetric failed: unknown result"); + } + + public NettyMetric getNettyMetric(String topologyName, int pos) throws org.apache.thrift.TException + { + send_getNettyMetric(topologyName, pos); + return recv_getNettyMetric(); + } + + public void send_getNettyMetric(String topologyName, int pos) throws org.apache.thrift.TException + { + getNettyMetric_args args = new getNettyMetric_args(); + args.set_topologyName(topologyName); + args.set_pos(pos); + sendBase("getNettyMetric", args); + } + + public NettyMetric recv_getNettyMetric() throws org.apache.thrift.TException + { + getNettyMetric_result result = new getNettyMetric_result(); + receiveBase(result, "getNettyMetric"); + if (result.is_set_success()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getNettyMetric failed: unknown result"); + } + + public NettyMetric getServerNettyMetric(String topologyName, String serverName) throws org.apache.thrift.TException + { + send_getServerNettyMetric(topologyName, serverName); + return recv_getServerNettyMetric(); + } + + public void send_getServerNettyMetric(String topologyName, String serverName) throws org.apache.thrift.TException + { + getServerNettyMetric_args args = new getServerNettyMetric_args(); + args.set_topologyName(topologyName); + args.set_serverName(serverName); + sendBase("getServerNettyMetric", args); + } + + public NettyMetric recv_getServerNettyMetric() throws org.apache.thrift.TException + { + getServerNettyMetric_result result = new getServerNettyMetric_result(); + receiveBase(result, "getServerNettyMetric"); + if (result.is_set_success()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getServerNettyMetric failed: unknown result"); + } + + public String getVersion() throws org.apache.thrift.TException + { + send_getVersion(); + return recv_getVersion(); + } + + public void send_getVersion() throws org.apache.thrift.TException + { + getVersion_args args = new getVersion_args(); + sendBase("getVersion", args); + } + + public String recv_getVersion() throws org.apache.thrift.TException + { + getVersion_result result = new getVersion_result(); + receiveBase(result, "getVersion"); + if (result.is_set_success()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getVersion failed: unknown result"); + } + + public void updateConf(String name, String conf) throws NotAliveException, InvalidTopologyException, org.apache.thrift.TException + { + send_updateConf(name, conf); + recv_updateConf(); + } + + public void send_updateConf(String name, String conf) throws org.apache.thrift.TException + { + updateConf_args args = new updateConf_args(); + args.set_name(name); + args.set_conf(conf); + sendBase("updateConf", args); + } + + public void recv_updateConf() throws NotAliveException, InvalidTopologyException, org.apache.thrift.TException + { + updateConf_result result = new updateConf_result(); + receiveBase(result, "updateConf"); + if (result.e != null) { + throw result.e; + } + if (result.ite != null) { + throw result.ite; + } + return; + } + + } + public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface { + public static class Factory implements org.apache.thrift.async.TAsyncClientFactory { + private org.apache.thrift.async.TAsyncClientManager clientManager; + private org.apache.thrift.protocol.TProtocolFactory protocolFactory; + public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) { + this.clientManager = clientManager; + this.protocolFactory = protocolFactory; + } + public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) { + return new AsyncClient(protocolFactory, clientManager, transport); + } + } + + public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) { + super(protocolFactory, clientManager, transport); + } + + public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + submitTopology_call method_call = new submitTopology_call(name, uploadedJarLocation, jsonConf, topology, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class submitTopology_call extends org.apache.thrift.async.TAsyncMethodCall { + private String name; + private String uploadedJarLocation; + private String jsonConf; + private StormTopology topology; + public submitTopology_call(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.name = name; + this.uploadedJarLocation = uploadedJarLocation; + this.jsonConf = jsonConf; + this.topology = topology; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("submitTopology", org.apache.thrift.protocol.TMessageType.CALL, 0)); + submitTopology_args args = new submitTopology_args(); + args.set_name(name); + args.set_uploadedJarLocation(uploadedJarLocation); + args.set_jsonConf(jsonConf); + args.set_topology(topology); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws AlreadyAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_submitTopology(); + } + } + + public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + submitTopologyWithOpts_call method_call = new submitTopologyWithOpts_call(name, uploadedJarLocation, jsonConf, topology, options, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class submitTopologyWithOpts_call extends org.apache.thrift.async.TAsyncMethodCall { + private String name; + private String uploadedJarLocation; + private String jsonConf; + private StormTopology topology; + private SubmitOptions options; + public submitTopologyWithOpts_call(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, SubmitOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.name = name; + this.uploadedJarLocation = uploadedJarLocation; + this.jsonConf = jsonConf; + this.topology = topology; + this.options = options; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("submitTopologyWithOpts", org.apache.thrift.protocol.TMessageType.CALL, 0)); + submitTopologyWithOpts_args args = new submitTopologyWithOpts_args(); + args.set_name(name); + args.set_uploadedJarLocation(uploadedJarLocation); + args.set_jsonConf(jsonConf); + args.set_topology(topology); + args.set_options(options); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws AlreadyAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_submitTopologyWithOpts(); + } + } + + public void killTopology(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + killTopology_call method_call = new killTopology_call(name, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class killTopology_call extends org.apache.thrift.async.TAsyncMethodCall { + private String name; + public killTopology_call(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.name = name; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("killTopology", org.apache.thrift.protocol.TMessageType.CALL, 0)); + killTopology_args args = new killTopology_args(); + args.set_name(name); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws NotAliveException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_killTopology(); + } + } + + public void killTopologyWithOpts(String name, KillOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + killTopologyWithOpts_call method_call = new killTopologyWithOpts_call(name, options, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class killTopologyWithOpts_call extends org.apache.thrift.async.TAsyncMethodCall { + private String name; + private KillOptions options; + public killTopologyWithOpts_call(String name, KillOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.name = name; + this.options = options; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("killTopologyWithOpts", org.apache.thrift.protocol.TMessageType.CALL, 0)); + killTopologyWithOpts_args args = new killTopologyWithOpts_args(); + args.set_name(name); + args.set_options(options); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws NotAliveException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_killTopologyWithOpts(); + } + } + + public void activate(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + activate_call method_call = new activate_call(name, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class activate_call extends org.apache.thrift.async.TAsyncMethodCall { + private String name; + public activate_call(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.name = name; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("activate", org.apache.thrift.protocol.TMessageType.CALL, 0)); + activate_args args = new activate_args(); + args.set_name(name); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws NotAliveException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_activate(); + } + } + + public void deactivate(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + deactivate_call method_call = new deactivate_call(name, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class deactivate_call extends org.apache.thrift.async.TAsyncMethodCall { + private String name; + public deactivate_call(String name, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.name = name; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("deactivate", org.apache.thrift.protocol.TMessageType.CALL, 0)); + deactivate_args args = new deactivate_args(); + args.set_name(name); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws NotAliveException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_deactivate(); + } + } + + public void rebalance(String name, RebalanceOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + rebalance_call method_call = new rebalance_call(name, options, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class rebalance_call extends org.apache.thrift.async.TAsyncMethodCall { + private String name; + private RebalanceOptions options; + public rebalance_call(String name, RebalanceOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.name = name; + this.options = options; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("rebalance", org.apache.thrift.protocol.TMessageType.CALL, 0)); + rebalance_args args = new rebalance_args(); + args.set_name(name); + args.set_options(options); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws NotAliveException, InvalidTopologyException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_rebalance(); + } + } + + public void metricMonitor(String name, MonitorOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + metricMonitor_call method_call = new metricMonitor_call(name, options, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class metricMonitor_call extends org.apache.thrift.async.TAsyncMethodCall { + private String name; + private MonitorOptions options; + public metricMonitor_call(String name, MonitorOptions options, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.name = name; + this.options = options; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("metricMonitor", org.apache.thrift.protocol.TMessageType.CALL, 0)); + metricMonitor_args args = new metricMonitor_args(); + args.set_name(name); + args.set_options(options); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws NotAliveException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_metricMonitor(); + } + } + + public void restart(String name, String jsonConf, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + restart_call method_call = new restart_call(name, jsonConf, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class restart_call extends org.apache.thrift.async.TAsyncMethodCall { + private String name; + private String jsonConf; + public restart_call(String name, String jsonConf, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.name = name; + this.jsonConf = jsonConf; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("restart", org.apache.thrift.protocol.TMessageType.CALL, 0)); + restart_args args = new restart_args(); + args.set_name(name); + args.set_jsonConf(jsonConf); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws NotAliveException, InvalidTopologyException, TopologyAssignException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_restart(); + } + } + + public void beginLibUpload(String libName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + beginLibUpload_call method_call = new beginLibUpload_call(libName, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class beginLibUpload_call extends org.apache.thrift.async.TAsyncMethodCall { + private String libName; + public beginLibUpload_call(String libName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.libName = libName; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("beginLibUpload", org.apache.thrift.protocol.TMessageType.CALL, 0)); + beginLibUpload_args args = new beginLibUpload_args(); + args.set_libName(libName); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_beginLibUpload(); + } + } + + public void beginFileUpload(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + beginFileUpload_call method_call = new beginFileUpload_call(resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class beginFileUpload_call extends org.apache.thrift.async.TAsyncMethodCall { + public beginFileUpload_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("beginFileUpload", org.apache.thrift.protocol.TMessageType.CALL, 0)); + beginFileUpload_args args = new beginFileUpload_args(); + args.write(prot); + prot.writeMessageEnd(); + } + + public String getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_beginFileUpload(); + } + } + + public void uploadChunk(String location, ByteBuffer chunk, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + uploadChunk_call method_call = new uploadChunk_call(location, chunk, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class uploadChunk_call extends org.apache.thrift.async.TAsyncMethodCall { + private String location; + private ByteBuffer chunk; + public uploadChunk_call(String location, ByteBuffer chunk, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.location = location; + this.chunk = chunk; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("uploadChunk", org.apache.thrift.protocol.TMessageType.CALL, 0)); + uploadChunk_args args = new uploadChunk_args(); + args.set_location(location); + args.set_chunk(chunk); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_uploadChunk(); + } + } + + public void finishFileUpload(String location, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + finishFileUpload_call method_call = new finishFileUpload_call(location, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class finishFileUpload_call extends org.apache.thrift.async.TAsyncMethodCall { + private String location; + public finishFileUpload_call(String location, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.location = location; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("finishFileUpload", org.apache.thrift.protocol.TMessageType.CALL, 0)); + finishFileUpload_args args = new finishFileUpload_args(); + args.set_location(location); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_finishFileUpload(); + } + } + + public void beginFileDownload(String file, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + beginFileDownload_call method_call = new beginFileDownload_call(file, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class beginFileDownload_call extends org.apache.thrift.async.TAsyncMethodCall { + private String file; + public beginFileDownload_call(String file, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.file = file; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("beginFileDownload", org.apache.thrift.protocol.TMessageType.CALL, 0)); + beginFileDownload_args args = new beginFileDownload_args(); + args.set_file(file); + args.write(prot); + prot.writeMessageEnd(); + } + + public String getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_beginFileDownload(); + } + } + + public void downloadChunk(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + downloadChunk_call method_call = new downloadChunk_call(id, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class downloadChunk_call extends org.apache.thrift.async.TAsyncMethodCall { + private String id; + public downloadChunk_call(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.id = id; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("downloadChunk", org.apache.thrift.protocol.TMessageType.CALL, 0)); + downloadChunk_args args = new downloadChunk_args(); + args.set_id(id); + args.write(prot); + prot.writeMessageEnd(); + } + + public ByteBuffer getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_downloadChunk(); + } + } + + public void finishFileDownload(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + finishFileDownload_call method_call = new finishFileDownload_call(id, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class finishFileDownload_call extends org.apache.thrift.async.TAsyncMethodCall { + private String id; + public finishFileDownload_call(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.id = id; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("finishFileDownload", org.apache.thrift.protocol.TMessageType.CALL, 0)); + finishFileDownload_args args = new finishFileDownload_args(); + args.set_id(id); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_finishFileDownload(); + } + } + + public void getNimbusConf(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getNimbusConf_call method_call = new getNimbusConf_call(resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getNimbusConf_call extends org.apache.thrift.async.TAsyncMethodCall { + public getNimbusConf_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getNimbusConf", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getNimbusConf_args args = new getNimbusConf_args(); + args.write(prot); + prot.writeMessageEnd(); + } + + public String getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_getNimbusConf(); + } + } + + public void getTopologyConf(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getTopologyConf_call method_call = new getTopologyConf_call(id, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getTopologyConf_call extends org.apache.thrift.async.TAsyncMethodCall { + private String id; + public getTopologyConf_call(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.id = id; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTopologyConf", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getTopologyConf_args args = new getTopologyConf_args(); + args.set_id(id); + args.write(prot); + prot.writeMessageEnd(); + } + + public String getResult() throws NotAliveException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_getTopologyConf(); + } + } + + public void getTopologyId(String topologyName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getTopologyId_call method_call = new getTopologyId_call(topologyName, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getTopologyId_call extends org.apache.thrift.async.TAsyncMethodCall { + private String topologyName; + public getTopologyId_call(String topologyName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.topologyName = topologyName; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTopologyId", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getTopologyId_args args = new getTopologyId_args(); + args.set_topologyName(topologyName); + args.write(prot); + prot.writeMessageEnd(); + } + + public String getResult() throws NotAliveException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_getTopologyId(); + } + } + + public void getClusterInfo(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getClusterInfo_call method_call = new getClusterInfo_call(resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getClusterInfo_call extends org.apache.thrift.async.TAsyncMethodCall { + public getClusterInfo_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getClusterInfo", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getClusterInfo_args args = new getClusterInfo_args(); + args.write(prot); + prot.writeMessageEnd(); + } + + public ClusterSummary getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_getClusterInfo(); + } + } + + public void getSupervisorWorkers(String host, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getSupervisorWorkers_call method_call = new getSupervisorWorkers_call(host, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getSupervisorWorkers_call extends org.apache.thrift.async.TAsyncMethodCall { + private String host; + public getSupervisorWorkers_call(String host, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.host = host; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getSupervisorWorkers", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getSupervisorWorkers_args args = new getSupervisorWorkers_args(); + args.set_host(host); + args.write(prot); + prot.writeMessageEnd(); + } + + public SupervisorWorkers getResult() throws NotAliveException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_getSupervisorWorkers(); + } + } + + public void getTopologyInfo(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getTopologyInfo_call method_call = new getTopologyInfo_call(id, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getTopologyInfo_call extends org.apache.thrift.async.TAsyncMethodCall { + private String id; + public getTopologyInfo_call(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.id = id; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTopologyInfo", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getTopologyInfo_args args = new getTopologyInfo_args(); + args.set_id(id); + args.write(prot); + prot.writeMessageEnd(); + } + + public TopologyInfo getResult() throws NotAliveException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_getTopologyInfo(); + } + } + + public void getTopologyInfoByName(String topologyName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getTopologyInfoByName_call method_call = new getTopologyInfoByName_call(topologyName, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getTopologyInfoByName_call extends org.apache.thrift.async.TAsyncMethodCall { + private String topologyName; + public getTopologyInfoByName_call(String topologyName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.topologyName = topologyName; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTopologyInfoByName", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getTopologyInfoByName_args args = new getTopologyInfoByName_args(); + args.set_topologyName(topologyName); + args.write(prot); + prot.writeMessageEnd(); + } + + public TopologyInfo getResult() throws NotAliveException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_getTopologyInfoByName(); + } + } + + public void getTopology(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getTopology_call method_call = new getTopology_call(id, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getTopology_call extends org.apache.thrift.async.TAsyncMethodCall { + private String id; + public getTopology_call(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.id = id; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTopology", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getTopology_args args = new getTopology_args(); + args.set_id(id); + args.write(prot); + prot.writeMessageEnd(); + } + + public StormTopology getResult() throws NotAliveException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_getTopology(); + } + } + + public void getUserTopology(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getUserTopology_call method_call = new getUserTopology_call(id, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getUserTopology_call extends org.apache.thrift.async.TAsyncMethodCall { + private String id; + public getUserTopology_call(String id, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.id = id; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getUserTopology", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getUserTopology_args args = new getUserTopology_args(); + args.set_id(id); + args.write(prot); + prot.writeMessageEnd(); + } + + public StormTopology getResult() throws NotAliveException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_getUserTopology(); + } + } + + public void workerUploadMetric(WorkerUploadMetrics uploadMetrics, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + workerUploadMetric_call method_call = new workerUploadMetric_call(uploadMetrics, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class workerUploadMetric_call extends org.apache.thrift.async.TAsyncMethodCall { + private WorkerUploadMetrics uploadMetrics; + public workerUploadMetric_call(WorkerUploadMetrics uploadMetrics, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.uploadMetrics = uploadMetrics; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("workerUploadMetric", org.apache.thrift.protocol.TMessageType.CALL, 0)); + workerUploadMetric_args args = new workerUploadMetric_args(); + args.set_uploadMetrics(uploadMetrics); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_workerUploadMetric(); + } + } + + public void getTopologyMetric(String topologyName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getTopologyMetric_call method_call = new getTopologyMetric_call(topologyName, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getTopologyMetric_call extends org.apache.thrift.async.TAsyncMethodCall { + private String topologyName; + public getTopologyMetric_call(String topologyName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.topologyName = topologyName; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getTopologyMetric", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getTopologyMetric_args args = new getTopologyMetric_args(); + args.set_topologyName(topologyName); + args.write(prot); + prot.writeMessageEnd(); + } + + public TopologyMetric getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_getTopologyMetric(); + } + } + + public void getNettyMetric(String topologyName, int pos, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getNettyMetric_call method_call = new getNettyMetric_call(topologyName, pos, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getNettyMetric_call extends org.apache.thrift.async.TAsyncMethodCall { + private String topologyName; + private int pos; + public getNettyMetric_call(String topologyName, int pos, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.topologyName = topologyName; + this.pos = pos; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getNettyMetric", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getNettyMetric_args args = new getNettyMetric_args(); + args.set_topologyName(topologyName); + args.set_pos(pos); + args.write(prot); + prot.writeMessageEnd(); + } + + public NettyMetric getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_getNettyMetric(); + } + } + + public void getServerNettyMetric(String topologyName, String serverName, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getServerNettyMetric_call method_call = new getServerNettyMetric_call(topologyName, serverName, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getServerNettyMetric_call extends org.apache.thrift.async.TAsyncMethodCall { + private String topologyName; + private String serverName; + public getServerNettyMetric_call(String topologyName, String serverName, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.topologyName = topologyName; + this.serverName = serverName; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getServerNettyMetric", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getServerNettyMetric_args args = new getServerNettyMetric_args(); + args.set_topologyName(topologyName); + args.set_serverName(serverName); + args.write(prot); + prot.writeMessageEnd(); + } + + public NettyMetric getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_getServerNettyMetric(); + } + } + + public void getVersion(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + getVersion_call method_call = new getVersion_call(resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class getVersion_call extends org.apache.thrift.async.TAsyncMethodCall { + public getVersion_call(org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("getVersion", org.apache.thrift.protocol.TMessageType.CALL, 0)); + getVersion_args args = new getVersion_args(); + args.write(prot); + prot.writeMessageEnd(); + } + + public String getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_getVersion(); + } + } + + public void updateConf(String name, String conf, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + updateConf_call method_call = new updateConf_call(name, conf, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class updateConf_call extends org.apache.thrift.async.TAsyncMethodCall { + private String name; + private String conf; + public updateConf_call(String name, String conf, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.name = name; + this.conf = conf; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("updateConf", org.apache.thrift.protocol.TMessageType.CALL, 0)); + updateConf_args args = new updateConf_args(); + args.set_name(name); + args.set_conf(conf); + args.write(prot); + prot.writeMessageEnd(); + } + + public void getResult() throws NotAliveException, InvalidTopologyException, org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + (new Client(prot)).recv_updateConf(); + } + } + + } + + public static class Processor extends org.apache.thrift.TBaseProcessor implements org.apache.thrift.TProcessor { + private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName()); + public Processor(I iface) { + super(iface, getProcessMap(new HashMap>())); + } + + protected Processor(I iface, Map> processMap) { + super(iface, getProcessMap(processMap)); + } + + private static Map> getProcessMap(Map> processMap) { + processMap.put("submitTopology", new submitTopology()); + processMap.put("submitTopologyWithOpts", new submitTopologyWithOpts()); + processMap.put("killTopology", new killTopology()); + processMap.put("killTopologyWithOpts", new killTopologyWithOpts()); + processMap.put("activate", new activate()); + processMap.put("deactivate", new deactivate()); + processMap.put("rebalance", new rebalance()); + processMap.put("metricMonitor", new metricMonitor()); + processMap.put("restart", new restart()); + processMap.put("beginLibUpload", new beginLibUpload()); + processMap.put("beginFileUpload", new beginFileUpload()); + processMap.put("uploadChunk", new uploadChunk()); + processMap.put("finishFileUpload", new finishFileUpload()); + processMap.put("beginFileDownload", new beginFileDownload()); + processMap.put("downloadChunk", new downloadChunk()); + processMap.put("finishFileDownload", new finishFileDownload()); + processMap.put("getNimbusConf", new getNimbusConf()); + processMap.put("getTopologyConf", new getTopologyConf()); + processMap.put("getTopologyId", new getTopologyId()); + processMap.put("getClusterInfo", new getClusterInfo()); + processMap.put("getSupervisorWorkers", new getSupervisorWorkers()); + processMap.put("getTopologyInfo", new getTopologyInfo()); + processMap.put("getTopologyInfoByName", new getTopologyInfoByName()); + processMap.put("getTopology", new getTopology()); + processMap.put("getUserTopology", new getUserTopology()); + processMap.put("workerUploadMetric", new workerUploadMetric()); + processMap.put("getTopologyMetric", new getTopologyMetric()); + processMap.put("getNettyMetric", new getNettyMetric()); + processMap.put("getServerNettyMetric", new getServerNettyMetric()); + processMap.put("getVersion", new getVersion()); + processMap.put("updateConf", new updateConf()); + return processMap; + } + + public static class submitTopology extends org.apache.thrift.ProcessFunction { + public submitTopology() { + super("submitTopology"); + } + + public submitTopology_args getEmptyArgsInstance() { + return new submitTopology_args(); + } + + protected boolean isOneway() { + return false; + } + + public submitTopology_result getResult(I iface, submitTopology_args args) throws org.apache.thrift.TException { + submitTopology_result result = new submitTopology_result(); + try { + iface.submitTopology(args.name, args.uploadedJarLocation, args.jsonConf, args.topology); + } catch (AlreadyAliveException e) { + result.e = e; + } catch (InvalidTopologyException ite) { + result.ite = ite; + } catch (TopologyAssignException tae) { + result.tae = tae; + } + return result; + } + } + + public static class submitTopologyWithOpts extends org.apache.thrift.ProcessFunction { + public submitTopologyWithOpts() { + super("submitTopologyWithOpts"); + } + + public submitTopologyWithOpts_args getEmptyArgsInstance() { + return new submitTopologyWithOpts_args(); + } + + protected boolean isOneway() { + return false; + } + + public submitTopologyWithOpts_result getResult(I iface, submitTopologyWithOpts_args args) throws org.apache.thrift.TException { + submitTopologyWithOpts_result result = new submitTopologyWithOpts_result(); + try { + iface.submitTopologyWithOpts(args.name, args.uploadedJarLocation, args.jsonConf, args.topology, args.options); + } catch (AlreadyAliveException e) { + result.e = e; + } catch (InvalidTopologyException ite) { + result.ite = ite; + } catch (TopologyAssignException tae) { + result.tae = tae; + } + return result; + } + } + + public static class killTopology extends org.apache.thrift.ProcessFunction { + public killTopology() { + super("killTopology"); + } + + public killTopology_args getEmptyArgsInstance() { + return new killTopology_args(); + } + + protected boolean isOneway() { + return false; + } + + public killTopology_result getResult(I iface, killTopology_args args) throws org.apache.thrift.TException { + killTopology_result result = new killTopology_result(); + try { + iface.killTopology(args.name); + } catch (NotAliveException e) { + result.e = e; + } + return result; + } + } + + public static class killTopologyWithOpts extends org.apache.thrift.ProcessFunction { + public killTopologyWithOpts() { + super("killTopologyWithOpts"); + } + + public killTopologyWithOpts_args getEmptyArgsInstance() { + return new killTopologyWithOpts_args(); + } + + protected boolean isOneway() { + return false; + } + + public killTopologyWithOpts_result getResult(I iface, killTopologyWithOpts_args args) throws org.apache.thrift.TException { + killTopologyWithOpts_result result = new killTopologyWithOpts_result(); + try { + iface.killTopologyWithOpts(args.name, args.options); + } catch (NotAliveException e) { + result.e = e; + } + return result; + } + } + + public static class activate extends org.apache.thrift.ProcessFunction { + public activate() { + super("activate"); + } + + public activate_args getEmptyArgsInstance() { + return new activate_args(); + } + + protected boolean isOneway() { + return false; + } + + public activate_result getResult(I iface, activate_args args) throws org.apache.thrift.TException { + activate_result result = new activate_result(); + try { + iface.activate(args.name); + } catch (NotAliveException e) { + result.e = e; + } + return result; + } + } + + public static class deactivate extends org.apache.thrift.ProcessFunction { + public deactivate() { + super("deactivate"); + } + + public deactivate_args getEmptyArgsInstance() { + return new deactivate_args(); + } + + protected boolean isOneway() { + return false; + } + + public deactivate_result getResult(I iface, deactivate_args args) throws org.apache.thrift.TException { + deactivate_result result = new deactivate_result(); + try { + iface.deactivate(args.name); + } catch (NotAliveException e) { + result.e = e; + } + return result; + } + } + + public static class rebalance extends org.apache.thrift.ProcessFunction { + public rebalance() { + super("rebalance"); + } + + public rebalance_args getEmptyArgsInstance() { + return new rebalance_args(); + } + + protected boolean isOneway() { + return false; + } + + public rebalance_result getResult(I iface, rebalance_args args) throws org.apache.thrift.TException { + rebalance_result result = new rebalance_result(); + try { + iface.rebalance(args.name, args.options); + } catch (NotAliveException e) { + result.e = e; + } catch (InvalidTopologyException ite) { + result.ite = ite; + } + return result; + } + } + + public static class metricMonitor extends org.apache.thrift.ProcessFunction { + public metricMonitor() { + super("metricMonitor"); + } + + public metricMonitor_args getEmptyArgsInstance() { + return new metricMonitor_args(); + } + + protected boolean isOneway() { + return false; + } + + public metricMonitor_result getResult(I iface, metricMonitor_args args) throws org.apache.thrift.TException { + metricMonitor_result result = new metricMonitor_result(); + try { + iface.metricMonitor(args.name, args.options); + } catch (NotAliveException e) { + result.e = e; + } + return result; + } + } + + public static class restart extends org.apache.thrift.ProcessFunction { + public restart() { + super("restart"); + } + + public restart_args getEmptyArgsInstance() { + return new restart_args(); + } + + protected boolean isOneway() { + return false; + } + + public restart_result getResult(I iface, restart_args args) throws org.apache.thrift.TException { + restart_result result = new restart_result(); + try { + iface.restart(args.name, args.jsonConf); + } catch (NotAliveException e) { + result.e = e; + } catch (InvalidTopologyException ite) { + result.ite = ite; + } catch (TopologyAssignException tae) { + result.tae = tae; + } + return result; + } + } + + public static class beginLibUpload extends org.apache.thrift.ProcessFunction { + public beginLibUpload() { + super("beginLibUpload"); + } + + public beginLibUpload_args getEmptyArgsInstance() { + return new beginLibUpload_args(); + } + + protected boolean isOneway() { + return false; + } + + public beginLibUpload_result getResult(I iface, beginLibUpload_args args) throws org.apache.thrift.TException { + beginLibUpload_result result = new beginLibUpload_result(); + iface.beginLibUpload(args.libName); + return result; + } + } + + public static class beginFileUpload extends org.apache.thrift.ProcessFunction { + public beginFileUpload() { + super("beginFileUpload"); + } + + public beginFileUpload_args getEmptyArgsInstance() { + return new beginFileUpload_args(); + } + + protected boolean isOneway() { + return false; + } + + public beginFileUpload_result getResult(I iface, beginFileUpload_args args) throws org.apache.thrift.TException { + beginFileUpload_result result = new beginFileUpload_result(); + result.success = iface.beginFileUpload(); + return result; + } + } + + public static class uploadChunk extends org.apache.thrift.ProcessFunction { + public uploadChunk() { + super("uploadChunk"); + } + + public uploadChunk_args getEmptyArgsInstance() { + return new uploadChunk_args(); + } + + protected boolean isOneway() { + return false; + } + + public uploadChunk_result getResult(I iface, uploadChunk_args args) throws org.apache.thrift.TException { + uploadChunk_result result = new uploadChunk_result(); + iface.uploadChunk(args.location, args.chunk); + return result; + } + } + + public static class finishFileUpload extends org.apache.thrift.ProcessFunction { + public finishFileUpload() { + super("finishFileUpload"); + } + + public finishFileUpload_args getEmptyArgsInstance() { + return new finishFileUpload_args(); + } + + protected boolean isOneway() { + return false; + } + + public finishFileUpload_result getResult(I iface, finishFileUpload_args args) throws org.apache.thrift.TException { + finishFileUpload_result result = new finishFileUpload_result(); + iface.finishFileUpload(args.location); + return result; + } + } + + public static class beginFileDownload extends org.apache.thrift.ProcessFunction { + public beginFileDownload() { + super("beginFileDownload"); + } + + public beginFileDownload_args getEmptyArgsInstance() { + return new beginFileDownload_args(); + } + + protected boolean isOneway() { + return false; + } + + public beginFileDownload_result getResult(I iface, beginFileDownload_args args) throws org.apache.thrift.TException { + beginFileDownload_result result = new beginFileDownload_result(); + result.success = iface.beginFileDownload(args.file); + return result; + } + } + + public static class downloadChunk extends org.apache.thrift.ProcessFunction { + public downloadChunk() { + super("downloadChunk"); + } + + public downloadChunk_args getEmptyArgsInstance() { + return new downloadChunk_args(); + } + + protected boolean isOneway() { + return false; + } + + public downloadChunk_result getResult(I iface, downloadChunk_args args) throws org.apache.thrift.TException { + downloadChunk_result result = new downloadChunk_result(); + result.success = iface.downloadChunk(args.id); + return result; + } + } + + public static class finishFileDownload extends org.apache.thrift.ProcessFunction { + public finishFileDownload() { + super("finishFileDownload"); + } + + public finishFileDownload_args getEmptyArgsInstance() { + return new finishFileDownload_args(); + } + + protected boolean isOneway() { + return false; + } + + public finishFileDownload_result getResult(I iface, finishFileDownload_args args) throws org.apache.thrift.TException { + finishFileDownload_result result = new finishFileDownload_result(); + iface.finishFileDownload(args.id); + return result; + } + } + + public static class getNimbusConf extends org.apache.thrift.ProcessFunction { + public getNimbusConf() { + super("getNimbusConf"); + } + + public getNimbusConf_args getEmptyArgsInstance() { + return new getNimbusConf_args(); + } + + protected boolean isOneway() { + return false; + } + + public getNimbusConf_result getResult(I iface, getNimbusConf_args args) throws org.apache.thrift.TException { + getNimbusConf_result result = new getNimbusConf_result(); + result.success = iface.getNimbusConf(); + return result; + } + } + + public static class getTopologyConf extends org.apache.thrift.ProcessFunction { + public getTopologyConf() { + super("getTopologyConf"); + } + + public getTopologyConf_args getEmptyArgsInstance() { + return new getTopologyConf_args(); + } + + protected boolean isOneway() { + return false; + } + + public getTopologyConf_result getResult(I iface, getTopologyConf_args args) throws org.apache.thrift.TException { + getTopologyConf_result result = new getTopologyConf_result(); + try { + result.success = iface.getTopologyConf(args.id); + } catch (NotAliveException e) { + result.e = e; + } + return result; + } + } + + public static class getTopologyId extends org.apache.thrift.ProcessFunction { + public getTopologyId() { + super("getTopologyId"); + } + + public getTopologyId_args getEmptyArgsInstance() { + return new getTopologyId_args(); + } + + protected boolean isOneway() { + return false; + } + + public getTopologyId_result getResult(I iface, getTopologyId_args args) throws org.apache.thrift.TException { + getTopologyId_result result = new getTopologyId_result(); + try { + result.success = iface.getTopologyId(args.topologyName); + } catch (NotAliveException e) { + result.e = e; + } + return result; + } + } + + public static class getClusterInfo extends org.apache.thrift.ProcessFunction { + public getClusterInfo() { + super("getClusterInfo"); + } + + public getClusterInfo_args getEmptyArgsInstance() { + return new getClusterInfo_args(); + } + + protected boolean isOneway() { + return false; + } + + public getClusterInfo_result getResult(I iface, getClusterInfo_args args) throws org.apache.thrift.TException { + getClusterInfo_result result = new getClusterInfo_result(); + result.success = iface.getClusterInfo(); + return result; + } + } + + public static class getSupervisorWorkers extends org.apache.thrift.ProcessFunction { + public getSupervisorWorkers() { + super("getSupervisorWorkers"); + } + + public getSupervisorWorkers_args getEmptyArgsInstance() { + return new getSupervisorWorkers_args(); + } + + protected boolean isOneway() { + return false; + } + + public getSupervisorWorkers_result getResult(I iface, getSupervisorWorkers_args args) throws org.apache.thrift.TException { + getSupervisorWorkers_result result = new getSupervisorWorkers_result(); + try { + result.success = iface.getSupervisorWorkers(args.host); + } catch (NotAliveException e) { + result.e = e; + } + return result; + } + } + + public static class getTopologyInfo extends org.apache.thrift.ProcessFunction { + public getTopologyInfo() { + super("getTopologyInfo"); + } + + public getTopologyInfo_args getEmptyArgsInstance() { + return new getTopologyInfo_args(); + } + + protected boolean isOneway() { + return false; + } + + public getTopologyInfo_result getResult(I iface, getTopologyInfo_args args) throws org.apache.thrift.TException { + getTopologyInfo_result result = new getTopologyInfo_result(); + try { + result.success = iface.getTopologyInfo(args.id); + } catch (NotAliveException e) { + result.e = e; + } + return result; + } + } + + public static class getTopologyInfoByName extends org.apache.thrift.ProcessFunction { + public getTopologyInfoByName() { + super("getTopologyInfoByName"); + } + + public getTopologyInfoByName_args getEmptyArgsInstance() { + return new getTopologyInfoByName_args(); + } + + protected boolean isOneway() { + return false; + } + + public getTopologyInfoByName_result getResult(I iface, getTopologyInfoByName_args args) throws org.apache.thrift.TException { + getTopologyInfoByName_result result = new getTopologyInfoByName_result(); + try { + result.success = iface.getTopologyInfoByName(args.topologyName); + } catch (NotAliveException e) { + result.e = e; + } + return result; + } + } + + public static class getTopology extends org.apache.thrift.ProcessFunction { + public getTopology() { + super("getTopology"); + } + + public getTopology_args getEmptyArgsInstance() { + return new getTopology_args(); + } + + protected boolean isOneway() { + return false; + } + + public getTopology_result getResult(I iface, getTopology_args args) throws org.apache.thrift.TException { + getTopology_result result = new getTopology_result(); + try { + result.success = iface.getTopology(args.id); + } catch (NotAliveException e) { + result.e = e; + } + return result; + } + } + + public static class getUserTopology extends org.apache.thrift.ProcessFunction { + public getUserTopology() { + super("getUserTopology"); + } + + public getUserTopology_args getEmptyArgsInstance() { + return new getUserTopology_args(); + } + + protected boolean isOneway() { + return false; + } + + public getUserTopology_result getResult(I iface, getUserTopology_args args) throws org.apache.thrift.TException { + getUserTopology_result result = new getUserTopology_result(); + try { + result.success = iface.getUserTopology(args.id); + } catch (NotAliveException e) { + result.e = e; + } + return result; + } + } + + public static class workerUploadMetric extends org.apache.thrift.ProcessFunction { + public workerUploadMetric() { + super("workerUploadMetric"); + } + + public workerUploadMetric_args getEmptyArgsInstance() { + return new workerUploadMetric_args(); + } + + protected boolean isOneway() { + return false; + } + + public workerUploadMetric_result getResult(I iface, workerUploadMetric_args args) throws org.apache.thrift.TException { + workerUploadMetric_result result = new workerUploadMetric_result(); + iface.workerUploadMetric(args.uploadMetrics); + return result; + } + } + + public static class getTopologyMetric extends org.apache.thrift.ProcessFunction { + public getTopologyMetric() { + super("getTopologyMetric"); + } + + public getTopologyMetric_args getEmptyArgsInstance() { + return new getTopologyMetric_args(); + } + + protected boolean isOneway() { + return false; + } + + public getTopologyMetric_result getResult(I iface, getTopologyMetric_args args) throws org.apache.thrift.TException { + getTopologyMetric_result result = new getTopologyMetric_result(); + result.success = iface.getTopologyMetric(args.topologyName); + return result; + } + } + + public static class getNettyMetric extends org.apache.thrift.ProcessFunction { + public getNettyMetric() { + super("getNettyMetric"); + } + + public getNettyMetric_args getEmptyArgsInstance() { + return new getNettyMetric_args(); + } + + protected boolean isOneway() { + return false; + } + + public getNettyMetric_result getResult(I iface, getNettyMetric_args args) throws org.apache.thrift.TException { + getNettyMetric_result result = new getNettyMetric_result(); + result.success = iface.getNettyMetric(args.topologyName, args.pos); + return result; + } + } + + public static class getServerNettyMetric extends org.apache.thrift.ProcessFunction { + public getServerNettyMetric() { + super("getServerNettyMetric"); + } + + public getServerNettyMetric_args getEmptyArgsInstance() { + return new getServerNettyMetric_args(); + } + + protected boolean isOneway() { + return false; + } + + public getServerNettyMetric_result getResult(I iface, getServerNettyMetric_args args) throws org.apache.thrift.TException { + getServerNettyMetric_result result = new getServerNettyMetric_result(); + result.success = iface.getServerNettyMetric(args.topologyName, args.serverName); + return result; + } + } + + public static class getVersion extends org.apache.thrift.ProcessFunction { + public getVersion() { + super("getVersion"); + } + + public getVersion_args getEmptyArgsInstance() { + return new getVersion_args(); + } + + protected boolean isOneway() { + return false; + } + + public getVersion_result getResult(I iface, getVersion_args args) throws org.apache.thrift.TException { + getVersion_result result = new getVersion_result(); + result.success = iface.getVersion(); + return result; + } + } + + public static class updateConf extends org.apache.thrift.ProcessFunction { + public updateConf() { + super("updateConf"); + } + + public updateConf_args getEmptyArgsInstance() { + return new updateConf_args(); + } + + protected boolean isOneway() { + return false; + } + + public updateConf_result getResult(I iface, updateConf_args args) throws org.apache.thrift.TException { + updateConf_result result = new updateConf_result(); + try { + iface.updateConf(args.name, args.conf); + } catch (NotAliveException e) { + result.e = e; + } catch (InvalidTopologyException ite) { + result.ite = ite; + } + return result; + } + } + + } + + public static class AsyncProcessor extends org.apache.thrift.TBaseAsyncProcessor { + private static final Logger LOGGER = LoggerFactory.getLogger(AsyncProcessor.class.getName()); + public AsyncProcessor(I iface) { + super(iface, getProcessMap(new HashMap>())); + } + + protected AsyncProcessor(I iface, Map> processMap) { + super(iface, getProcessMap(processMap)); + } + + private static Map> getProcessMap(Map> processMap) { + processMap.put("submitTopology", new submitTopology()); + processMap.put("submitTopologyWithOpts", new submitTopologyWithOpts()); + processMap.put("killTopology", new killTopology()); + processMap.put("killTopologyWithOpts", new killTopologyWithOpts()); + processMap.put("activate", new activate()); + processMap.put("deactivate", new deactivate()); + processMap.put("rebalance", new rebalance()); + processMap.put("metricMonitor", new metricMonitor()); + processMap.put("restart", new restart()); + processMap.put("beginLibUpload", new beginLibUpload()); + processMap.put("beginFileUpload", new beginFileUpload()); + processMap.put("uploadChunk", new uploadChunk()); + processMap.put("finishFileUpload", new finishFileUpload()); + processMap.put("beginFileDownload", new beginFileDownload()); + processMap.put("downloadChunk", new downloadChunk()); + processMap.put("finishFileDownload", new finishFileDownload()); + processMap.put("getNimbusConf", new getNimbusConf()); + processMap.put("getTopologyConf", new getTopologyConf()); + processMap.put("getTopologyId", new getTopologyId()); + processMap.put("getClusterInfo", new getClusterInfo()); + processMap.put("getSupervisorWorkers", new getSupervisorWorkers()); + processMap.put("getTopologyInfo", new getTopologyInfo()); + processMap.put("getTopologyInfoByName", new getTopologyInfoByName()); + processMap.put("getTopology", new getTopology()); + processMap.put("getUserTopology", new getUserTopology()); + processMap.put("workerUploadMetric", new workerUploadMetric()); + processMap.put("getTopologyMetric", new getTopologyMetric()); + processMap.put("getNettyMetric", new getNettyMetric()); + processMap.put("getServerNettyMetric", new getServerNettyMetric()); + processMap.put("getVersion", new getVersion()); + processMap.put("updateConf", new updateConf()); + return processMap; + } + + public static class submitTopology extends org.apache.thrift.AsyncProcessFunction { + public submitTopology() { + super("submitTopology"); + } + + public submitTopology_args getEmptyArgsInstance() { + return new submitTopology_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + submitTopology_result result = new submitTopology_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + submitTopology_result result = new submitTopology_result(); + if (e instanceof AlreadyAliveException) { + result.e = (AlreadyAliveException) e; + result.set_e_isSet(true); + msg = result; + } + else if (e instanceof InvalidTopologyException) { + result.ite = (InvalidTopologyException) e; + result.set_ite_isSet(true); + msg = result; + } + else if (e instanceof TopologyAssignException) { + result.tae = (TopologyAssignException) e; + result.set_tae_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, submitTopology_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.submitTopology(args.name, args.uploadedJarLocation, args.jsonConf, args.topology,resultHandler); + } + } + + public static class submitTopologyWithOpts extends org.apache.thrift.AsyncProcessFunction { + public submitTopologyWithOpts() { + super("submitTopologyWithOpts"); + } + + public submitTopologyWithOpts_args getEmptyArgsInstance() { + return new submitTopologyWithOpts_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + submitTopologyWithOpts_result result = new submitTopologyWithOpts_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + submitTopologyWithOpts_result result = new submitTopologyWithOpts_result(); + if (e instanceof AlreadyAliveException) { + result.e = (AlreadyAliveException) e; + result.set_e_isSet(true); + msg = result; + } + else if (e instanceof InvalidTopologyException) { + result.ite = (InvalidTopologyException) e; + result.set_ite_isSet(true); + msg = result; + } + else if (e instanceof TopologyAssignException) { + result.tae = (TopologyAssignException) e; + result.set_tae_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, submitTopologyWithOpts_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.submitTopologyWithOpts(args.name, args.uploadedJarLocation, args.jsonConf, args.topology, args.options,resultHandler); + } + } + + public static class killTopology extends org.apache.thrift.AsyncProcessFunction { + public killTopology() { + super("killTopology"); + } + + public killTopology_args getEmptyArgsInstance() { + return new killTopology_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + killTopology_result result = new killTopology_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + killTopology_result result = new killTopology_result(); + if (e instanceof NotAliveException) { + result.e = (NotAliveException) e; + result.set_e_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, killTopology_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.killTopology(args.name,resultHandler); + } + } + + public static class killTopologyWithOpts extends org.apache.thrift.AsyncProcessFunction { + public killTopologyWithOpts() { + super("killTopologyWithOpts"); + } + + public killTopologyWithOpts_args getEmptyArgsInstance() { + return new killTopologyWithOpts_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + killTopologyWithOpts_result result = new killTopologyWithOpts_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + killTopologyWithOpts_result result = new killTopologyWithOpts_result(); + if (e instanceof NotAliveException) { + result.e = (NotAliveException) e; + result.set_e_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, killTopologyWithOpts_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.killTopologyWithOpts(args.name, args.options,resultHandler); + } + } + + public static class activate extends org.apache.thrift.AsyncProcessFunction { + public activate() { + super("activate"); + } + + public activate_args getEmptyArgsInstance() { + return new activate_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + activate_result result = new activate_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + activate_result result = new activate_result(); + if (e instanceof NotAliveException) { + result.e = (NotAliveException) e; + result.set_e_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, activate_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.activate(args.name,resultHandler); + } + } + + public static class deactivate extends org.apache.thrift.AsyncProcessFunction { + public deactivate() { + super("deactivate"); + } + + public deactivate_args getEmptyArgsInstance() { + return new deactivate_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + deactivate_result result = new deactivate_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + deactivate_result result = new deactivate_result(); + if (e instanceof NotAliveException) { + result.e = (NotAliveException) e; + result.set_e_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, deactivate_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.deactivate(args.name,resultHandler); + } + } + + public static class rebalance extends org.apache.thrift.AsyncProcessFunction { + public rebalance() { + super("rebalance"); + } + + public rebalance_args getEmptyArgsInstance() { + return new rebalance_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + rebalance_result result = new rebalance_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + rebalance_result result = new rebalance_result(); + if (e instanceof NotAliveException) { + result.e = (NotAliveException) e; + result.set_e_isSet(true); + msg = result; + } + else if (e instanceof InvalidTopologyException) { + result.ite = (InvalidTopologyException) e; + result.set_ite_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, rebalance_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.rebalance(args.name, args.options,resultHandler); + } + } + + public static class metricMonitor extends org.apache.thrift.AsyncProcessFunction { + public metricMonitor() { + super("metricMonitor"); + } + + public metricMonitor_args getEmptyArgsInstance() { + return new metricMonitor_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + metricMonitor_result result = new metricMonitor_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + metricMonitor_result result = new metricMonitor_result(); + if (e instanceof NotAliveException) { + result.e = (NotAliveException) e; + result.set_e_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, metricMonitor_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.metricMonitor(args.name, args.options,resultHandler); + } + } + + public static class restart extends org.apache.thrift.AsyncProcessFunction { + public restart() { + super("restart"); + } + + public restart_args getEmptyArgsInstance() { + return new restart_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + restart_result result = new restart_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + restart_result result = new restart_result(); + if (e instanceof NotAliveException) { + result.e = (NotAliveException) e; + result.set_e_isSet(true); + msg = result; + } + else if (e instanceof InvalidTopologyException) { + result.ite = (InvalidTopologyException) e; + result.set_ite_isSet(true); + msg = result; + } + else if (e instanceof TopologyAssignException) { + result.tae = (TopologyAssignException) e; + result.set_tae_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, restart_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.restart(args.name, args.jsonConf,resultHandler); + } + } + + public static class beginLibUpload extends org.apache.thrift.AsyncProcessFunction { + public beginLibUpload() { + super("beginLibUpload"); + } + + public beginLibUpload_args getEmptyArgsInstance() { + return new beginLibUpload_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + beginLibUpload_result result = new beginLibUpload_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + beginLibUpload_result result = new beginLibUpload_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, beginLibUpload_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.beginLibUpload(args.libName,resultHandler); + } + } + + public static class beginFileUpload extends org.apache.thrift.AsyncProcessFunction { + public beginFileUpload() { + super("beginFileUpload"); + } + + public beginFileUpload_args getEmptyArgsInstance() { + return new beginFileUpload_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(String o) { + beginFileUpload_result result = new beginFileUpload_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + beginFileUpload_result result = new beginFileUpload_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, beginFileUpload_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.beginFileUpload(resultHandler); + } + } + + public static class uploadChunk extends org.apache.thrift.AsyncProcessFunction { + public uploadChunk() { + super("uploadChunk"); + } + + public uploadChunk_args getEmptyArgsInstance() { + return new uploadChunk_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + uploadChunk_result result = new uploadChunk_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + uploadChunk_result result = new uploadChunk_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, uploadChunk_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.uploadChunk(args.location, args.chunk,resultHandler); + } + } + + public static class finishFileUpload extends org.apache.thrift.AsyncProcessFunction { + public finishFileUpload() { + super("finishFileUpload"); + } + + public finishFileUpload_args getEmptyArgsInstance() { + return new finishFileUpload_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + finishFileUpload_result result = new finishFileUpload_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + finishFileUpload_result result = new finishFileUpload_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, finishFileUpload_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.finishFileUpload(args.location,resultHandler); + } + } + + public static class beginFileDownload extends org.apache.thrift.AsyncProcessFunction { + public beginFileDownload() { + super("beginFileDownload"); + } + + public beginFileDownload_args getEmptyArgsInstance() { + return new beginFileDownload_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(String o) { + beginFileDownload_result result = new beginFileDownload_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + beginFileDownload_result result = new beginFileDownload_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, beginFileDownload_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.beginFileDownload(args.file,resultHandler); + } + } + + public static class downloadChunk extends org.apache.thrift.AsyncProcessFunction { + public downloadChunk() { + super("downloadChunk"); + } + + public downloadChunk_args getEmptyArgsInstance() { + return new downloadChunk_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(ByteBuffer o) { + downloadChunk_result result = new downloadChunk_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + downloadChunk_result result = new downloadChunk_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, downloadChunk_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.downloadChunk(args.id,resultHandler); + } + } + + public static class finishFileDownload extends org.apache.thrift.AsyncProcessFunction { + public finishFileDownload() { + super("finishFileDownload"); + } + + public finishFileDownload_args getEmptyArgsInstance() { + return new finishFileDownload_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + finishFileDownload_result result = new finishFileDownload_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + finishFileDownload_result result = new finishFileDownload_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, finishFileDownload_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.finishFileDownload(args.id,resultHandler); + } + } + + public static class getNimbusConf extends org.apache.thrift.AsyncProcessFunction { + public getNimbusConf() { + super("getNimbusConf"); + } + + public getNimbusConf_args getEmptyArgsInstance() { + return new getNimbusConf_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(String o) { + getNimbusConf_result result = new getNimbusConf_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getNimbusConf_result result = new getNimbusConf_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getNimbusConf_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getNimbusConf(resultHandler); + } + } + + public static class getTopologyConf extends org.apache.thrift.AsyncProcessFunction { + public getTopologyConf() { + super("getTopologyConf"); + } + + public getTopologyConf_args getEmptyArgsInstance() { + return new getTopologyConf_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(String o) { + getTopologyConf_result result = new getTopologyConf_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getTopologyConf_result result = new getTopologyConf_result(); + if (e instanceof NotAliveException) { + result.e = (NotAliveException) e; + result.set_e_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getTopologyConf_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getTopologyConf(args.id,resultHandler); + } + } + + public static class getTopologyId extends org.apache.thrift.AsyncProcessFunction { + public getTopologyId() { + super("getTopologyId"); + } + + public getTopologyId_args getEmptyArgsInstance() { + return new getTopologyId_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(String o) { + getTopologyId_result result = new getTopologyId_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getTopologyId_result result = new getTopologyId_result(); + if (e instanceof NotAliveException) { + result.e = (NotAliveException) e; + result.set_e_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getTopologyId_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getTopologyId(args.topologyName,resultHandler); + } + } + + public static class getClusterInfo extends org.apache.thrift.AsyncProcessFunction { + public getClusterInfo() { + super("getClusterInfo"); + } + + public getClusterInfo_args getEmptyArgsInstance() { + return new getClusterInfo_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(ClusterSummary o) { + getClusterInfo_result result = new getClusterInfo_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getClusterInfo_result result = new getClusterInfo_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getClusterInfo_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getClusterInfo(resultHandler); + } + } + + public static class getSupervisorWorkers extends org.apache.thrift.AsyncProcessFunction { + public getSupervisorWorkers() { + super("getSupervisorWorkers"); + } + + public getSupervisorWorkers_args getEmptyArgsInstance() { + return new getSupervisorWorkers_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(SupervisorWorkers o) { + getSupervisorWorkers_result result = new getSupervisorWorkers_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getSupervisorWorkers_result result = new getSupervisorWorkers_result(); + if (e instanceof NotAliveException) { + result.e = (NotAliveException) e; + result.set_e_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getSupervisorWorkers_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getSupervisorWorkers(args.host,resultHandler); + } + } + + public static class getTopologyInfo extends org.apache.thrift.AsyncProcessFunction { + public getTopologyInfo() { + super("getTopologyInfo"); + } + + public getTopologyInfo_args getEmptyArgsInstance() { + return new getTopologyInfo_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TopologyInfo o) { + getTopologyInfo_result result = new getTopologyInfo_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getTopologyInfo_result result = new getTopologyInfo_result(); + if (e instanceof NotAliveException) { + result.e = (NotAliveException) e; + result.set_e_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getTopologyInfo_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getTopologyInfo(args.id,resultHandler); + } + } + + public static class getTopologyInfoByName extends org.apache.thrift.AsyncProcessFunction { + public getTopologyInfoByName() { + super("getTopologyInfoByName"); + } + + public getTopologyInfoByName_args getEmptyArgsInstance() { + return new getTopologyInfoByName_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TopologyInfo o) { + getTopologyInfoByName_result result = new getTopologyInfoByName_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getTopologyInfoByName_result result = new getTopologyInfoByName_result(); + if (e instanceof NotAliveException) { + result.e = (NotAliveException) e; + result.set_e_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getTopologyInfoByName_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getTopologyInfoByName(args.topologyName,resultHandler); + } + } + + public static class getTopology extends org.apache.thrift.AsyncProcessFunction { + public getTopology() { + super("getTopology"); + } + + public getTopology_args getEmptyArgsInstance() { + return new getTopology_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(StormTopology o) { + getTopology_result result = new getTopology_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getTopology_result result = new getTopology_result(); + if (e instanceof NotAliveException) { + result.e = (NotAliveException) e; + result.set_e_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getTopology_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getTopology(args.id,resultHandler); + } + } + + public static class getUserTopology extends org.apache.thrift.AsyncProcessFunction { + public getUserTopology() { + super("getUserTopology"); + } + + public getUserTopology_args getEmptyArgsInstance() { + return new getUserTopology_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(StormTopology o) { + getUserTopology_result result = new getUserTopology_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getUserTopology_result result = new getUserTopology_result(); + if (e instanceof NotAliveException) { + result.e = (NotAliveException) e; + result.set_e_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getUserTopology_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getUserTopology(args.id,resultHandler); + } + } + + public static class workerUploadMetric extends org.apache.thrift.AsyncProcessFunction { + public workerUploadMetric() { + super("workerUploadMetric"); + } + + public workerUploadMetric_args getEmptyArgsInstance() { + return new workerUploadMetric_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + workerUploadMetric_result result = new workerUploadMetric_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + workerUploadMetric_result result = new workerUploadMetric_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, workerUploadMetric_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.workerUploadMetric(args.uploadMetrics,resultHandler); + } + } + + public static class getTopologyMetric extends org.apache.thrift.AsyncProcessFunction { + public getTopologyMetric() { + super("getTopologyMetric"); + } + + public getTopologyMetric_args getEmptyArgsInstance() { + return new getTopologyMetric_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TopologyMetric o) { + getTopologyMetric_result result = new getTopologyMetric_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getTopologyMetric_result result = new getTopologyMetric_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getTopologyMetric_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getTopologyMetric(args.topologyName,resultHandler); + } + } + + public static class getNettyMetric extends org.apache.thrift.AsyncProcessFunction { + public getNettyMetric() { + super("getNettyMetric"); + } + + public getNettyMetric_args getEmptyArgsInstance() { + return new getNettyMetric_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(NettyMetric o) { + getNettyMetric_result result = new getNettyMetric_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getNettyMetric_result result = new getNettyMetric_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getNettyMetric_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getNettyMetric(args.topologyName, args.pos,resultHandler); + } + } + + public static class getServerNettyMetric extends org.apache.thrift.AsyncProcessFunction { + public getServerNettyMetric() { + super("getServerNettyMetric"); + } + + public getServerNettyMetric_args getEmptyArgsInstance() { + return new getServerNettyMetric_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(NettyMetric o) { + getServerNettyMetric_result result = new getServerNettyMetric_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getServerNettyMetric_result result = new getServerNettyMetric_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getServerNettyMetric_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getServerNettyMetric(args.topologyName, args.serverName,resultHandler); + } + } + + public static class getVersion extends org.apache.thrift.AsyncProcessFunction { + public getVersion() { + super("getVersion"); + } + + public getVersion_args getEmptyArgsInstance() { + return new getVersion_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(String o) { + getVersion_result result = new getVersion_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + getVersion_result result = new getVersion_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, getVersion_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.getVersion(resultHandler); + } + } + + public static class updateConf extends org.apache.thrift.AsyncProcessFunction { + public updateConf() { + super("updateConf"); + } + + public updateConf_args getEmptyArgsInstance() { + return new updateConf_args(); + } + + public AsyncMethodCallback getResultHandler(final AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(Void o) { + updateConf_result result = new updateConf_result(); + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + updateConf_result result = new updateConf_result(); + if (e instanceof NotAliveException) { + result.e = (NotAliveException) e; + result.set_e_isSet(true); + msg = result; + } + else if (e instanceof InvalidTopologyException) { + result.ite = (InvalidTopologyException) e; + result.set_ite_isSet(true); + msg = result; + } + else + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, updateConf_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.updateConf(args.name, args.conf,resultHandler); + } + } + + } + + public static class submitTopology_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("submitTopology_args"); + + private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField UPLOADED_JAR_LOCATION_FIELD_DESC = new org.apache.thrift.protocol.TField("uploadedJarLocation", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField JSON_CONF_FIELD_DESC = new org.apache.thrift.protocol.TField("jsonConf", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField TOPOLOGY_FIELD_DESC = new org.apache.thrift.protocol.TField("topology", org.apache.thrift.protocol.TType.STRUCT, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new submitTopology_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new submitTopology_argsTupleSchemeFactory()); + } + + private String name; // required + private String uploadedJarLocation; // required + private String jsonConf; // required + private StormTopology topology; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NAME((short)1, "name"), + UPLOADED_JAR_LOCATION((short)2, "uploadedJarLocation"), + JSON_CONF((short)3, "jsonConf"), + TOPOLOGY((short)4, "topology"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NAME + return NAME; + case 2: // UPLOADED_JAR_LOCATION + return UPLOADED_JAR_LOCATION; + case 3: // JSON_CONF + return JSON_CONF; + case 4: // TOPOLOGY + return TOPOLOGY; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.UPLOADED_JAR_LOCATION, new org.apache.thrift.meta_data.FieldMetaData("uploadedJarLocation", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.JSON_CONF, new org.apache.thrift.meta_data.FieldMetaData("jsonConf", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TOPOLOGY, new org.apache.thrift.meta_data.FieldMetaData("topology", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, StormTopology.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(submitTopology_args.class, metaDataMap); + } + + public submitTopology_args() { + } + + public submitTopology_args( + String name, + String uploadedJarLocation, + String jsonConf, + StormTopology topology) + { + this(); + this.name = name; + this.uploadedJarLocation = uploadedJarLocation; + this.jsonConf = jsonConf; + this.topology = topology; + } + + /** + * Performs a deep copy on other. + */ + public submitTopology_args(submitTopology_args other) { + if (other.is_set_name()) { + this.name = other.name; + } + if (other.is_set_uploadedJarLocation()) { + this.uploadedJarLocation = other.uploadedJarLocation; + } + if (other.is_set_jsonConf()) { + this.jsonConf = other.jsonConf; + } + if (other.is_set_topology()) { + this.topology = new StormTopology(other.topology); + } + } + + public submitTopology_args deepCopy() { + return new submitTopology_args(this); + } + + @Override + public void clear() { + this.name = null; + this.uploadedJarLocation = null; + this.jsonConf = null; + this.topology = null; + } + + public String get_name() { + return this.name; + } + + public void set_name(String name) { + this.name = name; + } + + public void unset_name() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean is_set_name() { + return this.name != null; + } + + public void set_name_isSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public String get_uploadedJarLocation() { + return this.uploadedJarLocation; + } + + public void set_uploadedJarLocation(String uploadedJarLocation) { + this.uploadedJarLocation = uploadedJarLocation; + } + + public void unset_uploadedJarLocation() { + this.uploadedJarLocation = null; + } + + /** Returns true if field uploadedJarLocation is set (has been assigned a value) and false otherwise */ + public boolean is_set_uploadedJarLocation() { + return this.uploadedJarLocation != null; + } + + public void set_uploadedJarLocation_isSet(boolean value) { + if (!value) { + this.uploadedJarLocation = null; + } + } + + public String get_jsonConf() { + return this.jsonConf; + } + + public void set_jsonConf(String jsonConf) { + this.jsonConf = jsonConf; + } + + public void unset_jsonConf() { + this.jsonConf = null; + } + + /** Returns true if field jsonConf is set (has been assigned a value) and false otherwise */ + public boolean is_set_jsonConf() { + return this.jsonConf != null; + } + + public void set_jsonConf_isSet(boolean value) { + if (!value) { + this.jsonConf = null; + } + } + + public StormTopology get_topology() { + return this.topology; + } + + public void set_topology(StormTopology topology) { + this.topology = topology; + } + + public void unset_topology() { + this.topology = null; + } + + /** Returns true if field topology is set (has been assigned a value) and false otherwise */ + public boolean is_set_topology() { + return this.topology != null; + } + + public void set_topology_isSet(boolean value) { + if (!value) { + this.topology = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME: + if (value == null) { + unset_name(); + } else { + set_name((String)value); + } + break; + + case UPLOADED_JAR_LOCATION: + if (value == null) { + unset_uploadedJarLocation(); + } else { + set_uploadedJarLocation((String)value); + } + break; + + case JSON_CONF: + if (value == null) { + unset_jsonConf(); + } else { + set_jsonConf((String)value); + } + break; + + case TOPOLOGY: + if (value == null) { + unset_topology(); + } else { + set_topology((StormTopology)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME: + return get_name(); + + case UPLOADED_JAR_LOCATION: + return get_uploadedJarLocation(); + + case JSON_CONF: + return get_jsonConf(); + + case TOPOLOGY: + return get_topology(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NAME: + return is_set_name(); + case UPLOADED_JAR_LOCATION: + return is_set_uploadedJarLocation(); + case JSON_CONF: + return is_set_jsonConf(); + case TOPOLOGY: + return is_set_topology(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof submitTopology_args) + return this.equals((submitTopology_args)that); + return false; + } + + public boolean equals(submitTopology_args that) { + if (that == null) + return false; + + boolean this_present_name = true && this.is_set_name(); + boolean that_present_name = true && that.is_set_name(); + if (this_present_name || that_present_name) { + if (!(this_present_name && that_present_name)) + return false; + if (!this.name.equals(that.name)) + return false; + } + + boolean this_present_uploadedJarLocation = true && this.is_set_uploadedJarLocation(); + boolean that_present_uploadedJarLocation = true && that.is_set_uploadedJarLocation(); + if (this_present_uploadedJarLocation || that_present_uploadedJarLocation) { + if (!(this_present_uploadedJarLocation && that_present_uploadedJarLocation)) + return false; + if (!this.uploadedJarLocation.equals(that.uploadedJarLocation)) + return false; + } + + boolean this_present_jsonConf = true && this.is_set_jsonConf(); + boolean that_present_jsonConf = true && that.is_set_jsonConf(); + if (this_present_jsonConf || that_present_jsonConf) { + if (!(this_present_jsonConf && that_present_jsonConf)) + return false; + if (!this.jsonConf.equals(that.jsonConf)) + return false; + } + + boolean this_present_topology = true && this.is_set_topology(); + boolean that_present_topology = true && that.is_set_topology(); + if (this_present_topology || that_present_topology) { + if (!(this_present_topology && that_present_topology)) + return false; + if (!this.topology.equals(that.topology)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_name = true && (is_set_name()); + list.add(present_name); + if (present_name) + list.add(name); + + boolean present_uploadedJarLocation = true && (is_set_uploadedJarLocation()); + list.add(present_uploadedJarLocation); + if (present_uploadedJarLocation) + list.add(uploadedJarLocation); + + boolean present_jsonConf = true && (is_set_jsonConf()); + list.add(present_jsonConf); + if (present_jsonConf) + list.add(jsonConf); + + boolean present_topology = true && (is_set_topology()); + list.add(present_topology); + if (present_topology) + list.add(topology); + + return list.hashCode(); + } + + @Override + public int compareTo(submitTopology_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_uploadedJarLocation()).compareTo(other.is_set_uploadedJarLocation()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_uploadedJarLocation()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uploadedJarLocation, other.uploadedJarLocation); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_jsonConf()).compareTo(other.is_set_jsonConf()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_jsonConf()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.jsonConf, other.jsonConf); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_topology()).compareTo(other.is_set_topology()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_topology()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology, other.topology); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("submitTopology_args("); + boolean first = true; + + sb.append("name:"); + if (this.name == null) { + sb.append("null"); + } else { + sb.append(this.name); + } + first = false; + if (!first) sb.append(", "); + sb.append("uploadedJarLocation:"); + if (this.uploadedJarLocation == null) { + sb.append("null"); + } else { + sb.append(this.uploadedJarLocation); + } + first = false; + if (!first) sb.append(", "); + sb.append("jsonConf:"); + if (this.jsonConf == null) { + sb.append("null"); + } else { + sb.append(this.jsonConf); + } + first = false; + if (!first) sb.append(", "); + sb.append("topology:"); + if (this.topology == null) { + sb.append("null"); + } else { + sb.append(this.topology); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (topology != null) { + topology.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class submitTopology_argsStandardSchemeFactory implements SchemeFactory { + public submitTopology_argsStandardScheme getScheme() { + return new submitTopology_argsStandardScheme(); + } + } + + private static class submitTopology_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, submitTopology_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // UPLOADED_JAR_LOCATION + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.uploadedJarLocation = iprot.readString(); + struct.set_uploadedJarLocation_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // JSON_CONF + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.jsonConf = iprot.readString(); + struct.set_jsonConf_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // TOPOLOGY + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.topology = new StormTopology(); + struct.topology.read(iprot); + struct.set_topology_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, submitTopology_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.name != null) { + oprot.writeFieldBegin(NAME_FIELD_DESC); + oprot.writeString(struct.name); + oprot.writeFieldEnd(); + } + if (struct.uploadedJarLocation != null) { + oprot.writeFieldBegin(UPLOADED_JAR_LOCATION_FIELD_DESC); + oprot.writeString(struct.uploadedJarLocation); + oprot.writeFieldEnd(); + } + if (struct.jsonConf != null) { + oprot.writeFieldBegin(JSON_CONF_FIELD_DESC); + oprot.writeString(struct.jsonConf); + oprot.writeFieldEnd(); + } + if (struct.topology != null) { + oprot.writeFieldBegin(TOPOLOGY_FIELD_DESC); + struct.topology.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class submitTopology_argsTupleSchemeFactory implements SchemeFactory { + public submitTopology_argsTupleScheme getScheme() { + return new submitTopology_argsTupleScheme(); + } + } + + private static class submitTopology_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, submitTopology_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_name()) { + optionals.set(0); + } + if (struct.is_set_uploadedJarLocation()) { + optionals.set(1); + } + if (struct.is_set_jsonConf()) { + optionals.set(2); + } + if (struct.is_set_topology()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.is_set_name()) { + oprot.writeString(struct.name); + } + if (struct.is_set_uploadedJarLocation()) { + oprot.writeString(struct.uploadedJarLocation); + } + if (struct.is_set_jsonConf()) { + oprot.writeString(struct.jsonConf); + } + if (struct.is_set_topology()) { + struct.topology.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, submitTopology_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(4); + if (incoming.get(0)) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } + if (incoming.get(1)) { + struct.uploadedJarLocation = iprot.readString(); + struct.set_uploadedJarLocation_isSet(true); + } + if (incoming.get(2)) { + struct.jsonConf = iprot.readString(); + struct.set_jsonConf_isSet(true); + } + if (incoming.get(3)) { + struct.topology = new StormTopology(); + struct.topology.read(iprot); + struct.set_topology_isSet(true); + } + } + } + + } + + public static class submitTopology_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("submitTopology_result"); + + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField ITE_FIELD_DESC = new org.apache.thrift.protocol.TField("ite", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField TAE_FIELD_DESC = new org.apache.thrift.protocol.TField("tae", org.apache.thrift.protocol.TType.STRUCT, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new submitTopology_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new submitTopology_resultTupleSchemeFactory()); + } + + private AlreadyAliveException e; // required + private InvalidTopologyException ite; // required + private TopologyAssignException tae; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + E((short)1, "e"), + ITE((short)2, "ite"), + TAE((short)3, "tae"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // E + return E; + case 2: // ITE + return ITE; + case 3: // TAE + return TAE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.ITE, new org.apache.thrift.meta_data.FieldMetaData("ite", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.TAE, new org.apache.thrift.meta_data.FieldMetaData("tae", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(submitTopology_result.class, metaDataMap); + } + + public submitTopology_result() { + } + + public submitTopology_result( + AlreadyAliveException e, + InvalidTopologyException ite, + TopologyAssignException tae) + { + this(); + this.e = e; + this.ite = ite; + this.tae = tae; + } + + /** + * Performs a deep copy on other. + */ + public submitTopology_result(submitTopology_result other) { + if (other.is_set_e()) { + this.e = new AlreadyAliveException(other.e); + } + if (other.is_set_ite()) { + this.ite = new InvalidTopologyException(other.ite); + } + if (other.is_set_tae()) { + this.tae = new TopologyAssignException(other.tae); + } + } + + public submitTopology_result deepCopy() { + return new submitTopology_result(this); + } + + @Override + public void clear() { + this.e = null; + this.ite = null; + this.tae = null; + } + + public AlreadyAliveException get_e() { + return this.e; + } + + public void set_e(AlreadyAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public InvalidTopologyException get_ite() { + return this.ite; + } + + public void set_ite(InvalidTopologyException ite) { + this.ite = ite; + } + + public void unset_ite() { + this.ite = null; + } + + /** Returns true if field ite is set (has been assigned a value) and false otherwise */ + public boolean is_set_ite() { + return this.ite != null; + } + + public void set_ite_isSet(boolean value) { + if (!value) { + this.ite = null; + } + } + + public TopologyAssignException get_tae() { + return this.tae; + } + + public void set_tae(TopologyAssignException tae) { + this.tae = tae; + } + + public void unset_tae() { + this.tae = null; + } + + /** Returns true if field tae is set (has been assigned a value) and false otherwise */ + public boolean is_set_tae() { + return this.tae != null; + } + + public void set_tae_isSet(boolean value) { + if (!value) { + this.tae = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case E: + if (value == null) { + unset_e(); + } else { + set_e((AlreadyAliveException)value); + } + break; + + case ITE: + if (value == null) { + unset_ite(); + } else { + set_ite((InvalidTopologyException)value); + } + break; + + case TAE: + if (value == null) { + unset_tae(); + } else { + set_tae((TopologyAssignException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case E: + return get_e(); + + case ITE: + return get_ite(); + + case TAE: + return get_tae(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case E: + return is_set_e(); + case ITE: + return is_set_ite(); + case TAE: + return is_set_tae(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof submitTopology_result) + return this.equals((submitTopology_result)that); + return false; + } + + public boolean equals(submitTopology_result that) { + if (that == null) + return false; + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + boolean this_present_ite = true && this.is_set_ite(); + boolean that_present_ite = true && that.is_set_ite(); + if (this_present_ite || that_present_ite) { + if (!(this_present_ite && that_present_ite)) + return false; + if (!this.ite.equals(that.ite)) + return false; + } + + boolean this_present_tae = true && this.is_set_tae(); + boolean that_present_tae = true && that.is_set_tae(); + if (this_present_tae || that_present_tae) { + if (!(this_present_tae && that_present_tae)) + return false; + if (!this.tae.equals(that.tae)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + boolean present_ite = true && (is_set_ite()); + list.add(present_ite); + if (present_ite) + list.add(ite); + + boolean present_tae = true && (is_set_tae()); + list.add(present_tae); + if (present_tae) + list.add(tae); + + return list.hashCode(); + } + + @Override + public int compareTo(submitTopology_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_ite()).compareTo(other.is_set_ite()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_ite()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ite, other.ite); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_tae()).compareTo(other.is_set_tae()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_tae()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tae, other.tae); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("submitTopology_result("); + boolean first = true; + + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + if (!first) sb.append(", "); + sb.append("ite:"); + if (this.ite == null) { + sb.append("null"); + } else { + sb.append(this.ite); + } + first = false; + if (!first) sb.append(", "); + sb.append("tae:"); + if (this.tae == null) { + sb.append("null"); + } else { + sb.append(this.tae); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class submitTopology_resultStandardSchemeFactory implements SchemeFactory { + public submitTopology_resultStandardScheme getScheme() { + return new submitTopology_resultStandardScheme(); + } + } + + private static class submitTopology_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, submitTopology_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new AlreadyAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // ITE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.ite = new InvalidTopologyException(); + struct.ite.read(iprot); + struct.set_ite_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // TAE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.tae = new TopologyAssignException(); + struct.tae.read(iprot); + struct.set_tae_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, submitTopology_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.ite != null) { + oprot.writeFieldBegin(ITE_FIELD_DESC); + struct.ite.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.tae != null) { + oprot.writeFieldBegin(TAE_FIELD_DESC); + struct.tae.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class submitTopology_resultTupleSchemeFactory implements SchemeFactory { + public submitTopology_resultTupleScheme getScheme() { + return new submitTopology_resultTupleScheme(); + } + } + + private static class submitTopology_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, submitTopology_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_e()) { + optionals.set(0); + } + if (struct.is_set_ite()) { + optionals.set(1); + } + if (struct.is_set_tae()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.is_set_e()) { + struct.e.write(oprot); + } + if (struct.is_set_ite()) { + struct.ite.write(oprot); + } + if (struct.is_set_tae()) { + struct.tae.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, submitTopology_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.e = new AlreadyAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + if (incoming.get(1)) { + struct.ite = new InvalidTopologyException(); + struct.ite.read(iprot); + struct.set_ite_isSet(true); + } + if (incoming.get(2)) { + struct.tae = new TopologyAssignException(); + struct.tae.read(iprot); + struct.set_tae_isSet(true); + } + } + } + + } + + public static class submitTopologyWithOpts_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("submitTopologyWithOpts_args"); + + private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField UPLOADED_JAR_LOCATION_FIELD_DESC = new org.apache.thrift.protocol.TField("uploadedJarLocation", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField JSON_CONF_FIELD_DESC = new org.apache.thrift.protocol.TField("jsonConf", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField TOPOLOGY_FIELD_DESC = new org.apache.thrift.protocol.TField("topology", org.apache.thrift.protocol.TType.STRUCT, (short)4); + private static final org.apache.thrift.protocol.TField OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("options", org.apache.thrift.protocol.TType.STRUCT, (short)5); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new submitTopologyWithOpts_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new submitTopologyWithOpts_argsTupleSchemeFactory()); + } + + private String name; // required + private String uploadedJarLocation; // required + private String jsonConf; // required + private StormTopology topology; // required + private SubmitOptions options; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NAME((short)1, "name"), + UPLOADED_JAR_LOCATION((short)2, "uploadedJarLocation"), + JSON_CONF((short)3, "jsonConf"), + TOPOLOGY((short)4, "topology"), + OPTIONS((short)5, "options"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NAME + return NAME; + case 2: // UPLOADED_JAR_LOCATION + return UPLOADED_JAR_LOCATION; + case 3: // JSON_CONF + return JSON_CONF; + case 4: // TOPOLOGY + return TOPOLOGY; + case 5: // OPTIONS + return OPTIONS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.UPLOADED_JAR_LOCATION, new org.apache.thrift.meta_data.FieldMetaData("uploadedJarLocation", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.JSON_CONF, new org.apache.thrift.meta_data.FieldMetaData("jsonConf", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TOPOLOGY, new org.apache.thrift.meta_data.FieldMetaData("topology", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, StormTopology.class))); + tmpMap.put(_Fields.OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("options", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SubmitOptions.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(submitTopologyWithOpts_args.class, metaDataMap); + } + + public submitTopologyWithOpts_args() { + } + + public submitTopologyWithOpts_args( + String name, + String uploadedJarLocation, + String jsonConf, + StormTopology topology, + SubmitOptions options) + { + this(); + this.name = name; + this.uploadedJarLocation = uploadedJarLocation; + this.jsonConf = jsonConf; + this.topology = topology; + this.options = options; + } + + /** + * Performs a deep copy on other. + */ + public submitTopologyWithOpts_args(submitTopologyWithOpts_args other) { + if (other.is_set_name()) { + this.name = other.name; + } + if (other.is_set_uploadedJarLocation()) { + this.uploadedJarLocation = other.uploadedJarLocation; + } + if (other.is_set_jsonConf()) { + this.jsonConf = other.jsonConf; + } + if (other.is_set_topology()) { + this.topology = new StormTopology(other.topology); + } + if (other.is_set_options()) { + this.options = new SubmitOptions(other.options); + } + } + + public submitTopologyWithOpts_args deepCopy() { + return new submitTopologyWithOpts_args(this); + } + + @Override + public void clear() { + this.name = null; + this.uploadedJarLocation = null; + this.jsonConf = null; + this.topology = null; + this.options = null; + } + + public String get_name() { + return this.name; + } + + public void set_name(String name) { + this.name = name; + } + + public void unset_name() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean is_set_name() { + return this.name != null; + } + + public void set_name_isSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public String get_uploadedJarLocation() { + return this.uploadedJarLocation; + } + + public void set_uploadedJarLocation(String uploadedJarLocation) { + this.uploadedJarLocation = uploadedJarLocation; + } + + public void unset_uploadedJarLocation() { + this.uploadedJarLocation = null; + } + + /** Returns true if field uploadedJarLocation is set (has been assigned a value) and false otherwise */ + public boolean is_set_uploadedJarLocation() { + return this.uploadedJarLocation != null; + } + + public void set_uploadedJarLocation_isSet(boolean value) { + if (!value) { + this.uploadedJarLocation = null; + } + } + + public String get_jsonConf() { + return this.jsonConf; + } + + public void set_jsonConf(String jsonConf) { + this.jsonConf = jsonConf; + } + + public void unset_jsonConf() { + this.jsonConf = null; + } + + /** Returns true if field jsonConf is set (has been assigned a value) and false otherwise */ + public boolean is_set_jsonConf() { + return this.jsonConf != null; + } + + public void set_jsonConf_isSet(boolean value) { + if (!value) { + this.jsonConf = null; + } + } + + public StormTopology get_topology() { + return this.topology; + } + + public void set_topology(StormTopology topology) { + this.topology = topology; + } + + public void unset_topology() { + this.topology = null; + } + + /** Returns true if field topology is set (has been assigned a value) and false otherwise */ + public boolean is_set_topology() { + return this.topology != null; + } + + public void set_topology_isSet(boolean value) { + if (!value) { + this.topology = null; + } + } + + public SubmitOptions get_options() { + return this.options; + } + + public void set_options(SubmitOptions options) { + this.options = options; + } + + public void unset_options() { + this.options = null; + } + + /** Returns true if field options is set (has been assigned a value) and false otherwise */ + public boolean is_set_options() { + return this.options != null; + } + + public void set_options_isSet(boolean value) { + if (!value) { + this.options = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME: + if (value == null) { + unset_name(); + } else { + set_name((String)value); + } + break; + + case UPLOADED_JAR_LOCATION: + if (value == null) { + unset_uploadedJarLocation(); + } else { + set_uploadedJarLocation((String)value); + } + break; + + case JSON_CONF: + if (value == null) { + unset_jsonConf(); + } else { + set_jsonConf((String)value); + } + break; + + case TOPOLOGY: + if (value == null) { + unset_topology(); + } else { + set_topology((StormTopology)value); + } + break; + + case OPTIONS: + if (value == null) { + unset_options(); + } else { + set_options((SubmitOptions)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME: + return get_name(); + + case UPLOADED_JAR_LOCATION: + return get_uploadedJarLocation(); + + case JSON_CONF: + return get_jsonConf(); + + case TOPOLOGY: + return get_topology(); + + case OPTIONS: + return get_options(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NAME: + return is_set_name(); + case UPLOADED_JAR_LOCATION: + return is_set_uploadedJarLocation(); + case JSON_CONF: + return is_set_jsonConf(); + case TOPOLOGY: + return is_set_topology(); + case OPTIONS: + return is_set_options(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof submitTopologyWithOpts_args) + return this.equals((submitTopologyWithOpts_args)that); + return false; + } + + public boolean equals(submitTopologyWithOpts_args that) { + if (that == null) + return false; + + boolean this_present_name = true && this.is_set_name(); + boolean that_present_name = true && that.is_set_name(); + if (this_present_name || that_present_name) { + if (!(this_present_name && that_present_name)) + return false; + if (!this.name.equals(that.name)) + return false; + } + + boolean this_present_uploadedJarLocation = true && this.is_set_uploadedJarLocation(); + boolean that_present_uploadedJarLocation = true && that.is_set_uploadedJarLocation(); + if (this_present_uploadedJarLocation || that_present_uploadedJarLocation) { + if (!(this_present_uploadedJarLocation && that_present_uploadedJarLocation)) + return false; + if (!this.uploadedJarLocation.equals(that.uploadedJarLocation)) + return false; + } + + boolean this_present_jsonConf = true && this.is_set_jsonConf(); + boolean that_present_jsonConf = true && that.is_set_jsonConf(); + if (this_present_jsonConf || that_present_jsonConf) { + if (!(this_present_jsonConf && that_present_jsonConf)) + return false; + if (!this.jsonConf.equals(that.jsonConf)) + return false; + } + + boolean this_present_topology = true && this.is_set_topology(); + boolean that_present_topology = true && that.is_set_topology(); + if (this_present_topology || that_present_topology) { + if (!(this_present_topology && that_present_topology)) + return false; + if (!this.topology.equals(that.topology)) + return false; + } + + boolean this_present_options = true && this.is_set_options(); + boolean that_present_options = true && that.is_set_options(); + if (this_present_options || that_present_options) { + if (!(this_present_options && that_present_options)) + return false; + if (!this.options.equals(that.options)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_name = true && (is_set_name()); + list.add(present_name); + if (present_name) + list.add(name); + + boolean present_uploadedJarLocation = true && (is_set_uploadedJarLocation()); + list.add(present_uploadedJarLocation); + if (present_uploadedJarLocation) + list.add(uploadedJarLocation); + + boolean present_jsonConf = true && (is_set_jsonConf()); + list.add(present_jsonConf); + if (present_jsonConf) + list.add(jsonConf); + + boolean present_topology = true && (is_set_topology()); + list.add(present_topology); + if (present_topology) + list.add(topology); + + boolean present_options = true && (is_set_options()); + list.add(present_options); + if (present_options) + list.add(options); + + return list.hashCode(); + } + + @Override + public int compareTo(submitTopologyWithOpts_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_uploadedJarLocation()).compareTo(other.is_set_uploadedJarLocation()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_uploadedJarLocation()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uploadedJarLocation, other.uploadedJarLocation); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_jsonConf()).compareTo(other.is_set_jsonConf()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_jsonConf()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.jsonConf, other.jsonConf); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_topology()).compareTo(other.is_set_topology()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_topology()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology, other.topology); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_options()).compareTo(other.is_set_options()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_options()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.options, other.options); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("submitTopologyWithOpts_args("); + boolean first = true; + + sb.append("name:"); + if (this.name == null) { + sb.append("null"); + } else { + sb.append(this.name); + } + first = false; + if (!first) sb.append(", "); + sb.append("uploadedJarLocation:"); + if (this.uploadedJarLocation == null) { + sb.append("null"); + } else { + sb.append(this.uploadedJarLocation); + } + first = false; + if (!first) sb.append(", "); + sb.append("jsonConf:"); + if (this.jsonConf == null) { + sb.append("null"); + } else { + sb.append(this.jsonConf); + } + first = false; + if (!first) sb.append(", "); + sb.append("topology:"); + if (this.topology == null) { + sb.append("null"); + } else { + sb.append(this.topology); + } + first = false; + if (!first) sb.append(", "); + sb.append("options:"); + if (this.options == null) { + sb.append("null"); + } else { + sb.append(this.options); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (topology != null) { + topology.validate(); + } + if (options != null) { + options.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class submitTopologyWithOpts_argsStandardSchemeFactory implements SchemeFactory { + public submitTopologyWithOpts_argsStandardScheme getScheme() { + return new submitTopologyWithOpts_argsStandardScheme(); + } + } + + private static class submitTopologyWithOpts_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, submitTopologyWithOpts_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // UPLOADED_JAR_LOCATION + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.uploadedJarLocation = iprot.readString(); + struct.set_uploadedJarLocation_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // JSON_CONF + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.jsonConf = iprot.readString(); + struct.set_jsonConf_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // TOPOLOGY + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.topology = new StormTopology(); + struct.topology.read(iprot); + struct.set_topology_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // OPTIONS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.options = new SubmitOptions(); + struct.options.read(iprot); + struct.set_options_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, submitTopologyWithOpts_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.name != null) { + oprot.writeFieldBegin(NAME_FIELD_DESC); + oprot.writeString(struct.name); + oprot.writeFieldEnd(); + } + if (struct.uploadedJarLocation != null) { + oprot.writeFieldBegin(UPLOADED_JAR_LOCATION_FIELD_DESC); + oprot.writeString(struct.uploadedJarLocation); + oprot.writeFieldEnd(); + } + if (struct.jsonConf != null) { + oprot.writeFieldBegin(JSON_CONF_FIELD_DESC); + oprot.writeString(struct.jsonConf); + oprot.writeFieldEnd(); + } + if (struct.topology != null) { + oprot.writeFieldBegin(TOPOLOGY_FIELD_DESC); + struct.topology.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.options != null) { + oprot.writeFieldBegin(OPTIONS_FIELD_DESC); + struct.options.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class submitTopologyWithOpts_argsTupleSchemeFactory implements SchemeFactory { + public submitTopologyWithOpts_argsTupleScheme getScheme() { + return new submitTopologyWithOpts_argsTupleScheme(); + } + } + + private static class submitTopologyWithOpts_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, submitTopologyWithOpts_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_name()) { + optionals.set(0); + } + if (struct.is_set_uploadedJarLocation()) { + optionals.set(1); + } + if (struct.is_set_jsonConf()) { + optionals.set(2); + } + if (struct.is_set_topology()) { + optionals.set(3); + } + if (struct.is_set_options()) { + optionals.set(4); + } + oprot.writeBitSet(optionals, 5); + if (struct.is_set_name()) { + oprot.writeString(struct.name); + } + if (struct.is_set_uploadedJarLocation()) { + oprot.writeString(struct.uploadedJarLocation); + } + if (struct.is_set_jsonConf()) { + oprot.writeString(struct.jsonConf); + } + if (struct.is_set_topology()) { + struct.topology.write(oprot); + } + if (struct.is_set_options()) { + struct.options.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, submitTopologyWithOpts_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(5); + if (incoming.get(0)) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } + if (incoming.get(1)) { + struct.uploadedJarLocation = iprot.readString(); + struct.set_uploadedJarLocation_isSet(true); + } + if (incoming.get(2)) { + struct.jsonConf = iprot.readString(); + struct.set_jsonConf_isSet(true); + } + if (incoming.get(3)) { + struct.topology = new StormTopology(); + struct.topology.read(iprot); + struct.set_topology_isSet(true); + } + if (incoming.get(4)) { + struct.options = new SubmitOptions(); + struct.options.read(iprot); + struct.set_options_isSet(true); + } + } + } + + } + + public static class submitTopologyWithOpts_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("submitTopologyWithOpts_result"); + + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField ITE_FIELD_DESC = new org.apache.thrift.protocol.TField("ite", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField TAE_FIELD_DESC = new org.apache.thrift.protocol.TField("tae", org.apache.thrift.protocol.TType.STRUCT, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new submitTopologyWithOpts_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new submitTopologyWithOpts_resultTupleSchemeFactory()); + } + + private AlreadyAliveException e; // required + private InvalidTopologyException ite; // required + private TopologyAssignException tae; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + E((short)1, "e"), + ITE((short)2, "ite"), + TAE((short)3, "tae"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // E + return E; + case 2: // ITE + return ITE; + case 3: // TAE + return TAE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.ITE, new org.apache.thrift.meta_data.FieldMetaData("ite", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.TAE, new org.apache.thrift.meta_data.FieldMetaData("tae", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(submitTopologyWithOpts_result.class, metaDataMap); + } + + public submitTopologyWithOpts_result() { + } + + public submitTopologyWithOpts_result( + AlreadyAliveException e, + InvalidTopologyException ite, + TopologyAssignException tae) + { + this(); + this.e = e; + this.ite = ite; + this.tae = tae; + } + + /** + * Performs a deep copy on other. + */ + public submitTopologyWithOpts_result(submitTopologyWithOpts_result other) { + if (other.is_set_e()) { + this.e = new AlreadyAliveException(other.e); + } + if (other.is_set_ite()) { + this.ite = new InvalidTopologyException(other.ite); + } + if (other.is_set_tae()) { + this.tae = new TopologyAssignException(other.tae); + } + } + + public submitTopologyWithOpts_result deepCopy() { + return new submitTopologyWithOpts_result(this); + } + + @Override + public void clear() { + this.e = null; + this.ite = null; + this.tae = null; + } + + public AlreadyAliveException get_e() { + return this.e; + } + + public void set_e(AlreadyAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public InvalidTopologyException get_ite() { + return this.ite; + } + + public void set_ite(InvalidTopologyException ite) { + this.ite = ite; + } + + public void unset_ite() { + this.ite = null; + } + + /** Returns true if field ite is set (has been assigned a value) and false otherwise */ + public boolean is_set_ite() { + return this.ite != null; + } + + public void set_ite_isSet(boolean value) { + if (!value) { + this.ite = null; + } + } + + public TopologyAssignException get_tae() { + return this.tae; + } + + public void set_tae(TopologyAssignException tae) { + this.tae = tae; + } + + public void unset_tae() { + this.tae = null; + } + + /** Returns true if field tae is set (has been assigned a value) and false otherwise */ + public boolean is_set_tae() { + return this.tae != null; + } + + public void set_tae_isSet(boolean value) { + if (!value) { + this.tae = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case E: + if (value == null) { + unset_e(); + } else { + set_e((AlreadyAliveException)value); + } + break; + + case ITE: + if (value == null) { + unset_ite(); + } else { + set_ite((InvalidTopologyException)value); + } + break; + + case TAE: + if (value == null) { + unset_tae(); + } else { + set_tae((TopologyAssignException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case E: + return get_e(); + + case ITE: + return get_ite(); + + case TAE: + return get_tae(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case E: + return is_set_e(); + case ITE: + return is_set_ite(); + case TAE: + return is_set_tae(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof submitTopologyWithOpts_result) + return this.equals((submitTopologyWithOpts_result)that); + return false; + } + + public boolean equals(submitTopologyWithOpts_result that) { + if (that == null) + return false; + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + boolean this_present_ite = true && this.is_set_ite(); + boolean that_present_ite = true && that.is_set_ite(); + if (this_present_ite || that_present_ite) { + if (!(this_present_ite && that_present_ite)) + return false; + if (!this.ite.equals(that.ite)) + return false; + } + + boolean this_present_tae = true && this.is_set_tae(); + boolean that_present_tae = true && that.is_set_tae(); + if (this_present_tae || that_present_tae) { + if (!(this_present_tae && that_present_tae)) + return false; + if (!this.tae.equals(that.tae)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + boolean present_ite = true && (is_set_ite()); + list.add(present_ite); + if (present_ite) + list.add(ite); + + boolean present_tae = true && (is_set_tae()); + list.add(present_tae); + if (present_tae) + list.add(tae); + + return list.hashCode(); + } + + @Override + public int compareTo(submitTopologyWithOpts_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_ite()).compareTo(other.is_set_ite()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_ite()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ite, other.ite); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_tae()).compareTo(other.is_set_tae()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_tae()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tae, other.tae); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("submitTopologyWithOpts_result("); + boolean first = true; + + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + if (!first) sb.append(", "); + sb.append("ite:"); + if (this.ite == null) { + sb.append("null"); + } else { + sb.append(this.ite); + } + first = false; + if (!first) sb.append(", "); + sb.append("tae:"); + if (this.tae == null) { + sb.append("null"); + } else { + sb.append(this.tae); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class submitTopologyWithOpts_resultStandardSchemeFactory implements SchemeFactory { + public submitTopologyWithOpts_resultStandardScheme getScheme() { + return new submitTopologyWithOpts_resultStandardScheme(); + } + } + + private static class submitTopologyWithOpts_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, submitTopologyWithOpts_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new AlreadyAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // ITE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.ite = new InvalidTopologyException(); + struct.ite.read(iprot); + struct.set_ite_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // TAE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.tae = new TopologyAssignException(); + struct.tae.read(iprot); + struct.set_tae_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, submitTopologyWithOpts_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.ite != null) { + oprot.writeFieldBegin(ITE_FIELD_DESC); + struct.ite.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.tae != null) { + oprot.writeFieldBegin(TAE_FIELD_DESC); + struct.tae.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class submitTopologyWithOpts_resultTupleSchemeFactory implements SchemeFactory { + public submitTopologyWithOpts_resultTupleScheme getScheme() { + return new submitTopologyWithOpts_resultTupleScheme(); + } + } + + private static class submitTopologyWithOpts_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, submitTopologyWithOpts_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_e()) { + optionals.set(0); + } + if (struct.is_set_ite()) { + optionals.set(1); + } + if (struct.is_set_tae()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.is_set_e()) { + struct.e.write(oprot); + } + if (struct.is_set_ite()) { + struct.ite.write(oprot); + } + if (struct.is_set_tae()) { + struct.tae.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, submitTopologyWithOpts_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.e = new AlreadyAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + if (incoming.get(1)) { + struct.ite = new InvalidTopologyException(); + struct.ite.read(iprot); + struct.set_ite_isSet(true); + } + if (incoming.get(2)) { + struct.tae = new TopologyAssignException(); + struct.tae.read(iprot); + struct.set_tae_isSet(true); + } + } + } + + } + + public static class killTopology_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("killTopology_args"); + + private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new killTopology_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new killTopology_argsTupleSchemeFactory()); + } + + private String name; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NAME((short)1, "name"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NAME + return NAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(killTopology_args.class, metaDataMap); + } + + public killTopology_args() { + } + + public killTopology_args( + String name) + { + this(); + this.name = name; + } + + /** + * Performs a deep copy on other. + */ + public killTopology_args(killTopology_args other) { + if (other.is_set_name()) { + this.name = other.name; + } + } + + public killTopology_args deepCopy() { + return new killTopology_args(this); + } + + @Override + public void clear() { + this.name = null; + } + + public String get_name() { + return this.name; + } + + public void set_name(String name) { + this.name = name; + } + + public void unset_name() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean is_set_name() { + return this.name != null; + } + + public void set_name_isSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME: + if (value == null) { + unset_name(); + } else { + set_name((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME: + return get_name(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NAME: + return is_set_name(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof killTopology_args) + return this.equals((killTopology_args)that); + return false; + } + + public boolean equals(killTopology_args that) { + if (that == null) + return false; + + boolean this_present_name = true && this.is_set_name(); + boolean that_present_name = true && that.is_set_name(); + if (this_present_name || that_present_name) { + if (!(this_present_name && that_present_name)) + return false; + if (!this.name.equals(that.name)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_name = true && (is_set_name()); + list.add(present_name); + if (present_name) + list.add(name); + + return list.hashCode(); + } + + @Override + public int compareTo(killTopology_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("killTopology_args("); + boolean first = true; + + sb.append("name:"); + if (this.name == null) { + sb.append("null"); + } else { + sb.append(this.name); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class killTopology_argsStandardSchemeFactory implements SchemeFactory { + public killTopology_argsStandardScheme getScheme() { + return new killTopology_argsStandardScheme(); + } + } + + private static class killTopology_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, killTopology_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, killTopology_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.name != null) { + oprot.writeFieldBegin(NAME_FIELD_DESC); + oprot.writeString(struct.name); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class killTopology_argsTupleSchemeFactory implements SchemeFactory { + public killTopology_argsTupleScheme getScheme() { + return new killTopology_argsTupleScheme(); + } + } + + private static class killTopology_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, killTopology_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_name()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_name()) { + oprot.writeString(struct.name); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, killTopology_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } + } + } + + } + + public static class killTopology_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("killTopology_result"); + + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new killTopology_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new killTopology_resultTupleSchemeFactory()); + } + + private NotAliveException e; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + E((short)1, "e"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // E + return E; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(killTopology_result.class, metaDataMap); + } + + public killTopology_result() { + } + + public killTopology_result( + NotAliveException e) + { + this(); + this.e = e; + } + + /** + * Performs a deep copy on other. + */ + public killTopology_result(killTopology_result other) { + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + } + + public killTopology_result deepCopy() { + return new killTopology_result(this); + } + + @Override + public void clear() { + this.e = null; + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case E: + return get_e(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case E: + return is_set_e(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof killTopology_result) + return this.equals((killTopology_result)that); + return false; + } + + public boolean equals(killTopology_result that) { + if (that == null) + return false; + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + return list.hashCode(); + } + + @Override + public int compareTo(killTopology_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("killTopology_result("); + boolean first = true; + + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class killTopology_resultStandardSchemeFactory implements SchemeFactory { + public killTopology_resultStandardScheme getScheme() { + return new killTopology_resultStandardScheme(); + } + } + + private static class killTopology_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, killTopology_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, killTopology_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class killTopology_resultTupleSchemeFactory implements SchemeFactory { + public killTopology_resultTupleScheme getScheme() { + return new killTopology_resultTupleScheme(); + } + } + + private static class killTopology_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, killTopology_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_e()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_e()) { + struct.e.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, killTopology_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + } + } + + } + + public static class killTopologyWithOpts_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("killTopologyWithOpts_args"); + + private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("options", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new killTopologyWithOpts_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new killTopologyWithOpts_argsTupleSchemeFactory()); + } + + private String name; // required + private KillOptions options; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NAME((short)1, "name"), + OPTIONS((short)2, "options"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NAME + return NAME; + case 2: // OPTIONS + return OPTIONS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("options", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, KillOptions.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(killTopologyWithOpts_args.class, metaDataMap); + } + + public killTopologyWithOpts_args() { + } + + public killTopologyWithOpts_args( + String name, + KillOptions options) + { + this(); + this.name = name; + this.options = options; + } + + /** + * Performs a deep copy on other. + */ + public killTopologyWithOpts_args(killTopologyWithOpts_args other) { + if (other.is_set_name()) { + this.name = other.name; + } + if (other.is_set_options()) { + this.options = new KillOptions(other.options); + } + } + + public killTopologyWithOpts_args deepCopy() { + return new killTopologyWithOpts_args(this); + } + + @Override + public void clear() { + this.name = null; + this.options = null; + } + + public String get_name() { + return this.name; + } + + public void set_name(String name) { + this.name = name; + } + + public void unset_name() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean is_set_name() { + return this.name != null; + } + + public void set_name_isSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public KillOptions get_options() { + return this.options; + } + + public void set_options(KillOptions options) { + this.options = options; + } + + public void unset_options() { + this.options = null; + } + + /** Returns true if field options is set (has been assigned a value) and false otherwise */ + public boolean is_set_options() { + return this.options != null; + } + + public void set_options_isSet(boolean value) { + if (!value) { + this.options = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME: + if (value == null) { + unset_name(); + } else { + set_name((String)value); + } + break; + + case OPTIONS: + if (value == null) { + unset_options(); + } else { + set_options((KillOptions)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME: + return get_name(); + + case OPTIONS: + return get_options(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NAME: + return is_set_name(); + case OPTIONS: + return is_set_options(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof killTopologyWithOpts_args) + return this.equals((killTopologyWithOpts_args)that); + return false; + } + + public boolean equals(killTopologyWithOpts_args that) { + if (that == null) + return false; + + boolean this_present_name = true && this.is_set_name(); + boolean that_present_name = true && that.is_set_name(); + if (this_present_name || that_present_name) { + if (!(this_present_name && that_present_name)) + return false; + if (!this.name.equals(that.name)) + return false; + } + + boolean this_present_options = true && this.is_set_options(); + boolean that_present_options = true && that.is_set_options(); + if (this_present_options || that_present_options) { + if (!(this_present_options && that_present_options)) + return false; + if (!this.options.equals(that.options)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_name = true && (is_set_name()); + list.add(present_name); + if (present_name) + list.add(name); + + boolean present_options = true && (is_set_options()); + list.add(present_options); + if (present_options) + list.add(options); + + return list.hashCode(); + } + + @Override + public int compareTo(killTopologyWithOpts_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_options()).compareTo(other.is_set_options()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_options()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.options, other.options); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("killTopologyWithOpts_args("); + boolean first = true; + + sb.append("name:"); + if (this.name == null) { + sb.append("null"); + } else { + sb.append(this.name); + } + first = false; + if (!first) sb.append(", "); + sb.append("options:"); + if (this.options == null) { + sb.append("null"); + } else { + sb.append(this.options); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (options != null) { + options.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class killTopologyWithOpts_argsStandardSchemeFactory implements SchemeFactory { + public killTopologyWithOpts_argsStandardScheme getScheme() { + return new killTopologyWithOpts_argsStandardScheme(); + } + } + + private static class killTopologyWithOpts_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, killTopologyWithOpts_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPTIONS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.options = new KillOptions(); + struct.options.read(iprot); + struct.set_options_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, killTopologyWithOpts_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.name != null) { + oprot.writeFieldBegin(NAME_FIELD_DESC); + oprot.writeString(struct.name); + oprot.writeFieldEnd(); + } + if (struct.options != null) { + oprot.writeFieldBegin(OPTIONS_FIELD_DESC); + struct.options.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class killTopologyWithOpts_argsTupleSchemeFactory implements SchemeFactory { + public killTopologyWithOpts_argsTupleScheme getScheme() { + return new killTopologyWithOpts_argsTupleScheme(); + } + } + + private static class killTopologyWithOpts_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, killTopologyWithOpts_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_name()) { + optionals.set(0); + } + if (struct.is_set_options()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_name()) { + oprot.writeString(struct.name); + } + if (struct.is_set_options()) { + struct.options.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, killTopologyWithOpts_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } + if (incoming.get(1)) { + struct.options = new KillOptions(); + struct.options.read(iprot); + struct.set_options_isSet(true); + } + } + } + + } + + public static class killTopologyWithOpts_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("killTopologyWithOpts_result"); + + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new killTopologyWithOpts_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new killTopologyWithOpts_resultTupleSchemeFactory()); + } + + private NotAliveException e; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + E((short)1, "e"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // E + return E; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(killTopologyWithOpts_result.class, metaDataMap); + } + + public killTopologyWithOpts_result() { + } + + public killTopologyWithOpts_result( + NotAliveException e) + { + this(); + this.e = e; + } + + /** + * Performs a deep copy on other. + */ + public killTopologyWithOpts_result(killTopologyWithOpts_result other) { + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + } + + public killTopologyWithOpts_result deepCopy() { + return new killTopologyWithOpts_result(this); + } + + @Override + public void clear() { + this.e = null; + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case E: + return get_e(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case E: + return is_set_e(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof killTopologyWithOpts_result) + return this.equals((killTopologyWithOpts_result)that); + return false; + } + + public boolean equals(killTopologyWithOpts_result that) { + if (that == null) + return false; + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + return list.hashCode(); + } + + @Override + public int compareTo(killTopologyWithOpts_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("killTopologyWithOpts_result("); + boolean first = true; + + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class killTopologyWithOpts_resultStandardSchemeFactory implements SchemeFactory { + public killTopologyWithOpts_resultStandardScheme getScheme() { + return new killTopologyWithOpts_resultStandardScheme(); + } + } + + private static class killTopologyWithOpts_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, killTopologyWithOpts_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, killTopologyWithOpts_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class killTopologyWithOpts_resultTupleSchemeFactory implements SchemeFactory { + public killTopologyWithOpts_resultTupleScheme getScheme() { + return new killTopologyWithOpts_resultTupleScheme(); + } + } + + private static class killTopologyWithOpts_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, killTopologyWithOpts_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_e()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_e()) { + struct.e.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, killTopologyWithOpts_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + } + } + + } + + public static class activate_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("activate_args"); + + private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new activate_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new activate_argsTupleSchemeFactory()); + } + + private String name; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NAME((short)1, "name"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NAME + return NAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(activate_args.class, metaDataMap); + } + + public activate_args() { + } + + public activate_args( + String name) + { + this(); + this.name = name; + } + + /** + * Performs a deep copy on other. + */ + public activate_args(activate_args other) { + if (other.is_set_name()) { + this.name = other.name; + } + } + + public activate_args deepCopy() { + return new activate_args(this); + } + + @Override + public void clear() { + this.name = null; + } + + public String get_name() { + return this.name; + } + + public void set_name(String name) { + this.name = name; + } + + public void unset_name() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean is_set_name() { + return this.name != null; + } + + public void set_name_isSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME: + if (value == null) { + unset_name(); + } else { + set_name((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME: + return get_name(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NAME: + return is_set_name(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof activate_args) + return this.equals((activate_args)that); + return false; + } + + public boolean equals(activate_args that) { + if (that == null) + return false; + + boolean this_present_name = true && this.is_set_name(); + boolean that_present_name = true && that.is_set_name(); + if (this_present_name || that_present_name) { + if (!(this_present_name && that_present_name)) + return false; + if (!this.name.equals(that.name)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_name = true && (is_set_name()); + list.add(present_name); + if (present_name) + list.add(name); + + return list.hashCode(); + } + + @Override + public int compareTo(activate_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("activate_args("); + boolean first = true; + + sb.append("name:"); + if (this.name == null) { + sb.append("null"); + } else { + sb.append(this.name); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class activate_argsStandardSchemeFactory implements SchemeFactory { + public activate_argsStandardScheme getScheme() { + return new activate_argsStandardScheme(); + } + } + + private static class activate_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, activate_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, activate_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.name != null) { + oprot.writeFieldBegin(NAME_FIELD_DESC); + oprot.writeString(struct.name); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class activate_argsTupleSchemeFactory implements SchemeFactory { + public activate_argsTupleScheme getScheme() { + return new activate_argsTupleScheme(); + } + } + + private static class activate_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, activate_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_name()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_name()) { + oprot.writeString(struct.name); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, activate_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } + } + } + + } + + public static class activate_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("activate_result"); + + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new activate_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new activate_resultTupleSchemeFactory()); + } + + private NotAliveException e; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + E((short)1, "e"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // E + return E; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(activate_result.class, metaDataMap); + } + + public activate_result() { + } + + public activate_result( + NotAliveException e) + { + this(); + this.e = e; + } + + /** + * Performs a deep copy on other. + */ + public activate_result(activate_result other) { + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + } + + public activate_result deepCopy() { + return new activate_result(this); + } + + @Override + public void clear() { + this.e = null; + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case E: + return get_e(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case E: + return is_set_e(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof activate_result) + return this.equals((activate_result)that); + return false; + } + + public boolean equals(activate_result that) { + if (that == null) + return false; + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + return list.hashCode(); + } + + @Override + public int compareTo(activate_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("activate_result("); + boolean first = true; + + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class activate_resultStandardSchemeFactory implements SchemeFactory { + public activate_resultStandardScheme getScheme() { + return new activate_resultStandardScheme(); + } + } + + private static class activate_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, activate_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, activate_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class activate_resultTupleSchemeFactory implements SchemeFactory { + public activate_resultTupleScheme getScheme() { + return new activate_resultTupleScheme(); + } + } + + private static class activate_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, activate_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_e()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_e()) { + struct.e.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, activate_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + } + } + + } + + public static class deactivate_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deactivate_args"); + + private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new deactivate_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new deactivate_argsTupleSchemeFactory()); + } + + private String name; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NAME((short)1, "name"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NAME + return NAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deactivate_args.class, metaDataMap); + } + + public deactivate_args() { + } + + public deactivate_args( + String name) + { + this(); + this.name = name; + } + + /** + * Performs a deep copy on other. + */ + public deactivate_args(deactivate_args other) { + if (other.is_set_name()) { + this.name = other.name; + } + } + + public deactivate_args deepCopy() { + return new deactivate_args(this); + } + + @Override + public void clear() { + this.name = null; + } + + public String get_name() { + return this.name; + } + + public void set_name(String name) { + this.name = name; + } + + public void unset_name() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean is_set_name() { + return this.name != null; + } + + public void set_name_isSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME: + if (value == null) { + unset_name(); + } else { + set_name((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME: + return get_name(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NAME: + return is_set_name(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof deactivate_args) + return this.equals((deactivate_args)that); + return false; + } + + public boolean equals(deactivate_args that) { + if (that == null) + return false; + + boolean this_present_name = true && this.is_set_name(); + boolean that_present_name = true && that.is_set_name(); + if (this_present_name || that_present_name) { + if (!(this_present_name && that_present_name)) + return false; + if (!this.name.equals(that.name)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_name = true && (is_set_name()); + list.add(present_name); + if (present_name) + list.add(name); + + return list.hashCode(); + } + + @Override + public int compareTo(deactivate_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("deactivate_args("); + boolean first = true; + + sb.append("name:"); + if (this.name == null) { + sb.append("null"); + } else { + sb.append(this.name); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class deactivate_argsStandardSchemeFactory implements SchemeFactory { + public deactivate_argsStandardScheme getScheme() { + return new deactivate_argsStandardScheme(); + } + } + + private static class deactivate_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, deactivate_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, deactivate_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.name != null) { + oprot.writeFieldBegin(NAME_FIELD_DESC); + oprot.writeString(struct.name); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class deactivate_argsTupleSchemeFactory implements SchemeFactory { + public deactivate_argsTupleScheme getScheme() { + return new deactivate_argsTupleScheme(); + } + } + + private static class deactivate_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, deactivate_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_name()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_name()) { + oprot.writeString(struct.name); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, deactivate_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } + } + } + + } + + public static class deactivate_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("deactivate_result"); + + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new deactivate_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new deactivate_resultTupleSchemeFactory()); + } + + private NotAliveException e; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + E((short)1, "e"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // E + return E; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(deactivate_result.class, metaDataMap); + } + + public deactivate_result() { + } + + public deactivate_result( + NotAliveException e) + { + this(); + this.e = e; + } + + /** + * Performs a deep copy on other. + */ + public deactivate_result(deactivate_result other) { + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + } + + public deactivate_result deepCopy() { + return new deactivate_result(this); + } + + @Override + public void clear() { + this.e = null; + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case E: + return get_e(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case E: + return is_set_e(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof deactivate_result) + return this.equals((deactivate_result)that); + return false; + } + + public boolean equals(deactivate_result that) { + if (that == null) + return false; + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + return list.hashCode(); + } + + @Override + public int compareTo(deactivate_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("deactivate_result("); + boolean first = true; + + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class deactivate_resultStandardSchemeFactory implements SchemeFactory { + public deactivate_resultStandardScheme getScheme() { + return new deactivate_resultStandardScheme(); + } + } + + private static class deactivate_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, deactivate_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, deactivate_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class deactivate_resultTupleSchemeFactory implements SchemeFactory { + public deactivate_resultTupleScheme getScheme() { + return new deactivate_resultTupleScheme(); + } + } + + private static class deactivate_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, deactivate_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_e()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_e()) { + struct.e.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, deactivate_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + } + } + + } + + public static class rebalance_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("rebalance_args"); + + private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("options", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new rebalance_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new rebalance_argsTupleSchemeFactory()); + } + + private String name; // required + private RebalanceOptions options; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NAME((short)1, "name"), + OPTIONS((short)2, "options"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NAME + return NAME; + case 2: // OPTIONS + return OPTIONS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("options", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, RebalanceOptions.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(rebalance_args.class, metaDataMap); + } + + public rebalance_args() { + } + + public rebalance_args( + String name, + RebalanceOptions options) + { + this(); + this.name = name; + this.options = options; + } + + /** + * Performs a deep copy on other. + */ + public rebalance_args(rebalance_args other) { + if (other.is_set_name()) { + this.name = other.name; + } + if (other.is_set_options()) { + this.options = new RebalanceOptions(other.options); + } + } + + public rebalance_args deepCopy() { + return new rebalance_args(this); + } + + @Override + public void clear() { + this.name = null; + this.options = null; + } + + public String get_name() { + return this.name; + } + + public void set_name(String name) { + this.name = name; + } + + public void unset_name() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean is_set_name() { + return this.name != null; + } + + public void set_name_isSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public RebalanceOptions get_options() { + return this.options; + } + + public void set_options(RebalanceOptions options) { + this.options = options; + } + + public void unset_options() { + this.options = null; + } + + /** Returns true if field options is set (has been assigned a value) and false otherwise */ + public boolean is_set_options() { + return this.options != null; + } + + public void set_options_isSet(boolean value) { + if (!value) { + this.options = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME: + if (value == null) { + unset_name(); + } else { + set_name((String)value); + } + break; + + case OPTIONS: + if (value == null) { + unset_options(); + } else { + set_options((RebalanceOptions)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME: + return get_name(); + + case OPTIONS: + return get_options(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NAME: + return is_set_name(); + case OPTIONS: + return is_set_options(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof rebalance_args) + return this.equals((rebalance_args)that); + return false; + } + + public boolean equals(rebalance_args that) { + if (that == null) + return false; + + boolean this_present_name = true && this.is_set_name(); + boolean that_present_name = true && that.is_set_name(); + if (this_present_name || that_present_name) { + if (!(this_present_name && that_present_name)) + return false; + if (!this.name.equals(that.name)) + return false; + } + + boolean this_present_options = true && this.is_set_options(); + boolean that_present_options = true && that.is_set_options(); + if (this_present_options || that_present_options) { + if (!(this_present_options && that_present_options)) + return false; + if (!this.options.equals(that.options)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_name = true && (is_set_name()); + list.add(present_name); + if (present_name) + list.add(name); + + boolean present_options = true && (is_set_options()); + list.add(present_options); + if (present_options) + list.add(options); + + return list.hashCode(); + } + + @Override + public int compareTo(rebalance_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_options()).compareTo(other.is_set_options()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_options()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.options, other.options); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("rebalance_args("); + boolean first = true; + + sb.append("name:"); + if (this.name == null) { + sb.append("null"); + } else { + sb.append(this.name); + } + first = false; + if (!first) sb.append(", "); + sb.append("options:"); + if (this.options == null) { + sb.append("null"); + } else { + sb.append(this.options); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (options != null) { + options.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class rebalance_argsStandardSchemeFactory implements SchemeFactory { + public rebalance_argsStandardScheme getScheme() { + return new rebalance_argsStandardScheme(); + } + } + + private static class rebalance_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, rebalance_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPTIONS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.options = new RebalanceOptions(); + struct.options.read(iprot); + struct.set_options_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, rebalance_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.name != null) { + oprot.writeFieldBegin(NAME_FIELD_DESC); + oprot.writeString(struct.name); + oprot.writeFieldEnd(); + } + if (struct.options != null) { + oprot.writeFieldBegin(OPTIONS_FIELD_DESC); + struct.options.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class rebalance_argsTupleSchemeFactory implements SchemeFactory { + public rebalance_argsTupleScheme getScheme() { + return new rebalance_argsTupleScheme(); + } + } + + private static class rebalance_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, rebalance_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_name()) { + optionals.set(0); + } + if (struct.is_set_options()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_name()) { + oprot.writeString(struct.name); + } + if (struct.is_set_options()) { + struct.options.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, rebalance_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } + if (incoming.get(1)) { + struct.options = new RebalanceOptions(); + struct.options.read(iprot); + struct.set_options_isSet(true); + } + } + } + + } + + public static class rebalance_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("rebalance_result"); + + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField ITE_FIELD_DESC = new org.apache.thrift.protocol.TField("ite", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new rebalance_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new rebalance_resultTupleSchemeFactory()); + } + + private NotAliveException e; // required + private InvalidTopologyException ite; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + E((short)1, "e"), + ITE((short)2, "ite"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // E + return E; + case 2: // ITE + return ITE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.ITE, new org.apache.thrift.meta_data.FieldMetaData("ite", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(rebalance_result.class, metaDataMap); + } + + public rebalance_result() { + } + + public rebalance_result( + NotAliveException e, + InvalidTopologyException ite) + { + this(); + this.e = e; + this.ite = ite; + } + + /** + * Performs a deep copy on other. + */ + public rebalance_result(rebalance_result other) { + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + if (other.is_set_ite()) { + this.ite = new InvalidTopologyException(other.ite); + } + } + + public rebalance_result deepCopy() { + return new rebalance_result(this); + } + + @Override + public void clear() { + this.e = null; + this.ite = null; + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public InvalidTopologyException get_ite() { + return this.ite; + } + + public void set_ite(InvalidTopologyException ite) { + this.ite = ite; + } + + public void unset_ite() { + this.ite = null; + } + + /** Returns true if field ite is set (has been assigned a value) and false otherwise */ + public boolean is_set_ite() { + return this.ite != null; + } + + public void set_ite_isSet(boolean value) { + if (!value) { + this.ite = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + case ITE: + if (value == null) { + unset_ite(); + } else { + set_ite((InvalidTopologyException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case E: + return get_e(); + + case ITE: + return get_ite(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case E: + return is_set_e(); + case ITE: + return is_set_ite(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof rebalance_result) + return this.equals((rebalance_result)that); + return false; + } + + public boolean equals(rebalance_result that) { + if (that == null) + return false; + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + boolean this_present_ite = true && this.is_set_ite(); + boolean that_present_ite = true && that.is_set_ite(); + if (this_present_ite || that_present_ite) { + if (!(this_present_ite && that_present_ite)) + return false; + if (!this.ite.equals(that.ite)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + boolean present_ite = true && (is_set_ite()); + list.add(present_ite); + if (present_ite) + list.add(ite); + + return list.hashCode(); + } + + @Override + public int compareTo(rebalance_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_ite()).compareTo(other.is_set_ite()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_ite()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ite, other.ite); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("rebalance_result("); + boolean first = true; + + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + if (!first) sb.append(", "); + sb.append("ite:"); + if (this.ite == null) { + sb.append("null"); + } else { + sb.append(this.ite); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class rebalance_resultStandardSchemeFactory implements SchemeFactory { + public rebalance_resultStandardScheme getScheme() { + return new rebalance_resultStandardScheme(); + } + } + + private static class rebalance_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, rebalance_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // ITE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.ite = new InvalidTopologyException(); + struct.ite.read(iprot); + struct.set_ite_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, rebalance_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.ite != null) { + oprot.writeFieldBegin(ITE_FIELD_DESC); + struct.ite.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class rebalance_resultTupleSchemeFactory implements SchemeFactory { + public rebalance_resultTupleScheme getScheme() { + return new rebalance_resultTupleScheme(); + } + } + + private static class rebalance_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, rebalance_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_e()) { + optionals.set(0); + } + if (struct.is_set_ite()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_e()) { + struct.e.write(oprot); + } + if (struct.is_set_ite()) { + struct.ite.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, rebalance_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + if (incoming.get(1)) { + struct.ite = new InvalidTopologyException(); + struct.ite.read(iprot); + struct.set_ite_isSet(true); + } + } + } + + } + + public static class metricMonitor_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("metricMonitor_args"); + + private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField OPTIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("options", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new metricMonitor_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new metricMonitor_argsTupleSchemeFactory()); + } + + private String name; // required + private MonitorOptions options; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NAME((short)1, "name"), + OPTIONS((short)2, "options"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NAME + return NAME; + case 2: // OPTIONS + return OPTIONS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.OPTIONS, new org.apache.thrift.meta_data.FieldMetaData("options", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, MonitorOptions.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(metricMonitor_args.class, metaDataMap); + } + + public metricMonitor_args() { + } + + public metricMonitor_args( + String name, + MonitorOptions options) + { + this(); + this.name = name; + this.options = options; + } + + /** + * Performs a deep copy on other. + */ + public metricMonitor_args(metricMonitor_args other) { + if (other.is_set_name()) { + this.name = other.name; + } + if (other.is_set_options()) { + this.options = new MonitorOptions(other.options); + } + } + + public metricMonitor_args deepCopy() { + return new metricMonitor_args(this); + } + + @Override + public void clear() { + this.name = null; + this.options = null; + } + + public String get_name() { + return this.name; + } + + public void set_name(String name) { + this.name = name; + } + + public void unset_name() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean is_set_name() { + return this.name != null; + } + + public void set_name_isSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public MonitorOptions get_options() { + return this.options; + } + + public void set_options(MonitorOptions options) { + this.options = options; + } + + public void unset_options() { + this.options = null; + } + + /** Returns true if field options is set (has been assigned a value) and false otherwise */ + public boolean is_set_options() { + return this.options != null; + } + + public void set_options_isSet(boolean value) { + if (!value) { + this.options = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME: + if (value == null) { + unset_name(); + } else { + set_name((String)value); + } + break; + + case OPTIONS: + if (value == null) { + unset_options(); + } else { + set_options((MonitorOptions)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME: + return get_name(); + + case OPTIONS: + return get_options(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NAME: + return is_set_name(); + case OPTIONS: + return is_set_options(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof metricMonitor_args) + return this.equals((metricMonitor_args)that); + return false; + } + + public boolean equals(metricMonitor_args that) { + if (that == null) + return false; + + boolean this_present_name = true && this.is_set_name(); + boolean that_present_name = true && that.is_set_name(); + if (this_present_name || that_present_name) { + if (!(this_present_name && that_present_name)) + return false; + if (!this.name.equals(that.name)) + return false; + } + + boolean this_present_options = true && this.is_set_options(); + boolean that_present_options = true && that.is_set_options(); + if (this_present_options || that_present_options) { + if (!(this_present_options && that_present_options)) + return false; + if (!this.options.equals(that.options)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_name = true && (is_set_name()); + list.add(present_name); + if (present_name) + list.add(name); + + boolean present_options = true && (is_set_options()); + list.add(present_options); + if (present_options) + list.add(options); + + return list.hashCode(); + } + + @Override + public int compareTo(metricMonitor_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_options()).compareTo(other.is_set_options()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_options()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.options, other.options); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("metricMonitor_args("); + boolean first = true; + + sb.append("name:"); + if (this.name == null) { + sb.append("null"); + } else { + sb.append(this.name); + } + first = false; + if (!first) sb.append(", "); + sb.append("options:"); + if (this.options == null) { + sb.append("null"); + } else { + sb.append(this.options); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (options != null) { + options.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class metricMonitor_argsStandardSchemeFactory implements SchemeFactory { + public metricMonitor_argsStandardScheme getScheme() { + return new metricMonitor_argsStandardScheme(); + } + } + + private static class metricMonitor_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, metricMonitor_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPTIONS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.options = new MonitorOptions(); + struct.options.read(iprot); + struct.set_options_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, metricMonitor_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.name != null) { + oprot.writeFieldBegin(NAME_FIELD_DESC); + oprot.writeString(struct.name); + oprot.writeFieldEnd(); + } + if (struct.options != null) { + oprot.writeFieldBegin(OPTIONS_FIELD_DESC); + struct.options.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class metricMonitor_argsTupleSchemeFactory implements SchemeFactory { + public metricMonitor_argsTupleScheme getScheme() { + return new metricMonitor_argsTupleScheme(); + } + } + + private static class metricMonitor_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, metricMonitor_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_name()) { + optionals.set(0); + } + if (struct.is_set_options()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_name()) { + oprot.writeString(struct.name); + } + if (struct.is_set_options()) { + struct.options.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, metricMonitor_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } + if (incoming.get(1)) { + struct.options = new MonitorOptions(); + struct.options.read(iprot); + struct.set_options_isSet(true); + } + } + } + + } + + public static class metricMonitor_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("metricMonitor_result"); + + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new metricMonitor_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new metricMonitor_resultTupleSchemeFactory()); + } + + private NotAliveException e; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + E((short)1, "e"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // E + return E; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(metricMonitor_result.class, metaDataMap); + } + + public metricMonitor_result() { + } + + public metricMonitor_result( + NotAliveException e) + { + this(); + this.e = e; + } + + /** + * Performs a deep copy on other. + */ + public metricMonitor_result(metricMonitor_result other) { + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + } + + public metricMonitor_result deepCopy() { + return new metricMonitor_result(this); + } + + @Override + public void clear() { + this.e = null; + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case E: + return get_e(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case E: + return is_set_e(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof metricMonitor_result) + return this.equals((metricMonitor_result)that); + return false; + } + + public boolean equals(metricMonitor_result that) { + if (that == null) + return false; + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + return list.hashCode(); + } + + @Override + public int compareTo(metricMonitor_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("metricMonitor_result("); + boolean first = true; + + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class metricMonitor_resultStandardSchemeFactory implements SchemeFactory { + public metricMonitor_resultStandardScheme getScheme() { + return new metricMonitor_resultStandardScheme(); + } + } + + private static class metricMonitor_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, metricMonitor_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, metricMonitor_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class metricMonitor_resultTupleSchemeFactory implements SchemeFactory { + public metricMonitor_resultTupleScheme getScheme() { + return new metricMonitor_resultTupleScheme(); + } + } + + private static class metricMonitor_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, metricMonitor_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_e()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_e()) { + struct.e.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, metricMonitor_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + } + } + + } + + public static class restart_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("restart_args"); + + private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField JSON_CONF_FIELD_DESC = new org.apache.thrift.protocol.TField("jsonConf", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new restart_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new restart_argsTupleSchemeFactory()); + } + + private String name; // required + private String jsonConf; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NAME((short)1, "name"), + JSON_CONF((short)2, "jsonConf"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NAME + return NAME; + case 2: // JSON_CONF + return JSON_CONF; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.JSON_CONF, new org.apache.thrift.meta_data.FieldMetaData("jsonConf", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(restart_args.class, metaDataMap); + } + + public restart_args() { + } + + public restart_args( + String name, + String jsonConf) + { + this(); + this.name = name; + this.jsonConf = jsonConf; + } + + /** + * Performs a deep copy on other. + */ + public restart_args(restart_args other) { + if (other.is_set_name()) { + this.name = other.name; + } + if (other.is_set_jsonConf()) { + this.jsonConf = other.jsonConf; + } + } + + public restart_args deepCopy() { + return new restart_args(this); + } + + @Override + public void clear() { + this.name = null; + this.jsonConf = null; + } + + public String get_name() { + return this.name; + } + + public void set_name(String name) { + this.name = name; + } + + public void unset_name() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean is_set_name() { + return this.name != null; + } + + public void set_name_isSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public String get_jsonConf() { + return this.jsonConf; + } + + public void set_jsonConf(String jsonConf) { + this.jsonConf = jsonConf; + } + + public void unset_jsonConf() { + this.jsonConf = null; + } + + /** Returns true if field jsonConf is set (has been assigned a value) and false otherwise */ + public boolean is_set_jsonConf() { + return this.jsonConf != null; + } + + public void set_jsonConf_isSet(boolean value) { + if (!value) { + this.jsonConf = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME: + if (value == null) { + unset_name(); + } else { + set_name((String)value); + } + break; + + case JSON_CONF: + if (value == null) { + unset_jsonConf(); + } else { + set_jsonConf((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME: + return get_name(); + + case JSON_CONF: + return get_jsonConf(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NAME: + return is_set_name(); + case JSON_CONF: + return is_set_jsonConf(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof restart_args) + return this.equals((restart_args)that); + return false; + } + + public boolean equals(restart_args that) { + if (that == null) + return false; + + boolean this_present_name = true && this.is_set_name(); + boolean that_present_name = true && that.is_set_name(); + if (this_present_name || that_present_name) { + if (!(this_present_name && that_present_name)) + return false; + if (!this.name.equals(that.name)) + return false; + } + + boolean this_present_jsonConf = true && this.is_set_jsonConf(); + boolean that_present_jsonConf = true && that.is_set_jsonConf(); + if (this_present_jsonConf || that_present_jsonConf) { + if (!(this_present_jsonConf && that_present_jsonConf)) + return false; + if (!this.jsonConf.equals(that.jsonConf)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_name = true && (is_set_name()); + list.add(present_name); + if (present_name) + list.add(name); + + boolean present_jsonConf = true && (is_set_jsonConf()); + list.add(present_jsonConf); + if (present_jsonConf) + list.add(jsonConf); + + return list.hashCode(); + } + + @Override + public int compareTo(restart_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_jsonConf()).compareTo(other.is_set_jsonConf()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_jsonConf()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.jsonConf, other.jsonConf); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("restart_args("); + boolean first = true; + + sb.append("name:"); + if (this.name == null) { + sb.append("null"); + } else { + sb.append(this.name); + } + first = false; + if (!first) sb.append(", "); + sb.append("jsonConf:"); + if (this.jsonConf == null) { + sb.append("null"); + } else { + sb.append(this.jsonConf); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class restart_argsStandardSchemeFactory implements SchemeFactory { + public restart_argsStandardScheme getScheme() { + return new restart_argsStandardScheme(); + } + } + + private static class restart_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, restart_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // JSON_CONF + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.jsonConf = iprot.readString(); + struct.set_jsonConf_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, restart_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.name != null) { + oprot.writeFieldBegin(NAME_FIELD_DESC); + oprot.writeString(struct.name); + oprot.writeFieldEnd(); + } + if (struct.jsonConf != null) { + oprot.writeFieldBegin(JSON_CONF_FIELD_DESC); + oprot.writeString(struct.jsonConf); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class restart_argsTupleSchemeFactory implements SchemeFactory { + public restart_argsTupleScheme getScheme() { + return new restart_argsTupleScheme(); + } + } + + private static class restart_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, restart_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_name()) { + optionals.set(0); + } + if (struct.is_set_jsonConf()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_name()) { + oprot.writeString(struct.name); + } + if (struct.is_set_jsonConf()) { + oprot.writeString(struct.jsonConf); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, restart_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } + if (incoming.get(1)) { + struct.jsonConf = iprot.readString(); + struct.set_jsonConf_isSet(true); + } + } + } + + } + + public static class restart_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("restart_result"); + + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField ITE_FIELD_DESC = new org.apache.thrift.protocol.TField("ite", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField TAE_FIELD_DESC = new org.apache.thrift.protocol.TField("tae", org.apache.thrift.protocol.TType.STRUCT, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new restart_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new restart_resultTupleSchemeFactory()); + } + + private NotAliveException e; // required + private InvalidTopologyException ite; // required + private TopologyAssignException tae; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + E((short)1, "e"), + ITE((short)2, "ite"), + TAE((short)3, "tae"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // E + return E; + case 2: // ITE + return ITE; + case 3: // TAE + return TAE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.ITE, new org.apache.thrift.meta_data.FieldMetaData("ite", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.TAE, new org.apache.thrift.meta_data.FieldMetaData("tae", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(restart_result.class, metaDataMap); + } + + public restart_result() { + } + + public restart_result( + NotAliveException e, + InvalidTopologyException ite, + TopologyAssignException tae) + { + this(); + this.e = e; + this.ite = ite; + this.tae = tae; + } + + /** + * Performs a deep copy on other. + */ + public restart_result(restart_result other) { + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + if (other.is_set_ite()) { + this.ite = new InvalidTopologyException(other.ite); + } + if (other.is_set_tae()) { + this.tae = new TopologyAssignException(other.tae); + } + } + + public restart_result deepCopy() { + return new restart_result(this); + } + + @Override + public void clear() { + this.e = null; + this.ite = null; + this.tae = null; + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public InvalidTopologyException get_ite() { + return this.ite; + } + + public void set_ite(InvalidTopologyException ite) { + this.ite = ite; + } + + public void unset_ite() { + this.ite = null; + } + + /** Returns true if field ite is set (has been assigned a value) and false otherwise */ + public boolean is_set_ite() { + return this.ite != null; + } + + public void set_ite_isSet(boolean value) { + if (!value) { + this.ite = null; + } + } + + public TopologyAssignException get_tae() { + return this.tae; + } + + public void set_tae(TopologyAssignException tae) { + this.tae = tae; + } + + public void unset_tae() { + this.tae = null; + } + + /** Returns true if field tae is set (has been assigned a value) and false otherwise */ + public boolean is_set_tae() { + return this.tae != null; + } + + public void set_tae_isSet(boolean value) { + if (!value) { + this.tae = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + case ITE: + if (value == null) { + unset_ite(); + } else { + set_ite((InvalidTopologyException)value); + } + break; + + case TAE: + if (value == null) { + unset_tae(); + } else { + set_tae((TopologyAssignException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case E: + return get_e(); + + case ITE: + return get_ite(); + + case TAE: + return get_tae(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case E: + return is_set_e(); + case ITE: + return is_set_ite(); + case TAE: + return is_set_tae(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof restart_result) + return this.equals((restart_result)that); + return false; + } + + public boolean equals(restart_result that) { + if (that == null) + return false; + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + boolean this_present_ite = true && this.is_set_ite(); + boolean that_present_ite = true && that.is_set_ite(); + if (this_present_ite || that_present_ite) { + if (!(this_present_ite && that_present_ite)) + return false; + if (!this.ite.equals(that.ite)) + return false; + } + + boolean this_present_tae = true && this.is_set_tae(); + boolean that_present_tae = true && that.is_set_tae(); + if (this_present_tae || that_present_tae) { + if (!(this_present_tae && that_present_tae)) + return false; + if (!this.tae.equals(that.tae)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + boolean present_ite = true && (is_set_ite()); + list.add(present_ite); + if (present_ite) + list.add(ite); + + boolean present_tae = true && (is_set_tae()); + list.add(present_tae); + if (present_tae) + list.add(tae); + + return list.hashCode(); + } + + @Override + public int compareTo(restart_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_ite()).compareTo(other.is_set_ite()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_ite()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ite, other.ite); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_tae()).compareTo(other.is_set_tae()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_tae()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tae, other.tae); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("restart_result("); + boolean first = true; + + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + if (!first) sb.append(", "); + sb.append("ite:"); + if (this.ite == null) { + sb.append("null"); + } else { + sb.append(this.ite); + } + first = false; + if (!first) sb.append(", "); + sb.append("tae:"); + if (this.tae == null) { + sb.append("null"); + } else { + sb.append(this.tae); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class restart_resultStandardSchemeFactory implements SchemeFactory { + public restart_resultStandardScheme getScheme() { + return new restart_resultStandardScheme(); + } + } + + private static class restart_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, restart_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // ITE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.ite = new InvalidTopologyException(); + struct.ite.read(iprot); + struct.set_ite_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // TAE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.tae = new TopologyAssignException(); + struct.tae.read(iprot); + struct.set_tae_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, restart_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.ite != null) { + oprot.writeFieldBegin(ITE_FIELD_DESC); + struct.ite.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.tae != null) { + oprot.writeFieldBegin(TAE_FIELD_DESC); + struct.tae.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class restart_resultTupleSchemeFactory implements SchemeFactory { + public restart_resultTupleScheme getScheme() { + return new restart_resultTupleScheme(); + } + } + + private static class restart_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, restart_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_e()) { + optionals.set(0); + } + if (struct.is_set_ite()) { + optionals.set(1); + } + if (struct.is_set_tae()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.is_set_e()) { + struct.e.write(oprot); + } + if (struct.is_set_ite()) { + struct.ite.write(oprot); + } + if (struct.is_set_tae()) { + struct.tae.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, restart_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + if (incoming.get(1)) { + struct.ite = new InvalidTopologyException(); + struct.ite.read(iprot); + struct.set_ite_isSet(true); + } + if (incoming.get(2)) { + struct.tae = new TopologyAssignException(); + struct.tae.read(iprot); + struct.set_tae_isSet(true); + } + } + } + + } + + public static class beginLibUpload_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("beginLibUpload_args"); + + private static final org.apache.thrift.protocol.TField LIB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("libName", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new beginLibUpload_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new beginLibUpload_argsTupleSchemeFactory()); + } + + private String libName; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + LIB_NAME((short)1, "libName"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // LIB_NAME + return LIB_NAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.LIB_NAME, new org.apache.thrift.meta_data.FieldMetaData("libName", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(beginLibUpload_args.class, metaDataMap); + } + + public beginLibUpload_args() { + } + + public beginLibUpload_args( + String libName) + { + this(); + this.libName = libName; + } + + /** + * Performs a deep copy on other. + */ + public beginLibUpload_args(beginLibUpload_args other) { + if (other.is_set_libName()) { + this.libName = other.libName; + } + } + + public beginLibUpload_args deepCopy() { + return new beginLibUpload_args(this); + } + + @Override + public void clear() { + this.libName = null; + } + + public String get_libName() { + return this.libName; + } + + public void set_libName(String libName) { + this.libName = libName; + } + + public void unset_libName() { + this.libName = null; + } + + /** Returns true if field libName is set (has been assigned a value) and false otherwise */ + public boolean is_set_libName() { + return this.libName != null; + } + + public void set_libName_isSet(boolean value) { + if (!value) { + this.libName = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case LIB_NAME: + if (value == null) { + unset_libName(); + } else { + set_libName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case LIB_NAME: + return get_libName(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case LIB_NAME: + return is_set_libName(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof beginLibUpload_args) + return this.equals((beginLibUpload_args)that); + return false; + } + + public boolean equals(beginLibUpload_args that) { + if (that == null) + return false; + + boolean this_present_libName = true && this.is_set_libName(); + boolean that_present_libName = true && that.is_set_libName(); + if (this_present_libName || that_present_libName) { + if (!(this_present_libName && that_present_libName)) + return false; + if (!this.libName.equals(that.libName)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_libName = true && (is_set_libName()); + list.add(present_libName); + if (present_libName) + list.add(libName); + + return list.hashCode(); + } + + @Override + public int compareTo(beginLibUpload_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_libName()).compareTo(other.is_set_libName()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_libName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.libName, other.libName); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("beginLibUpload_args("); + boolean first = true; + + sb.append("libName:"); + if (this.libName == null) { + sb.append("null"); + } else { + sb.append(this.libName); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class beginLibUpload_argsStandardSchemeFactory implements SchemeFactory { + public beginLibUpload_argsStandardScheme getScheme() { + return new beginLibUpload_argsStandardScheme(); + } + } + + private static class beginLibUpload_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, beginLibUpload_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // LIB_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.libName = iprot.readString(); + struct.set_libName_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, beginLibUpload_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.libName != null) { + oprot.writeFieldBegin(LIB_NAME_FIELD_DESC); + oprot.writeString(struct.libName); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class beginLibUpload_argsTupleSchemeFactory implements SchemeFactory { + public beginLibUpload_argsTupleScheme getScheme() { + return new beginLibUpload_argsTupleScheme(); + } + } + + private static class beginLibUpload_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, beginLibUpload_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_libName()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_libName()) { + oprot.writeString(struct.libName); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, beginLibUpload_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.libName = iprot.readString(); + struct.set_libName_isSet(true); + } + } + } + + } + + public static class beginLibUpload_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("beginLibUpload_result"); + + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new beginLibUpload_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new beginLibUpload_resultTupleSchemeFactory()); + } + + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { +; + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(beginLibUpload_result.class, metaDataMap); + } + + public beginLibUpload_result() { + } + + /** + * Performs a deep copy on other. + */ + public beginLibUpload_result(beginLibUpload_result other) { + } + + public beginLibUpload_result deepCopy() { + return new beginLibUpload_result(this); + } + + @Override + public void clear() { + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof beginLibUpload_result) + return this.equals((beginLibUpload_result)that); + return false; + } + + public boolean equals(beginLibUpload_result that) { + if (that == null) + return false; + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + return list.hashCode(); + } + + @Override + public int compareTo(beginLibUpload_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("beginLibUpload_result("); + boolean first = true; + + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class beginLibUpload_resultStandardSchemeFactory implements SchemeFactory { + public beginLibUpload_resultStandardScheme getScheme() { + return new beginLibUpload_resultStandardScheme(); + } + } + + private static class beginLibUpload_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, beginLibUpload_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, beginLibUpload_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class beginLibUpload_resultTupleSchemeFactory implements SchemeFactory { + public beginLibUpload_resultTupleScheme getScheme() { + return new beginLibUpload_resultTupleScheme(); + } + } + + private static class beginLibUpload_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, beginLibUpload_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, beginLibUpload_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + } + } + + } + + public static class beginFileUpload_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("beginFileUpload_args"); + + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new beginFileUpload_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new beginFileUpload_argsTupleSchemeFactory()); + } + + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { +; + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(beginFileUpload_args.class, metaDataMap); + } + + public beginFileUpload_args() { + } + + /** + * Performs a deep copy on other. + */ + public beginFileUpload_args(beginFileUpload_args other) { + } + + public beginFileUpload_args deepCopy() { + return new beginFileUpload_args(this); + } + + @Override + public void clear() { + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof beginFileUpload_args) + return this.equals((beginFileUpload_args)that); + return false; + } + + public boolean equals(beginFileUpload_args that) { + if (that == null) + return false; + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + return list.hashCode(); + } + + @Override + public int compareTo(beginFileUpload_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("beginFileUpload_args("); + boolean first = true; + + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class beginFileUpload_argsStandardSchemeFactory implements SchemeFactory { + public beginFileUpload_argsStandardScheme getScheme() { + return new beginFileUpload_argsStandardScheme(); + } + } + + private static class beginFileUpload_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, beginFileUpload_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, beginFileUpload_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class beginFileUpload_argsTupleSchemeFactory implements SchemeFactory { + public beginFileUpload_argsTupleScheme getScheme() { + return new beginFileUpload_argsTupleScheme(); + } + } + + private static class beginFileUpload_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, beginFileUpload_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, beginFileUpload_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + } + } + + } + + public static class beginFileUpload_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("beginFileUpload_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new beginFileUpload_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new beginFileUpload_resultTupleSchemeFactory()); + } + + private String success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(beginFileUpload_result.class, metaDataMap); + } + + public beginFileUpload_result() { + } + + public beginFileUpload_result( + String success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public beginFileUpload_result(beginFileUpload_result other) { + if (other.is_set_success()) { + this.success = other.success; + } + } + + public beginFileUpload_result deepCopy() { + return new beginFileUpload_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public String get_success() { + return this.success; + } + + public void set_success(String success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof beginFileUpload_result) + return this.equals((beginFileUpload_result)that); + return false; + } + + public boolean equals(beginFileUpload_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(beginFileUpload_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("beginFileUpload_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class beginFileUpload_resultStandardSchemeFactory implements SchemeFactory { + public beginFileUpload_resultStandardScheme getScheme() { + return new beginFileUpload_resultStandardScheme(); + } + } + + private static class beginFileUpload_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, beginFileUpload_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.success = iprot.readString(); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, beginFileUpload_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeString(struct.success); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class beginFileUpload_resultTupleSchemeFactory implements SchemeFactory { + public beginFileUpload_resultTupleScheme getScheme() { + return new beginFileUpload_resultTupleScheme(); + } + } + + private static class beginFileUpload_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, beginFileUpload_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_success()) { + oprot.writeString(struct.success); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, beginFileUpload_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = iprot.readString(); + struct.set_success_isSet(true); + } + } + } + + } + + public static class uploadChunk_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("uploadChunk_args"); + + private static final org.apache.thrift.protocol.TField LOCATION_FIELD_DESC = new org.apache.thrift.protocol.TField("location", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField CHUNK_FIELD_DESC = new org.apache.thrift.protocol.TField("chunk", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new uploadChunk_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new uploadChunk_argsTupleSchemeFactory()); + } + + private String location; // required + private ByteBuffer chunk; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + LOCATION((short)1, "location"), + CHUNK((short)2, "chunk"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // LOCATION + return LOCATION; + case 2: // CHUNK + return CHUNK; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.LOCATION, new org.apache.thrift.meta_data.FieldMetaData("location", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.CHUNK, new org.apache.thrift.meta_data.FieldMetaData("chunk", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(uploadChunk_args.class, metaDataMap); + } + + public uploadChunk_args() { + } + + public uploadChunk_args( + String location, + ByteBuffer chunk) + { + this(); + this.location = location; + this.chunk = org.apache.thrift.TBaseHelper.copyBinary(chunk); + } + + /** + * Performs a deep copy on other. + */ + public uploadChunk_args(uploadChunk_args other) { + if (other.is_set_location()) { + this.location = other.location; + } + if (other.is_set_chunk()) { + this.chunk = org.apache.thrift.TBaseHelper.copyBinary(other.chunk); + } + } + + public uploadChunk_args deepCopy() { + return new uploadChunk_args(this); + } + + @Override + public void clear() { + this.location = null; + this.chunk = null; + } + + public String get_location() { + return this.location; + } + + public void set_location(String location) { + this.location = location; + } + + public void unset_location() { + this.location = null; + } + + /** Returns true if field location is set (has been assigned a value) and false otherwise */ + public boolean is_set_location() { + return this.location != null; + } + + public void set_location_isSet(boolean value) { + if (!value) { + this.location = null; + } + } + + public byte[] get_chunk() { + set_chunk(org.apache.thrift.TBaseHelper.rightSize(chunk)); + return chunk == null ? null : chunk.array(); + } + + public ByteBuffer buffer_for_chunk() { + return org.apache.thrift.TBaseHelper.copyBinary(chunk); + } + + public void set_chunk(byte[] chunk) { + this.chunk = chunk == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(chunk, chunk.length)); + } + + public void set_chunk(ByteBuffer chunk) { + this.chunk = org.apache.thrift.TBaseHelper.copyBinary(chunk); + } + + public void unset_chunk() { + this.chunk = null; + } + + /** Returns true if field chunk is set (has been assigned a value) and false otherwise */ + public boolean is_set_chunk() { + return this.chunk != null; + } + + public void set_chunk_isSet(boolean value) { + if (!value) { + this.chunk = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case LOCATION: + if (value == null) { + unset_location(); + } else { + set_location((String)value); + } + break; + + case CHUNK: + if (value == null) { + unset_chunk(); + } else { + set_chunk((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case LOCATION: + return get_location(); + + case CHUNK: + return get_chunk(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case LOCATION: + return is_set_location(); + case CHUNK: + return is_set_chunk(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof uploadChunk_args) + return this.equals((uploadChunk_args)that); + return false; + } + + public boolean equals(uploadChunk_args that) { + if (that == null) + return false; + + boolean this_present_location = true && this.is_set_location(); + boolean that_present_location = true && that.is_set_location(); + if (this_present_location || that_present_location) { + if (!(this_present_location && that_present_location)) + return false; + if (!this.location.equals(that.location)) + return false; + } + + boolean this_present_chunk = true && this.is_set_chunk(); + boolean that_present_chunk = true && that.is_set_chunk(); + if (this_present_chunk || that_present_chunk) { + if (!(this_present_chunk && that_present_chunk)) + return false; + if (!this.chunk.equals(that.chunk)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_location = true && (is_set_location()); + list.add(present_location); + if (present_location) + list.add(location); + + boolean present_chunk = true && (is_set_chunk()); + list.add(present_chunk); + if (present_chunk) + list.add(chunk); + + return list.hashCode(); + } + + @Override + public int compareTo(uploadChunk_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_location()).compareTo(other.is_set_location()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_location()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.location, other.location); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_chunk()).compareTo(other.is_set_chunk()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_chunk()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.chunk, other.chunk); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("uploadChunk_args("); + boolean first = true; + + sb.append("location:"); + if (this.location == null) { + sb.append("null"); + } else { + sb.append(this.location); + } + first = false; + if (!first) sb.append(", "); + sb.append("chunk:"); + if (this.chunk == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.chunk, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class uploadChunk_argsStandardSchemeFactory implements SchemeFactory { + public uploadChunk_argsStandardScheme getScheme() { + return new uploadChunk_argsStandardScheme(); + } + } + + private static class uploadChunk_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, uploadChunk_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // LOCATION + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.location = iprot.readString(); + struct.set_location_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // CHUNK + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.chunk = iprot.readBinary(); + struct.set_chunk_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, uploadChunk_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.location != null) { + oprot.writeFieldBegin(LOCATION_FIELD_DESC); + oprot.writeString(struct.location); + oprot.writeFieldEnd(); + } + if (struct.chunk != null) { + oprot.writeFieldBegin(CHUNK_FIELD_DESC); + oprot.writeBinary(struct.chunk); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class uploadChunk_argsTupleSchemeFactory implements SchemeFactory { + public uploadChunk_argsTupleScheme getScheme() { + return new uploadChunk_argsTupleScheme(); + } + } + + private static class uploadChunk_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, uploadChunk_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_location()) { + optionals.set(0); + } + if (struct.is_set_chunk()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_location()) { + oprot.writeString(struct.location); + } + if (struct.is_set_chunk()) { + oprot.writeBinary(struct.chunk); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, uploadChunk_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.location = iprot.readString(); + struct.set_location_isSet(true); + } + if (incoming.get(1)) { + struct.chunk = iprot.readBinary(); + struct.set_chunk_isSet(true); + } + } + } + + } + + public static class uploadChunk_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("uploadChunk_result"); + + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new uploadChunk_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new uploadChunk_resultTupleSchemeFactory()); + } + + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { +; + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(uploadChunk_result.class, metaDataMap); + } + + public uploadChunk_result() { + } + + /** + * Performs a deep copy on other. + */ + public uploadChunk_result(uploadChunk_result other) { + } + + public uploadChunk_result deepCopy() { + return new uploadChunk_result(this); + } + + @Override + public void clear() { + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof uploadChunk_result) + return this.equals((uploadChunk_result)that); + return false; + } + + public boolean equals(uploadChunk_result that) { + if (that == null) + return false; + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + return list.hashCode(); + } + + @Override + public int compareTo(uploadChunk_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("uploadChunk_result("); + boolean first = true; + + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class uploadChunk_resultStandardSchemeFactory implements SchemeFactory { + public uploadChunk_resultStandardScheme getScheme() { + return new uploadChunk_resultStandardScheme(); + } + } + + private static class uploadChunk_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, uploadChunk_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, uploadChunk_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class uploadChunk_resultTupleSchemeFactory implements SchemeFactory { + public uploadChunk_resultTupleScheme getScheme() { + return new uploadChunk_resultTupleScheme(); + } + } + + private static class uploadChunk_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, uploadChunk_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, uploadChunk_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + } + } + + } + + public static class finishFileUpload_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("finishFileUpload_args"); + + private static final org.apache.thrift.protocol.TField LOCATION_FIELD_DESC = new org.apache.thrift.protocol.TField("location", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new finishFileUpload_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new finishFileUpload_argsTupleSchemeFactory()); + } + + private String location; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + LOCATION((short)1, "location"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // LOCATION + return LOCATION; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.LOCATION, new org.apache.thrift.meta_data.FieldMetaData("location", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(finishFileUpload_args.class, metaDataMap); + } + + public finishFileUpload_args() { + } + + public finishFileUpload_args( + String location) + { + this(); + this.location = location; + } + + /** + * Performs a deep copy on other. + */ + public finishFileUpload_args(finishFileUpload_args other) { + if (other.is_set_location()) { + this.location = other.location; + } + } + + public finishFileUpload_args deepCopy() { + return new finishFileUpload_args(this); + } + + @Override + public void clear() { + this.location = null; + } + + public String get_location() { + return this.location; + } + + public void set_location(String location) { + this.location = location; + } + + public void unset_location() { + this.location = null; + } + + /** Returns true if field location is set (has been assigned a value) and false otherwise */ + public boolean is_set_location() { + return this.location != null; + } + + public void set_location_isSet(boolean value) { + if (!value) { + this.location = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case LOCATION: + if (value == null) { + unset_location(); + } else { + set_location((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case LOCATION: + return get_location(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case LOCATION: + return is_set_location(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof finishFileUpload_args) + return this.equals((finishFileUpload_args)that); + return false; + } + + public boolean equals(finishFileUpload_args that) { + if (that == null) + return false; + + boolean this_present_location = true && this.is_set_location(); + boolean that_present_location = true && that.is_set_location(); + if (this_present_location || that_present_location) { + if (!(this_present_location && that_present_location)) + return false; + if (!this.location.equals(that.location)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_location = true && (is_set_location()); + list.add(present_location); + if (present_location) + list.add(location); + + return list.hashCode(); + } + + @Override + public int compareTo(finishFileUpload_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_location()).compareTo(other.is_set_location()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_location()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.location, other.location); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("finishFileUpload_args("); + boolean first = true; + + sb.append("location:"); + if (this.location == null) { + sb.append("null"); + } else { + sb.append(this.location); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class finishFileUpload_argsStandardSchemeFactory implements SchemeFactory { + public finishFileUpload_argsStandardScheme getScheme() { + return new finishFileUpload_argsStandardScheme(); + } + } + + private static class finishFileUpload_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, finishFileUpload_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // LOCATION + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.location = iprot.readString(); + struct.set_location_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, finishFileUpload_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.location != null) { + oprot.writeFieldBegin(LOCATION_FIELD_DESC); + oprot.writeString(struct.location); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class finishFileUpload_argsTupleSchemeFactory implements SchemeFactory { + public finishFileUpload_argsTupleScheme getScheme() { + return new finishFileUpload_argsTupleScheme(); + } + } + + private static class finishFileUpload_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, finishFileUpload_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_location()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_location()) { + oprot.writeString(struct.location); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, finishFileUpload_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.location = iprot.readString(); + struct.set_location_isSet(true); + } + } + } + + } + + public static class finishFileUpload_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("finishFileUpload_result"); + + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new finishFileUpload_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new finishFileUpload_resultTupleSchemeFactory()); + } + + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { +; + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(finishFileUpload_result.class, metaDataMap); + } + + public finishFileUpload_result() { + } + + /** + * Performs a deep copy on other. + */ + public finishFileUpload_result(finishFileUpload_result other) { + } + + public finishFileUpload_result deepCopy() { + return new finishFileUpload_result(this); + } + + @Override + public void clear() { + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof finishFileUpload_result) + return this.equals((finishFileUpload_result)that); + return false; + } + + public boolean equals(finishFileUpload_result that) { + if (that == null) + return false; + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + return list.hashCode(); + } + + @Override + public int compareTo(finishFileUpload_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("finishFileUpload_result("); + boolean first = true; + + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class finishFileUpload_resultStandardSchemeFactory implements SchemeFactory { + public finishFileUpload_resultStandardScheme getScheme() { + return new finishFileUpload_resultStandardScheme(); + } + } + + private static class finishFileUpload_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, finishFileUpload_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, finishFileUpload_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class finishFileUpload_resultTupleSchemeFactory implements SchemeFactory { + public finishFileUpload_resultTupleScheme getScheme() { + return new finishFileUpload_resultTupleScheme(); + } + } + + private static class finishFileUpload_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, finishFileUpload_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, finishFileUpload_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + } + } + + } + + public static class beginFileDownload_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("beginFileDownload_args"); + + private static final org.apache.thrift.protocol.TField FILE_FIELD_DESC = new org.apache.thrift.protocol.TField("file", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new beginFileDownload_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new beginFileDownload_argsTupleSchemeFactory()); + } + + private String file; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + FILE((short)1, "file"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // FILE + return FILE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.FILE, new org.apache.thrift.meta_data.FieldMetaData("file", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(beginFileDownload_args.class, metaDataMap); + } + + public beginFileDownload_args() { + } + + public beginFileDownload_args( + String file) + { + this(); + this.file = file; + } + + /** + * Performs a deep copy on other. + */ + public beginFileDownload_args(beginFileDownload_args other) { + if (other.is_set_file()) { + this.file = other.file; + } + } + + public beginFileDownload_args deepCopy() { + return new beginFileDownload_args(this); + } + + @Override + public void clear() { + this.file = null; + } + + public String get_file() { + return this.file; + } + + public void set_file(String file) { + this.file = file; + } + + public void unset_file() { + this.file = null; + } + + /** Returns true if field file is set (has been assigned a value) and false otherwise */ + public boolean is_set_file() { + return this.file != null; + } + + public void set_file_isSet(boolean value) { + if (!value) { + this.file = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case FILE: + if (value == null) { + unset_file(); + } else { + set_file((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case FILE: + return get_file(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case FILE: + return is_set_file(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof beginFileDownload_args) + return this.equals((beginFileDownload_args)that); + return false; + } + + public boolean equals(beginFileDownload_args that) { + if (that == null) + return false; + + boolean this_present_file = true && this.is_set_file(); + boolean that_present_file = true && that.is_set_file(); + if (this_present_file || that_present_file) { + if (!(this_present_file && that_present_file)) + return false; + if (!this.file.equals(that.file)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_file = true && (is_set_file()); + list.add(present_file); + if (present_file) + list.add(file); + + return list.hashCode(); + } + + @Override + public int compareTo(beginFileDownload_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_file()).compareTo(other.is_set_file()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_file()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.file, other.file); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("beginFileDownload_args("); + boolean first = true; + + sb.append("file:"); + if (this.file == null) { + sb.append("null"); + } else { + sb.append(this.file); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class beginFileDownload_argsStandardSchemeFactory implements SchemeFactory { + public beginFileDownload_argsStandardScheme getScheme() { + return new beginFileDownload_argsStandardScheme(); + } + } + + private static class beginFileDownload_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, beginFileDownload_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // FILE + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.file = iprot.readString(); + struct.set_file_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, beginFileDownload_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.file != null) { + oprot.writeFieldBegin(FILE_FIELD_DESC); + oprot.writeString(struct.file); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class beginFileDownload_argsTupleSchemeFactory implements SchemeFactory { + public beginFileDownload_argsTupleScheme getScheme() { + return new beginFileDownload_argsTupleScheme(); + } + } + + private static class beginFileDownload_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, beginFileDownload_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_file()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_file()) { + oprot.writeString(struct.file); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, beginFileDownload_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.file = iprot.readString(); + struct.set_file_isSet(true); + } + } + } + + } + + public static class beginFileDownload_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("beginFileDownload_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new beginFileDownload_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new beginFileDownload_resultTupleSchemeFactory()); + } + + private String success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(beginFileDownload_result.class, metaDataMap); + } + + public beginFileDownload_result() { + } + + public beginFileDownload_result( + String success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public beginFileDownload_result(beginFileDownload_result other) { + if (other.is_set_success()) { + this.success = other.success; + } + } + + public beginFileDownload_result deepCopy() { + return new beginFileDownload_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public String get_success() { + return this.success; + } + + public void set_success(String success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof beginFileDownload_result) + return this.equals((beginFileDownload_result)that); + return false; + } + + public boolean equals(beginFileDownload_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(beginFileDownload_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("beginFileDownload_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class beginFileDownload_resultStandardSchemeFactory implements SchemeFactory { + public beginFileDownload_resultStandardScheme getScheme() { + return new beginFileDownload_resultStandardScheme(); + } + } + + private static class beginFileDownload_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, beginFileDownload_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.success = iprot.readString(); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, beginFileDownload_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeString(struct.success); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class beginFileDownload_resultTupleSchemeFactory implements SchemeFactory { + public beginFileDownload_resultTupleScheme getScheme() { + return new beginFileDownload_resultTupleScheme(); + } + } + + private static class beginFileDownload_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, beginFileDownload_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_success()) { + oprot.writeString(struct.success); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, beginFileDownload_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = iprot.readString(); + struct.set_success_isSet(true); + } + } + } + + } + + public static class downloadChunk_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("downloadChunk_args"); + + private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new downloadChunk_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new downloadChunk_argsTupleSchemeFactory()); + } + + private String id; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + ID((short)1, "id"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // ID + return ID; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(downloadChunk_args.class, metaDataMap); + } + + public downloadChunk_args() { + } + + public downloadChunk_args( + String id) + { + this(); + this.id = id; + } + + /** + * Performs a deep copy on other. + */ + public downloadChunk_args(downloadChunk_args other) { + if (other.is_set_id()) { + this.id = other.id; + } + } + + public downloadChunk_args deepCopy() { + return new downloadChunk_args(this); + } + + @Override + public void clear() { + this.id = null; + } + + public String get_id() { + return this.id; + } + + public void set_id(String id) { + this.id = id; + } + + public void unset_id() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean is_set_id() { + return this.id != null; + } + + public void set_id_isSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unset_id(); + } else { + set_id((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return get_id(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case ID: + return is_set_id(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof downloadChunk_args) + return this.equals((downloadChunk_args)that); + return false; + } + + public boolean equals(downloadChunk_args that) { + if (that == null) + return false; + + boolean this_present_id = true && this.is_set_id(); + boolean that_present_id = true && that.is_set_id(); + if (this_present_id || that_present_id) { + if (!(this_present_id && that_present_id)) + return false; + if (!this.id.equals(that.id)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_id = true && (is_set_id()); + list.add(present_id); + if (present_id) + list.add(id); + + return list.hashCode(); + } + + @Override + public int compareTo(downloadChunk_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("downloadChunk_args("); + boolean first = true; + + sb.append("id:"); + if (this.id == null) { + sb.append("null"); + } else { + sb.append(this.id); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class downloadChunk_argsStandardSchemeFactory implements SchemeFactory { + public downloadChunk_argsStandardScheme getScheme() { + return new downloadChunk_argsStandardScheme(); + } + } + + private static class downloadChunk_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, downloadChunk_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, downloadChunk_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.id != null) { + oprot.writeFieldBegin(ID_FIELD_DESC); + oprot.writeString(struct.id); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class downloadChunk_argsTupleSchemeFactory implements SchemeFactory { + public downloadChunk_argsTupleScheme getScheme() { + return new downloadChunk_argsTupleScheme(); + } + } + + private static class downloadChunk_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, downloadChunk_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_id()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_id()) { + oprot.writeString(struct.id); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, downloadChunk_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } + } + } + + } + + public static class downloadChunk_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("downloadChunk_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new downloadChunk_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new downloadChunk_resultTupleSchemeFactory()); + } + + private ByteBuffer success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(downloadChunk_result.class, metaDataMap); + } + + public downloadChunk_result() { + } + + public downloadChunk_result( + ByteBuffer success) + { + this(); + this.success = org.apache.thrift.TBaseHelper.copyBinary(success); + } + + /** + * Performs a deep copy on other. + */ + public downloadChunk_result(downloadChunk_result other) { + if (other.is_set_success()) { + this.success = org.apache.thrift.TBaseHelper.copyBinary(other.success); + } + } + + public downloadChunk_result deepCopy() { + return new downloadChunk_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public byte[] get_success() { + set_success(org.apache.thrift.TBaseHelper.rightSize(success)); + return success == null ? null : success.array(); + } + + public ByteBuffer buffer_for_success() { + return org.apache.thrift.TBaseHelper.copyBinary(success); + } + + public void set_success(byte[] success) { + this.success = success == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(success, success.length)); + } + + public void set_success(ByteBuffer success) { + this.success = org.apache.thrift.TBaseHelper.copyBinary(success); + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof downloadChunk_result) + return this.equals((downloadChunk_result)that); + return false; + } + + public boolean equals(downloadChunk_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(downloadChunk_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("downloadChunk_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.success, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class downloadChunk_resultStandardSchemeFactory implements SchemeFactory { + public downloadChunk_resultStandardScheme getScheme() { + return new downloadChunk_resultStandardScheme(); + } + } + + private static class downloadChunk_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, downloadChunk_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.success = iprot.readBinary(); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, downloadChunk_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeBinary(struct.success); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class downloadChunk_resultTupleSchemeFactory implements SchemeFactory { + public downloadChunk_resultTupleScheme getScheme() { + return new downloadChunk_resultTupleScheme(); + } + } + + private static class downloadChunk_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, downloadChunk_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_success()) { + oprot.writeBinary(struct.success); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, downloadChunk_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = iprot.readBinary(); + struct.set_success_isSet(true); + } + } + } + + } + + public static class finishFileDownload_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("finishFileDownload_args"); + + private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new finishFileDownload_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new finishFileDownload_argsTupleSchemeFactory()); + } + + private String id; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + ID((short)1, "id"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // ID + return ID; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(finishFileDownload_args.class, metaDataMap); + } + + public finishFileDownload_args() { + } + + public finishFileDownload_args( + String id) + { + this(); + this.id = id; + } + + /** + * Performs a deep copy on other. + */ + public finishFileDownload_args(finishFileDownload_args other) { + if (other.is_set_id()) { + this.id = other.id; + } + } + + public finishFileDownload_args deepCopy() { + return new finishFileDownload_args(this); + } + + @Override + public void clear() { + this.id = null; + } + + public String get_id() { + return this.id; + } + + public void set_id(String id) { + this.id = id; + } + + public void unset_id() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean is_set_id() { + return this.id != null; + } + + public void set_id_isSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unset_id(); + } else { + set_id((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return get_id(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case ID: + return is_set_id(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof finishFileDownload_args) + return this.equals((finishFileDownload_args)that); + return false; + } + + public boolean equals(finishFileDownload_args that) { + if (that == null) + return false; + + boolean this_present_id = true && this.is_set_id(); + boolean that_present_id = true && that.is_set_id(); + if (this_present_id || that_present_id) { + if (!(this_present_id && that_present_id)) + return false; + if (!this.id.equals(that.id)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_id = true && (is_set_id()); + list.add(present_id); + if (present_id) + list.add(id); + + return list.hashCode(); + } + + @Override + public int compareTo(finishFileDownload_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("finishFileDownload_args("); + boolean first = true; + + sb.append("id:"); + if (this.id == null) { + sb.append("null"); + } else { + sb.append(this.id); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class finishFileDownload_argsStandardSchemeFactory implements SchemeFactory { + public finishFileDownload_argsStandardScheme getScheme() { + return new finishFileDownload_argsStandardScheme(); + } + } + + private static class finishFileDownload_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, finishFileDownload_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, finishFileDownload_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.id != null) { + oprot.writeFieldBegin(ID_FIELD_DESC); + oprot.writeString(struct.id); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class finishFileDownload_argsTupleSchemeFactory implements SchemeFactory { + public finishFileDownload_argsTupleScheme getScheme() { + return new finishFileDownload_argsTupleScheme(); + } + } + + private static class finishFileDownload_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, finishFileDownload_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_id()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_id()) { + oprot.writeString(struct.id); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, finishFileDownload_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } + } + } + + } + + public static class finishFileDownload_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("finishFileDownload_result"); + + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new finishFileDownload_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new finishFileDownload_resultTupleSchemeFactory()); + } + + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { +; + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(finishFileDownload_result.class, metaDataMap); + } + + public finishFileDownload_result() { + } + + /** + * Performs a deep copy on other. + */ + public finishFileDownload_result(finishFileDownload_result other) { + } + + public finishFileDownload_result deepCopy() { + return new finishFileDownload_result(this); + } + + @Override + public void clear() { + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof finishFileDownload_result) + return this.equals((finishFileDownload_result)that); + return false; + } + + public boolean equals(finishFileDownload_result that) { + if (that == null) + return false; + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + return list.hashCode(); + } + + @Override + public int compareTo(finishFileDownload_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("finishFileDownload_result("); + boolean first = true; + + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class finishFileDownload_resultStandardSchemeFactory implements SchemeFactory { + public finishFileDownload_resultStandardScheme getScheme() { + return new finishFileDownload_resultStandardScheme(); + } + } + + private static class finishFileDownload_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, finishFileDownload_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, finishFileDownload_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class finishFileDownload_resultTupleSchemeFactory implements SchemeFactory { + public finishFileDownload_resultTupleScheme getScheme() { + return new finishFileDownload_resultTupleScheme(); + } + } + + private static class finishFileDownload_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, finishFileDownload_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, finishFileDownload_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + } + } + + } + + public static class getNimbusConf_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getNimbusConf_args"); + + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getNimbusConf_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getNimbusConf_argsTupleSchemeFactory()); + } + + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { +; + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getNimbusConf_args.class, metaDataMap); + } + + public getNimbusConf_args() { + } + + /** + * Performs a deep copy on other. + */ + public getNimbusConf_args(getNimbusConf_args other) { + } + + public getNimbusConf_args deepCopy() { + return new getNimbusConf_args(this); + } + + @Override + public void clear() { + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getNimbusConf_args) + return this.equals((getNimbusConf_args)that); + return false; + } + + public boolean equals(getNimbusConf_args that) { + if (that == null) + return false; + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + return list.hashCode(); + } + + @Override + public int compareTo(getNimbusConf_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getNimbusConf_args("); + boolean first = true; + + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getNimbusConf_argsStandardSchemeFactory implements SchemeFactory { + public getNimbusConf_argsStandardScheme getScheme() { + return new getNimbusConf_argsStandardScheme(); + } + } + + private static class getNimbusConf_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getNimbusConf_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getNimbusConf_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getNimbusConf_argsTupleSchemeFactory implements SchemeFactory { + public getNimbusConf_argsTupleScheme getScheme() { + return new getNimbusConf_argsTupleScheme(); + } + } + + private static class getNimbusConf_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getNimbusConf_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getNimbusConf_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + } + } + + } + + public static class getNimbusConf_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getNimbusConf_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getNimbusConf_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getNimbusConf_resultTupleSchemeFactory()); + } + + private String success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getNimbusConf_result.class, metaDataMap); + } + + public getNimbusConf_result() { + } + + public getNimbusConf_result( + String success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public getNimbusConf_result(getNimbusConf_result other) { + if (other.is_set_success()) { + this.success = other.success; + } + } + + public getNimbusConf_result deepCopy() { + return new getNimbusConf_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public String get_success() { + return this.success; + } + + public void set_success(String success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getNimbusConf_result) + return this.equals((getNimbusConf_result)that); + return false; + } + + public boolean equals(getNimbusConf_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(getNimbusConf_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getNimbusConf_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getNimbusConf_resultStandardSchemeFactory implements SchemeFactory { + public getNimbusConf_resultStandardScheme getScheme() { + return new getNimbusConf_resultStandardScheme(); + } + } + + private static class getNimbusConf_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getNimbusConf_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.success = iprot.readString(); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getNimbusConf_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeString(struct.success); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getNimbusConf_resultTupleSchemeFactory implements SchemeFactory { + public getNimbusConf_resultTupleScheme getScheme() { + return new getNimbusConf_resultTupleScheme(); + } + } + + private static class getNimbusConf_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getNimbusConf_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_success()) { + oprot.writeString(struct.success); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getNimbusConf_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = iprot.readString(); + struct.set_success_isSet(true); + } + } + } + + } + + public static class getTopologyConf_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyConf_args"); + + private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getTopologyConf_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getTopologyConf_argsTupleSchemeFactory()); + } + + private String id; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + ID((short)1, "id"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // ID + return ID; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyConf_args.class, metaDataMap); + } + + public getTopologyConf_args() { + } + + public getTopologyConf_args( + String id) + { + this(); + this.id = id; + } + + /** + * Performs a deep copy on other. + */ + public getTopologyConf_args(getTopologyConf_args other) { + if (other.is_set_id()) { + this.id = other.id; + } + } + + public getTopologyConf_args deepCopy() { + return new getTopologyConf_args(this); + } + + @Override + public void clear() { + this.id = null; + } + + public String get_id() { + return this.id; + } + + public void set_id(String id) { + this.id = id; + } + + public void unset_id() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean is_set_id() { + return this.id != null; + } + + public void set_id_isSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unset_id(); + } else { + set_id((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return get_id(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case ID: + return is_set_id(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getTopologyConf_args) + return this.equals((getTopologyConf_args)that); + return false; + } + + public boolean equals(getTopologyConf_args that) { + if (that == null) + return false; + + boolean this_present_id = true && this.is_set_id(); + boolean that_present_id = true && that.is_set_id(); + if (this_present_id || that_present_id) { + if (!(this_present_id && that_present_id)) + return false; + if (!this.id.equals(that.id)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_id = true && (is_set_id()); + list.add(present_id); + if (present_id) + list.add(id); + + return list.hashCode(); + } + + @Override + public int compareTo(getTopologyConf_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getTopologyConf_args("); + boolean first = true; + + sb.append("id:"); + if (this.id == null) { + sb.append("null"); + } else { + sb.append(this.id); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getTopologyConf_argsStandardSchemeFactory implements SchemeFactory { + public getTopologyConf_argsStandardScheme getScheme() { + return new getTopologyConf_argsStandardScheme(); + } + } + + private static class getTopologyConf_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyConf_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyConf_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.id != null) { + oprot.writeFieldBegin(ID_FIELD_DESC); + oprot.writeString(struct.id); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getTopologyConf_argsTupleSchemeFactory implements SchemeFactory { + public getTopologyConf_argsTupleScheme getScheme() { + return new getTopologyConf_argsTupleScheme(); + } + } + + private static class getTopologyConf_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyConf_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_id()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_id()) { + oprot.writeString(struct.id); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyConf_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } + } + } + + } + + public static class getTopologyConf_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyConf_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0); + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getTopologyConf_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getTopologyConf_resultTupleSchemeFactory()); + } + + private String success; // required + private NotAliveException e; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"), + E((short)1, "e"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // E + return E; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyConf_result.class, metaDataMap); + } + + public getTopologyConf_result() { + } + + public getTopologyConf_result( + String success, + NotAliveException e) + { + this(); + this.success = success; + this.e = e; + } + + /** + * Performs a deep copy on other. + */ + public getTopologyConf_result(getTopologyConf_result other) { + if (other.is_set_success()) { + this.success = other.success; + } + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + } + + public getTopologyConf_result deepCopy() { + return new getTopologyConf_result(this); + } + + @Override + public void clear() { + this.success = null; + this.e = null; + } + + public String get_success() { + return this.success; + } + + public void set_success(String success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((String)value); + } + break; + + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + case E: + return get_e(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + case E: + return is_set_e(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getTopologyConf_result) + return this.equals((getTopologyConf_result)that); + return false; + } + + public boolean equals(getTopologyConf_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + return list.hashCode(); + } + + @Override + public int compareTo(getTopologyConf_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getTopologyConf_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + if (!first) sb.append(", "); + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getTopologyConf_resultStandardSchemeFactory implements SchemeFactory { + public getTopologyConf_resultStandardScheme getScheme() { + return new getTopologyConf_resultStandardScheme(); + } + } + + private static class getTopologyConf_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyConf_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.success = iprot.readString(); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyConf_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeString(struct.success); + oprot.writeFieldEnd(); + } + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getTopologyConf_resultTupleSchemeFactory implements SchemeFactory { + public getTopologyConf_resultTupleScheme getScheme() { + return new getTopologyConf_resultTupleScheme(); + } + } + + private static class getTopologyConf_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyConf_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + if (struct.is_set_e()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_success()) { + oprot.writeString(struct.success); + } + if (struct.is_set_e()) { + struct.e.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyConf_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.success = iprot.readString(); + struct.set_success_isSet(true); + } + if (incoming.get(1)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + } + } + + } + + public static class getTopologyId_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyId_args"); + + private static final org.apache.thrift.protocol.TField TOPOLOGY_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("topologyName", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getTopologyId_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getTopologyId_argsTupleSchemeFactory()); + } + + private String topologyName; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TOPOLOGY_NAME((short)1, "topologyName"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TOPOLOGY_NAME + return TOPOLOGY_NAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TOPOLOGY_NAME, new org.apache.thrift.meta_data.FieldMetaData("topologyName", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyId_args.class, metaDataMap); + } + + public getTopologyId_args() { + } + + public getTopologyId_args( + String topologyName) + { + this(); + this.topologyName = topologyName; + } + + /** + * Performs a deep copy on other. + */ + public getTopologyId_args(getTopologyId_args other) { + if (other.is_set_topologyName()) { + this.topologyName = other.topologyName; + } + } + + public getTopologyId_args deepCopy() { + return new getTopologyId_args(this); + } + + @Override + public void clear() { + this.topologyName = null; + } + + public String get_topologyName() { + return this.topologyName; + } + + public void set_topologyName(String topologyName) { + this.topologyName = topologyName; + } + + public void unset_topologyName() { + this.topologyName = null; + } + + /** Returns true if field topologyName is set (has been assigned a value) and false otherwise */ + public boolean is_set_topologyName() { + return this.topologyName != null; + } + + public void set_topologyName_isSet(boolean value) { + if (!value) { + this.topologyName = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TOPOLOGY_NAME: + if (value == null) { + unset_topologyName(); + } else { + set_topologyName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TOPOLOGY_NAME: + return get_topologyName(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TOPOLOGY_NAME: + return is_set_topologyName(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getTopologyId_args) + return this.equals((getTopologyId_args)that); + return false; + } + + public boolean equals(getTopologyId_args that) { + if (that == null) + return false; + + boolean this_present_topologyName = true && this.is_set_topologyName(); + boolean that_present_topologyName = true && that.is_set_topologyName(); + if (this_present_topologyName || that_present_topologyName) { + if (!(this_present_topologyName && that_present_topologyName)) + return false; + if (!this.topologyName.equals(that.topologyName)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_topologyName = true && (is_set_topologyName()); + list.add(present_topologyName); + if (present_topologyName) + list.add(topologyName); + + return list.hashCode(); + } + + @Override + public int compareTo(getTopologyId_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_topologyName()).compareTo(other.is_set_topologyName()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_topologyName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topologyName, other.topologyName); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getTopologyId_args("); + boolean first = true; + + sb.append("topologyName:"); + if (this.topologyName == null) { + sb.append("null"); + } else { + sb.append(this.topologyName); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getTopologyId_argsStandardSchemeFactory implements SchemeFactory { + public getTopologyId_argsStandardScheme getScheme() { + return new getTopologyId_argsStandardScheme(); + } + } + + private static class getTopologyId_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyId_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TOPOLOGY_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.topologyName = iprot.readString(); + struct.set_topologyName_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyId_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.topologyName != null) { + oprot.writeFieldBegin(TOPOLOGY_NAME_FIELD_DESC); + oprot.writeString(struct.topologyName); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getTopologyId_argsTupleSchemeFactory implements SchemeFactory { + public getTopologyId_argsTupleScheme getScheme() { + return new getTopologyId_argsTupleScheme(); + } + } + + private static class getTopologyId_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyId_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_topologyName()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_topologyName()) { + oprot.writeString(struct.topologyName); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyId_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.topologyName = iprot.readString(); + struct.set_topologyName_isSet(true); + } + } + } + + } + + public static class getTopologyId_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyId_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0); + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getTopologyId_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getTopologyId_resultTupleSchemeFactory()); + } + + private String success; // required + private NotAliveException e; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"), + E((short)1, "e"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // E + return E; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyId_result.class, metaDataMap); + } + + public getTopologyId_result() { + } + + public getTopologyId_result( + String success, + NotAliveException e) + { + this(); + this.success = success; + this.e = e; + } + + /** + * Performs a deep copy on other. + */ + public getTopologyId_result(getTopologyId_result other) { + if (other.is_set_success()) { + this.success = other.success; + } + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + } + + public getTopologyId_result deepCopy() { + return new getTopologyId_result(this); + } + + @Override + public void clear() { + this.success = null; + this.e = null; + } + + public String get_success() { + return this.success; + } + + public void set_success(String success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((String)value); + } + break; + + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + case E: + return get_e(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + case E: + return is_set_e(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getTopologyId_result) + return this.equals((getTopologyId_result)that); + return false; + } + + public boolean equals(getTopologyId_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + return list.hashCode(); + } + + @Override + public int compareTo(getTopologyId_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getTopologyId_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + if (!first) sb.append(", "); + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getTopologyId_resultStandardSchemeFactory implements SchemeFactory { + public getTopologyId_resultStandardScheme getScheme() { + return new getTopologyId_resultStandardScheme(); + } + } + + private static class getTopologyId_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyId_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.success = iprot.readString(); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyId_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeString(struct.success); + oprot.writeFieldEnd(); + } + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getTopologyId_resultTupleSchemeFactory implements SchemeFactory { + public getTopologyId_resultTupleScheme getScheme() { + return new getTopologyId_resultTupleScheme(); + } + } + + private static class getTopologyId_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyId_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + if (struct.is_set_e()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_success()) { + oprot.writeString(struct.success); + } + if (struct.is_set_e()) { + struct.e.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyId_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.success = iprot.readString(); + struct.set_success_isSet(true); + } + if (incoming.get(1)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + } + } + + } + + public static class getClusterInfo_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getClusterInfo_args"); + + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getClusterInfo_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getClusterInfo_argsTupleSchemeFactory()); + } + + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { +; + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getClusterInfo_args.class, metaDataMap); + } + + public getClusterInfo_args() { + } + + /** + * Performs a deep copy on other. + */ + public getClusterInfo_args(getClusterInfo_args other) { + } + + public getClusterInfo_args deepCopy() { + return new getClusterInfo_args(this); + } + + @Override + public void clear() { + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getClusterInfo_args) + return this.equals((getClusterInfo_args)that); + return false; + } + + public boolean equals(getClusterInfo_args that) { + if (that == null) + return false; + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + return list.hashCode(); + } + + @Override + public int compareTo(getClusterInfo_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getClusterInfo_args("); + boolean first = true; + + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getClusterInfo_argsStandardSchemeFactory implements SchemeFactory { + public getClusterInfo_argsStandardScheme getScheme() { + return new getClusterInfo_argsStandardScheme(); + } + } + + private static class getClusterInfo_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getClusterInfo_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getClusterInfo_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getClusterInfo_argsTupleSchemeFactory implements SchemeFactory { + public getClusterInfo_argsTupleScheme getScheme() { + return new getClusterInfo_argsTupleScheme(); + } + } + + private static class getClusterInfo_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getClusterInfo_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getClusterInfo_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + } + } + + } + + public static class getClusterInfo_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getClusterInfo_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getClusterInfo_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getClusterInfo_resultTupleSchemeFactory()); + } + + private ClusterSummary success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ClusterSummary.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getClusterInfo_result.class, metaDataMap); + } + + public getClusterInfo_result() { + } + + public getClusterInfo_result( + ClusterSummary success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public getClusterInfo_result(getClusterInfo_result other) { + if (other.is_set_success()) { + this.success = new ClusterSummary(other.success); + } + } + + public getClusterInfo_result deepCopy() { + return new getClusterInfo_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public ClusterSummary get_success() { + return this.success; + } + + public void set_success(ClusterSummary success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((ClusterSummary)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getClusterInfo_result) + return this.equals((getClusterInfo_result)that); + return false; + } + + public boolean equals(getClusterInfo_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(getClusterInfo_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getClusterInfo_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getClusterInfo_resultStandardSchemeFactory implements SchemeFactory { + public getClusterInfo_resultStandardScheme getScheme() { + return new getClusterInfo_resultStandardScheme(); + } + } + + private static class getClusterInfo_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getClusterInfo_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new ClusterSummary(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getClusterInfo_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getClusterInfo_resultTupleSchemeFactory implements SchemeFactory { + public getClusterInfo_resultTupleScheme getScheme() { + return new getClusterInfo_resultTupleScheme(); + } + } + + private static class getClusterInfo_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getClusterInfo_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_success()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getClusterInfo_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new ClusterSummary(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } + } + } + + } + + public static class getSupervisorWorkers_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getSupervisorWorkers_args"); + + private static final org.apache.thrift.protocol.TField HOST_FIELD_DESC = new org.apache.thrift.protocol.TField("host", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getSupervisorWorkers_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getSupervisorWorkers_argsTupleSchemeFactory()); + } + + private String host; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + HOST((short)1, "host"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // HOST + return HOST; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.HOST, new org.apache.thrift.meta_data.FieldMetaData("host", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getSupervisorWorkers_args.class, metaDataMap); + } + + public getSupervisorWorkers_args() { + } + + public getSupervisorWorkers_args( + String host) + { + this(); + this.host = host; + } + + /** + * Performs a deep copy on other. + */ + public getSupervisorWorkers_args(getSupervisorWorkers_args other) { + if (other.is_set_host()) { + this.host = other.host; + } + } + + public getSupervisorWorkers_args deepCopy() { + return new getSupervisorWorkers_args(this); + } + + @Override + public void clear() { + this.host = null; + } + + public String get_host() { + return this.host; + } + + public void set_host(String host) { + this.host = host; + } + + public void unset_host() { + this.host = null; + } + + /** Returns true if field host is set (has been assigned a value) and false otherwise */ + public boolean is_set_host() { + return this.host != null; + } + + public void set_host_isSet(boolean value) { + if (!value) { + this.host = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case HOST: + if (value == null) { + unset_host(); + } else { + set_host((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case HOST: + return get_host(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case HOST: + return is_set_host(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getSupervisorWorkers_args) + return this.equals((getSupervisorWorkers_args)that); + return false; + } + + public boolean equals(getSupervisorWorkers_args that) { + if (that == null) + return false; + + boolean this_present_host = true && this.is_set_host(); + boolean that_present_host = true && that.is_set_host(); + if (this_present_host || that_present_host) { + if (!(this_present_host && that_present_host)) + return false; + if (!this.host.equals(that.host)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_host = true && (is_set_host()); + list.add(present_host); + if (present_host) + list.add(host); + + return list.hashCode(); + } + + @Override + public int compareTo(getSupervisorWorkers_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_host()).compareTo(other.is_set_host()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_host()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.host, other.host); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getSupervisorWorkers_args("); + boolean first = true; + + sb.append("host:"); + if (this.host == null) { + sb.append("null"); + } else { + sb.append(this.host); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getSupervisorWorkers_argsStandardSchemeFactory implements SchemeFactory { + public getSupervisorWorkers_argsStandardScheme getScheme() { + return new getSupervisorWorkers_argsStandardScheme(); + } + } + + private static class getSupervisorWorkers_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getSupervisorWorkers_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // HOST + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.host = iprot.readString(); + struct.set_host_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getSupervisorWorkers_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.host != null) { + oprot.writeFieldBegin(HOST_FIELD_DESC); + oprot.writeString(struct.host); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getSupervisorWorkers_argsTupleSchemeFactory implements SchemeFactory { + public getSupervisorWorkers_argsTupleScheme getScheme() { + return new getSupervisorWorkers_argsTupleScheme(); + } + } + + private static class getSupervisorWorkers_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getSupervisorWorkers_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_host()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_host()) { + oprot.writeString(struct.host); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getSupervisorWorkers_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.host = iprot.readString(); + struct.set_host_isSet(true); + } + } + } + + } + + public static class getSupervisorWorkers_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getSupervisorWorkers_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getSupervisorWorkers_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getSupervisorWorkers_resultTupleSchemeFactory()); + } + + private SupervisorWorkers success; // required + private NotAliveException e; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"), + E((short)1, "e"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // E + return E; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SupervisorWorkers.class))); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getSupervisorWorkers_result.class, metaDataMap); + } + + public getSupervisorWorkers_result() { + } + + public getSupervisorWorkers_result( + SupervisorWorkers success, + NotAliveException e) + { + this(); + this.success = success; + this.e = e; + } + + /** + * Performs a deep copy on other. + */ + public getSupervisorWorkers_result(getSupervisorWorkers_result other) { + if (other.is_set_success()) { + this.success = new SupervisorWorkers(other.success); + } + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + } + + public getSupervisorWorkers_result deepCopy() { + return new getSupervisorWorkers_result(this); + } + + @Override + public void clear() { + this.success = null; + this.e = null; + } + + public SupervisorWorkers get_success() { + return this.success; + } + + public void set_success(SupervisorWorkers success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((SupervisorWorkers)value); + } + break; + + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + case E: + return get_e(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + case E: + return is_set_e(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getSupervisorWorkers_result) + return this.equals((getSupervisorWorkers_result)that); + return false; + } + + public boolean equals(getSupervisorWorkers_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + return list.hashCode(); + } + + @Override + public int compareTo(getSupervisorWorkers_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getSupervisorWorkers_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + if (!first) sb.append(", "); + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getSupervisorWorkers_resultStandardSchemeFactory implements SchemeFactory { + public getSupervisorWorkers_resultStandardScheme getScheme() { + return new getSupervisorWorkers_resultStandardScheme(); + } + } + + private static class getSupervisorWorkers_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getSupervisorWorkers_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new SupervisorWorkers(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getSupervisorWorkers_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getSupervisorWorkers_resultTupleSchemeFactory implements SchemeFactory { + public getSupervisorWorkers_resultTupleScheme getScheme() { + return new getSupervisorWorkers_resultTupleScheme(); + } + } + + private static class getSupervisorWorkers_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getSupervisorWorkers_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + if (struct.is_set_e()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_success()) { + struct.success.write(oprot); + } + if (struct.is_set_e()) { + struct.e.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getSupervisorWorkers_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.success = new SupervisorWorkers(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } + if (incoming.get(1)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + } + } + + } + + public static class getTopologyInfo_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyInfo_args"); + + private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getTopologyInfo_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getTopologyInfo_argsTupleSchemeFactory()); + } + + private String id; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + ID((short)1, "id"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // ID + return ID; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyInfo_args.class, metaDataMap); + } + + public getTopologyInfo_args() { + } + + public getTopologyInfo_args( + String id) + { + this(); + this.id = id; + } + + /** + * Performs a deep copy on other. + */ + public getTopologyInfo_args(getTopologyInfo_args other) { + if (other.is_set_id()) { + this.id = other.id; + } + } + + public getTopologyInfo_args deepCopy() { + return new getTopologyInfo_args(this); + } + + @Override + public void clear() { + this.id = null; + } + + public String get_id() { + return this.id; + } + + public void set_id(String id) { + this.id = id; + } + + public void unset_id() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean is_set_id() { + return this.id != null; + } + + public void set_id_isSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unset_id(); + } else { + set_id((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return get_id(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case ID: + return is_set_id(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getTopologyInfo_args) + return this.equals((getTopologyInfo_args)that); + return false; + } + + public boolean equals(getTopologyInfo_args that) { + if (that == null) + return false; + + boolean this_present_id = true && this.is_set_id(); + boolean that_present_id = true && that.is_set_id(); + if (this_present_id || that_present_id) { + if (!(this_present_id && that_present_id)) + return false; + if (!this.id.equals(that.id)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_id = true && (is_set_id()); + list.add(present_id); + if (present_id) + list.add(id); + + return list.hashCode(); + } + + @Override + public int compareTo(getTopologyInfo_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getTopologyInfo_args("); + boolean first = true; + + sb.append("id:"); + if (this.id == null) { + sb.append("null"); + } else { + sb.append(this.id); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getTopologyInfo_argsStandardSchemeFactory implements SchemeFactory { + public getTopologyInfo_argsStandardScheme getScheme() { + return new getTopologyInfo_argsStandardScheme(); + } + } + + private static class getTopologyInfo_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyInfo_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyInfo_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.id != null) { + oprot.writeFieldBegin(ID_FIELD_DESC); + oprot.writeString(struct.id); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getTopologyInfo_argsTupleSchemeFactory implements SchemeFactory { + public getTopologyInfo_argsTupleScheme getScheme() { + return new getTopologyInfo_argsTupleScheme(); + } + } + + private static class getTopologyInfo_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyInfo_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_id()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_id()) { + oprot.writeString(struct.id); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyInfo_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } + } + } + + } + + public static class getTopologyInfo_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyInfo_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getTopologyInfo_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getTopologyInfo_resultTupleSchemeFactory()); + } + + private TopologyInfo success; // required + private NotAliveException e; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"), + E((short)1, "e"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // E + return E; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologyInfo.class))); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyInfo_result.class, metaDataMap); + } + + public getTopologyInfo_result() { + } + + public getTopologyInfo_result( + TopologyInfo success, + NotAliveException e) + { + this(); + this.success = success; + this.e = e; + } + + /** + * Performs a deep copy on other. + */ + public getTopologyInfo_result(getTopologyInfo_result other) { + if (other.is_set_success()) { + this.success = new TopologyInfo(other.success); + } + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + } + + public getTopologyInfo_result deepCopy() { + return new getTopologyInfo_result(this); + } + + @Override + public void clear() { + this.success = null; + this.e = null; + } + + public TopologyInfo get_success() { + return this.success; + } + + public void set_success(TopologyInfo success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((TopologyInfo)value); + } + break; + + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + case E: + return get_e(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + case E: + return is_set_e(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getTopologyInfo_result) + return this.equals((getTopologyInfo_result)that); + return false; + } + + public boolean equals(getTopologyInfo_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + return list.hashCode(); + } + + @Override + public int compareTo(getTopologyInfo_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getTopologyInfo_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + if (!first) sb.append(", "); + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getTopologyInfo_resultStandardSchemeFactory implements SchemeFactory { + public getTopologyInfo_resultStandardScheme getScheme() { + return new getTopologyInfo_resultStandardScheme(); + } + } + + private static class getTopologyInfo_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyInfo_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TopologyInfo(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyInfo_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getTopologyInfo_resultTupleSchemeFactory implements SchemeFactory { + public getTopologyInfo_resultTupleScheme getScheme() { + return new getTopologyInfo_resultTupleScheme(); + } + } + + private static class getTopologyInfo_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyInfo_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + if (struct.is_set_e()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_success()) { + struct.success.write(oprot); + } + if (struct.is_set_e()) { + struct.e.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyInfo_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.success = new TopologyInfo(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } + if (incoming.get(1)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + } + } + + } + + public static class getTopologyInfoByName_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyInfoByName_args"); + + private static final org.apache.thrift.protocol.TField TOPOLOGY_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("topologyName", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getTopologyInfoByName_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getTopologyInfoByName_argsTupleSchemeFactory()); + } + + private String topologyName; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TOPOLOGY_NAME((short)1, "topologyName"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TOPOLOGY_NAME + return TOPOLOGY_NAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TOPOLOGY_NAME, new org.apache.thrift.meta_data.FieldMetaData("topologyName", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyInfoByName_args.class, metaDataMap); + } + + public getTopologyInfoByName_args() { + } + + public getTopologyInfoByName_args( + String topologyName) + { + this(); + this.topologyName = topologyName; + } + + /** + * Performs a deep copy on other. + */ + public getTopologyInfoByName_args(getTopologyInfoByName_args other) { + if (other.is_set_topologyName()) { + this.topologyName = other.topologyName; + } + } + + public getTopologyInfoByName_args deepCopy() { + return new getTopologyInfoByName_args(this); + } + + @Override + public void clear() { + this.topologyName = null; + } + + public String get_topologyName() { + return this.topologyName; + } + + public void set_topologyName(String topologyName) { + this.topologyName = topologyName; + } + + public void unset_topologyName() { + this.topologyName = null; + } + + /** Returns true if field topologyName is set (has been assigned a value) and false otherwise */ + public boolean is_set_topologyName() { + return this.topologyName != null; + } + + public void set_topologyName_isSet(boolean value) { + if (!value) { + this.topologyName = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TOPOLOGY_NAME: + if (value == null) { + unset_topologyName(); + } else { + set_topologyName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TOPOLOGY_NAME: + return get_topologyName(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TOPOLOGY_NAME: + return is_set_topologyName(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getTopologyInfoByName_args) + return this.equals((getTopologyInfoByName_args)that); + return false; + } + + public boolean equals(getTopologyInfoByName_args that) { + if (that == null) + return false; + + boolean this_present_topologyName = true && this.is_set_topologyName(); + boolean that_present_topologyName = true && that.is_set_topologyName(); + if (this_present_topologyName || that_present_topologyName) { + if (!(this_present_topologyName && that_present_topologyName)) + return false; + if (!this.topologyName.equals(that.topologyName)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_topologyName = true && (is_set_topologyName()); + list.add(present_topologyName); + if (present_topologyName) + list.add(topologyName); + + return list.hashCode(); + } + + @Override + public int compareTo(getTopologyInfoByName_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_topologyName()).compareTo(other.is_set_topologyName()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_topologyName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topologyName, other.topologyName); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getTopologyInfoByName_args("); + boolean first = true; + + sb.append("topologyName:"); + if (this.topologyName == null) { + sb.append("null"); + } else { + sb.append(this.topologyName); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getTopologyInfoByName_argsStandardSchemeFactory implements SchemeFactory { + public getTopologyInfoByName_argsStandardScheme getScheme() { + return new getTopologyInfoByName_argsStandardScheme(); + } + } + + private static class getTopologyInfoByName_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyInfoByName_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TOPOLOGY_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.topologyName = iprot.readString(); + struct.set_topologyName_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyInfoByName_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.topologyName != null) { + oprot.writeFieldBegin(TOPOLOGY_NAME_FIELD_DESC); + oprot.writeString(struct.topologyName); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getTopologyInfoByName_argsTupleSchemeFactory implements SchemeFactory { + public getTopologyInfoByName_argsTupleScheme getScheme() { + return new getTopologyInfoByName_argsTupleScheme(); + } + } + + private static class getTopologyInfoByName_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyInfoByName_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_topologyName()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_topologyName()) { + oprot.writeString(struct.topologyName); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyInfoByName_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.topologyName = iprot.readString(); + struct.set_topologyName_isSet(true); + } + } + } + + } + + public static class getTopologyInfoByName_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyInfoByName_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getTopologyInfoByName_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getTopologyInfoByName_resultTupleSchemeFactory()); + } + + private TopologyInfo success; // required + private NotAliveException e; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"), + E((short)1, "e"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // E + return E; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologyInfo.class))); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyInfoByName_result.class, metaDataMap); + } + + public getTopologyInfoByName_result() { + } + + public getTopologyInfoByName_result( + TopologyInfo success, + NotAliveException e) + { + this(); + this.success = success; + this.e = e; + } + + /** + * Performs a deep copy on other. + */ + public getTopologyInfoByName_result(getTopologyInfoByName_result other) { + if (other.is_set_success()) { + this.success = new TopologyInfo(other.success); + } + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + } + + public getTopologyInfoByName_result deepCopy() { + return new getTopologyInfoByName_result(this); + } + + @Override + public void clear() { + this.success = null; + this.e = null; + } + + public TopologyInfo get_success() { + return this.success; + } + + public void set_success(TopologyInfo success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((TopologyInfo)value); + } + break; + + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + case E: + return get_e(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + case E: + return is_set_e(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getTopologyInfoByName_result) + return this.equals((getTopologyInfoByName_result)that); + return false; + } + + public boolean equals(getTopologyInfoByName_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + return list.hashCode(); + } + + @Override + public int compareTo(getTopologyInfoByName_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getTopologyInfoByName_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + if (!first) sb.append(", "); + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getTopologyInfoByName_resultStandardSchemeFactory implements SchemeFactory { + public getTopologyInfoByName_resultStandardScheme getScheme() { + return new getTopologyInfoByName_resultStandardScheme(); + } + } + + private static class getTopologyInfoByName_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyInfoByName_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TopologyInfo(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyInfoByName_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getTopologyInfoByName_resultTupleSchemeFactory implements SchemeFactory { + public getTopologyInfoByName_resultTupleScheme getScheme() { + return new getTopologyInfoByName_resultTupleScheme(); + } + } + + private static class getTopologyInfoByName_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyInfoByName_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + if (struct.is_set_e()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_success()) { + struct.success.write(oprot); + } + if (struct.is_set_e()) { + struct.e.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyInfoByName_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.success = new TopologyInfo(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } + if (incoming.get(1)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + } + } + + } + + public static class getTopology_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopology_args"); + + private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getTopology_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getTopology_argsTupleSchemeFactory()); + } + + private String id; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + ID((short)1, "id"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // ID + return ID; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopology_args.class, metaDataMap); + } + + public getTopology_args() { + } + + public getTopology_args( + String id) + { + this(); + this.id = id; + } + + /** + * Performs a deep copy on other. + */ + public getTopology_args(getTopology_args other) { + if (other.is_set_id()) { + this.id = other.id; + } + } + + public getTopology_args deepCopy() { + return new getTopology_args(this); + } + + @Override + public void clear() { + this.id = null; + } + + public String get_id() { + return this.id; + } + + public void set_id(String id) { + this.id = id; + } + + public void unset_id() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean is_set_id() { + return this.id != null; + } + + public void set_id_isSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unset_id(); + } else { + set_id((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return get_id(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case ID: + return is_set_id(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getTopology_args) + return this.equals((getTopology_args)that); + return false; + } + + public boolean equals(getTopology_args that) { + if (that == null) + return false; + + boolean this_present_id = true && this.is_set_id(); + boolean that_present_id = true && that.is_set_id(); + if (this_present_id || that_present_id) { + if (!(this_present_id && that_present_id)) + return false; + if (!this.id.equals(that.id)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_id = true && (is_set_id()); + list.add(present_id); + if (present_id) + list.add(id); + + return list.hashCode(); + } + + @Override + public int compareTo(getTopology_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getTopology_args("); + boolean first = true; + + sb.append("id:"); + if (this.id == null) { + sb.append("null"); + } else { + sb.append(this.id); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getTopology_argsStandardSchemeFactory implements SchemeFactory { + public getTopology_argsStandardScheme getScheme() { + return new getTopology_argsStandardScheme(); + } + } + + private static class getTopology_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getTopology_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getTopology_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.id != null) { + oprot.writeFieldBegin(ID_FIELD_DESC); + oprot.writeString(struct.id); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getTopology_argsTupleSchemeFactory implements SchemeFactory { + public getTopology_argsTupleScheme getScheme() { + return new getTopology_argsTupleScheme(); + } + } + + private static class getTopology_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getTopology_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_id()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_id()) { + oprot.writeString(struct.id); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getTopology_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } + } + } + + } + + public static class getTopology_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopology_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getTopology_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getTopology_resultTupleSchemeFactory()); + } + + private StormTopology success; // required + private NotAliveException e; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"), + E((short)1, "e"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // E + return E; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, StormTopology.class))); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopology_result.class, metaDataMap); + } + + public getTopology_result() { + } + + public getTopology_result( + StormTopology success, + NotAliveException e) + { + this(); + this.success = success; + this.e = e; + } + + /** + * Performs a deep copy on other. + */ + public getTopology_result(getTopology_result other) { + if (other.is_set_success()) { + this.success = new StormTopology(other.success); + } + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + } + + public getTopology_result deepCopy() { + return new getTopology_result(this); + } + + @Override + public void clear() { + this.success = null; + this.e = null; + } + + public StormTopology get_success() { + return this.success; + } + + public void set_success(StormTopology success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((StormTopology)value); + } + break; + + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + case E: + return get_e(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + case E: + return is_set_e(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getTopology_result) + return this.equals((getTopology_result)that); + return false; + } + + public boolean equals(getTopology_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + return list.hashCode(); + } + + @Override + public int compareTo(getTopology_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getTopology_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + if (!first) sb.append(", "); + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getTopology_resultStandardSchemeFactory implements SchemeFactory { + public getTopology_resultStandardScheme getScheme() { + return new getTopology_resultStandardScheme(); + } + } + + private static class getTopology_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getTopology_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new StormTopology(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getTopology_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getTopology_resultTupleSchemeFactory implements SchemeFactory { + public getTopology_resultTupleScheme getScheme() { + return new getTopology_resultTupleScheme(); + } + } + + private static class getTopology_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getTopology_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + if (struct.is_set_e()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_success()) { + struct.success.write(oprot); + } + if (struct.is_set_e()) { + struct.e.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getTopology_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.success = new StormTopology(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } + if (incoming.get(1)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + } + } + + } + + public static class getUserTopology_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getUserTopology_args"); + + private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getUserTopology_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getUserTopology_argsTupleSchemeFactory()); + } + + private String id; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + ID((short)1, "id"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // ID + return ID; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getUserTopology_args.class, metaDataMap); + } + + public getUserTopology_args() { + } + + public getUserTopology_args( + String id) + { + this(); + this.id = id; + } + + /** + * Performs a deep copy on other. + */ + public getUserTopology_args(getUserTopology_args other) { + if (other.is_set_id()) { + this.id = other.id; + } + } + + public getUserTopology_args deepCopy() { + return new getUserTopology_args(this); + } + + @Override + public void clear() { + this.id = null; + } + + public String get_id() { + return this.id; + } + + public void set_id(String id) { + this.id = id; + } + + public void unset_id() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean is_set_id() { + return this.id != null; + } + + public void set_id_isSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unset_id(); + } else { + set_id((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return get_id(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case ID: + return is_set_id(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getUserTopology_args) + return this.equals((getUserTopology_args)that); + return false; + } + + public boolean equals(getUserTopology_args that) { + if (that == null) + return false; + + boolean this_present_id = true && this.is_set_id(); + boolean that_present_id = true && that.is_set_id(); + if (this_present_id || that_present_id) { + if (!(this_present_id && that_present_id)) + return false; + if (!this.id.equals(that.id)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_id = true && (is_set_id()); + list.add(present_id); + if (present_id) + list.add(id); + + return list.hashCode(); + } + + @Override + public int compareTo(getUserTopology_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getUserTopology_args("); + boolean first = true; + + sb.append("id:"); + if (this.id == null) { + sb.append("null"); + } else { + sb.append(this.id); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getUserTopology_argsStandardSchemeFactory implements SchemeFactory { + public getUserTopology_argsStandardScheme getScheme() { + return new getUserTopology_argsStandardScheme(); + } + } + + private static class getUserTopology_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getUserTopology_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getUserTopology_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.id != null) { + oprot.writeFieldBegin(ID_FIELD_DESC); + oprot.writeString(struct.id); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getUserTopology_argsTupleSchemeFactory implements SchemeFactory { + public getUserTopology_argsTupleScheme getScheme() { + return new getUserTopology_argsTupleScheme(); + } + } + + private static class getUserTopology_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getUserTopology_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_id()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_id()) { + oprot.writeString(struct.id); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getUserTopology_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } + } + } + + } + + public static class getUserTopology_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getUserTopology_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getUserTopology_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getUserTopology_resultTupleSchemeFactory()); + } + + private StormTopology success; // required + private NotAliveException e; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"), + E((short)1, "e"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + case 1: // E + return E; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, StormTopology.class))); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getUserTopology_result.class, metaDataMap); + } + + public getUserTopology_result() { + } + + public getUserTopology_result( + StormTopology success, + NotAliveException e) + { + this(); + this.success = success; + this.e = e; + } + + /** + * Performs a deep copy on other. + */ + public getUserTopology_result(getUserTopology_result other) { + if (other.is_set_success()) { + this.success = new StormTopology(other.success); + } + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + } + + public getUserTopology_result deepCopy() { + return new getUserTopology_result(this); + } + + @Override + public void clear() { + this.success = null; + this.e = null; + } + + public StormTopology get_success() { + return this.success; + } + + public void set_success(StormTopology success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((StormTopology)value); + } + break; + + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + case E: + return get_e(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + case E: + return is_set_e(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getUserTopology_result) + return this.equals((getUserTopology_result)that); + return false; + } + + public boolean equals(getUserTopology_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + return list.hashCode(); + } + + @Override + public int compareTo(getUserTopology_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getUserTopology_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + if (!first) sb.append(", "); + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getUserTopology_resultStandardSchemeFactory implements SchemeFactory { + public getUserTopology_resultStandardScheme getScheme() { + return new getUserTopology_resultStandardScheme(); + } + } + + private static class getUserTopology_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getUserTopology_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new StormTopology(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getUserTopology_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getUserTopology_resultTupleSchemeFactory implements SchemeFactory { + public getUserTopology_resultTupleScheme getScheme() { + return new getUserTopology_resultTupleScheme(); + } + } + + private static class getUserTopology_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getUserTopology_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + if (struct.is_set_e()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_success()) { + struct.success.write(oprot); + } + if (struct.is_set_e()) { + struct.e.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getUserTopology_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.success = new StormTopology(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } + if (incoming.get(1)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + } + } + + } + + public static class workerUploadMetric_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("workerUploadMetric_args"); + + private static final org.apache.thrift.protocol.TField UPLOAD_METRICS_FIELD_DESC = new org.apache.thrift.protocol.TField("uploadMetrics", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new workerUploadMetric_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new workerUploadMetric_argsTupleSchemeFactory()); + } + + private WorkerUploadMetrics uploadMetrics; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + UPLOAD_METRICS((short)1, "uploadMetrics"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // UPLOAD_METRICS + return UPLOAD_METRICS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.UPLOAD_METRICS, new org.apache.thrift.meta_data.FieldMetaData("uploadMetrics", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, WorkerUploadMetrics.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(workerUploadMetric_args.class, metaDataMap); + } + + public workerUploadMetric_args() { + } + + public workerUploadMetric_args( + WorkerUploadMetrics uploadMetrics) + { + this(); + this.uploadMetrics = uploadMetrics; + } + + /** + * Performs a deep copy on other. + */ + public workerUploadMetric_args(workerUploadMetric_args other) { + if (other.is_set_uploadMetrics()) { + this.uploadMetrics = new WorkerUploadMetrics(other.uploadMetrics); + } + } + + public workerUploadMetric_args deepCopy() { + return new workerUploadMetric_args(this); + } + + @Override + public void clear() { + this.uploadMetrics = null; + } + + public WorkerUploadMetrics get_uploadMetrics() { + return this.uploadMetrics; + } + + public void set_uploadMetrics(WorkerUploadMetrics uploadMetrics) { + this.uploadMetrics = uploadMetrics; + } + + public void unset_uploadMetrics() { + this.uploadMetrics = null; + } + + /** Returns true if field uploadMetrics is set (has been assigned a value) and false otherwise */ + public boolean is_set_uploadMetrics() { + return this.uploadMetrics != null; + } + + public void set_uploadMetrics_isSet(boolean value) { + if (!value) { + this.uploadMetrics = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case UPLOAD_METRICS: + if (value == null) { + unset_uploadMetrics(); + } else { + set_uploadMetrics((WorkerUploadMetrics)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case UPLOAD_METRICS: + return get_uploadMetrics(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case UPLOAD_METRICS: + return is_set_uploadMetrics(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof workerUploadMetric_args) + return this.equals((workerUploadMetric_args)that); + return false; + } + + public boolean equals(workerUploadMetric_args that) { + if (that == null) + return false; + + boolean this_present_uploadMetrics = true && this.is_set_uploadMetrics(); + boolean that_present_uploadMetrics = true && that.is_set_uploadMetrics(); + if (this_present_uploadMetrics || that_present_uploadMetrics) { + if (!(this_present_uploadMetrics && that_present_uploadMetrics)) + return false; + if (!this.uploadMetrics.equals(that.uploadMetrics)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_uploadMetrics = true && (is_set_uploadMetrics()); + list.add(present_uploadMetrics); + if (present_uploadMetrics) + list.add(uploadMetrics); + + return list.hashCode(); + } + + @Override + public int compareTo(workerUploadMetric_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_uploadMetrics()).compareTo(other.is_set_uploadMetrics()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_uploadMetrics()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uploadMetrics, other.uploadMetrics); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("workerUploadMetric_args("); + boolean first = true; + + sb.append("uploadMetrics:"); + if (this.uploadMetrics == null) { + sb.append("null"); + } else { + sb.append(this.uploadMetrics); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (uploadMetrics != null) { + uploadMetrics.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class workerUploadMetric_argsStandardSchemeFactory implements SchemeFactory { + public workerUploadMetric_argsStandardScheme getScheme() { + return new workerUploadMetric_argsStandardScheme(); + } + } + + private static class workerUploadMetric_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, workerUploadMetric_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // UPLOAD_METRICS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.uploadMetrics = new WorkerUploadMetrics(); + struct.uploadMetrics.read(iprot); + struct.set_uploadMetrics_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, workerUploadMetric_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.uploadMetrics != null) { + oprot.writeFieldBegin(UPLOAD_METRICS_FIELD_DESC); + struct.uploadMetrics.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class workerUploadMetric_argsTupleSchemeFactory implements SchemeFactory { + public workerUploadMetric_argsTupleScheme getScheme() { + return new workerUploadMetric_argsTupleScheme(); + } + } + + private static class workerUploadMetric_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, workerUploadMetric_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_uploadMetrics()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_uploadMetrics()) { + struct.uploadMetrics.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, workerUploadMetric_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.uploadMetrics = new WorkerUploadMetrics(); + struct.uploadMetrics.read(iprot); + struct.set_uploadMetrics_isSet(true); + } + } + } + + } + + public static class workerUploadMetric_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("workerUploadMetric_result"); + + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new workerUploadMetric_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new workerUploadMetric_resultTupleSchemeFactory()); + } + + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { +; + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(workerUploadMetric_result.class, metaDataMap); + } + + public workerUploadMetric_result() { + } + + /** + * Performs a deep copy on other. + */ + public workerUploadMetric_result(workerUploadMetric_result other) { + } + + public workerUploadMetric_result deepCopy() { + return new workerUploadMetric_result(this); + } + + @Override + public void clear() { + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof workerUploadMetric_result) + return this.equals((workerUploadMetric_result)that); + return false; + } + + public boolean equals(workerUploadMetric_result that) { + if (that == null) + return false; + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + return list.hashCode(); + } + + @Override + public int compareTo(workerUploadMetric_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("workerUploadMetric_result("); + boolean first = true; + + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class workerUploadMetric_resultStandardSchemeFactory implements SchemeFactory { + public workerUploadMetric_resultStandardScheme getScheme() { + return new workerUploadMetric_resultStandardScheme(); + } + } + + private static class workerUploadMetric_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, workerUploadMetric_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, workerUploadMetric_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class workerUploadMetric_resultTupleSchemeFactory implements SchemeFactory { + public workerUploadMetric_resultTupleScheme getScheme() { + return new workerUploadMetric_resultTupleScheme(); + } + } + + private static class workerUploadMetric_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, workerUploadMetric_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, workerUploadMetric_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + } + } + + } + + public static class getTopologyMetric_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyMetric_args"); + + private static final org.apache.thrift.protocol.TField TOPOLOGY_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("topologyName", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getTopologyMetric_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getTopologyMetric_argsTupleSchemeFactory()); + } + + private String topologyName; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TOPOLOGY_NAME((short)1, "topologyName"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TOPOLOGY_NAME + return TOPOLOGY_NAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TOPOLOGY_NAME, new org.apache.thrift.meta_data.FieldMetaData("topologyName", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyMetric_args.class, metaDataMap); + } + + public getTopologyMetric_args() { + } + + public getTopologyMetric_args( + String topologyName) + { + this(); + this.topologyName = topologyName; + } + + /** + * Performs a deep copy on other. + */ + public getTopologyMetric_args(getTopologyMetric_args other) { + if (other.is_set_topologyName()) { + this.topologyName = other.topologyName; + } + } + + public getTopologyMetric_args deepCopy() { + return new getTopologyMetric_args(this); + } + + @Override + public void clear() { + this.topologyName = null; + } + + public String get_topologyName() { + return this.topologyName; + } + + public void set_topologyName(String topologyName) { + this.topologyName = topologyName; + } + + public void unset_topologyName() { + this.topologyName = null; + } + + /** Returns true if field topologyName is set (has been assigned a value) and false otherwise */ + public boolean is_set_topologyName() { + return this.topologyName != null; + } + + public void set_topologyName_isSet(boolean value) { + if (!value) { + this.topologyName = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TOPOLOGY_NAME: + if (value == null) { + unset_topologyName(); + } else { + set_topologyName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TOPOLOGY_NAME: + return get_topologyName(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TOPOLOGY_NAME: + return is_set_topologyName(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getTopologyMetric_args) + return this.equals((getTopologyMetric_args)that); + return false; + } + + public boolean equals(getTopologyMetric_args that) { + if (that == null) + return false; + + boolean this_present_topologyName = true && this.is_set_topologyName(); + boolean that_present_topologyName = true && that.is_set_topologyName(); + if (this_present_topologyName || that_present_topologyName) { + if (!(this_present_topologyName && that_present_topologyName)) + return false; + if (!this.topologyName.equals(that.topologyName)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_topologyName = true && (is_set_topologyName()); + list.add(present_topologyName); + if (present_topologyName) + list.add(topologyName); + + return list.hashCode(); + } + + @Override + public int compareTo(getTopologyMetric_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_topologyName()).compareTo(other.is_set_topologyName()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_topologyName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topologyName, other.topologyName); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getTopologyMetric_args("); + boolean first = true; + + sb.append("topologyName:"); + if (this.topologyName == null) { + sb.append("null"); + } else { + sb.append(this.topologyName); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getTopologyMetric_argsStandardSchemeFactory implements SchemeFactory { + public getTopologyMetric_argsStandardScheme getScheme() { + return new getTopologyMetric_argsStandardScheme(); + } + } + + private static class getTopologyMetric_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyMetric_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TOPOLOGY_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.topologyName = iprot.readString(); + struct.set_topologyName_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyMetric_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.topologyName != null) { + oprot.writeFieldBegin(TOPOLOGY_NAME_FIELD_DESC); + oprot.writeString(struct.topologyName); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getTopologyMetric_argsTupleSchemeFactory implements SchemeFactory { + public getTopologyMetric_argsTupleScheme getScheme() { + return new getTopologyMetric_argsTupleScheme(); + } + } + + private static class getTopologyMetric_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyMetric_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_topologyName()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_topologyName()) { + oprot.writeString(struct.topologyName); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyMetric_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.topologyName = iprot.readString(); + struct.set_topologyName_isSet(true); + } + } + } + + } + + public static class getTopologyMetric_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getTopologyMetric_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getTopologyMetric_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getTopologyMetric_resultTupleSchemeFactory()); + } + + private TopologyMetric success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologyMetric.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getTopologyMetric_result.class, metaDataMap); + } + + public getTopologyMetric_result() { + } + + public getTopologyMetric_result( + TopologyMetric success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public getTopologyMetric_result(getTopologyMetric_result other) { + if (other.is_set_success()) { + this.success = new TopologyMetric(other.success); + } + } + + public getTopologyMetric_result deepCopy() { + return new getTopologyMetric_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TopologyMetric get_success() { + return this.success; + } + + public void set_success(TopologyMetric success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((TopologyMetric)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getTopologyMetric_result) + return this.equals((getTopologyMetric_result)that); + return false; + } + + public boolean equals(getTopologyMetric_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(getTopologyMetric_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getTopologyMetric_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getTopologyMetric_resultStandardSchemeFactory implements SchemeFactory { + public getTopologyMetric_resultStandardScheme getScheme() { + return new getTopologyMetric_resultStandardScheme(); + } + } + + private static class getTopologyMetric_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getTopologyMetric_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TopologyMetric(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getTopologyMetric_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getTopologyMetric_resultTupleSchemeFactory implements SchemeFactory { + public getTopologyMetric_resultTupleScheme getScheme() { + return new getTopologyMetric_resultTupleScheme(); + } + } + + private static class getTopologyMetric_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getTopologyMetric_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_success()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getTopologyMetric_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TopologyMetric(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } + } + } + + } + + public static class getNettyMetric_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getNettyMetric_args"); + + private static final org.apache.thrift.protocol.TField TOPOLOGY_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("topologyName", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField POS_FIELD_DESC = new org.apache.thrift.protocol.TField("pos", org.apache.thrift.protocol.TType.I32, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getNettyMetric_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getNettyMetric_argsTupleSchemeFactory()); + } + + private String topologyName; // required + private int pos; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TOPOLOGY_NAME((short)1, "topologyName"), + POS((short)2, "pos"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TOPOLOGY_NAME + return TOPOLOGY_NAME; + case 2: // POS + return POS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __POS_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TOPOLOGY_NAME, new org.apache.thrift.meta_data.FieldMetaData("topologyName", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.POS, new org.apache.thrift.meta_data.FieldMetaData("pos", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getNettyMetric_args.class, metaDataMap); + } + + public getNettyMetric_args() { + } + + public getNettyMetric_args( + String topologyName, + int pos) + { + this(); + this.topologyName = topologyName; + this.pos = pos; + set_pos_isSet(true); + } + + /** + * Performs a deep copy on other. + */ + public getNettyMetric_args(getNettyMetric_args other) { + __isset_bitfield = other.__isset_bitfield; + if (other.is_set_topologyName()) { + this.topologyName = other.topologyName; + } + this.pos = other.pos; + } + + public getNettyMetric_args deepCopy() { + return new getNettyMetric_args(this); + } + + @Override + public void clear() { + this.topologyName = null; + set_pos_isSet(false); + this.pos = 0; + } + + public String get_topologyName() { + return this.topologyName; + } + + public void set_topologyName(String topologyName) { + this.topologyName = topologyName; + } + + public void unset_topologyName() { + this.topologyName = null; + } + + /** Returns true if field topologyName is set (has been assigned a value) and false otherwise */ + public boolean is_set_topologyName() { + return this.topologyName != null; + } + + public void set_topologyName_isSet(boolean value) { + if (!value) { + this.topologyName = null; + } + } + + public int get_pos() { + return this.pos; + } + + public void set_pos(int pos) { + this.pos = pos; + set_pos_isSet(true); + } + + public void unset_pos() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __POS_ISSET_ID); + } + + /** Returns true if field pos is set (has been assigned a value) and false otherwise */ + public boolean is_set_pos() { + return EncodingUtils.testBit(__isset_bitfield, __POS_ISSET_ID); + } + + public void set_pos_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __POS_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TOPOLOGY_NAME: + if (value == null) { + unset_topologyName(); + } else { + set_topologyName((String)value); + } + break; + + case POS: + if (value == null) { + unset_pos(); + } else { + set_pos((Integer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TOPOLOGY_NAME: + return get_topologyName(); + + case POS: + return Integer.valueOf(get_pos()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TOPOLOGY_NAME: + return is_set_topologyName(); + case POS: + return is_set_pos(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getNettyMetric_args) + return this.equals((getNettyMetric_args)that); + return false; + } + + public boolean equals(getNettyMetric_args that) { + if (that == null) + return false; + + boolean this_present_topologyName = true && this.is_set_topologyName(); + boolean that_present_topologyName = true && that.is_set_topologyName(); + if (this_present_topologyName || that_present_topologyName) { + if (!(this_present_topologyName && that_present_topologyName)) + return false; + if (!this.topologyName.equals(that.topologyName)) + return false; + } + + boolean this_present_pos = true; + boolean that_present_pos = true; + if (this_present_pos || that_present_pos) { + if (!(this_present_pos && that_present_pos)) + return false; + if (this.pos != that.pos) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_topologyName = true && (is_set_topologyName()); + list.add(present_topologyName); + if (present_topologyName) + list.add(topologyName); + + boolean present_pos = true; + list.add(present_pos); + if (present_pos) + list.add(pos); + + return list.hashCode(); + } + + @Override + public int compareTo(getNettyMetric_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_topologyName()).compareTo(other.is_set_topologyName()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_topologyName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topologyName, other.topologyName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_pos()).compareTo(other.is_set_pos()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_pos()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.pos, other.pos); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getNettyMetric_args("); + boolean first = true; + + sb.append("topologyName:"); + if (this.topologyName == null) { + sb.append("null"); + } else { + sb.append(this.topologyName); + } + first = false; + if (!first) sb.append(", "); + sb.append("pos:"); + sb.append(this.pos); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getNettyMetric_argsStandardSchemeFactory implements SchemeFactory { + public getNettyMetric_argsStandardScheme getScheme() { + return new getNettyMetric_argsStandardScheme(); + } + } + + private static class getNettyMetric_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getNettyMetric_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TOPOLOGY_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.topologyName = iprot.readString(); + struct.set_topologyName_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // POS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.pos = iprot.readI32(); + struct.set_pos_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getNettyMetric_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.topologyName != null) { + oprot.writeFieldBegin(TOPOLOGY_NAME_FIELD_DESC); + oprot.writeString(struct.topologyName); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(POS_FIELD_DESC); + oprot.writeI32(struct.pos); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getNettyMetric_argsTupleSchemeFactory implements SchemeFactory { + public getNettyMetric_argsTupleScheme getScheme() { + return new getNettyMetric_argsTupleScheme(); + } + } + + private static class getNettyMetric_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getNettyMetric_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_topologyName()) { + optionals.set(0); + } + if (struct.is_set_pos()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_topologyName()) { + oprot.writeString(struct.topologyName); + } + if (struct.is_set_pos()) { + oprot.writeI32(struct.pos); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getNettyMetric_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.topologyName = iprot.readString(); + struct.set_topologyName_isSet(true); + } + if (incoming.get(1)) { + struct.pos = iprot.readI32(); + struct.set_pos_isSet(true); + } + } + } + + } + + public static class getNettyMetric_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getNettyMetric_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getNettyMetric_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getNettyMetric_resultTupleSchemeFactory()); + } + + private NettyMetric success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NettyMetric.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getNettyMetric_result.class, metaDataMap); + } + + public getNettyMetric_result() { + } + + public getNettyMetric_result( + NettyMetric success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public getNettyMetric_result(getNettyMetric_result other) { + if (other.is_set_success()) { + this.success = new NettyMetric(other.success); + } + } + + public getNettyMetric_result deepCopy() { + return new getNettyMetric_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public NettyMetric get_success() { + return this.success; + } + + public void set_success(NettyMetric success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((NettyMetric)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getNettyMetric_result) + return this.equals((getNettyMetric_result)that); + return false; + } + + public boolean equals(getNettyMetric_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(getNettyMetric_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getNettyMetric_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getNettyMetric_resultStandardSchemeFactory implements SchemeFactory { + public getNettyMetric_resultStandardScheme getScheme() { + return new getNettyMetric_resultStandardScheme(); + } + } + + private static class getNettyMetric_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getNettyMetric_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new NettyMetric(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getNettyMetric_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getNettyMetric_resultTupleSchemeFactory implements SchemeFactory { + public getNettyMetric_resultTupleScheme getScheme() { + return new getNettyMetric_resultTupleScheme(); + } + } + + private static class getNettyMetric_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getNettyMetric_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_success()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getNettyMetric_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new NettyMetric(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } + } + } + + } + + public static class getServerNettyMetric_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getServerNettyMetric_args"); + + private static final org.apache.thrift.protocol.TField TOPOLOGY_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("topologyName", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField SERVER_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("serverName", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getServerNettyMetric_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getServerNettyMetric_argsTupleSchemeFactory()); + } + + private String topologyName; // required + private String serverName; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TOPOLOGY_NAME((short)1, "topologyName"), + SERVER_NAME((short)2, "serverName"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TOPOLOGY_NAME + return TOPOLOGY_NAME; + case 2: // SERVER_NAME + return SERVER_NAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TOPOLOGY_NAME, new org.apache.thrift.meta_data.FieldMetaData("topologyName", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.SERVER_NAME, new org.apache.thrift.meta_data.FieldMetaData("serverName", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getServerNettyMetric_args.class, metaDataMap); + } + + public getServerNettyMetric_args() { + } + + public getServerNettyMetric_args( + String topologyName, + String serverName) + { + this(); + this.topologyName = topologyName; + this.serverName = serverName; + } + + /** + * Performs a deep copy on other. + */ + public getServerNettyMetric_args(getServerNettyMetric_args other) { + if (other.is_set_topologyName()) { + this.topologyName = other.topologyName; + } + if (other.is_set_serverName()) { + this.serverName = other.serverName; + } + } + + public getServerNettyMetric_args deepCopy() { + return new getServerNettyMetric_args(this); + } + + @Override + public void clear() { + this.topologyName = null; + this.serverName = null; + } + + public String get_topologyName() { + return this.topologyName; + } + + public void set_topologyName(String topologyName) { + this.topologyName = topologyName; + } + + public void unset_topologyName() { + this.topologyName = null; + } + + /** Returns true if field topologyName is set (has been assigned a value) and false otherwise */ + public boolean is_set_topologyName() { + return this.topologyName != null; + } + + public void set_topologyName_isSet(boolean value) { + if (!value) { + this.topologyName = null; + } + } + + public String get_serverName() { + return this.serverName; + } + + public void set_serverName(String serverName) { + this.serverName = serverName; + } + + public void unset_serverName() { + this.serverName = null; + } + + /** Returns true if field serverName is set (has been assigned a value) and false otherwise */ + public boolean is_set_serverName() { + return this.serverName != null; + } + + public void set_serverName_isSet(boolean value) { + if (!value) { + this.serverName = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TOPOLOGY_NAME: + if (value == null) { + unset_topologyName(); + } else { + set_topologyName((String)value); + } + break; + + case SERVER_NAME: + if (value == null) { + unset_serverName(); + } else { + set_serverName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TOPOLOGY_NAME: + return get_topologyName(); + + case SERVER_NAME: + return get_serverName(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TOPOLOGY_NAME: + return is_set_topologyName(); + case SERVER_NAME: + return is_set_serverName(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getServerNettyMetric_args) + return this.equals((getServerNettyMetric_args)that); + return false; + } + + public boolean equals(getServerNettyMetric_args that) { + if (that == null) + return false; + + boolean this_present_topologyName = true && this.is_set_topologyName(); + boolean that_present_topologyName = true && that.is_set_topologyName(); + if (this_present_topologyName || that_present_topologyName) { + if (!(this_present_topologyName && that_present_topologyName)) + return false; + if (!this.topologyName.equals(that.topologyName)) + return false; + } + + boolean this_present_serverName = true && this.is_set_serverName(); + boolean that_present_serverName = true && that.is_set_serverName(); + if (this_present_serverName || that_present_serverName) { + if (!(this_present_serverName && that_present_serverName)) + return false; + if (!this.serverName.equals(that.serverName)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_topologyName = true && (is_set_topologyName()); + list.add(present_topologyName); + if (present_topologyName) + list.add(topologyName); + + boolean present_serverName = true && (is_set_serverName()); + list.add(present_serverName); + if (present_serverName) + list.add(serverName); + + return list.hashCode(); + } + + @Override + public int compareTo(getServerNettyMetric_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_topologyName()).compareTo(other.is_set_topologyName()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_topologyName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topologyName, other.topologyName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_serverName()).compareTo(other.is_set_serverName()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_serverName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.serverName, other.serverName); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getServerNettyMetric_args("); + boolean first = true; + + sb.append("topologyName:"); + if (this.topologyName == null) { + sb.append("null"); + } else { + sb.append(this.topologyName); + } + first = false; + if (!first) sb.append(", "); + sb.append("serverName:"); + if (this.serverName == null) { + sb.append("null"); + } else { + sb.append(this.serverName); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getServerNettyMetric_argsStandardSchemeFactory implements SchemeFactory { + public getServerNettyMetric_argsStandardScheme getScheme() { + return new getServerNettyMetric_argsStandardScheme(); + } + } + + private static class getServerNettyMetric_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getServerNettyMetric_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TOPOLOGY_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.topologyName = iprot.readString(); + struct.set_topologyName_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // SERVER_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.serverName = iprot.readString(); + struct.set_serverName_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getServerNettyMetric_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.topologyName != null) { + oprot.writeFieldBegin(TOPOLOGY_NAME_FIELD_DESC); + oprot.writeString(struct.topologyName); + oprot.writeFieldEnd(); + } + if (struct.serverName != null) { + oprot.writeFieldBegin(SERVER_NAME_FIELD_DESC); + oprot.writeString(struct.serverName); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getServerNettyMetric_argsTupleSchemeFactory implements SchemeFactory { + public getServerNettyMetric_argsTupleScheme getScheme() { + return new getServerNettyMetric_argsTupleScheme(); + } + } + + private static class getServerNettyMetric_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getServerNettyMetric_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_topologyName()) { + optionals.set(0); + } + if (struct.is_set_serverName()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_topologyName()) { + oprot.writeString(struct.topologyName); + } + if (struct.is_set_serverName()) { + oprot.writeString(struct.serverName); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getServerNettyMetric_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.topologyName = iprot.readString(); + struct.set_topologyName_isSet(true); + } + if (incoming.get(1)) { + struct.serverName = iprot.readString(); + struct.set_serverName_isSet(true); + } + } + } + + } + + public static class getServerNettyMetric_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getServerNettyMetric_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getServerNettyMetric_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getServerNettyMetric_resultTupleSchemeFactory()); + } + + private NettyMetric success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NettyMetric.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getServerNettyMetric_result.class, metaDataMap); + } + + public getServerNettyMetric_result() { + } + + public getServerNettyMetric_result( + NettyMetric success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public getServerNettyMetric_result(getServerNettyMetric_result other) { + if (other.is_set_success()) { + this.success = new NettyMetric(other.success); + } + } + + public getServerNettyMetric_result deepCopy() { + return new getServerNettyMetric_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public NettyMetric get_success() { + return this.success; + } + + public void set_success(NettyMetric success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((NettyMetric)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getServerNettyMetric_result) + return this.equals((getServerNettyMetric_result)that); + return false; + } + + public boolean equals(getServerNettyMetric_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(getServerNettyMetric_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getServerNettyMetric_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getServerNettyMetric_resultStandardSchemeFactory implements SchemeFactory { + public getServerNettyMetric_resultStandardScheme getScheme() { + return new getServerNettyMetric_resultStandardScheme(); + } + } + + private static class getServerNettyMetric_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getServerNettyMetric_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new NettyMetric(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getServerNettyMetric_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getServerNettyMetric_resultTupleSchemeFactory implements SchemeFactory { + public getServerNettyMetric_resultTupleScheme getScheme() { + return new getServerNettyMetric_resultTupleScheme(); + } + } + + private static class getServerNettyMetric_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getServerNettyMetric_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_success()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getServerNettyMetric_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new NettyMetric(); + struct.success.read(iprot); + struct.set_success_isSet(true); + } + } + } + + } + + public static class getVersion_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getVersion_args"); + + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getVersion_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getVersion_argsTupleSchemeFactory()); + } + + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { +; + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getVersion_args.class, metaDataMap); + } + + public getVersion_args() { + } + + /** + * Performs a deep copy on other. + */ + public getVersion_args(getVersion_args other) { + } + + public getVersion_args deepCopy() { + return new getVersion_args(this); + } + + @Override + public void clear() { + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getVersion_args) + return this.equals((getVersion_args)that); + return false; + } + + public boolean equals(getVersion_args that) { + if (that == null) + return false; + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + return list.hashCode(); + } + + @Override + public int compareTo(getVersion_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getVersion_args("); + boolean first = true; + + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getVersion_argsStandardSchemeFactory implements SchemeFactory { + public getVersion_argsStandardScheme getScheme() { + return new getVersion_argsStandardScheme(); + } + } + + private static class getVersion_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getVersion_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getVersion_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getVersion_argsTupleSchemeFactory implements SchemeFactory { + public getVersion_argsTupleScheme getScheme() { + return new getVersion_argsTupleScheme(); + } + } + + private static class getVersion_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getVersion_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getVersion_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + } + } + + } + + public static class getVersion_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getVersion_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRING, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new getVersion_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new getVersion_resultTupleSchemeFactory()); + } + + private String success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(getVersion_result.class, metaDataMap); + } + + public getVersion_result() { + } + + public getVersion_result( + String success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public getVersion_result(getVersion_result other) { + if (other.is_set_success()) { + this.success = other.success; + } + } + + public getVersion_result deepCopy() { + return new getVersion_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public String get_success() { + return this.success; + } + + public void set_success(String success) { + this.success = success; + } + + public void unset_success() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean is_set_success() { + return this.success != null; + } + + public void set_success_isSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unset_success(); + } else { + set_success((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return get_success(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return is_set_success(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof getVersion_result) + return this.equals((getVersion_result)that); + return false; + } + + public boolean equals(getVersion_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.is_set_success(); + boolean that_present_success = true && that.is_set_success(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (is_set_success()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(getVersion_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_success()).compareTo(other.is_set_success()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_success()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("getVersion_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class getVersion_resultStandardSchemeFactory implements SchemeFactory { + public getVersion_resultStandardScheme getScheme() { + return new getVersion_resultStandardScheme(); + } + } + + private static class getVersion_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, getVersion_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.success = iprot.readString(); + struct.set_success_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, getVersion_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + oprot.writeString(struct.success); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class getVersion_resultTupleSchemeFactory implements SchemeFactory { + public getVersion_resultTupleScheme getScheme() { + return new getVersion_resultTupleScheme(); + } + } + + private static class getVersion_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, getVersion_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_success()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_success()) { + oprot.writeString(struct.success); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, getVersion_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = iprot.readString(); + struct.set_success_isSet(true); + } + } + } + + } + + public static class updateConf_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("updateConf_args"); + + private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField CONF_FIELD_DESC = new org.apache.thrift.protocol.TField("conf", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new updateConf_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new updateConf_argsTupleSchemeFactory()); + } + + private String name; // required + private String conf; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NAME((short)1, "name"), + CONF((short)2, "conf"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NAME + return NAME; + case 2: // CONF + return CONF; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.CONF, new org.apache.thrift.meta_data.FieldMetaData("conf", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(updateConf_args.class, metaDataMap); + } + + public updateConf_args() { + } + + public updateConf_args( + String name, + String conf) + { + this(); + this.name = name; + this.conf = conf; + } + + /** + * Performs a deep copy on other. + */ + public updateConf_args(updateConf_args other) { + if (other.is_set_name()) { + this.name = other.name; + } + if (other.is_set_conf()) { + this.conf = other.conf; + } + } + + public updateConf_args deepCopy() { + return new updateConf_args(this); + } + + @Override + public void clear() { + this.name = null; + this.conf = null; + } + + public String get_name() { + return this.name; + } + + public void set_name(String name) { + this.name = name; + } + + public void unset_name() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean is_set_name() { + return this.name != null; + } + + public void set_name_isSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public String get_conf() { + return this.conf; + } + + public void set_conf(String conf) { + this.conf = conf; + } + + public void unset_conf() { + this.conf = null; + } + + /** Returns true if field conf is set (has been assigned a value) and false otherwise */ + public boolean is_set_conf() { + return this.conf != null; + } + + public void set_conf_isSet(boolean value) { + if (!value) { + this.conf = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME: + if (value == null) { + unset_name(); + } else { + set_name((String)value); + } + break; + + case CONF: + if (value == null) { + unset_conf(); + } else { + set_conf((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME: + return get_name(); + + case CONF: + return get_conf(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NAME: + return is_set_name(); + case CONF: + return is_set_conf(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof updateConf_args) + return this.equals((updateConf_args)that); + return false; + } + + public boolean equals(updateConf_args that) { + if (that == null) + return false; + + boolean this_present_name = true && this.is_set_name(); + boolean that_present_name = true && that.is_set_name(); + if (this_present_name || that_present_name) { + if (!(this_present_name && that_present_name)) + return false; + if (!this.name.equals(that.name)) + return false; + } + + boolean this_present_conf = true && this.is_set_conf(); + boolean that_present_conf = true && that.is_set_conf(); + if (this_present_conf || that_present_conf) { + if (!(this_present_conf && that_present_conf)) + return false; + if (!this.conf.equals(that.conf)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_name = true && (is_set_name()); + list.add(present_name); + if (present_name) + list.add(name); + + boolean present_conf = true && (is_set_conf()); + list.add(present_conf); + if (present_conf) + list.add(conf); + + return list.hashCode(); + } + + @Override + public int compareTo(updateConf_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_conf()).compareTo(other.is_set_conf()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_conf()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.conf, other.conf); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("updateConf_args("); + boolean first = true; + + sb.append("name:"); + if (this.name == null) { + sb.append("null"); + } else { + sb.append(this.name); + } + first = false; + if (!first) sb.append(", "); + sb.append("conf:"); + if (this.conf == null) { + sb.append("null"); + } else { + sb.append(this.conf); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class updateConf_argsStandardSchemeFactory implements SchemeFactory { + public updateConf_argsStandardScheme getScheme() { + return new updateConf_argsStandardScheme(); + } + } + + private static class updateConf_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, updateConf_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // CONF + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.conf = iprot.readString(); + struct.set_conf_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, updateConf_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.name != null) { + oprot.writeFieldBegin(NAME_FIELD_DESC); + oprot.writeString(struct.name); + oprot.writeFieldEnd(); + } + if (struct.conf != null) { + oprot.writeFieldBegin(CONF_FIELD_DESC); + oprot.writeString(struct.conf); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class updateConf_argsTupleSchemeFactory implements SchemeFactory { + public updateConf_argsTupleScheme getScheme() { + return new updateConf_argsTupleScheme(); + } + } + + private static class updateConf_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, updateConf_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_name()) { + optionals.set(0); + } + if (struct.is_set_conf()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_name()) { + oprot.writeString(struct.name); + } + if (struct.is_set_conf()) { + oprot.writeString(struct.conf); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, updateConf_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } + if (incoming.get(1)) { + struct.conf = iprot.readString(); + struct.set_conf_isSet(true); + } + } + } + + } + + public static class updateConf_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("updateConf_result"); + + private static final org.apache.thrift.protocol.TField E_FIELD_DESC = new org.apache.thrift.protocol.TField("e", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField ITE_FIELD_DESC = new org.apache.thrift.protocol.TField("ite", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new updateConf_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new updateConf_resultTupleSchemeFactory()); + } + + private NotAliveException e; // required + private InvalidTopologyException ite; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + E((short)1, "e"), + ITE((short)2, "ite"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // E + return E; + case 2: // ITE + return ITE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.E, new org.apache.thrift.meta_data.FieldMetaData("e", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + tmpMap.put(_Fields.ITE, new org.apache.thrift.meta_data.FieldMetaData("ite", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(updateConf_result.class, metaDataMap); + } + + public updateConf_result() { + } + + public updateConf_result( + NotAliveException e, + InvalidTopologyException ite) + { + this(); + this.e = e; + this.ite = ite; + } + + /** + * Performs a deep copy on other. + */ + public updateConf_result(updateConf_result other) { + if (other.is_set_e()) { + this.e = new NotAliveException(other.e); + } + if (other.is_set_ite()) { + this.ite = new InvalidTopologyException(other.ite); + } + } + + public updateConf_result deepCopy() { + return new updateConf_result(this); + } + + @Override + public void clear() { + this.e = null; + this.ite = null; + } + + public NotAliveException get_e() { + return this.e; + } + + public void set_e(NotAliveException e) { + this.e = e; + } + + public void unset_e() { + this.e = null; + } + + /** Returns true if field e is set (has been assigned a value) and false otherwise */ + public boolean is_set_e() { + return this.e != null; + } + + public void set_e_isSet(boolean value) { + if (!value) { + this.e = null; + } + } + + public InvalidTopologyException get_ite() { + return this.ite; + } + + public void set_ite(InvalidTopologyException ite) { + this.ite = ite; + } + + public void unset_ite() { + this.ite = null; + } + + /** Returns true if field ite is set (has been assigned a value) and false otherwise */ + public boolean is_set_ite() { + return this.ite != null; + } + + public void set_ite_isSet(boolean value) { + if (!value) { + this.ite = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case E: + if (value == null) { + unset_e(); + } else { + set_e((NotAliveException)value); + } + break; + + case ITE: + if (value == null) { + unset_ite(); + } else { + set_ite((InvalidTopologyException)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case E: + return get_e(); + + case ITE: + return get_ite(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case E: + return is_set_e(); + case ITE: + return is_set_ite(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof updateConf_result) + return this.equals((updateConf_result)that); + return false; + } + + public boolean equals(updateConf_result that) { + if (that == null) + return false; + + boolean this_present_e = true && this.is_set_e(); + boolean that_present_e = true && that.is_set_e(); + if (this_present_e || that_present_e) { + if (!(this_present_e && that_present_e)) + return false; + if (!this.e.equals(that.e)) + return false; + } + + boolean this_present_ite = true && this.is_set_ite(); + boolean that_present_ite = true && that.is_set_ite(); + if (this_present_ite || that_present_ite) { + if (!(this_present_ite && that_present_ite)) + return false; + if (!this.ite.equals(that.ite)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_e = true && (is_set_e()); + list.add(present_e); + if (present_e) + list.add(e); + + boolean present_ite = true && (is_set_ite()); + list.add(present_ite); + if (present_ite) + list.add(ite); + + return list.hashCode(); + } + + @Override + public int compareTo(updateConf_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_e()).compareTo(other.is_set_e()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_e()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.e, other.e); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_ite()).compareTo(other.is_set_ite()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_ite()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.ite, other.ite); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("updateConf_result("); + boolean first = true; + + sb.append("e:"); + if (this.e == null) { + sb.append("null"); + } else { + sb.append(this.e); + } + first = false; + if (!first) sb.append(", "); + sb.append("ite:"); + if (this.ite == null) { + sb.append("null"); + } else { + sb.append(this.ite); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class updateConf_resultStandardSchemeFactory implements SchemeFactory { + public updateConf_resultStandardScheme getScheme() { + return new updateConf_resultStandardScheme(); + } + } + + private static class updateConf_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, updateConf_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // E + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // ITE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.ite = new InvalidTopologyException(); + struct.ite.read(iprot); + struct.set_ite_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, updateConf_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.e != null) { + oprot.writeFieldBegin(E_FIELD_DESC); + struct.e.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.ite != null) { + oprot.writeFieldBegin(ITE_FIELD_DESC); + struct.ite.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class updateConf_resultTupleSchemeFactory implements SchemeFactory { + public updateConf_resultTupleScheme getScheme() { + return new updateConf_resultTupleScheme(); + } + } + + private static class updateConf_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, updateConf_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_e()) { + optionals.set(0); + } + if (struct.is_set_ite()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_e()) { + struct.e.write(oprot); + } + if (struct.is_set_ite()) { + struct.ite.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, updateConf_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.e = new NotAliveException(); + struct.e.read(iprot); + struct.set_e_isSet(true); + } + if (incoming.get(1)) { + struct.ite = new InvalidTopologyException(); + struct.ite.read(iprot); + struct.set_ite_isSet(true); + } + } + } + + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/generated/NimbusStat.java b/jstorm-core/src/main/java/backtype/storm/generated/NimbusStat.java new file mode 100644 index 000000000..90badb68a --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/NimbusStat.java @@ -0,0 +1,490 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class NimbusStat implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NimbusStat"); + + private static final org.apache.thrift.protocol.TField HOST_FIELD_DESC = new org.apache.thrift.protocol.TField("host", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime_secs", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new NimbusStatStandardSchemeFactory()); + schemes.put(TupleScheme.class, new NimbusStatTupleSchemeFactory()); + } + + private String host; // required + private String uptime_secs; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + HOST((short)1, "host"), + UPTIME_SECS((short)2, "uptime_secs"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // HOST + return HOST; + case 2: // UPTIME_SECS + return UPTIME_SECS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.HOST, new org.apache.thrift.meta_data.FieldMetaData("host", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("uptime_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(NimbusStat.class, metaDataMap); + } + + public NimbusStat() { + } + + public NimbusStat( + String host, + String uptime_secs) + { + this(); + this.host = host; + this.uptime_secs = uptime_secs; + } + + /** + * Performs a deep copy on other. + */ + public NimbusStat(NimbusStat other) { + if (other.is_set_host()) { + this.host = other.host; + } + if (other.is_set_uptime_secs()) { + this.uptime_secs = other.uptime_secs; + } + } + + public NimbusStat deepCopy() { + return new NimbusStat(this); + } + + @Override + public void clear() { + this.host = null; + this.uptime_secs = null; + } + + public String get_host() { + return this.host; + } + + public void set_host(String host) { + this.host = host; + } + + public void unset_host() { + this.host = null; + } + + /** Returns true if field host is set (has been assigned a value) and false otherwise */ + public boolean is_set_host() { + return this.host != null; + } + + public void set_host_isSet(boolean value) { + if (!value) { + this.host = null; + } + } + + public String get_uptime_secs() { + return this.uptime_secs; + } + + public void set_uptime_secs(String uptime_secs) { + this.uptime_secs = uptime_secs; + } + + public void unset_uptime_secs() { + this.uptime_secs = null; + } + + /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */ + public boolean is_set_uptime_secs() { + return this.uptime_secs != null; + } + + public void set_uptime_secs_isSet(boolean value) { + if (!value) { + this.uptime_secs = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case HOST: + if (value == null) { + unset_host(); + } else { + set_host((String)value); + } + break; + + case UPTIME_SECS: + if (value == null) { + unset_uptime_secs(); + } else { + set_uptime_secs((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case HOST: + return get_host(); + + case UPTIME_SECS: + return get_uptime_secs(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case HOST: + return is_set_host(); + case UPTIME_SECS: + return is_set_uptime_secs(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof NimbusStat) + return this.equals((NimbusStat)that); + return false; + } + + public boolean equals(NimbusStat that) { + if (that == null) + return false; + + boolean this_present_host = true && this.is_set_host(); + boolean that_present_host = true && that.is_set_host(); + if (this_present_host || that_present_host) { + if (!(this_present_host && that_present_host)) + return false; + if (!this.host.equals(that.host)) + return false; + } + + boolean this_present_uptime_secs = true && this.is_set_uptime_secs(); + boolean that_present_uptime_secs = true && that.is_set_uptime_secs(); + if (this_present_uptime_secs || that_present_uptime_secs) { + if (!(this_present_uptime_secs && that_present_uptime_secs)) + return false; + if (!this.uptime_secs.equals(that.uptime_secs)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_host = true && (is_set_host()); + list.add(present_host); + if (present_host) + list.add(host); + + boolean present_uptime_secs = true && (is_set_uptime_secs()); + list.add(present_uptime_secs); + if (present_uptime_secs) + list.add(uptime_secs); + + return list.hashCode(); + } + + @Override + public int compareTo(NimbusStat other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_host()).compareTo(other.is_set_host()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_host()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.host, other.host); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(other.is_set_uptime_secs()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_uptime_secs()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, other.uptime_secs); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("NimbusStat("); + boolean first = true; + + sb.append("host:"); + if (this.host == null) { + sb.append("null"); + } else { + sb.append(this.host); + } + first = false; + if (!first) sb.append(", "); + sb.append("uptime_secs:"); + if (this.uptime_secs == null) { + sb.append("null"); + } else { + sb.append(this.uptime_secs); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_host()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'host' is unset! Struct:" + toString()); + } + + if (!is_set_uptime_secs()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class NimbusStatStandardSchemeFactory implements SchemeFactory { + public NimbusStatStandardScheme getScheme() { + return new NimbusStatStandardScheme(); + } + } + + private static class NimbusStatStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, NimbusStat struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // HOST + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.host = iprot.readString(); + struct.set_host_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // UPTIME_SECS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.uptime_secs = iprot.readString(); + struct.set_uptime_secs_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, NimbusStat struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.host != null) { + oprot.writeFieldBegin(HOST_FIELD_DESC); + oprot.writeString(struct.host); + oprot.writeFieldEnd(); + } + if (struct.uptime_secs != null) { + oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC); + oprot.writeString(struct.uptime_secs); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class NimbusStatTupleSchemeFactory implements SchemeFactory { + public NimbusStatTupleScheme getScheme() { + return new NimbusStatTupleScheme(); + } + } + + private static class NimbusStatTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, NimbusStat struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.host); + oprot.writeString(struct.uptime_secs); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, NimbusStat struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.host = iprot.readString(); + struct.set_host_isSet(true); + struct.uptime_secs = iprot.readString(); + struct.set_uptime_secs_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/NimbusSummary.java b/jstorm-core/src/main/java/backtype/storm/generated/NimbusSummary.java new file mode 100644 index 000000000..55b6e35f5 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/NimbusSummary.java @@ -0,0 +1,1034 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class NimbusSummary implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NimbusSummary"); + + private static final org.apache.thrift.protocol.TField NIMBUS_MASTER_FIELD_DESC = new org.apache.thrift.protocol.TField("nimbus_master", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField NIMBUS_SLAVES_FIELD_DESC = new org.apache.thrift.protocol.TField("nimbus_slaves", org.apache.thrift.protocol.TType.LIST, (short)2); + private static final org.apache.thrift.protocol.TField SUPERVISOR_NUM_FIELD_DESC = new org.apache.thrift.protocol.TField("supervisor_num", org.apache.thrift.protocol.TType.I32, (short)3); + private static final org.apache.thrift.protocol.TField TOTAL_PORT_NUM_FIELD_DESC = new org.apache.thrift.protocol.TField("total_port_num", org.apache.thrift.protocol.TType.I32, (short)4); + private static final org.apache.thrift.protocol.TField USED_PORT_NUM_FIELD_DESC = new org.apache.thrift.protocol.TField("used_port_num", org.apache.thrift.protocol.TType.I32, (short)5); + private static final org.apache.thrift.protocol.TField FREE_PORT_NUM_FIELD_DESC = new org.apache.thrift.protocol.TField("free_port_num", org.apache.thrift.protocol.TType.I32, (short)6); + private static final org.apache.thrift.protocol.TField VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("version", org.apache.thrift.protocol.TType.STRING, (short)7); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new NimbusSummaryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new NimbusSummaryTupleSchemeFactory()); + } + + private NimbusStat nimbus_master; // required + private List nimbus_slaves; // required + private int supervisor_num; // required + private int total_port_num; // required + private int used_port_num; // required + private int free_port_num; // required + private String version; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NIMBUS_MASTER((short)1, "nimbus_master"), + NIMBUS_SLAVES((short)2, "nimbus_slaves"), + SUPERVISOR_NUM((short)3, "supervisor_num"), + TOTAL_PORT_NUM((short)4, "total_port_num"), + USED_PORT_NUM((short)5, "used_port_num"), + FREE_PORT_NUM((short)6, "free_port_num"), + VERSION((short)7, "version"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NIMBUS_MASTER + return NIMBUS_MASTER; + case 2: // NIMBUS_SLAVES + return NIMBUS_SLAVES; + case 3: // SUPERVISOR_NUM + return SUPERVISOR_NUM; + case 4: // TOTAL_PORT_NUM + return TOTAL_PORT_NUM; + case 5: // USED_PORT_NUM + return USED_PORT_NUM; + case 6: // FREE_PORT_NUM + return FREE_PORT_NUM; + case 7: // VERSION + return VERSION; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __SUPERVISOR_NUM_ISSET_ID = 0; + private static final int __TOTAL_PORT_NUM_ISSET_ID = 1; + private static final int __USED_PORT_NUM_ISSET_ID = 2; + private static final int __FREE_PORT_NUM_ISSET_ID = 3; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NIMBUS_MASTER, new org.apache.thrift.meta_data.FieldMetaData("nimbus_master", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NimbusStat.class))); + tmpMap.put(_Fields.NIMBUS_SLAVES, new org.apache.thrift.meta_data.FieldMetaData("nimbus_slaves", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NimbusStat.class)))); + tmpMap.put(_Fields.SUPERVISOR_NUM, new org.apache.thrift.meta_data.FieldMetaData("supervisor_num", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.TOTAL_PORT_NUM, new org.apache.thrift.meta_data.FieldMetaData("total_port_num", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.USED_PORT_NUM, new org.apache.thrift.meta_data.FieldMetaData("used_port_num", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.FREE_PORT_NUM, new org.apache.thrift.meta_data.FieldMetaData("free_port_num", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.VERSION, new org.apache.thrift.meta_data.FieldMetaData("version", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(NimbusSummary.class, metaDataMap); + } + + public NimbusSummary() { + } + + public NimbusSummary( + NimbusStat nimbus_master, + List nimbus_slaves, + int supervisor_num, + int total_port_num, + int used_port_num, + int free_port_num, + String version) + { + this(); + this.nimbus_master = nimbus_master; + this.nimbus_slaves = nimbus_slaves; + this.supervisor_num = supervisor_num; + set_supervisor_num_isSet(true); + this.total_port_num = total_port_num; + set_total_port_num_isSet(true); + this.used_port_num = used_port_num; + set_used_port_num_isSet(true); + this.free_port_num = free_port_num; + set_free_port_num_isSet(true); + this.version = version; + } + + /** + * Performs a deep copy on other. + */ + public NimbusSummary(NimbusSummary other) { + __isset_bitfield = other.__isset_bitfield; + if (other.is_set_nimbus_master()) { + this.nimbus_master = new NimbusStat(other.nimbus_master); + } + if (other.is_set_nimbus_slaves()) { + List __this__nimbus_slaves = new ArrayList(other.nimbus_slaves.size()); + for (NimbusStat other_element : other.nimbus_slaves) { + __this__nimbus_slaves.add(new NimbusStat(other_element)); + } + this.nimbus_slaves = __this__nimbus_slaves; + } + this.supervisor_num = other.supervisor_num; + this.total_port_num = other.total_port_num; + this.used_port_num = other.used_port_num; + this.free_port_num = other.free_port_num; + if (other.is_set_version()) { + this.version = other.version; + } + } + + public NimbusSummary deepCopy() { + return new NimbusSummary(this); + } + + @Override + public void clear() { + this.nimbus_master = null; + this.nimbus_slaves = null; + set_supervisor_num_isSet(false); + this.supervisor_num = 0; + set_total_port_num_isSet(false); + this.total_port_num = 0; + set_used_port_num_isSet(false); + this.used_port_num = 0; + set_free_port_num_isSet(false); + this.free_port_num = 0; + this.version = null; + } + + public NimbusStat get_nimbus_master() { + return this.nimbus_master; + } + + public void set_nimbus_master(NimbusStat nimbus_master) { + this.nimbus_master = nimbus_master; + } + + public void unset_nimbus_master() { + this.nimbus_master = null; + } + + /** Returns true if field nimbus_master is set (has been assigned a value) and false otherwise */ + public boolean is_set_nimbus_master() { + return this.nimbus_master != null; + } + + public void set_nimbus_master_isSet(boolean value) { + if (!value) { + this.nimbus_master = null; + } + } + + public int get_nimbus_slaves_size() { + return (this.nimbus_slaves == null) ? 0 : this.nimbus_slaves.size(); + } + + public java.util.Iterator get_nimbus_slaves_iterator() { + return (this.nimbus_slaves == null) ? null : this.nimbus_slaves.iterator(); + } + + public void add_to_nimbus_slaves(NimbusStat elem) { + if (this.nimbus_slaves == null) { + this.nimbus_slaves = new ArrayList(); + } + this.nimbus_slaves.add(elem); + } + + public List get_nimbus_slaves() { + return this.nimbus_slaves; + } + + public void set_nimbus_slaves(List nimbus_slaves) { + this.nimbus_slaves = nimbus_slaves; + } + + public void unset_nimbus_slaves() { + this.nimbus_slaves = null; + } + + /** Returns true if field nimbus_slaves is set (has been assigned a value) and false otherwise */ + public boolean is_set_nimbus_slaves() { + return this.nimbus_slaves != null; + } + + public void set_nimbus_slaves_isSet(boolean value) { + if (!value) { + this.nimbus_slaves = null; + } + } + + public int get_supervisor_num() { + return this.supervisor_num; + } + + public void set_supervisor_num(int supervisor_num) { + this.supervisor_num = supervisor_num; + set_supervisor_num_isSet(true); + } + + public void unset_supervisor_num() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SUPERVISOR_NUM_ISSET_ID); + } + + /** Returns true if field supervisor_num is set (has been assigned a value) and false otherwise */ + public boolean is_set_supervisor_num() { + return EncodingUtils.testBit(__isset_bitfield, __SUPERVISOR_NUM_ISSET_ID); + } + + public void set_supervisor_num_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SUPERVISOR_NUM_ISSET_ID, value); + } + + public int get_total_port_num() { + return this.total_port_num; + } + + public void set_total_port_num(int total_port_num) { + this.total_port_num = total_port_num; + set_total_port_num_isSet(true); + } + + public void unset_total_port_num() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TOTAL_PORT_NUM_ISSET_ID); + } + + /** Returns true if field total_port_num is set (has been assigned a value) and false otherwise */ + public boolean is_set_total_port_num() { + return EncodingUtils.testBit(__isset_bitfield, __TOTAL_PORT_NUM_ISSET_ID); + } + + public void set_total_port_num_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TOTAL_PORT_NUM_ISSET_ID, value); + } + + public int get_used_port_num() { + return this.used_port_num; + } + + public void set_used_port_num(int used_port_num) { + this.used_port_num = used_port_num; + set_used_port_num_isSet(true); + } + + public void unset_used_port_num() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __USED_PORT_NUM_ISSET_ID); + } + + /** Returns true if field used_port_num is set (has been assigned a value) and false otherwise */ + public boolean is_set_used_port_num() { + return EncodingUtils.testBit(__isset_bitfield, __USED_PORT_NUM_ISSET_ID); + } + + public void set_used_port_num_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __USED_PORT_NUM_ISSET_ID, value); + } + + public int get_free_port_num() { + return this.free_port_num; + } + + public void set_free_port_num(int free_port_num) { + this.free_port_num = free_port_num; + set_free_port_num_isSet(true); + } + + public void unset_free_port_num() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __FREE_PORT_NUM_ISSET_ID); + } + + /** Returns true if field free_port_num is set (has been assigned a value) and false otherwise */ + public boolean is_set_free_port_num() { + return EncodingUtils.testBit(__isset_bitfield, __FREE_PORT_NUM_ISSET_ID); + } + + public void set_free_port_num_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __FREE_PORT_NUM_ISSET_ID, value); + } + + public String get_version() { + return this.version; + } + + public void set_version(String version) { + this.version = version; + } + + public void unset_version() { + this.version = null; + } + + /** Returns true if field version is set (has been assigned a value) and false otherwise */ + public boolean is_set_version() { + return this.version != null; + } + + public void set_version_isSet(boolean value) { + if (!value) { + this.version = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NIMBUS_MASTER: + if (value == null) { + unset_nimbus_master(); + } else { + set_nimbus_master((NimbusStat)value); + } + break; + + case NIMBUS_SLAVES: + if (value == null) { + unset_nimbus_slaves(); + } else { + set_nimbus_slaves((List)value); + } + break; + + case SUPERVISOR_NUM: + if (value == null) { + unset_supervisor_num(); + } else { + set_supervisor_num((Integer)value); + } + break; + + case TOTAL_PORT_NUM: + if (value == null) { + unset_total_port_num(); + } else { + set_total_port_num((Integer)value); + } + break; + + case USED_PORT_NUM: + if (value == null) { + unset_used_port_num(); + } else { + set_used_port_num((Integer)value); + } + break; + + case FREE_PORT_NUM: + if (value == null) { + unset_free_port_num(); + } else { + set_free_port_num((Integer)value); + } + break; + + case VERSION: + if (value == null) { + unset_version(); + } else { + set_version((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NIMBUS_MASTER: + return get_nimbus_master(); + + case NIMBUS_SLAVES: + return get_nimbus_slaves(); + + case SUPERVISOR_NUM: + return Integer.valueOf(get_supervisor_num()); + + case TOTAL_PORT_NUM: + return Integer.valueOf(get_total_port_num()); + + case USED_PORT_NUM: + return Integer.valueOf(get_used_port_num()); + + case FREE_PORT_NUM: + return Integer.valueOf(get_free_port_num()); + + case VERSION: + return get_version(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NIMBUS_MASTER: + return is_set_nimbus_master(); + case NIMBUS_SLAVES: + return is_set_nimbus_slaves(); + case SUPERVISOR_NUM: + return is_set_supervisor_num(); + case TOTAL_PORT_NUM: + return is_set_total_port_num(); + case USED_PORT_NUM: + return is_set_used_port_num(); + case FREE_PORT_NUM: + return is_set_free_port_num(); + case VERSION: + return is_set_version(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof NimbusSummary) + return this.equals((NimbusSummary)that); + return false; + } + + public boolean equals(NimbusSummary that) { + if (that == null) + return false; + + boolean this_present_nimbus_master = true && this.is_set_nimbus_master(); + boolean that_present_nimbus_master = true && that.is_set_nimbus_master(); + if (this_present_nimbus_master || that_present_nimbus_master) { + if (!(this_present_nimbus_master && that_present_nimbus_master)) + return false; + if (!this.nimbus_master.equals(that.nimbus_master)) + return false; + } + + boolean this_present_nimbus_slaves = true && this.is_set_nimbus_slaves(); + boolean that_present_nimbus_slaves = true && that.is_set_nimbus_slaves(); + if (this_present_nimbus_slaves || that_present_nimbus_slaves) { + if (!(this_present_nimbus_slaves && that_present_nimbus_slaves)) + return false; + if (!this.nimbus_slaves.equals(that.nimbus_slaves)) + return false; + } + + boolean this_present_supervisor_num = true; + boolean that_present_supervisor_num = true; + if (this_present_supervisor_num || that_present_supervisor_num) { + if (!(this_present_supervisor_num && that_present_supervisor_num)) + return false; + if (this.supervisor_num != that.supervisor_num) + return false; + } + + boolean this_present_total_port_num = true; + boolean that_present_total_port_num = true; + if (this_present_total_port_num || that_present_total_port_num) { + if (!(this_present_total_port_num && that_present_total_port_num)) + return false; + if (this.total_port_num != that.total_port_num) + return false; + } + + boolean this_present_used_port_num = true; + boolean that_present_used_port_num = true; + if (this_present_used_port_num || that_present_used_port_num) { + if (!(this_present_used_port_num && that_present_used_port_num)) + return false; + if (this.used_port_num != that.used_port_num) + return false; + } + + boolean this_present_free_port_num = true; + boolean that_present_free_port_num = true; + if (this_present_free_port_num || that_present_free_port_num) { + if (!(this_present_free_port_num && that_present_free_port_num)) + return false; + if (this.free_port_num != that.free_port_num) + return false; + } + + boolean this_present_version = true && this.is_set_version(); + boolean that_present_version = true && that.is_set_version(); + if (this_present_version || that_present_version) { + if (!(this_present_version && that_present_version)) + return false; + if (!this.version.equals(that.version)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_nimbus_master = true && (is_set_nimbus_master()); + list.add(present_nimbus_master); + if (present_nimbus_master) + list.add(nimbus_master); + + boolean present_nimbus_slaves = true && (is_set_nimbus_slaves()); + list.add(present_nimbus_slaves); + if (present_nimbus_slaves) + list.add(nimbus_slaves); + + boolean present_supervisor_num = true; + list.add(present_supervisor_num); + if (present_supervisor_num) + list.add(supervisor_num); + + boolean present_total_port_num = true; + list.add(present_total_port_num); + if (present_total_port_num) + list.add(total_port_num); + + boolean present_used_port_num = true; + list.add(present_used_port_num); + if (present_used_port_num) + list.add(used_port_num); + + boolean present_free_port_num = true; + list.add(present_free_port_num); + if (present_free_port_num) + list.add(free_port_num); + + boolean present_version = true && (is_set_version()); + list.add(present_version); + if (present_version) + list.add(version); + + return list.hashCode(); + } + + @Override + public int compareTo(NimbusSummary other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_nimbus_master()).compareTo(other.is_set_nimbus_master()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_nimbus_master()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nimbus_master, other.nimbus_master); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_nimbus_slaves()).compareTo(other.is_set_nimbus_slaves()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_nimbus_slaves()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nimbus_slaves, other.nimbus_slaves); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_supervisor_num()).compareTo(other.is_set_supervisor_num()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_supervisor_num()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.supervisor_num, other.supervisor_num); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_total_port_num()).compareTo(other.is_set_total_port_num()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_total_port_num()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.total_port_num, other.total_port_num); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_used_port_num()).compareTo(other.is_set_used_port_num()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_used_port_num()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.used_port_num, other.used_port_num); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_free_port_num()).compareTo(other.is_set_free_port_num()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_free_port_num()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.free_port_num, other.free_port_num); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_version()).compareTo(other.is_set_version()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_version()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.version, other.version); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("NimbusSummary("); + boolean first = true; + + sb.append("nimbus_master:"); + if (this.nimbus_master == null) { + sb.append("null"); + } else { + sb.append(this.nimbus_master); + } + first = false; + if (!first) sb.append(", "); + sb.append("nimbus_slaves:"); + if (this.nimbus_slaves == null) { + sb.append("null"); + } else { + sb.append(this.nimbus_slaves); + } + first = false; + if (!first) sb.append(", "); + sb.append("supervisor_num:"); + sb.append(this.supervisor_num); + first = false; + if (!first) sb.append(", "); + sb.append("total_port_num:"); + sb.append(this.total_port_num); + first = false; + if (!first) sb.append(", "); + sb.append("used_port_num:"); + sb.append(this.used_port_num); + first = false; + if (!first) sb.append(", "); + sb.append("free_port_num:"); + sb.append(this.free_port_num); + first = false; + if (!first) sb.append(", "); + sb.append("version:"); + if (this.version == null) { + sb.append("null"); + } else { + sb.append(this.version); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_nimbus_master()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nimbus_master' is unset! Struct:" + toString()); + } + + if (!is_set_nimbus_slaves()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nimbus_slaves' is unset! Struct:" + toString()); + } + + if (!is_set_supervisor_num()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'supervisor_num' is unset! Struct:" + toString()); + } + + if (!is_set_total_port_num()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'total_port_num' is unset! Struct:" + toString()); + } + + if (!is_set_used_port_num()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'used_port_num' is unset! Struct:" + toString()); + } + + if (!is_set_free_port_num()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'free_port_num' is unset! Struct:" + toString()); + } + + if (!is_set_version()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'version' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (nimbus_master != null) { + nimbus_master.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class NimbusSummaryStandardSchemeFactory implements SchemeFactory { + public NimbusSummaryStandardScheme getScheme() { + return new NimbusSummaryStandardScheme(); + } + } + + private static class NimbusSummaryStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, NimbusSummary struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NIMBUS_MASTER + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.nimbus_master = new NimbusStat(); + struct.nimbus_master.read(iprot); + struct.set_nimbus_master_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NIMBUS_SLAVES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list74 = iprot.readListBegin(); + struct.nimbus_slaves = new ArrayList(_list74.size); + NimbusStat _elem75; + for (int _i76 = 0; _i76 < _list74.size; ++_i76) + { + _elem75 = new NimbusStat(); + _elem75.read(iprot); + struct.nimbus_slaves.add(_elem75); + } + iprot.readListEnd(); + } + struct.set_nimbus_slaves_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SUPERVISOR_NUM + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.supervisor_num = iprot.readI32(); + struct.set_supervisor_num_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // TOTAL_PORT_NUM + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.total_port_num = iprot.readI32(); + struct.set_total_port_num_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // USED_PORT_NUM + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.used_port_num = iprot.readI32(); + struct.set_used_port_num_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 6: // FREE_PORT_NUM + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.free_port_num = iprot.readI32(); + struct.set_free_port_num_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 7: // VERSION + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.version = iprot.readString(); + struct.set_version_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, NimbusSummary struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.nimbus_master != null) { + oprot.writeFieldBegin(NIMBUS_MASTER_FIELD_DESC); + struct.nimbus_master.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.nimbus_slaves != null) { + oprot.writeFieldBegin(NIMBUS_SLAVES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.nimbus_slaves.size())); + for (NimbusStat _iter77 : struct.nimbus_slaves) + { + _iter77.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(SUPERVISOR_NUM_FIELD_DESC); + oprot.writeI32(struct.supervisor_num); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(TOTAL_PORT_NUM_FIELD_DESC); + oprot.writeI32(struct.total_port_num); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(USED_PORT_NUM_FIELD_DESC); + oprot.writeI32(struct.used_port_num); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(FREE_PORT_NUM_FIELD_DESC); + oprot.writeI32(struct.free_port_num); + oprot.writeFieldEnd(); + if (struct.version != null) { + oprot.writeFieldBegin(VERSION_FIELD_DESC); + oprot.writeString(struct.version); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class NimbusSummaryTupleSchemeFactory implements SchemeFactory { + public NimbusSummaryTupleScheme getScheme() { + return new NimbusSummaryTupleScheme(); + } + } + + private static class NimbusSummaryTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, NimbusSummary struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.nimbus_master.write(oprot); + { + oprot.writeI32(struct.nimbus_slaves.size()); + for (NimbusStat _iter78 : struct.nimbus_slaves) + { + _iter78.write(oprot); + } + } + oprot.writeI32(struct.supervisor_num); + oprot.writeI32(struct.total_port_num); + oprot.writeI32(struct.used_port_num); + oprot.writeI32(struct.free_port_num); + oprot.writeString(struct.version); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, NimbusSummary struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.nimbus_master = new NimbusStat(); + struct.nimbus_master.read(iprot); + struct.set_nimbus_master_isSet(true); + { + org.apache.thrift.protocol.TList _list79 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.nimbus_slaves = new ArrayList(_list79.size); + NimbusStat _elem80; + for (int _i81 = 0; _i81 < _list79.size; ++_i81) + { + _elem80 = new NimbusStat(); + _elem80.read(iprot); + struct.nimbus_slaves.add(_elem80); + } + } + struct.set_nimbus_slaves_isSet(true); + struct.supervisor_num = iprot.readI32(); + struct.set_supervisor_num_isSet(true); + struct.total_port_num = iprot.readI32(); + struct.set_total_port_num_isSet(true); + struct.used_port_num = iprot.readI32(); + struct.set_used_port_num_isSet(true); + struct.free_port_num = iprot.readI32(); + struct.set_free_port_num_isSet(true); + struct.version = iprot.readString(); + struct.set_version_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/NotAliveException.java b/jstorm-core/src/main/java/backtype/storm/generated/NotAliveException.java new file mode 100644 index 000000000..ae0f056f9 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/NotAliveException.java @@ -0,0 +1,389 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class NotAliveException extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NotAliveException"); + + private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new NotAliveExceptionStandardSchemeFactory()); + schemes.put(TupleScheme.class, new NotAliveExceptionTupleSchemeFactory()); + } + + private String msg; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + MSG((short)1, "msg"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // MSG + return MSG; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(NotAliveException.class, metaDataMap); + } + + public NotAliveException() { + } + + public NotAliveException( + String msg) + { + this(); + this.msg = msg; + } + + /** + * Performs a deep copy on other. + */ + public NotAliveException(NotAliveException other) { + if (other.is_set_msg()) { + this.msg = other.msg; + } + } + + public NotAliveException deepCopy() { + return new NotAliveException(this); + } + + @Override + public void clear() { + this.msg = null; + } + + public String get_msg() { + return this.msg; + } + + public void set_msg(String msg) { + this.msg = msg; + } + + public void unset_msg() { + this.msg = null; + } + + /** Returns true if field msg is set (has been assigned a value) and false otherwise */ + public boolean is_set_msg() { + return this.msg != null; + } + + public void set_msg_isSet(boolean value) { + if (!value) { + this.msg = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case MSG: + if (value == null) { + unset_msg(); + } else { + set_msg((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case MSG: + return get_msg(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case MSG: + return is_set_msg(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof NotAliveException) + return this.equals((NotAliveException)that); + return false; + } + + public boolean equals(NotAliveException that) { + if (that == null) + return false; + + boolean this_present_msg = true && this.is_set_msg(); + boolean that_present_msg = true && that.is_set_msg(); + if (this_present_msg || that_present_msg) { + if (!(this_present_msg && that_present_msg)) + return false; + if (!this.msg.equals(that.msg)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_msg = true && (is_set_msg()); + list.add(present_msg); + if (present_msg) + list.add(msg); + + return list.hashCode(); + } + + @Override + public int compareTo(NotAliveException other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_msg()).compareTo(other.is_set_msg()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_msg()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, other.msg); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("NotAliveException("); + boolean first = true; + + sb.append("msg:"); + if (this.msg == null) { + sb.append("null"); + } else { + sb.append(this.msg); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_msg()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class NotAliveExceptionStandardSchemeFactory implements SchemeFactory { + public NotAliveExceptionStandardScheme getScheme() { + return new NotAliveExceptionStandardScheme(); + } + } + + private static class NotAliveExceptionStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, NotAliveException struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // MSG + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.msg = iprot.readString(); + struct.set_msg_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, NotAliveException struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.msg != null) { + oprot.writeFieldBegin(MSG_FIELD_DESC); + oprot.writeString(struct.msg); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class NotAliveExceptionTupleSchemeFactory implements SchemeFactory { + public NotAliveExceptionTupleScheme getScheme() { + return new NotAliveExceptionTupleScheme(); + } + } + + private static class NotAliveExceptionTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, NotAliveException struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.msg); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, NotAliveException struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.msg = iprot.readString(); + struct.set_msg_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/NullStruct.java b/jstorm-core/src/main/java/backtype/storm/generated/NullStruct.java new file mode 100644 index 000000000..0494eb2cc --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/NullStruct.java @@ -0,0 +1,283 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class NullStruct implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("NullStruct"); + + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new NullStructStandardSchemeFactory()); + schemes.put(TupleScheme.class, new NullStructTupleSchemeFactory()); + } + + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { +; + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(NullStruct.class, metaDataMap); + } + + public NullStruct() { + } + + /** + * Performs a deep copy on other. + */ + public NullStruct(NullStruct other) { + } + + public NullStruct deepCopy() { + return new NullStruct(this); + } + + @Override + public void clear() { + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof NullStruct) + return this.equals((NullStruct)that); + return false; + } + + public boolean equals(NullStruct that) { + if (that == null) + return false; + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + return list.hashCode(); + } + + @Override + public int compareTo(NullStruct other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("NullStruct("); + boolean first = true; + + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class NullStructStandardSchemeFactory implements SchemeFactory { + public NullStructStandardScheme getScheme() { + return new NullStructStandardScheme(); + } + } + + private static class NullStructStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, NullStruct struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, NullStruct struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class NullStructTupleSchemeFactory implements SchemeFactory { + public NullStructTupleScheme getScheme() { + return new NullStructTupleScheme(); + } + } + + private static class NullStructTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, NullStruct struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, NullStruct struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/RebalanceOptions.java b/jstorm-core/src/main/java/backtype/storm/generated/RebalanceOptions.java new file mode 100644 index 000000000..33481891a --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/RebalanceOptions.java @@ -0,0 +1,595 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class RebalanceOptions implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RebalanceOptions"); + + private static final org.apache.thrift.protocol.TField WAIT_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("wait_secs", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField REASSIGN_FIELD_DESC = new org.apache.thrift.protocol.TField("reassign", org.apache.thrift.protocol.TType.BOOL, (short)2); + private static final org.apache.thrift.protocol.TField CONF_FIELD_DESC = new org.apache.thrift.protocol.TField("conf", org.apache.thrift.protocol.TType.STRING, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new RebalanceOptionsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new RebalanceOptionsTupleSchemeFactory()); + } + + private int wait_secs; // optional + private boolean reassign; // optional + private String conf; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + WAIT_SECS((short)1, "wait_secs"), + REASSIGN((short)2, "reassign"), + CONF((short)3, "conf"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // WAIT_SECS + return WAIT_SECS; + case 2: // REASSIGN + return REASSIGN; + case 3: // CONF + return CONF; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __WAIT_SECS_ISSET_ID = 0; + private static final int __REASSIGN_ISSET_ID = 1; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.WAIT_SECS,_Fields.REASSIGN,_Fields.CONF}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.WAIT_SECS, new org.apache.thrift.meta_data.FieldMetaData("wait_secs", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.REASSIGN, new org.apache.thrift.meta_data.FieldMetaData("reassign", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + tmpMap.put(_Fields.CONF, new org.apache.thrift.meta_data.FieldMetaData("conf", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(RebalanceOptions.class, metaDataMap); + } + + public RebalanceOptions() { + } + + /** + * Performs a deep copy on other. + */ + public RebalanceOptions(RebalanceOptions other) { + __isset_bitfield = other.__isset_bitfield; + this.wait_secs = other.wait_secs; + this.reassign = other.reassign; + if (other.is_set_conf()) { + this.conf = other.conf; + } + } + + public RebalanceOptions deepCopy() { + return new RebalanceOptions(this); + } + + @Override + public void clear() { + set_wait_secs_isSet(false); + this.wait_secs = 0; + set_reassign_isSet(false); + this.reassign = false; + this.conf = null; + } + + public int get_wait_secs() { + return this.wait_secs; + } + + public void set_wait_secs(int wait_secs) { + this.wait_secs = wait_secs; + set_wait_secs_isSet(true); + } + + public void unset_wait_secs() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __WAIT_SECS_ISSET_ID); + } + + /** Returns true if field wait_secs is set (has been assigned a value) and false otherwise */ + public boolean is_set_wait_secs() { + return EncodingUtils.testBit(__isset_bitfield, __WAIT_SECS_ISSET_ID); + } + + public void set_wait_secs_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __WAIT_SECS_ISSET_ID, value); + } + + public boolean is_reassign() { + return this.reassign; + } + + public void set_reassign(boolean reassign) { + this.reassign = reassign; + set_reassign_isSet(true); + } + + public void unset_reassign() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __REASSIGN_ISSET_ID); + } + + /** Returns true if field reassign is set (has been assigned a value) and false otherwise */ + public boolean is_set_reassign() { + return EncodingUtils.testBit(__isset_bitfield, __REASSIGN_ISSET_ID); + } + + public void set_reassign_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REASSIGN_ISSET_ID, value); + } + + public String get_conf() { + return this.conf; + } + + public void set_conf(String conf) { + this.conf = conf; + } + + public void unset_conf() { + this.conf = null; + } + + /** Returns true if field conf is set (has been assigned a value) and false otherwise */ + public boolean is_set_conf() { + return this.conf != null; + } + + public void set_conf_isSet(boolean value) { + if (!value) { + this.conf = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case WAIT_SECS: + if (value == null) { + unset_wait_secs(); + } else { + set_wait_secs((Integer)value); + } + break; + + case REASSIGN: + if (value == null) { + unset_reassign(); + } else { + set_reassign((Boolean)value); + } + break; + + case CONF: + if (value == null) { + unset_conf(); + } else { + set_conf((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case WAIT_SECS: + return Integer.valueOf(get_wait_secs()); + + case REASSIGN: + return Boolean.valueOf(is_reassign()); + + case CONF: + return get_conf(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case WAIT_SECS: + return is_set_wait_secs(); + case REASSIGN: + return is_set_reassign(); + case CONF: + return is_set_conf(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof RebalanceOptions) + return this.equals((RebalanceOptions)that); + return false; + } + + public boolean equals(RebalanceOptions that) { + if (that == null) + return false; + + boolean this_present_wait_secs = true && this.is_set_wait_secs(); + boolean that_present_wait_secs = true && that.is_set_wait_secs(); + if (this_present_wait_secs || that_present_wait_secs) { + if (!(this_present_wait_secs && that_present_wait_secs)) + return false; + if (this.wait_secs != that.wait_secs) + return false; + } + + boolean this_present_reassign = true && this.is_set_reassign(); + boolean that_present_reassign = true && that.is_set_reassign(); + if (this_present_reassign || that_present_reassign) { + if (!(this_present_reassign && that_present_reassign)) + return false; + if (this.reassign != that.reassign) + return false; + } + + boolean this_present_conf = true && this.is_set_conf(); + boolean that_present_conf = true && that.is_set_conf(); + if (this_present_conf || that_present_conf) { + if (!(this_present_conf && that_present_conf)) + return false; + if (!this.conf.equals(that.conf)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_wait_secs = true && (is_set_wait_secs()); + list.add(present_wait_secs); + if (present_wait_secs) + list.add(wait_secs); + + boolean present_reassign = true && (is_set_reassign()); + list.add(present_reassign); + if (present_reassign) + list.add(reassign); + + boolean present_conf = true && (is_set_conf()); + list.add(present_conf); + if (present_conf) + list.add(conf); + + return list.hashCode(); + } + + @Override + public int compareTo(RebalanceOptions other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_wait_secs()).compareTo(other.is_set_wait_secs()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_wait_secs()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.wait_secs, other.wait_secs); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_reassign()).compareTo(other.is_set_reassign()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_reassign()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.reassign, other.reassign); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_conf()).compareTo(other.is_set_conf()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_conf()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.conf, other.conf); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("RebalanceOptions("); + boolean first = true; + + if (is_set_wait_secs()) { + sb.append("wait_secs:"); + sb.append(this.wait_secs); + first = false; + } + if (is_set_reassign()) { + if (!first) sb.append(", "); + sb.append("reassign:"); + sb.append(this.reassign); + first = false; + } + if (is_set_conf()) { + if (!first) sb.append(", "); + sb.append("conf:"); + if (this.conf == null) { + sb.append("null"); + } else { + sb.append(this.conf); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class RebalanceOptionsStandardSchemeFactory implements SchemeFactory { + public RebalanceOptionsStandardScheme getScheme() { + return new RebalanceOptionsStandardScheme(); + } + } + + private static class RebalanceOptionsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, RebalanceOptions struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // WAIT_SECS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.wait_secs = iprot.readI32(); + struct.set_wait_secs_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // REASSIGN + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.reassign = iprot.readBool(); + struct.set_reassign_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // CONF + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.conf = iprot.readString(); + struct.set_conf_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, RebalanceOptions struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.is_set_wait_secs()) { + oprot.writeFieldBegin(WAIT_SECS_FIELD_DESC); + oprot.writeI32(struct.wait_secs); + oprot.writeFieldEnd(); + } + if (struct.is_set_reassign()) { + oprot.writeFieldBegin(REASSIGN_FIELD_DESC); + oprot.writeBool(struct.reassign); + oprot.writeFieldEnd(); + } + if (struct.conf != null) { + if (struct.is_set_conf()) { + oprot.writeFieldBegin(CONF_FIELD_DESC); + oprot.writeString(struct.conf); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class RebalanceOptionsTupleSchemeFactory implements SchemeFactory { + public RebalanceOptionsTupleScheme getScheme() { + return new RebalanceOptionsTupleScheme(); + } + } + + private static class RebalanceOptionsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, RebalanceOptions struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_wait_secs()) { + optionals.set(0); + } + if (struct.is_set_reassign()) { + optionals.set(1); + } + if (struct.is_set_conf()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.is_set_wait_secs()) { + oprot.writeI32(struct.wait_secs); + } + if (struct.is_set_reassign()) { + oprot.writeBool(struct.reassign); + } + if (struct.is_set_conf()) { + oprot.writeString(struct.conf); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, RebalanceOptions struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.wait_secs = iprot.readI32(); + struct.set_wait_secs_isSet(true); + } + if (incoming.get(1)) { + struct.reassign = iprot.readBool(); + struct.set_reassign_isSet(true); + } + if (incoming.get(2)) { + struct.conf = iprot.readString(); + struct.set_conf_isSet(true); + } + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/ShellComponent.java b/jstorm-core/src/main/java/backtype/storm/generated/ShellComponent.java new file mode 100644 index 000000000..1f9e827ff --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/ShellComponent.java @@ -0,0 +1,499 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class ShellComponent implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ShellComponent"); + + private static final org.apache.thrift.protocol.TField EXECUTION_COMMAND_FIELD_DESC = new org.apache.thrift.protocol.TField("execution_command", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField SCRIPT_FIELD_DESC = new org.apache.thrift.protocol.TField("script", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ShellComponentStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ShellComponentTupleSchemeFactory()); + } + + private String execution_command; // required + private String script; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + EXECUTION_COMMAND((short)1, "execution_command"), + SCRIPT((short)2, "script"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // EXECUTION_COMMAND + return EXECUTION_COMMAND; + case 2: // SCRIPT + return SCRIPT; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.EXECUTION_COMMAND, new org.apache.thrift.meta_data.FieldMetaData("execution_command", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.SCRIPT, new org.apache.thrift.meta_data.FieldMetaData("script", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ShellComponent.class, metaDataMap); + } + + public ShellComponent() { + } + + public ShellComponent( + String execution_command, + String script) + { + this(); + this.execution_command = execution_command; + this.script = script; + } + + /** + * Performs a deep copy on other. + */ + public ShellComponent(ShellComponent other) { + if (other.is_set_execution_command()) { + this.execution_command = other.execution_command; + } + if (other.is_set_script()) { + this.script = other.script; + } + } + + public ShellComponent deepCopy() { + return new ShellComponent(this); + } + + @Override + public void clear() { + this.execution_command = null; + this.script = null; + } + + public String get_execution_command() { + return this.execution_command; + } + + public void set_execution_command(String execution_command) { + this.execution_command = execution_command; + } + + public void unset_execution_command() { + this.execution_command = null; + } + + /** Returns true if field execution_command is set (has been assigned a value) and false otherwise */ + public boolean is_set_execution_command() { + return this.execution_command != null; + } + + public void set_execution_command_isSet(boolean value) { + if (!value) { + this.execution_command = null; + } + } + + public String get_script() { + return this.script; + } + + public void set_script(String script) { + this.script = script; + } + + public void unset_script() { + this.script = null; + } + + /** Returns true if field script is set (has been assigned a value) and false otherwise */ + public boolean is_set_script() { + return this.script != null; + } + + public void set_script_isSet(boolean value) { + if (!value) { + this.script = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case EXECUTION_COMMAND: + if (value == null) { + unset_execution_command(); + } else { + set_execution_command((String)value); + } + break; + + case SCRIPT: + if (value == null) { + unset_script(); + } else { + set_script((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case EXECUTION_COMMAND: + return get_execution_command(); + + case SCRIPT: + return get_script(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case EXECUTION_COMMAND: + return is_set_execution_command(); + case SCRIPT: + return is_set_script(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ShellComponent) + return this.equals((ShellComponent)that); + return false; + } + + public boolean equals(ShellComponent that) { + if (that == null) + return false; + + boolean this_present_execution_command = true && this.is_set_execution_command(); + boolean that_present_execution_command = true && that.is_set_execution_command(); + if (this_present_execution_command || that_present_execution_command) { + if (!(this_present_execution_command && that_present_execution_command)) + return false; + if (!this.execution_command.equals(that.execution_command)) + return false; + } + + boolean this_present_script = true && this.is_set_script(); + boolean that_present_script = true && that.is_set_script(); + if (this_present_script || that_present_script) { + if (!(this_present_script && that_present_script)) + return false; + if (!this.script.equals(that.script)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_execution_command = true && (is_set_execution_command()); + list.add(present_execution_command); + if (present_execution_command) + list.add(execution_command); + + boolean present_script = true && (is_set_script()); + list.add(present_script); + if (present_script) + list.add(script); + + return list.hashCode(); + } + + @Override + public int compareTo(ShellComponent other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_execution_command()).compareTo(other.is_set_execution_command()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_execution_command()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.execution_command, other.execution_command); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_script()).compareTo(other.is_set_script()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_script()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.script, other.script); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ShellComponent("); + boolean first = true; + + sb.append("execution_command:"); + if (this.execution_command == null) { + sb.append("null"); + } else { + sb.append(this.execution_command); + } + first = false; + if (!first) sb.append(", "); + sb.append("script:"); + if (this.script == null) { + sb.append("null"); + } else { + sb.append(this.script); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ShellComponentStandardSchemeFactory implements SchemeFactory { + public ShellComponentStandardScheme getScheme() { + return new ShellComponentStandardScheme(); + } + } + + private static class ShellComponentStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ShellComponent struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // EXECUTION_COMMAND + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.execution_command = iprot.readString(); + struct.set_execution_command_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // SCRIPT + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.script = iprot.readString(); + struct.set_script_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ShellComponent struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.execution_command != null) { + oprot.writeFieldBegin(EXECUTION_COMMAND_FIELD_DESC); + oprot.writeString(struct.execution_command); + oprot.writeFieldEnd(); + } + if (struct.script != null) { + oprot.writeFieldBegin(SCRIPT_FIELD_DESC); + oprot.writeString(struct.script); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ShellComponentTupleSchemeFactory implements SchemeFactory { + public ShellComponentTupleScheme getScheme() { + return new ShellComponentTupleScheme(); + } + } + + private static class ShellComponentTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ShellComponent struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.is_set_execution_command()) { + optionals.set(0); + } + if (struct.is_set_script()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.is_set_execution_command()) { + oprot.writeString(struct.execution_command); + } + if (struct.is_set_script()) { + oprot.writeString(struct.script); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ShellComponent struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.execution_command = iprot.readString(); + struct.set_execution_command_isSet(true); + } + if (incoming.get(1)) { + struct.script = iprot.readString(); + struct.set_script_isSet(true); + } + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/SpoutSpec.java b/jstorm-core/src/main/java/backtype/storm/generated/SpoutSpec.java new file mode 100644 index 000000000..722fc54a3 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/SpoutSpec.java @@ -0,0 +1,497 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class SpoutSpec implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SpoutSpec"); + + private static final org.apache.thrift.protocol.TField SPOUT_OBJECT_FIELD_DESC = new org.apache.thrift.protocol.TField("spout_object", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField COMMON_FIELD_DESC = new org.apache.thrift.protocol.TField("common", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new SpoutSpecStandardSchemeFactory()); + schemes.put(TupleScheme.class, new SpoutSpecTupleSchemeFactory()); + } + + private ComponentObject spout_object; // required + private ComponentCommon common; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SPOUT_OBJECT((short)1, "spout_object"), + COMMON((short)2, "common"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SPOUT_OBJECT + return SPOUT_OBJECT; + case 2: // COMMON + return COMMON; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SPOUT_OBJECT, new org.apache.thrift.meta_data.FieldMetaData("spout_object", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentObject.class))); + tmpMap.put(_Fields.COMMON, new org.apache.thrift.meta_data.FieldMetaData("common", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentCommon.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SpoutSpec.class, metaDataMap); + } + + public SpoutSpec() { + } + + public SpoutSpec( + ComponentObject spout_object, + ComponentCommon common) + { + this(); + this.spout_object = spout_object; + this.common = common; + } + + /** + * Performs a deep copy on other. + */ + public SpoutSpec(SpoutSpec other) { + if (other.is_set_spout_object()) { + this.spout_object = new ComponentObject(other.spout_object); + } + if (other.is_set_common()) { + this.common = new ComponentCommon(other.common); + } + } + + public SpoutSpec deepCopy() { + return new SpoutSpec(this); + } + + @Override + public void clear() { + this.spout_object = null; + this.common = null; + } + + public ComponentObject get_spout_object() { + return this.spout_object; + } + + public void set_spout_object(ComponentObject spout_object) { + this.spout_object = spout_object; + } + + public void unset_spout_object() { + this.spout_object = null; + } + + /** Returns true if field spout_object is set (has been assigned a value) and false otherwise */ + public boolean is_set_spout_object() { + return this.spout_object != null; + } + + public void set_spout_object_isSet(boolean value) { + if (!value) { + this.spout_object = null; + } + } + + public ComponentCommon get_common() { + return this.common; + } + + public void set_common(ComponentCommon common) { + this.common = common; + } + + public void unset_common() { + this.common = null; + } + + /** Returns true if field common is set (has been assigned a value) and false otherwise */ + public boolean is_set_common() { + return this.common != null; + } + + public void set_common_isSet(boolean value) { + if (!value) { + this.common = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SPOUT_OBJECT: + if (value == null) { + unset_spout_object(); + } else { + set_spout_object((ComponentObject)value); + } + break; + + case COMMON: + if (value == null) { + unset_common(); + } else { + set_common((ComponentCommon)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SPOUT_OBJECT: + return get_spout_object(); + + case COMMON: + return get_common(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SPOUT_OBJECT: + return is_set_spout_object(); + case COMMON: + return is_set_common(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof SpoutSpec) + return this.equals((SpoutSpec)that); + return false; + } + + public boolean equals(SpoutSpec that) { + if (that == null) + return false; + + boolean this_present_spout_object = true && this.is_set_spout_object(); + boolean that_present_spout_object = true && that.is_set_spout_object(); + if (this_present_spout_object || that_present_spout_object) { + if (!(this_present_spout_object && that_present_spout_object)) + return false; + if (!this.spout_object.equals(that.spout_object)) + return false; + } + + boolean this_present_common = true && this.is_set_common(); + boolean that_present_common = true && that.is_set_common(); + if (this_present_common || that_present_common) { + if (!(this_present_common && that_present_common)) + return false; + if (!this.common.equals(that.common)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_spout_object = true && (is_set_spout_object()); + list.add(present_spout_object); + if (present_spout_object) + list.add(spout_object); + + boolean present_common = true && (is_set_common()); + list.add(present_common); + if (present_common) + list.add(common); + + return list.hashCode(); + } + + @Override + public int compareTo(SpoutSpec other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_spout_object()).compareTo(other.is_set_spout_object()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_spout_object()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.spout_object, other.spout_object); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_common()).compareTo(other.is_set_common()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_common()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.common, other.common); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("SpoutSpec("); + boolean first = true; + + sb.append("spout_object:"); + if (this.spout_object == null) { + sb.append("null"); + } else { + sb.append(this.spout_object); + } + first = false; + if (!first) sb.append(", "); + sb.append("common:"); + if (this.common == null) { + sb.append("null"); + } else { + sb.append(this.common); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_spout_object()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'spout_object' is unset! Struct:" + toString()); + } + + if (!is_set_common()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'common' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (common != null) { + common.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class SpoutSpecStandardSchemeFactory implements SchemeFactory { + public SpoutSpecStandardScheme getScheme() { + return new SpoutSpecStandardScheme(); + } + } + + private static class SpoutSpecStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, SpoutSpec struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SPOUT_OBJECT + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.spout_object = new ComponentObject(); + struct.spout_object.read(iprot); + struct.set_spout_object_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // COMMON + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.common = new ComponentCommon(); + struct.common.read(iprot); + struct.set_common_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, SpoutSpec struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.spout_object != null) { + oprot.writeFieldBegin(SPOUT_OBJECT_FIELD_DESC); + struct.spout_object.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.common != null) { + oprot.writeFieldBegin(COMMON_FIELD_DESC); + struct.common.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class SpoutSpecTupleSchemeFactory implements SchemeFactory { + public SpoutSpecTupleScheme getScheme() { + return new SpoutSpecTupleScheme(); + } + } + + private static class SpoutSpecTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, SpoutSpec struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.spout_object.write(oprot); + struct.common.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, SpoutSpec struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.spout_object = new ComponentObject(); + struct.spout_object.read(iprot); + struct.set_spout_object_isSet(true); + struct.common = new ComponentCommon(); + struct.common.read(iprot); + struct.set_common_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/StateSpoutSpec.java b/jstorm-core/src/main/java/backtype/storm/generated/StateSpoutSpec.java new file mode 100644 index 000000000..66cc7352e --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/StateSpoutSpec.java @@ -0,0 +1,497 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class StateSpoutSpec implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StateSpoutSpec"); + + private static final org.apache.thrift.protocol.TField STATE_SPOUT_OBJECT_FIELD_DESC = new org.apache.thrift.protocol.TField("state_spout_object", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField COMMON_FIELD_DESC = new org.apache.thrift.protocol.TField("common", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new StateSpoutSpecStandardSchemeFactory()); + schemes.put(TupleScheme.class, new StateSpoutSpecTupleSchemeFactory()); + } + + private ComponentObject state_spout_object; // required + private ComponentCommon common; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATE_SPOUT_OBJECT((short)1, "state_spout_object"), + COMMON((short)2, "common"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATE_SPOUT_OBJECT + return STATE_SPOUT_OBJECT; + case 2: // COMMON + return COMMON; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATE_SPOUT_OBJECT, new org.apache.thrift.meta_data.FieldMetaData("state_spout_object", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentObject.class))); + tmpMap.put(_Fields.COMMON, new org.apache.thrift.meta_data.FieldMetaData("common", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentCommon.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(StateSpoutSpec.class, metaDataMap); + } + + public StateSpoutSpec() { + } + + public StateSpoutSpec( + ComponentObject state_spout_object, + ComponentCommon common) + { + this(); + this.state_spout_object = state_spout_object; + this.common = common; + } + + /** + * Performs a deep copy on other. + */ + public StateSpoutSpec(StateSpoutSpec other) { + if (other.is_set_state_spout_object()) { + this.state_spout_object = new ComponentObject(other.state_spout_object); + } + if (other.is_set_common()) { + this.common = new ComponentCommon(other.common); + } + } + + public StateSpoutSpec deepCopy() { + return new StateSpoutSpec(this); + } + + @Override + public void clear() { + this.state_spout_object = null; + this.common = null; + } + + public ComponentObject get_state_spout_object() { + return this.state_spout_object; + } + + public void set_state_spout_object(ComponentObject state_spout_object) { + this.state_spout_object = state_spout_object; + } + + public void unset_state_spout_object() { + this.state_spout_object = null; + } + + /** Returns true if field state_spout_object is set (has been assigned a value) and false otherwise */ + public boolean is_set_state_spout_object() { + return this.state_spout_object != null; + } + + public void set_state_spout_object_isSet(boolean value) { + if (!value) { + this.state_spout_object = null; + } + } + + public ComponentCommon get_common() { + return this.common; + } + + public void set_common(ComponentCommon common) { + this.common = common; + } + + public void unset_common() { + this.common = null; + } + + /** Returns true if field common is set (has been assigned a value) and false otherwise */ + public boolean is_set_common() { + return this.common != null; + } + + public void set_common_isSet(boolean value) { + if (!value) { + this.common = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATE_SPOUT_OBJECT: + if (value == null) { + unset_state_spout_object(); + } else { + set_state_spout_object((ComponentObject)value); + } + break; + + case COMMON: + if (value == null) { + unset_common(); + } else { + set_common((ComponentCommon)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATE_SPOUT_OBJECT: + return get_state_spout_object(); + + case COMMON: + return get_common(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATE_SPOUT_OBJECT: + return is_set_state_spout_object(); + case COMMON: + return is_set_common(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof StateSpoutSpec) + return this.equals((StateSpoutSpec)that); + return false; + } + + public boolean equals(StateSpoutSpec that) { + if (that == null) + return false; + + boolean this_present_state_spout_object = true && this.is_set_state_spout_object(); + boolean that_present_state_spout_object = true && that.is_set_state_spout_object(); + if (this_present_state_spout_object || that_present_state_spout_object) { + if (!(this_present_state_spout_object && that_present_state_spout_object)) + return false; + if (!this.state_spout_object.equals(that.state_spout_object)) + return false; + } + + boolean this_present_common = true && this.is_set_common(); + boolean that_present_common = true && that.is_set_common(); + if (this_present_common || that_present_common) { + if (!(this_present_common && that_present_common)) + return false; + if (!this.common.equals(that.common)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_state_spout_object = true && (is_set_state_spout_object()); + list.add(present_state_spout_object); + if (present_state_spout_object) + list.add(state_spout_object); + + boolean present_common = true && (is_set_common()); + list.add(present_common); + if (present_common) + list.add(common); + + return list.hashCode(); + } + + @Override + public int compareTo(StateSpoutSpec other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_state_spout_object()).compareTo(other.is_set_state_spout_object()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_state_spout_object()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.state_spout_object, other.state_spout_object); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_common()).compareTo(other.is_set_common()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_common()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.common, other.common); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("StateSpoutSpec("); + boolean first = true; + + sb.append("state_spout_object:"); + if (this.state_spout_object == null) { + sb.append("null"); + } else { + sb.append(this.state_spout_object); + } + first = false; + if (!first) sb.append(", "); + sb.append("common:"); + if (this.common == null) { + sb.append("null"); + } else { + sb.append(this.common); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_state_spout_object()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'state_spout_object' is unset! Struct:" + toString()); + } + + if (!is_set_common()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'common' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (common != null) { + common.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class StateSpoutSpecStandardSchemeFactory implements SchemeFactory { + public StateSpoutSpecStandardScheme getScheme() { + return new StateSpoutSpecStandardScheme(); + } + } + + private static class StateSpoutSpecStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, StateSpoutSpec struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATE_SPOUT_OBJECT + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.state_spout_object = new ComponentObject(); + struct.state_spout_object.read(iprot); + struct.set_state_spout_object_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // COMMON + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.common = new ComponentCommon(); + struct.common.read(iprot); + struct.set_common_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, StateSpoutSpec struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.state_spout_object != null) { + oprot.writeFieldBegin(STATE_SPOUT_OBJECT_FIELD_DESC); + struct.state_spout_object.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.common != null) { + oprot.writeFieldBegin(COMMON_FIELD_DESC); + struct.common.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class StateSpoutSpecTupleSchemeFactory implements SchemeFactory { + public StateSpoutSpecTupleScheme getScheme() { + return new StateSpoutSpecTupleScheme(); + } + } + + private static class StateSpoutSpecTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, StateSpoutSpec struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.state_spout_object.write(oprot); + struct.common.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, StateSpoutSpec struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.state_spout_object = new ComponentObject(); + struct.state_spout_object.read(iprot); + struct.set_state_spout_object_isSet(true); + struct.common = new ComponentCommon(); + struct.common.read(iprot); + struct.set_common_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/StormTopology.java b/jstorm-core/src/main/java/backtype/storm/generated/StormTopology.java new file mode 100644 index 000000000..205f73d50 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/StormTopology.java @@ -0,0 +1,786 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class StormTopology implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StormTopology"); + + private static final org.apache.thrift.protocol.TField SPOUTS_FIELD_DESC = new org.apache.thrift.protocol.TField("spouts", org.apache.thrift.protocol.TType.MAP, (short)1); + private static final org.apache.thrift.protocol.TField BOLTS_FIELD_DESC = new org.apache.thrift.protocol.TField("bolts", org.apache.thrift.protocol.TType.MAP, (short)2); + private static final org.apache.thrift.protocol.TField STATE_SPOUTS_FIELD_DESC = new org.apache.thrift.protocol.TField("state_spouts", org.apache.thrift.protocol.TType.MAP, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new StormTopologyStandardSchemeFactory()); + schemes.put(TupleScheme.class, new StormTopologyTupleSchemeFactory()); + } + + private Map spouts; // required + private Map bolts; // required + private Map state_spouts; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SPOUTS((short)1, "spouts"), + BOLTS((short)2, "bolts"), + STATE_SPOUTS((short)3, "state_spouts"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SPOUTS + return SPOUTS; + case 2: // BOLTS + return BOLTS; + case 3: // STATE_SPOUTS + return STATE_SPOUTS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SPOUTS, new org.apache.thrift.meta_data.FieldMetaData("spouts", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SpoutSpec.class)))); + tmpMap.put(_Fields.BOLTS, new org.apache.thrift.meta_data.FieldMetaData("bolts", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Bolt.class)))); + tmpMap.put(_Fields.STATE_SPOUTS, new org.apache.thrift.meta_data.FieldMetaData("state_spouts", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, StateSpoutSpec.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(StormTopology.class, metaDataMap); + } + + public StormTopology() { + } + + public StormTopology( + Map spouts, + Map bolts, + Map state_spouts) + { + this(); + this.spouts = spouts; + this.bolts = bolts; + this.state_spouts = state_spouts; + } + + /** + * Performs a deep copy on other. + */ + public StormTopology(StormTopology other) { + if (other.is_set_spouts()) { + Map __this__spouts = new HashMap(other.spouts.size()); + for (Map.Entry other_element : other.spouts.entrySet()) { + + String other_element_key = other_element.getKey(); + SpoutSpec other_element_value = other_element.getValue(); + + String __this__spouts_copy_key = other_element_key; + + SpoutSpec __this__spouts_copy_value = new SpoutSpec(other_element_value); + + __this__spouts.put(__this__spouts_copy_key, __this__spouts_copy_value); + } + this.spouts = __this__spouts; + } + if (other.is_set_bolts()) { + Map __this__bolts = new HashMap(other.bolts.size()); + for (Map.Entry other_element : other.bolts.entrySet()) { + + String other_element_key = other_element.getKey(); + Bolt other_element_value = other_element.getValue(); + + String __this__bolts_copy_key = other_element_key; + + Bolt __this__bolts_copy_value = new Bolt(other_element_value); + + __this__bolts.put(__this__bolts_copy_key, __this__bolts_copy_value); + } + this.bolts = __this__bolts; + } + if (other.is_set_state_spouts()) { + Map __this__state_spouts = new HashMap(other.state_spouts.size()); + for (Map.Entry other_element : other.state_spouts.entrySet()) { + + String other_element_key = other_element.getKey(); + StateSpoutSpec other_element_value = other_element.getValue(); + + String __this__state_spouts_copy_key = other_element_key; + + StateSpoutSpec __this__state_spouts_copy_value = new StateSpoutSpec(other_element_value); + + __this__state_spouts.put(__this__state_spouts_copy_key, __this__state_spouts_copy_value); + } + this.state_spouts = __this__state_spouts; + } + } + + public StormTopology deepCopy() { + return new StormTopology(this); + } + + @Override + public void clear() { + this.spouts = null; + this.bolts = null; + this.state_spouts = null; + } + + public int get_spouts_size() { + return (this.spouts == null) ? 0 : this.spouts.size(); + } + + public void put_to_spouts(String key, SpoutSpec val) { + if (this.spouts == null) { + this.spouts = new HashMap(); + } + this.spouts.put(key, val); + } + + public Map get_spouts() { + return this.spouts; + } + + public void set_spouts(Map spouts) { + this.spouts = spouts; + } + + public void unset_spouts() { + this.spouts = null; + } + + /** Returns true if field spouts is set (has been assigned a value) and false otherwise */ + public boolean is_set_spouts() { + return this.spouts != null; + } + + public void set_spouts_isSet(boolean value) { + if (!value) { + this.spouts = null; + } + } + + public int get_bolts_size() { + return (this.bolts == null) ? 0 : this.bolts.size(); + } + + public void put_to_bolts(String key, Bolt val) { + if (this.bolts == null) { + this.bolts = new HashMap(); + } + this.bolts.put(key, val); + } + + public Map get_bolts() { + return this.bolts; + } + + public void set_bolts(Map bolts) { + this.bolts = bolts; + } + + public void unset_bolts() { + this.bolts = null; + } + + /** Returns true if field bolts is set (has been assigned a value) and false otherwise */ + public boolean is_set_bolts() { + return this.bolts != null; + } + + public void set_bolts_isSet(boolean value) { + if (!value) { + this.bolts = null; + } + } + + public int get_state_spouts_size() { + return (this.state_spouts == null) ? 0 : this.state_spouts.size(); + } + + public void put_to_state_spouts(String key, StateSpoutSpec val) { + if (this.state_spouts == null) { + this.state_spouts = new HashMap(); + } + this.state_spouts.put(key, val); + } + + public Map get_state_spouts() { + return this.state_spouts; + } + + public void set_state_spouts(Map state_spouts) { + this.state_spouts = state_spouts; + } + + public void unset_state_spouts() { + this.state_spouts = null; + } + + /** Returns true if field state_spouts is set (has been assigned a value) and false otherwise */ + public boolean is_set_state_spouts() { + return this.state_spouts != null; + } + + public void set_state_spouts_isSet(boolean value) { + if (!value) { + this.state_spouts = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SPOUTS: + if (value == null) { + unset_spouts(); + } else { + set_spouts((Map)value); + } + break; + + case BOLTS: + if (value == null) { + unset_bolts(); + } else { + set_bolts((Map)value); + } + break; + + case STATE_SPOUTS: + if (value == null) { + unset_state_spouts(); + } else { + set_state_spouts((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SPOUTS: + return get_spouts(); + + case BOLTS: + return get_bolts(); + + case STATE_SPOUTS: + return get_state_spouts(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SPOUTS: + return is_set_spouts(); + case BOLTS: + return is_set_bolts(); + case STATE_SPOUTS: + return is_set_state_spouts(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof StormTopology) + return this.equals((StormTopology)that); + return false; + } + + public boolean equals(StormTopology that) { + if (that == null) + return false; + + boolean this_present_spouts = true && this.is_set_spouts(); + boolean that_present_spouts = true && that.is_set_spouts(); + if (this_present_spouts || that_present_spouts) { + if (!(this_present_spouts && that_present_spouts)) + return false; + if (!this.spouts.equals(that.spouts)) + return false; + } + + boolean this_present_bolts = true && this.is_set_bolts(); + boolean that_present_bolts = true && that.is_set_bolts(); + if (this_present_bolts || that_present_bolts) { + if (!(this_present_bolts && that_present_bolts)) + return false; + if (!this.bolts.equals(that.bolts)) + return false; + } + + boolean this_present_state_spouts = true && this.is_set_state_spouts(); + boolean that_present_state_spouts = true && that.is_set_state_spouts(); + if (this_present_state_spouts || that_present_state_spouts) { + if (!(this_present_state_spouts && that_present_state_spouts)) + return false; + if (!this.state_spouts.equals(that.state_spouts)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_spouts = true && (is_set_spouts()); + list.add(present_spouts); + if (present_spouts) + list.add(spouts); + + boolean present_bolts = true && (is_set_bolts()); + list.add(present_bolts); + if (present_bolts) + list.add(bolts); + + boolean present_state_spouts = true && (is_set_state_spouts()); + list.add(present_state_spouts); + if (present_state_spouts) + list.add(state_spouts); + + return list.hashCode(); + } + + @Override + public int compareTo(StormTopology other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_spouts()).compareTo(other.is_set_spouts()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_spouts()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.spouts, other.spouts); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_bolts()).compareTo(other.is_set_bolts()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_bolts()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.bolts, other.bolts); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_state_spouts()).compareTo(other.is_set_state_spouts()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_state_spouts()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.state_spouts, other.state_spouts); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("StormTopology("); + boolean first = true; + + sb.append("spouts:"); + if (this.spouts == null) { + sb.append("null"); + } else { + sb.append(this.spouts); + } + first = false; + if (!first) sb.append(", "); + sb.append("bolts:"); + if (this.bolts == null) { + sb.append("null"); + } else { + sb.append(this.bolts); + } + first = false; + if (!first) sb.append(", "); + sb.append("state_spouts:"); + if (this.state_spouts == null) { + sb.append("null"); + } else { + sb.append(this.state_spouts); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_spouts()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'spouts' is unset! Struct:" + toString()); + } + + if (!is_set_bolts()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'bolts' is unset! Struct:" + toString()); + } + + if (!is_set_state_spouts()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'state_spouts' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class StormTopologyStandardSchemeFactory implements SchemeFactory { + public StormTopologyStandardScheme getScheme() { + return new StormTopologyStandardScheme(); + } + } + + private static class StormTopologyStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, StormTopology struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SPOUTS + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map44 = iprot.readMapBegin(); + struct.spouts = new HashMap(2*_map44.size); + String _key45; + SpoutSpec _val46; + for (int _i47 = 0; _i47 < _map44.size; ++_i47) + { + _key45 = iprot.readString(); + _val46 = new SpoutSpec(); + _val46.read(iprot); + struct.spouts.put(_key45, _val46); + } + iprot.readMapEnd(); + } + struct.set_spouts_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // BOLTS + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map48 = iprot.readMapBegin(); + struct.bolts = new HashMap(2*_map48.size); + String _key49; + Bolt _val50; + for (int _i51 = 0; _i51 < _map48.size; ++_i51) + { + _key49 = iprot.readString(); + _val50 = new Bolt(); + _val50.read(iprot); + struct.bolts.put(_key49, _val50); + } + iprot.readMapEnd(); + } + struct.set_bolts_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // STATE_SPOUTS + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map52 = iprot.readMapBegin(); + struct.state_spouts = new HashMap(2*_map52.size); + String _key53; + StateSpoutSpec _val54; + for (int _i55 = 0; _i55 < _map52.size; ++_i55) + { + _key53 = iprot.readString(); + _val54 = new StateSpoutSpec(); + _val54.read(iprot); + struct.state_spouts.put(_key53, _val54); + } + iprot.readMapEnd(); + } + struct.set_state_spouts_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, StormTopology struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.spouts != null) { + oprot.writeFieldBegin(SPOUTS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.spouts.size())); + for (Map.Entry _iter56 : struct.spouts.entrySet()) + { + oprot.writeString(_iter56.getKey()); + _iter56.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.bolts != null) { + oprot.writeFieldBegin(BOLTS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.bolts.size())); + for (Map.Entry _iter57 : struct.bolts.entrySet()) + { + oprot.writeString(_iter57.getKey()); + _iter57.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.state_spouts != null) { + oprot.writeFieldBegin(STATE_SPOUTS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.state_spouts.size())); + for (Map.Entry _iter58 : struct.state_spouts.entrySet()) + { + oprot.writeString(_iter58.getKey()); + _iter58.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class StormTopologyTupleSchemeFactory implements SchemeFactory { + public StormTopologyTupleScheme getScheme() { + return new StormTopologyTupleScheme(); + } + } + + private static class StormTopologyTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, StormTopology struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.spouts.size()); + for (Map.Entry _iter59 : struct.spouts.entrySet()) + { + oprot.writeString(_iter59.getKey()); + _iter59.getValue().write(oprot); + } + } + { + oprot.writeI32(struct.bolts.size()); + for (Map.Entry _iter60 : struct.bolts.entrySet()) + { + oprot.writeString(_iter60.getKey()); + _iter60.getValue().write(oprot); + } + } + { + oprot.writeI32(struct.state_spouts.size()); + for (Map.Entry _iter61 : struct.state_spouts.entrySet()) + { + oprot.writeString(_iter61.getKey()); + _iter61.getValue().write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, StormTopology struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TMap _map62 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.spouts = new HashMap(2*_map62.size); + String _key63; + SpoutSpec _val64; + for (int _i65 = 0; _i65 < _map62.size; ++_i65) + { + _key63 = iprot.readString(); + _val64 = new SpoutSpec(); + _val64.read(iprot); + struct.spouts.put(_key63, _val64); + } + } + struct.set_spouts_isSet(true); + { + org.apache.thrift.protocol.TMap _map66 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.bolts = new HashMap(2*_map66.size); + String _key67; + Bolt _val68; + for (int _i69 = 0; _i69 < _map66.size; ++_i69) + { + _key67 = iprot.readString(); + _val68 = new Bolt(); + _val68.read(iprot); + struct.bolts.put(_key67, _val68); + } + } + struct.set_bolts_isSet(true); + { + org.apache.thrift.protocol.TMap _map70 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.state_spouts = new HashMap(2*_map70.size); + String _key71; + StateSpoutSpec _val72; + for (int _i73 = 0; _i73 < _map70.size; ++_i73) + { + _key71 = iprot.readString(); + _val72 = new StateSpoutSpec(); + _val72.read(iprot); + struct.state_spouts.put(_key71, _val72); + } + } + struct.set_state_spouts_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/StreamInfo.java b/jstorm-core/src/main/java/backtype/storm/generated/StreamInfo.java new file mode 100644 index 000000000..abed2eac9 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/StreamInfo.java @@ -0,0 +1,537 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class StreamInfo implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("StreamInfo"); + + private static final org.apache.thrift.protocol.TField OUTPUT_FIELDS_FIELD_DESC = new org.apache.thrift.protocol.TField("output_fields", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField DIRECT_FIELD_DESC = new org.apache.thrift.protocol.TField("direct", org.apache.thrift.protocol.TType.BOOL, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new StreamInfoStandardSchemeFactory()); + schemes.put(TupleScheme.class, new StreamInfoTupleSchemeFactory()); + } + + private List output_fields; // required + private boolean direct; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + OUTPUT_FIELDS((short)1, "output_fields"), + DIRECT((short)2, "direct"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // OUTPUT_FIELDS + return OUTPUT_FIELDS; + case 2: // DIRECT + return DIRECT; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __DIRECT_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.OUTPUT_FIELDS, new org.apache.thrift.meta_data.FieldMetaData("output_fields", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + tmpMap.put(_Fields.DIRECT, new org.apache.thrift.meta_data.FieldMetaData("direct", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(StreamInfo.class, metaDataMap); + } + + public StreamInfo() { + } + + public StreamInfo( + List output_fields, + boolean direct) + { + this(); + this.output_fields = output_fields; + this.direct = direct; + set_direct_isSet(true); + } + + /** + * Performs a deep copy on other. + */ + public StreamInfo(StreamInfo other) { + __isset_bitfield = other.__isset_bitfield; + if (other.is_set_output_fields()) { + List __this__output_fields = new ArrayList(other.output_fields); + this.output_fields = __this__output_fields; + } + this.direct = other.direct; + } + + public StreamInfo deepCopy() { + return new StreamInfo(this); + } + + @Override + public void clear() { + this.output_fields = null; + set_direct_isSet(false); + this.direct = false; + } + + public int get_output_fields_size() { + return (this.output_fields == null) ? 0 : this.output_fields.size(); + } + + public java.util.Iterator get_output_fields_iterator() { + return (this.output_fields == null) ? null : this.output_fields.iterator(); + } + + public void add_to_output_fields(String elem) { + if (this.output_fields == null) { + this.output_fields = new ArrayList(); + } + this.output_fields.add(elem); + } + + public List get_output_fields() { + return this.output_fields; + } + + public void set_output_fields(List output_fields) { + this.output_fields = output_fields; + } + + public void unset_output_fields() { + this.output_fields = null; + } + + /** Returns true if field output_fields is set (has been assigned a value) and false otherwise */ + public boolean is_set_output_fields() { + return this.output_fields != null; + } + + public void set_output_fields_isSet(boolean value) { + if (!value) { + this.output_fields = null; + } + } + + public boolean is_direct() { + return this.direct; + } + + public void set_direct(boolean direct) { + this.direct = direct; + set_direct_isSet(true); + } + + public void unset_direct() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __DIRECT_ISSET_ID); + } + + /** Returns true if field direct is set (has been assigned a value) and false otherwise */ + public boolean is_set_direct() { + return EncodingUtils.testBit(__isset_bitfield, __DIRECT_ISSET_ID); + } + + public void set_direct_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __DIRECT_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case OUTPUT_FIELDS: + if (value == null) { + unset_output_fields(); + } else { + set_output_fields((List)value); + } + break; + + case DIRECT: + if (value == null) { + unset_direct(); + } else { + set_direct((Boolean)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case OUTPUT_FIELDS: + return get_output_fields(); + + case DIRECT: + return Boolean.valueOf(is_direct()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case OUTPUT_FIELDS: + return is_set_output_fields(); + case DIRECT: + return is_set_direct(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof StreamInfo) + return this.equals((StreamInfo)that); + return false; + } + + public boolean equals(StreamInfo that) { + if (that == null) + return false; + + boolean this_present_output_fields = true && this.is_set_output_fields(); + boolean that_present_output_fields = true && that.is_set_output_fields(); + if (this_present_output_fields || that_present_output_fields) { + if (!(this_present_output_fields && that_present_output_fields)) + return false; + if (!this.output_fields.equals(that.output_fields)) + return false; + } + + boolean this_present_direct = true; + boolean that_present_direct = true; + if (this_present_direct || that_present_direct) { + if (!(this_present_direct && that_present_direct)) + return false; + if (this.direct != that.direct) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_output_fields = true && (is_set_output_fields()); + list.add(present_output_fields); + if (present_output_fields) + list.add(output_fields); + + boolean present_direct = true; + list.add(present_direct); + if (present_direct) + list.add(direct); + + return list.hashCode(); + } + + @Override + public int compareTo(StreamInfo other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_output_fields()).compareTo(other.is_set_output_fields()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_output_fields()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.output_fields, other.output_fields); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_direct()).compareTo(other.is_set_direct()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_direct()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.direct, other.direct); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("StreamInfo("); + boolean first = true; + + sb.append("output_fields:"); + if (this.output_fields == null) { + sb.append("null"); + } else { + sb.append(this.output_fields); + } + first = false; + if (!first) sb.append(", "); + sb.append("direct:"); + sb.append(this.direct); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_output_fields()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'output_fields' is unset! Struct:" + toString()); + } + + if (!is_set_direct()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'direct' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class StreamInfoStandardSchemeFactory implements SchemeFactory { + public StreamInfoStandardScheme getScheme() { + return new StreamInfoStandardScheme(); + } + } + + private static class StreamInfoStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, StreamInfo struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // OUTPUT_FIELDS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list16 = iprot.readListBegin(); + struct.output_fields = new ArrayList(_list16.size); + String _elem17; + for (int _i18 = 0; _i18 < _list16.size; ++_i18) + { + _elem17 = iprot.readString(); + struct.output_fields.add(_elem17); + } + iprot.readListEnd(); + } + struct.set_output_fields_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // DIRECT + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.direct = iprot.readBool(); + struct.set_direct_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, StreamInfo struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.output_fields != null) { + oprot.writeFieldBegin(OUTPUT_FIELDS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.output_fields.size())); + for (String _iter19 : struct.output_fields) + { + oprot.writeString(_iter19); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(DIRECT_FIELD_DESC); + oprot.writeBool(struct.direct); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class StreamInfoTupleSchemeFactory implements SchemeFactory { + public StreamInfoTupleScheme getScheme() { + return new StreamInfoTupleScheme(); + } + } + + private static class StreamInfoTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, StreamInfo struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.output_fields.size()); + for (String _iter20 : struct.output_fields) + { + oprot.writeString(_iter20); + } + } + oprot.writeBool(struct.direct); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, StreamInfo struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list21 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.output_fields = new ArrayList(_list21.size); + String _elem22; + for (int _i23 = 0; _i23 < _list21.size; ++_i23) + { + _elem22 = iprot.readString(); + struct.output_fields.add(_elem22); + } + } + struct.set_output_fields_isSet(true); + struct.direct = iprot.readBool(); + struct.set_direct_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/SubmitOptions.java b/jstorm-core/src/main/java/backtype/storm/generated/SubmitOptions.java new file mode 100644 index 000000000..b4343abf7 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/SubmitOptions.java @@ -0,0 +1,401 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class SubmitOptions implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SubmitOptions"); + + private static final org.apache.thrift.protocol.TField INITIAL_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("initial_status", org.apache.thrift.protocol.TType.I32, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new SubmitOptionsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new SubmitOptionsTupleSchemeFactory()); + } + + private TopologyInitialStatus initial_status; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + /** + * + * @see TopologyInitialStatus + */ + INITIAL_STATUS((short)1, "initial_status"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // INITIAL_STATUS + return INITIAL_STATUS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.INITIAL_STATUS, new org.apache.thrift.meta_data.FieldMetaData("initial_status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TopologyInitialStatus.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SubmitOptions.class, metaDataMap); + } + + public SubmitOptions() { + } + + public SubmitOptions( + TopologyInitialStatus initial_status) + { + this(); + this.initial_status = initial_status; + } + + /** + * Performs a deep copy on other. + */ + public SubmitOptions(SubmitOptions other) { + if (other.is_set_initial_status()) { + this.initial_status = other.initial_status; + } + } + + public SubmitOptions deepCopy() { + return new SubmitOptions(this); + } + + @Override + public void clear() { + this.initial_status = null; + } + + /** + * + * @see TopologyInitialStatus + */ + public TopologyInitialStatus get_initial_status() { + return this.initial_status; + } + + /** + * + * @see TopologyInitialStatus + */ + public void set_initial_status(TopologyInitialStatus initial_status) { + this.initial_status = initial_status; + } + + public void unset_initial_status() { + this.initial_status = null; + } + + /** Returns true if field initial_status is set (has been assigned a value) and false otherwise */ + public boolean is_set_initial_status() { + return this.initial_status != null; + } + + public void set_initial_status_isSet(boolean value) { + if (!value) { + this.initial_status = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case INITIAL_STATUS: + if (value == null) { + unset_initial_status(); + } else { + set_initial_status((TopologyInitialStatus)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case INITIAL_STATUS: + return get_initial_status(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case INITIAL_STATUS: + return is_set_initial_status(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof SubmitOptions) + return this.equals((SubmitOptions)that); + return false; + } + + public boolean equals(SubmitOptions that) { + if (that == null) + return false; + + boolean this_present_initial_status = true && this.is_set_initial_status(); + boolean that_present_initial_status = true && that.is_set_initial_status(); + if (this_present_initial_status || that_present_initial_status) { + if (!(this_present_initial_status && that_present_initial_status)) + return false; + if (!this.initial_status.equals(that.initial_status)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_initial_status = true && (is_set_initial_status()); + list.add(present_initial_status); + if (present_initial_status) + list.add(initial_status.getValue()); + + return list.hashCode(); + } + + @Override + public int compareTo(SubmitOptions other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_initial_status()).compareTo(other.is_set_initial_status()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_initial_status()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.initial_status, other.initial_status); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("SubmitOptions("); + boolean first = true; + + sb.append("initial_status:"); + if (this.initial_status == null) { + sb.append("null"); + } else { + sb.append(this.initial_status); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_initial_status()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'initial_status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class SubmitOptionsStandardSchemeFactory implements SchemeFactory { + public SubmitOptionsStandardScheme getScheme() { + return new SubmitOptionsStandardScheme(); + } + } + + private static class SubmitOptionsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, SubmitOptions struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // INITIAL_STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.initial_status = backtype.storm.generated.TopologyInitialStatus.findByValue(iprot.readI32()); + struct.set_initial_status_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, SubmitOptions struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.initial_status != null) { + oprot.writeFieldBegin(INITIAL_STATUS_FIELD_DESC); + oprot.writeI32(struct.initial_status.getValue()); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class SubmitOptionsTupleSchemeFactory implements SchemeFactory { + public SubmitOptionsTupleScheme getScheme() { + return new SubmitOptionsTupleScheme(); + } + } + + private static class SubmitOptionsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, SubmitOptions struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI32(struct.initial_status.getValue()); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, SubmitOptions struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.initial_status = backtype.storm.generated.TopologyInitialStatus.findByValue(iprot.readI32()); + struct.set_initial_status_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/SupervisorSummary.java b/jstorm-core/src/main/java/backtype/storm/generated/SupervisorSummary.java new file mode 100644 index 000000000..903dab0a0 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/SupervisorSummary.java @@ -0,0 +1,779 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class SupervisorSummary implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SupervisorSummary"); + + private static final org.apache.thrift.protocol.TField HOST_FIELD_DESC = new org.apache.thrift.protocol.TField("host", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField SUPERVISOR_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("supervisor_id", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime_secs", org.apache.thrift.protocol.TType.I32, (short)3); + private static final org.apache.thrift.protocol.TField NUM_WORKERS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_workers", org.apache.thrift.protocol.TType.I32, (short)4); + private static final org.apache.thrift.protocol.TField NUM_USED_WORKERS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_used_workers", org.apache.thrift.protocol.TType.I32, (short)5); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new SupervisorSummaryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new SupervisorSummaryTupleSchemeFactory()); + } + + private String host; // required + private String supervisor_id; // required + private int uptime_secs; // required + private int num_workers; // required + private int num_used_workers; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + HOST((short)1, "host"), + SUPERVISOR_ID((short)2, "supervisor_id"), + UPTIME_SECS((short)3, "uptime_secs"), + NUM_WORKERS((short)4, "num_workers"), + NUM_USED_WORKERS((short)5, "num_used_workers"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // HOST + return HOST; + case 2: // SUPERVISOR_ID + return SUPERVISOR_ID; + case 3: // UPTIME_SECS + return UPTIME_SECS; + case 4: // NUM_WORKERS + return NUM_WORKERS; + case 5: // NUM_USED_WORKERS + return NUM_USED_WORKERS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __UPTIME_SECS_ISSET_ID = 0; + private static final int __NUM_WORKERS_ISSET_ID = 1; + private static final int __NUM_USED_WORKERS_ISSET_ID = 2; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.HOST, new org.apache.thrift.meta_data.FieldMetaData("host", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.SUPERVISOR_ID, new org.apache.thrift.meta_data.FieldMetaData("supervisor_id", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("uptime_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.NUM_WORKERS, new org.apache.thrift.meta_data.FieldMetaData("num_workers", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.NUM_USED_WORKERS, new org.apache.thrift.meta_data.FieldMetaData("num_used_workers", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SupervisorSummary.class, metaDataMap); + } + + public SupervisorSummary() { + } + + public SupervisorSummary( + String host, + String supervisor_id, + int uptime_secs, + int num_workers, + int num_used_workers) + { + this(); + this.host = host; + this.supervisor_id = supervisor_id; + this.uptime_secs = uptime_secs; + set_uptime_secs_isSet(true); + this.num_workers = num_workers; + set_num_workers_isSet(true); + this.num_used_workers = num_used_workers; + set_num_used_workers_isSet(true); + } + + /** + * Performs a deep copy on other. + */ + public SupervisorSummary(SupervisorSummary other) { + __isset_bitfield = other.__isset_bitfield; + if (other.is_set_host()) { + this.host = other.host; + } + if (other.is_set_supervisor_id()) { + this.supervisor_id = other.supervisor_id; + } + this.uptime_secs = other.uptime_secs; + this.num_workers = other.num_workers; + this.num_used_workers = other.num_used_workers; + } + + public SupervisorSummary deepCopy() { + return new SupervisorSummary(this); + } + + @Override + public void clear() { + this.host = null; + this.supervisor_id = null; + set_uptime_secs_isSet(false); + this.uptime_secs = 0; + set_num_workers_isSet(false); + this.num_workers = 0; + set_num_used_workers_isSet(false); + this.num_used_workers = 0; + } + + public String get_host() { + return this.host; + } + + public void set_host(String host) { + this.host = host; + } + + public void unset_host() { + this.host = null; + } + + /** Returns true if field host is set (has been assigned a value) and false otherwise */ + public boolean is_set_host() { + return this.host != null; + } + + public void set_host_isSet(boolean value) { + if (!value) { + this.host = null; + } + } + + public String get_supervisor_id() { + return this.supervisor_id; + } + + public void set_supervisor_id(String supervisor_id) { + this.supervisor_id = supervisor_id; + } + + public void unset_supervisor_id() { + this.supervisor_id = null; + } + + /** Returns true if field supervisor_id is set (has been assigned a value) and false otherwise */ + public boolean is_set_supervisor_id() { + return this.supervisor_id != null; + } + + public void set_supervisor_id_isSet(boolean value) { + if (!value) { + this.supervisor_id = null; + } + } + + public int get_uptime_secs() { + return this.uptime_secs; + } + + public void set_uptime_secs(int uptime_secs) { + this.uptime_secs = uptime_secs; + set_uptime_secs_isSet(true); + } + + public void unset_uptime_secs() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID); + } + + /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */ + public boolean is_set_uptime_secs() { + return EncodingUtils.testBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID); + } + + public void set_uptime_secs_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID, value); + } + + public int get_num_workers() { + return this.num_workers; + } + + public void set_num_workers(int num_workers) { + this.num_workers = num_workers; + set_num_workers_isSet(true); + } + + public void unset_num_workers() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID); + } + + /** Returns true if field num_workers is set (has been assigned a value) and false otherwise */ + public boolean is_set_num_workers() { + return EncodingUtils.testBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID); + } + + public void set_num_workers_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID, value); + } + + public int get_num_used_workers() { + return this.num_used_workers; + } + + public void set_num_used_workers(int num_used_workers) { + this.num_used_workers = num_used_workers; + set_num_used_workers_isSet(true); + } + + public void unset_num_used_workers() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_USED_WORKERS_ISSET_ID); + } + + /** Returns true if field num_used_workers is set (has been assigned a value) and false otherwise */ + public boolean is_set_num_used_workers() { + return EncodingUtils.testBit(__isset_bitfield, __NUM_USED_WORKERS_ISSET_ID); + } + + public void set_num_used_workers_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_USED_WORKERS_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case HOST: + if (value == null) { + unset_host(); + } else { + set_host((String)value); + } + break; + + case SUPERVISOR_ID: + if (value == null) { + unset_supervisor_id(); + } else { + set_supervisor_id((String)value); + } + break; + + case UPTIME_SECS: + if (value == null) { + unset_uptime_secs(); + } else { + set_uptime_secs((Integer)value); + } + break; + + case NUM_WORKERS: + if (value == null) { + unset_num_workers(); + } else { + set_num_workers((Integer)value); + } + break; + + case NUM_USED_WORKERS: + if (value == null) { + unset_num_used_workers(); + } else { + set_num_used_workers((Integer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case HOST: + return get_host(); + + case SUPERVISOR_ID: + return get_supervisor_id(); + + case UPTIME_SECS: + return Integer.valueOf(get_uptime_secs()); + + case NUM_WORKERS: + return Integer.valueOf(get_num_workers()); + + case NUM_USED_WORKERS: + return Integer.valueOf(get_num_used_workers()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case HOST: + return is_set_host(); + case SUPERVISOR_ID: + return is_set_supervisor_id(); + case UPTIME_SECS: + return is_set_uptime_secs(); + case NUM_WORKERS: + return is_set_num_workers(); + case NUM_USED_WORKERS: + return is_set_num_used_workers(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof SupervisorSummary) + return this.equals((SupervisorSummary)that); + return false; + } + + public boolean equals(SupervisorSummary that) { + if (that == null) + return false; + + boolean this_present_host = true && this.is_set_host(); + boolean that_present_host = true && that.is_set_host(); + if (this_present_host || that_present_host) { + if (!(this_present_host && that_present_host)) + return false; + if (!this.host.equals(that.host)) + return false; + } + + boolean this_present_supervisor_id = true && this.is_set_supervisor_id(); + boolean that_present_supervisor_id = true && that.is_set_supervisor_id(); + if (this_present_supervisor_id || that_present_supervisor_id) { + if (!(this_present_supervisor_id && that_present_supervisor_id)) + return false; + if (!this.supervisor_id.equals(that.supervisor_id)) + return false; + } + + boolean this_present_uptime_secs = true; + boolean that_present_uptime_secs = true; + if (this_present_uptime_secs || that_present_uptime_secs) { + if (!(this_present_uptime_secs && that_present_uptime_secs)) + return false; + if (this.uptime_secs != that.uptime_secs) + return false; + } + + boolean this_present_num_workers = true; + boolean that_present_num_workers = true; + if (this_present_num_workers || that_present_num_workers) { + if (!(this_present_num_workers && that_present_num_workers)) + return false; + if (this.num_workers != that.num_workers) + return false; + } + + boolean this_present_num_used_workers = true; + boolean that_present_num_used_workers = true; + if (this_present_num_used_workers || that_present_num_used_workers) { + if (!(this_present_num_used_workers && that_present_num_used_workers)) + return false; + if (this.num_used_workers != that.num_used_workers) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_host = true && (is_set_host()); + list.add(present_host); + if (present_host) + list.add(host); + + boolean present_supervisor_id = true && (is_set_supervisor_id()); + list.add(present_supervisor_id); + if (present_supervisor_id) + list.add(supervisor_id); + + boolean present_uptime_secs = true; + list.add(present_uptime_secs); + if (present_uptime_secs) + list.add(uptime_secs); + + boolean present_num_workers = true; + list.add(present_num_workers); + if (present_num_workers) + list.add(num_workers); + + boolean present_num_used_workers = true; + list.add(present_num_used_workers); + if (present_num_used_workers) + list.add(num_used_workers); + + return list.hashCode(); + } + + @Override + public int compareTo(SupervisorSummary other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_host()).compareTo(other.is_set_host()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_host()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.host, other.host); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_supervisor_id()).compareTo(other.is_set_supervisor_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_supervisor_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.supervisor_id, other.supervisor_id); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(other.is_set_uptime_secs()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_uptime_secs()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, other.uptime_secs); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_num_workers()).compareTo(other.is_set_num_workers()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_num_workers()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_workers, other.num_workers); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_num_used_workers()).compareTo(other.is_set_num_used_workers()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_num_used_workers()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_used_workers, other.num_used_workers); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("SupervisorSummary("); + boolean first = true; + + sb.append("host:"); + if (this.host == null) { + sb.append("null"); + } else { + sb.append(this.host); + } + first = false; + if (!first) sb.append(", "); + sb.append("supervisor_id:"); + if (this.supervisor_id == null) { + sb.append("null"); + } else { + sb.append(this.supervisor_id); + } + first = false; + if (!first) sb.append(", "); + sb.append("uptime_secs:"); + sb.append(this.uptime_secs); + first = false; + if (!first) sb.append(", "); + sb.append("num_workers:"); + sb.append(this.num_workers); + first = false; + if (!first) sb.append(", "); + sb.append("num_used_workers:"); + sb.append(this.num_used_workers); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_host()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'host' is unset! Struct:" + toString()); + } + + if (!is_set_supervisor_id()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'supervisor_id' is unset! Struct:" + toString()); + } + + if (!is_set_uptime_secs()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString()); + } + + if (!is_set_num_workers()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'num_workers' is unset! Struct:" + toString()); + } + + if (!is_set_num_used_workers()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'num_used_workers' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class SupervisorSummaryStandardSchemeFactory implements SchemeFactory { + public SupervisorSummaryStandardScheme getScheme() { + return new SupervisorSummaryStandardScheme(); + } + } + + private static class SupervisorSummaryStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorSummary struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // HOST + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.host = iprot.readString(); + struct.set_host_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // SUPERVISOR_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.supervisor_id = iprot.readString(); + struct.set_supervisor_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // UPTIME_SECS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.uptime_secs = iprot.readI32(); + struct.set_uptime_secs_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // NUM_WORKERS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.num_workers = iprot.readI32(); + struct.set_num_workers_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // NUM_USED_WORKERS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.num_used_workers = iprot.readI32(); + struct.set_num_used_workers_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, SupervisorSummary struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.host != null) { + oprot.writeFieldBegin(HOST_FIELD_DESC); + oprot.writeString(struct.host); + oprot.writeFieldEnd(); + } + if (struct.supervisor_id != null) { + oprot.writeFieldBegin(SUPERVISOR_ID_FIELD_DESC); + oprot.writeString(struct.supervisor_id); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC); + oprot.writeI32(struct.uptime_secs); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC); + oprot.writeI32(struct.num_workers); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(NUM_USED_WORKERS_FIELD_DESC); + oprot.writeI32(struct.num_used_workers); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class SupervisorSummaryTupleSchemeFactory implements SchemeFactory { + public SupervisorSummaryTupleScheme getScheme() { + return new SupervisorSummaryTupleScheme(); + } + } + + private static class SupervisorSummaryTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, SupervisorSummary struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.host); + oprot.writeString(struct.supervisor_id); + oprot.writeI32(struct.uptime_secs); + oprot.writeI32(struct.num_workers); + oprot.writeI32(struct.num_used_workers); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, SupervisorSummary struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.host = iprot.readString(); + struct.set_host_isSet(true); + struct.supervisor_id = iprot.readString(); + struct.set_supervisor_id_isSet(true); + struct.uptime_secs = iprot.readI32(); + struct.set_uptime_secs_isSet(true); + struct.num_workers = iprot.readI32(); + struct.set_num_workers_isSet(true); + struct.num_used_workers = iprot.readI32(); + struct.set_num_used_workers_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/SupervisorWorkers.java b/jstorm-core/src/main/java/backtype/storm/generated/SupervisorWorkers.java new file mode 100644 index 000000000..93ec504df --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/SupervisorWorkers.java @@ -0,0 +1,715 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class SupervisorWorkers implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SupervisorWorkers"); + + private static final org.apache.thrift.protocol.TField SUPERVISOR_FIELD_DESC = new org.apache.thrift.protocol.TField("supervisor", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField WORKERS_FIELD_DESC = new org.apache.thrift.protocol.TField("workers", org.apache.thrift.protocol.TType.LIST, (short)2); + private static final org.apache.thrift.protocol.TField WORKER_METRIC_FIELD_DESC = new org.apache.thrift.protocol.TField("workerMetric", org.apache.thrift.protocol.TType.MAP, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new SupervisorWorkersStandardSchemeFactory()); + schemes.put(TupleScheme.class, new SupervisorWorkersTupleSchemeFactory()); + } + + private SupervisorSummary supervisor; // required + private List workers; // required + private Map workerMetric; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUPERVISOR((short)1, "supervisor"), + WORKERS((short)2, "workers"), + WORKER_METRIC((short)3, "workerMetric"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SUPERVISOR + return SUPERVISOR; + case 2: // WORKERS + return WORKERS; + case 3: // WORKER_METRIC + return WORKER_METRIC; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUPERVISOR, new org.apache.thrift.meta_data.FieldMetaData("supervisor", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, SupervisorSummary.class))); + tmpMap.put(_Fields.WORKERS, new org.apache.thrift.meta_data.FieldMetaData("workers", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, WorkerSummary.class)))); + tmpMap.put(_Fields.WORKER_METRIC, new org.apache.thrift.meta_data.FieldMetaData("workerMetric", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, MetricInfo.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SupervisorWorkers.class, metaDataMap); + } + + public SupervisorWorkers() { + } + + public SupervisorWorkers( + SupervisorSummary supervisor, + List workers, + Map workerMetric) + { + this(); + this.supervisor = supervisor; + this.workers = workers; + this.workerMetric = workerMetric; + } + + /** + * Performs a deep copy on other. + */ + public SupervisorWorkers(SupervisorWorkers other) { + if (other.is_set_supervisor()) { + this.supervisor = new SupervisorSummary(other.supervisor); + } + if (other.is_set_workers()) { + List __this__workers = new ArrayList(other.workers.size()); + for (WorkerSummary other_element : other.workers) { + __this__workers.add(new WorkerSummary(other_element)); + } + this.workers = __this__workers; + } + if (other.is_set_workerMetric()) { + Map __this__workerMetric = new HashMap(other.workerMetric.size()); + for (Map.Entry other_element : other.workerMetric.entrySet()) { + + String other_element_key = other_element.getKey(); + MetricInfo other_element_value = other_element.getValue(); + + String __this__workerMetric_copy_key = other_element_key; + + MetricInfo __this__workerMetric_copy_value = new MetricInfo(other_element_value); + + __this__workerMetric.put(__this__workerMetric_copy_key, __this__workerMetric_copy_value); + } + this.workerMetric = __this__workerMetric; + } + } + + public SupervisorWorkers deepCopy() { + return new SupervisorWorkers(this); + } + + @Override + public void clear() { + this.supervisor = null; + this.workers = null; + this.workerMetric = null; + } + + public SupervisorSummary get_supervisor() { + return this.supervisor; + } + + public void set_supervisor(SupervisorSummary supervisor) { + this.supervisor = supervisor; + } + + public void unset_supervisor() { + this.supervisor = null; + } + + /** Returns true if field supervisor is set (has been assigned a value) and false otherwise */ + public boolean is_set_supervisor() { + return this.supervisor != null; + } + + public void set_supervisor_isSet(boolean value) { + if (!value) { + this.supervisor = null; + } + } + + public int get_workers_size() { + return (this.workers == null) ? 0 : this.workers.size(); + } + + public java.util.Iterator get_workers_iterator() { + return (this.workers == null) ? null : this.workers.iterator(); + } + + public void add_to_workers(WorkerSummary elem) { + if (this.workers == null) { + this.workers = new ArrayList(); + } + this.workers.add(elem); + } + + public List get_workers() { + return this.workers; + } + + public void set_workers(List workers) { + this.workers = workers; + } + + public void unset_workers() { + this.workers = null; + } + + /** Returns true if field workers is set (has been assigned a value) and false otherwise */ + public boolean is_set_workers() { + return this.workers != null; + } + + public void set_workers_isSet(boolean value) { + if (!value) { + this.workers = null; + } + } + + public int get_workerMetric_size() { + return (this.workerMetric == null) ? 0 : this.workerMetric.size(); + } + + public void put_to_workerMetric(String key, MetricInfo val) { + if (this.workerMetric == null) { + this.workerMetric = new HashMap(); + } + this.workerMetric.put(key, val); + } + + public Map get_workerMetric() { + return this.workerMetric; + } + + public void set_workerMetric(Map workerMetric) { + this.workerMetric = workerMetric; + } + + public void unset_workerMetric() { + this.workerMetric = null; + } + + /** Returns true if field workerMetric is set (has been assigned a value) and false otherwise */ + public boolean is_set_workerMetric() { + return this.workerMetric != null; + } + + public void set_workerMetric_isSet(boolean value) { + if (!value) { + this.workerMetric = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUPERVISOR: + if (value == null) { + unset_supervisor(); + } else { + set_supervisor((SupervisorSummary)value); + } + break; + + case WORKERS: + if (value == null) { + unset_workers(); + } else { + set_workers((List)value); + } + break; + + case WORKER_METRIC: + if (value == null) { + unset_workerMetric(); + } else { + set_workerMetric((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUPERVISOR: + return get_supervisor(); + + case WORKERS: + return get_workers(); + + case WORKER_METRIC: + return get_workerMetric(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUPERVISOR: + return is_set_supervisor(); + case WORKERS: + return is_set_workers(); + case WORKER_METRIC: + return is_set_workerMetric(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof SupervisorWorkers) + return this.equals((SupervisorWorkers)that); + return false; + } + + public boolean equals(SupervisorWorkers that) { + if (that == null) + return false; + + boolean this_present_supervisor = true && this.is_set_supervisor(); + boolean that_present_supervisor = true && that.is_set_supervisor(); + if (this_present_supervisor || that_present_supervisor) { + if (!(this_present_supervisor && that_present_supervisor)) + return false; + if (!this.supervisor.equals(that.supervisor)) + return false; + } + + boolean this_present_workers = true && this.is_set_workers(); + boolean that_present_workers = true && that.is_set_workers(); + if (this_present_workers || that_present_workers) { + if (!(this_present_workers && that_present_workers)) + return false; + if (!this.workers.equals(that.workers)) + return false; + } + + boolean this_present_workerMetric = true && this.is_set_workerMetric(); + boolean that_present_workerMetric = true && that.is_set_workerMetric(); + if (this_present_workerMetric || that_present_workerMetric) { + if (!(this_present_workerMetric && that_present_workerMetric)) + return false; + if (!this.workerMetric.equals(that.workerMetric)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_supervisor = true && (is_set_supervisor()); + list.add(present_supervisor); + if (present_supervisor) + list.add(supervisor); + + boolean present_workers = true && (is_set_workers()); + list.add(present_workers); + if (present_workers) + list.add(workers); + + boolean present_workerMetric = true && (is_set_workerMetric()); + list.add(present_workerMetric); + if (present_workerMetric) + list.add(workerMetric); + + return list.hashCode(); + } + + @Override + public int compareTo(SupervisorWorkers other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_supervisor()).compareTo(other.is_set_supervisor()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_supervisor()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.supervisor, other.supervisor); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_workers()).compareTo(other.is_set_workers()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_workers()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.workers, other.workers); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_workerMetric()).compareTo(other.is_set_workerMetric()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_workerMetric()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.workerMetric, other.workerMetric); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("SupervisorWorkers("); + boolean first = true; + + sb.append("supervisor:"); + if (this.supervisor == null) { + sb.append("null"); + } else { + sb.append(this.supervisor); + } + first = false; + if (!first) sb.append(", "); + sb.append("workers:"); + if (this.workers == null) { + sb.append("null"); + } else { + sb.append(this.workers); + } + first = false; + if (!first) sb.append(", "); + sb.append("workerMetric:"); + if (this.workerMetric == null) { + sb.append("null"); + } else { + sb.append(this.workerMetric); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_supervisor()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'supervisor' is unset! Struct:" + toString()); + } + + if (!is_set_workers()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'workers' is unset! Struct:" + toString()); + } + + if (!is_set_workerMetric()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'workerMetric' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (supervisor != null) { + supervisor.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class SupervisorWorkersStandardSchemeFactory implements SchemeFactory { + public SupervisorWorkersStandardScheme getScheme() { + return new SupervisorWorkersStandardScheme(); + } + } + + private static class SupervisorWorkersStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, SupervisorWorkers struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SUPERVISOR + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.supervisor = new SupervisorSummary(); + struct.supervisor.read(iprot); + struct.set_supervisor_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // WORKERS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list196 = iprot.readListBegin(); + struct.workers = new ArrayList(_list196.size); + WorkerSummary _elem197; + for (int _i198 = 0; _i198 < _list196.size; ++_i198) + { + _elem197 = new WorkerSummary(); + _elem197.read(iprot); + struct.workers.add(_elem197); + } + iprot.readListEnd(); + } + struct.set_workers_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // WORKER_METRIC + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map199 = iprot.readMapBegin(); + struct.workerMetric = new HashMap(2*_map199.size); + String _key200; + MetricInfo _val201; + for (int _i202 = 0; _i202 < _map199.size; ++_i202) + { + _key200 = iprot.readString(); + _val201 = new MetricInfo(); + _val201.read(iprot); + struct.workerMetric.put(_key200, _val201); + } + iprot.readMapEnd(); + } + struct.set_workerMetric_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, SupervisorWorkers struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.supervisor != null) { + oprot.writeFieldBegin(SUPERVISOR_FIELD_DESC); + struct.supervisor.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.workers != null) { + oprot.writeFieldBegin(WORKERS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.workers.size())); + for (WorkerSummary _iter203 : struct.workers) + { + _iter203.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.workerMetric != null) { + oprot.writeFieldBegin(WORKER_METRIC_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.workerMetric.size())); + for (Map.Entry _iter204 : struct.workerMetric.entrySet()) + { + oprot.writeString(_iter204.getKey()); + _iter204.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class SupervisorWorkersTupleSchemeFactory implements SchemeFactory { + public SupervisorWorkersTupleScheme getScheme() { + return new SupervisorWorkersTupleScheme(); + } + } + + private static class SupervisorWorkersTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, SupervisorWorkers struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.supervisor.write(oprot); + { + oprot.writeI32(struct.workers.size()); + for (WorkerSummary _iter205 : struct.workers) + { + _iter205.write(oprot); + } + } + { + oprot.writeI32(struct.workerMetric.size()); + for (Map.Entry _iter206 : struct.workerMetric.entrySet()) + { + oprot.writeString(_iter206.getKey()); + _iter206.getValue().write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, SupervisorWorkers struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.supervisor = new SupervisorSummary(); + struct.supervisor.read(iprot); + struct.set_supervisor_isSet(true); + { + org.apache.thrift.protocol.TList _list207 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.workers = new ArrayList(_list207.size); + WorkerSummary _elem208; + for (int _i209 = 0; _i209 < _list207.size; ++_i209) + { + _elem208 = new WorkerSummary(); + _elem208.read(iprot); + struct.workers.add(_elem208); + } + } + struct.set_workers_isSet(true); + { + org.apache.thrift.protocol.TMap _map210 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.workerMetric = new HashMap(2*_map210.size); + String _key211; + MetricInfo _val212; + for (int _i213 = 0; _i213 < _map210.size; ++_i213) + { + _key211 = iprot.readString(); + _val212 = new MetricInfo(); + _val212.read(iprot); + struct.workerMetric.put(_key211, _val212); + } + } + struct.set_workerMetric_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/TaskComponent.java b/jstorm-core/src/main/java/backtype/storm/generated/TaskComponent.java new file mode 100644 index 000000000..53603fe66 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/TaskComponent.java @@ -0,0 +1,488 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class TaskComponent implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TaskComponent"); + + private static final org.apache.thrift.protocol.TField TASK_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("taskId", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField COMPONENT_FIELD_DESC = new org.apache.thrift.protocol.TField("component", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TaskComponentStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TaskComponentTupleSchemeFactory()); + } + + private int taskId; // required + private String component; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TASK_ID((short)1, "taskId"), + COMPONENT((short)2, "component"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TASK_ID + return TASK_ID; + case 2: // COMPONENT + return COMPONENT; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __TASKID_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TASK_ID, new org.apache.thrift.meta_data.FieldMetaData("taskId", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.COMPONENT, new org.apache.thrift.meta_data.FieldMetaData("component", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TaskComponent.class, metaDataMap); + } + + public TaskComponent() { + } + + public TaskComponent( + int taskId, + String component) + { + this(); + this.taskId = taskId; + set_taskId_isSet(true); + this.component = component; + } + + /** + * Performs a deep copy on other. + */ + public TaskComponent(TaskComponent other) { + __isset_bitfield = other.__isset_bitfield; + this.taskId = other.taskId; + if (other.is_set_component()) { + this.component = other.component; + } + } + + public TaskComponent deepCopy() { + return new TaskComponent(this); + } + + @Override + public void clear() { + set_taskId_isSet(false); + this.taskId = 0; + this.component = null; + } + + public int get_taskId() { + return this.taskId; + } + + public void set_taskId(int taskId) { + this.taskId = taskId; + set_taskId_isSet(true); + } + + public void unset_taskId() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TASKID_ISSET_ID); + } + + /** Returns true if field taskId is set (has been assigned a value) and false otherwise */ + public boolean is_set_taskId() { + return EncodingUtils.testBit(__isset_bitfield, __TASKID_ISSET_ID); + } + + public void set_taskId_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TASKID_ISSET_ID, value); + } + + public String get_component() { + return this.component; + } + + public void set_component(String component) { + this.component = component; + } + + public void unset_component() { + this.component = null; + } + + /** Returns true if field component is set (has been assigned a value) and false otherwise */ + public boolean is_set_component() { + return this.component != null; + } + + public void set_component_isSet(boolean value) { + if (!value) { + this.component = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TASK_ID: + if (value == null) { + unset_taskId(); + } else { + set_taskId((Integer)value); + } + break; + + case COMPONENT: + if (value == null) { + unset_component(); + } else { + set_component((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TASK_ID: + return Integer.valueOf(get_taskId()); + + case COMPONENT: + return get_component(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TASK_ID: + return is_set_taskId(); + case COMPONENT: + return is_set_component(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TaskComponent) + return this.equals((TaskComponent)that); + return false; + } + + public boolean equals(TaskComponent that) { + if (that == null) + return false; + + boolean this_present_taskId = true; + boolean that_present_taskId = true; + if (this_present_taskId || that_present_taskId) { + if (!(this_present_taskId && that_present_taskId)) + return false; + if (this.taskId != that.taskId) + return false; + } + + boolean this_present_component = true && this.is_set_component(); + boolean that_present_component = true && that.is_set_component(); + if (this_present_component || that_present_component) { + if (!(this_present_component && that_present_component)) + return false; + if (!this.component.equals(that.component)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_taskId = true; + list.add(present_taskId); + if (present_taskId) + list.add(taskId); + + boolean present_component = true && (is_set_component()); + list.add(present_component); + if (present_component) + list.add(component); + + return list.hashCode(); + } + + @Override + public int compareTo(TaskComponent other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_taskId()).compareTo(other.is_set_taskId()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_taskId()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.taskId, other.taskId); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_component()).compareTo(other.is_set_component()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_component()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.component, other.component); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TaskComponent("); + boolean first = true; + + sb.append("taskId:"); + sb.append(this.taskId); + first = false; + if (!first) sb.append(", "); + sb.append("component:"); + if (this.component == null) { + sb.append("null"); + } else { + sb.append(this.component); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_taskId()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'taskId' is unset! Struct:" + toString()); + } + + if (!is_set_component()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'component' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TaskComponentStandardSchemeFactory implements SchemeFactory { + public TaskComponentStandardScheme getScheme() { + return new TaskComponentStandardScheme(); + } + } + + private static class TaskComponentStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TaskComponent struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TASK_ID + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.taskId = iprot.readI32(); + struct.set_taskId_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // COMPONENT + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.component = iprot.readString(); + struct.set_component_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TaskComponent struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(TASK_ID_FIELD_DESC); + oprot.writeI32(struct.taskId); + oprot.writeFieldEnd(); + if (struct.component != null) { + oprot.writeFieldBegin(COMPONENT_FIELD_DESC); + oprot.writeString(struct.component); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TaskComponentTupleSchemeFactory implements SchemeFactory { + public TaskComponentTupleScheme getScheme() { + return new TaskComponentTupleScheme(); + } + } + + private static class TaskComponentTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TaskComponent struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI32(struct.taskId); + oprot.writeString(struct.component); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TaskComponent struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.taskId = iprot.readI32(); + struct.set_taskId_isSet(true); + struct.component = iprot.readString(); + struct.set_component_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/TaskSummary.java b/jstorm-core/src/main/java/backtype/storm/generated/TaskSummary.java new file mode 100644 index 000000000..9abf19758 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/TaskSummary.java @@ -0,0 +1,943 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class TaskSummary implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TaskSummary"); + + private static final org.apache.thrift.protocol.TField TASK_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("task_id", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField UPTIME_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime", org.apache.thrift.protocol.TType.I32, (short)2); + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField HOST_FIELD_DESC = new org.apache.thrift.protocol.TField("host", org.apache.thrift.protocol.TType.STRING, (short)4); + private static final org.apache.thrift.protocol.TField PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("port", org.apache.thrift.protocol.TType.I32, (short)5); + private static final org.apache.thrift.protocol.TField ERRORS_FIELD_DESC = new org.apache.thrift.protocol.TField("errors", org.apache.thrift.protocol.TType.LIST, (short)6); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TaskSummaryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TaskSummaryTupleSchemeFactory()); + } + + private int task_id; // required + private int uptime; // required + private String status; // required + private String host; // required + private int port; // required + private List errors; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TASK_ID((short)1, "task_id"), + UPTIME((short)2, "uptime"), + STATUS((short)3, "status"), + HOST((short)4, "host"), + PORT((short)5, "port"), + ERRORS((short)6, "errors"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TASK_ID + return TASK_ID; + case 2: // UPTIME + return UPTIME; + case 3: // STATUS + return STATUS; + case 4: // HOST + return HOST; + case 5: // PORT + return PORT; + case 6: // ERRORS + return ERRORS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __TASK_ID_ISSET_ID = 0; + private static final int __UPTIME_ISSET_ID = 1; + private static final int __PORT_ISSET_ID = 2; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.ERRORS}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TASK_ID, new org.apache.thrift.meta_data.FieldMetaData("task_id", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.UPTIME, new org.apache.thrift.meta_data.FieldMetaData("uptime", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.HOST, new org.apache.thrift.meta_data.FieldMetaData("host", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.PORT, new org.apache.thrift.meta_data.FieldMetaData("port", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.ERRORS, new org.apache.thrift.meta_data.FieldMetaData("errors", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ErrorInfo.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TaskSummary.class, metaDataMap); + } + + public TaskSummary() { + } + + public TaskSummary( + int task_id, + int uptime, + String status, + String host, + int port) + { + this(); + this.task_id = task_id; + set_task_id_isSet(true); + this.uptime = uptime; + set_uptime_isSet(true); + this.status = status; + this.host = host; + this.port = port; + set_port_isSet(true); + } + + /** + * Performs a deep copy on other. + */ + public TaskSummary(TaskSummary other) { + __isset_bitfield = other.__isset_bitfield; + this.task_id = other.task_id; + this.uptime = other.uptime; + if (other.is_set_status()) { + this.status = other.status; + } + if (other.is_set_host()) { + this.host = other.host; + } + this.port = other.port; + if (other.is_set_errors()) { + List __this__errors = new ArrayList(other.errors.size()); + for (ErrorInfo other_element : other.errors) { + __this__errors.add(new ErrorInfo(other_element)); + } + this.errors = __this__errors; + } + } + + public TaskSummary deepCopy() { + return new TaskSummary(this); + } + + @Override + public void clear() { + set_task_id_isSet(false); + this.task_id = 0; + set_uptime_isSet(false); + this.uptime = 0; + this.status = null; + this.host = null; + set_port_isSet(false); + this.port = 0; + this.errors = null; + } + + public int get_task_id() { + return this.task_id; + } + + public void set_task_id(int task_id) { + this.task_id = task_id; + set_task_id_isSet(true); + } + + public void unset_task_id() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TASK_ID_ISSET_ID); + } + + /** Returns true if field task_id is set (has been assigned a value) and false otherwise */ + public boolean is_set_task_id() { + return EncodingUtils.testBit(__isset_bitfield, __TASK_ID_ISSET_ID); + } + + public void set_task_id_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TASK_ID_ISSET_ID, value); + } + + public int get_uptime() { + return this.uptime; + } + + public void set_uptime(int uptime) { + this.uptime = uptime; + set_uptime_isSet(true); + } + + public void unset_uptime() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __UPTIME_ISSET_ID); + } + + /** Returns true if field uptime is set (has been assigned a value) and false otherwise */ + public boolean is_set_uptime() { + return EncodingUtils.testBit(__isset_bitfield, __UPTIME_ISSET_ID); + } + + public void set_uptime_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_ISSET_ID, value); + } + + public String get_status() { + return this.status; + } + + public void set_status(String status) { + this.status = status; + } + + public void unset_status() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean is_set_status() { + return this.status != null; + } + + public void set_status_isSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public String get_host() { + return this.host; + } + + public void set_host(String host) { + this.host = host; + } + + public void unset_host() { + this.host = null; + } + + /** Returns true if field host is set (has been assigned a value) and false otherwise */ + public boolean is_set_host() { + return this.host != null; + } + + public void set_host_isSet(boolean value) { + if (!value) { + this.host = null; + } + } + + public int get_port() { + return this.port; + } + + public void set_port(int port) { + this.port = port; + set_port_isSet(true); + } + + public void unset_port() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PORT_ISSET_ID); + } + + /** Returns true if field port is set (has been assigned a value) and false otherwise */ + public boolean is_set_port() { + return EncodingUtils.testBit(__isset_bitfield, __PORT_ISSET_ID); + } + + public void set_port_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PORT_ISSET_ID, value); + } + + public int get_errors_size() { + return (this.errors == null) ? 0 : this.errors.size(); + } + + public java.util.Iterator get_errors_iterator() { + return (this.errors == null) ? null : this.errors.iterator(); + } + + public void add_to_errors(ErrorInfo elem) { + if (this.errors == null) { + this.errors = new ArrayList(); + } + this.errors.add(elem); + } + + public List get_errors() { + return this.errors; + } + + public void set_errors(List errors) { + this.errors = errors; + } + + public void unset_errors() { + this.errors = null; + } + + /** Returns true if field errors is set (has been assigned a value) and false otherwise */ + public boolean is_set_errors() { + return this.errors != null; + } + + public void set_errors_isSet(boolean value) { + if (!value) { + this.errors = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TASK_ID: + if (value == null) { + unset_task_id(); + } else { + set_task_id((Integer)value); + } + break; + + case UPTIME: + if (value == null) { + unset_uptime(); + } else { + set_uptime((Integer)value); + } + break; + + case STATUS: + if (value == null) { + unset_status(); + } else { + set_status((String)value); + } + break; + + case HOST: + if (value == null) { + unset_host(); + } else { + set_host((String)value); + } + break; + + case PORT: + if (value == null) { + unset_port(); + } else { + set_port((Integer)value); + } + break; + + case ERRORS: + if (value == null) { + unset_errors(); + } else { + set_errors((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TASK_ID: + return Integer.valueOf(get_task_id()); + + case UPTIME: + return Integer.valueOf(get_uptime()); + + case STATUS: + return get_status(); + + case HOST: + return get_host(); + + case PORT: + return Integer.valueOf(get_port()); + + case ERRORS: + return get_errors(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TASK_ID: + return is_set_task_id(); + case UPTIME: + return is_set_uptime(); + case STATUS: + return is_set_status(); + case HOST: + return is_set_host(); + case PORT: + return is_set_port(); + case ERRORS: + return is_set_errors(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TaskSummary) + return this.equals((TaskSummary)that); + return false; + } + + public boolean equals(TaskSummary that) { + if (that == null) + return false; + + boolean this_present_task_id = true; + boolean that_present_task_id = true; + if (this_present_task_id || that_present_task_id) { + if (!(this_present_task_id && that_present_task_id)) + return false; + if (this.task_id != that.task_id) + return false; + } + + boolean this_present_uptime = true; + boolean that_present_uptime = true; + if (this_present_uptime || that_present_uptime) { + if (!(this_present_uptime && that_present_uptime)) + return false; + if (this.uptime != that.uptime) + return false; + } + + boolean this_present_status = true && this.is_set_status(); + boolean that_present_status = true && that.is_set_status(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_host = true && this.is_set_host(); + boolean that_present_host = true && that.is_set_host(); + if (this_present_host || that_present_host) { + if (!(this_present_host && that_present_host)) + return false; + if (!this.host.equals(that.host)) + return false; + } + + boolean this_present_port = true; + boolean that_present_port = true; + if (this_present_port || that_present_port) { + if (!(this_present_port && that_present_port)) + return false; + if (this.port != that.port) + return false; + } + + boolean this_present_errors = true && this.is_set_errors(); + boolean that_present_errors = true && that.is_set_errors(); + if (this_present_errors || that_present_errors) { + if (!(this_present_errors && that_present_errors)) + return false; + if (!this.errors.equals(that.errors)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_task_id = true; + list.add(present_task_id); + if (present_task_id) + list.add(task_id); + + boolean present_uptime = true; + list.add(present_uptime); + if (present_uptime) + list.add(uptime); + + boolean present_status = true && (is_set_status()); + list.add(present_status); + if (present_status) + list.add(status); + + boolean present_host = true && (is_set_host()); + list.add(present_host); + if (present_host) + list.add(host); + + boolean present_port = true; + list.add(present_port); + if (present_port) + list.add(port); + + boolean present_errors = true && (is_set_errors()); + list.add(present_errors); + if (present_errors) + list.add(errors); + + return list.hashCode(); + } + + @Override + public int compareTo(TaskSummary other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_task_id()).compareTo(other.is_set_task_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_task_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.task_id, other.task_id); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_uptime()).compareTo(other.is_set_uptime()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_uptime()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime, other.uptime); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_status()).compareTo(other.is_set_status()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_status()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_host()).compareTo(other.is_set_host()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_host()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.host, other.host); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_port()).compareTo(other.is_set_port()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_port()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.port, other.port); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_errors()).compareTo(other.is_set_errors()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_errors()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errors, other.errors); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TaskSummary("); + boolean first = true; + + sb.append("task_id:"); + sb.append(this.task_id); + first = false; + if (!first) sb.append(", "); + sb.append("uptime:"); + sb.append(this.uptime); + first = false; + if (!first) sb.append(", "); + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (!first) sb.append(", "); + sb.append("host:"); + if (this.host == null) { + sb.append("null"); + } else { + sb.append(this.host); + } + first = false; + if (!first) sb.append(", "); + sb.append("port:"); + sb.append(this.port); + first = false; + if (is_set_errors()) { + if (!first) sb.append(", "); + sb.append("errors:"); + if (this.errors == null) { + sb.append("null"); + } else { + sb.append(this.errors); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_task_id()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'task_id' is unset! Struct:" + toString()); + } + + if (!is_set_uptime()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'uptime' is unset! Struct:" + toString()); + } + + if (!is_set_status()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + if (!is_set_host()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'host' is unset! Struct:" + toString()); + } + + if (!is_set_port()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'port' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TaskSummaryStandardSchemeFactory implements SchemeFactory { + public TaskSummaryStandardScheme getScheme() { + return new TaskSummaryStandardScheme(); + } + } + + private static class TaskSummaryStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TaskSummary struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TASK_ID + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.task_id = iprot.readI32(); + struct.set_task_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // UPTIME + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.uptime = iprot.readI32(); + struct.set_uptime_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.status = iprot.readString(); + struct.set_status_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // HOST + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.host = iprot.readString(); + struct.set_host_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // PORT + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.port = iprot.readI32(); + struct.set_port_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 6: // ERRORS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list230 = iprot.readListBegin(); + struct.errors = new ArrayList(_list230.size); + ErrorInfo _elem231; + for (int _i232 = 0; _i232 < _list230.size; ++_i232) + { + _elem231 = new ErrorInfo(); + _elem231.read(iprot); + struct.errors.add(_elem231); + } + iprot.readListEnd(); + } + struct.set_errors_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TaskSummary struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(TASK_ID_FIELD_DESC); + oprot.writeI32(struct.task_id); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(UPTIME_FIELD_DESC); + oprot.writeI32(struct.uptime); + oprot.writeFieldEnd(); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + oprot.writeString(struct.status); + oprot.writeFieldEnd(); + } + if (struct.host != null) { + oprot.writeFieldBegin(HOST_FIELD_DESC); + oprot.writeString(struct.host); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(PORT_FIELD_DESC); + oprot.writeI32(struct.port); + oprot.writeFieldEnd(); + if (struct.errors != null) { + if (struct.is_set_errors()) { + oprot.writeFieldBegin(ERRORS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.errors.size())); + for (ErrorInfo _iter233 : struct.errors) + { + _iter233.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TaskSummaryTupleSchemeFactory implements SchemeFactory { + public TaskSummaryTupleScheme getScheme() { + return new TaskSummaryTupleScheme(); + } + } + + private static class TaskSummaryTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TaskSummary struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI32(struct.task_id); + oprot.writeI32(struct.uptime); + oprot.writeString(struct.status); + oprot.writeString(struct.host); + oprot.writeI32(struct.port); + BitSet optionals = new BitSet(); + if (struct.is_set_errors()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_errors()) { + { + oprot.writeI32(struct.errors.size()); + for (ErrorInfo _iter234 : struct.errors) + { + _iter234.write(oprot); + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TaskSummary struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.task_id = iprot.readI32(); + struct.set_task_id_isSet(true); + struct.uptime = iprot.readI32(); + struct.set_uptime_isSet(true); + struct.status = iprot.readString(); + struct.set_status_isSet(true); + struct.host = iprot.readString(); + struct.set_host_isSet(true); + struct.port = iprot.readI32(); + struct.set_port_isSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + { + org.apache.thrift.protocol.TList _list235 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.errors = new ArrayList(_list235.size); + ErrorInfo _elem236; + for (int _i237 = 0; _i237 < _list235.size; ++_i237) + { + _elem236 = new ErrorInfo(); + _elem236.read(iprot); + struct.errors.add(_elem236); + } + } + struct.set_errors_isSet(true); + } + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/ThriftSerializedObject.java b/jstorm-core/src/main/java/backtype/storm/generated/ThriftSerializedObject.java new file mode 100644 index 000000000..a753d7ff2 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/ThriftSerializedObject.java @@ -0,0 +1,499 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class ThriftSerializedObject implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ThriftSerializedObject"); + + private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField BITS_FIELD_DESC = new org.apache.thrift.protocol.TField("bits", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ThriftSerializedObjectStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ThriftSerializedObjectTupleSchemeFactory()); + } + + private String name; // required + private ByteBuffer bits; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NAME((short)1, "name"), + BITS((short)2, "bits"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NAME + return NAME; + case 2: // BITS + return BITS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.BITS, new org.apache.thrift.meta_data.FieldMetaData("bits", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ThriftSerializedObject.class, metaDataMap); + } + + public ThriftSerializedObject() { + } + + public ThriftSerializedObject( + String name, + ByteBuffer bits) + { + this(); + this.name = name; + this.bits = org.apache.thrift.TBaseHelper.copyBinary(bits); + } + + /** + * Performs a deep copy on other. + */ + public ThriftSerializedObject(ThriftSerializedObject other) { + if (other.is_set_name()) { + this.name = other.name; + } + if (other.is_set_bits()) { + this.bits = org.apache.thrift.TBaseHelper.copyBinary(other.bits); + } + } + + public ThriftSerializedObject deepCopy() { + return new ThriftSerializedObject(this); + } + + @Override + public void clear() { + this.name = null; + this.bits = null; + } + + public String get_name() { + return this.name; + } + + public void set_name(String name) { + this.name = name; + } + + public void unset_name() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean is_set_name() { + return this.name != null; + } + + public void set_name_isSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public byte[] get_bits() { + set_bits(org.apache.thrift.TBaseHelper.rightSize(bits)); + return bits == null ? null : bits.array(); + } + + public ByteBuffer buffer_for_bits() { + return org.apache.thrift.TBaseHelper.copyBinary(bits); + } + + public void set_bits(byte[] bits) { + this.bits = bits == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(bits, bits.length)); + } + + public void set_bits(ByteBuffer bits) { + this.bits = org.apache.thrift.TBaseHelper.copyBinary(bits); + } + + public void unset_bits() { + this.bits = null; + } + + /** Returns true if field bits is set (has been assigned a value) and false otherwise */ + public boolean is_set_bits() { + return this.bits != null; + } + + public void set_bits_isSet(boolean value) { + if (!value) { + this.bits = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME: + if (value == null) { + unset_name(); + } else { + set_name((String)value); + } + break; + + case BITS: + if (value == null) { + unset_bits(); + } else { + set_bits((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME: + return get_name(); + + case BITS: + return get_bits(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NAME: + return is_set_name(); + case BITS: + return is_set_bits(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ThriftSerializedObject) + return this.equals((ThriftSerializedObject)that); + return false; + } + + public boolean equals(ThriftSerializedObject that) { + if (that == null) + return false; + + boolean this_present_name = true && this.is_set_name(); + boolean that_present_name = true && that.is_set_name(); + if (this_present_name || that_present_name) { + if (!(this_present_name && that_present_name)) + return false; + if (!this.name.equals(that.name)) + return false; + } + + boolean this_present_bits = true && this.is_set_bits(); + boolean that_present_bits = true && that.is_set_bits(); + if (this_present_bits || that_present_bits) { + if (!(this_present_bits && that_present_bits)) + return false; + if (!this.bits.equals(that.bits)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_name = true && (is_set_name()); + list.add(present_name); + if (present_name) + list.add(name); + + boolean present_bits = true && (is_set_bits()); + list.add(present_bits); + if (present_bits) + list.add(bits); + + return list.hashCode(); + } + + @Override + public int compareTo(ThriftSerializedObject other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_bits()).compareTo(other.is_set_bits()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_bits()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.bits, other.bits); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ThriftSerializedObject("); + boolean first = true; + + sb.append("name:"); + if (this.name == null) { + sb.append("null"); + } else { + sb.append(this.name); + } + first = false; + if (!first) sb.append(", "); + sb.append("bits:"); + if (this.bits == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.bits, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_name()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'name' is unset! Struct:" + toString()); + } + + if (!is_set_bits()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'bits' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ThriftSerializedObjectStandardSchemeFactory implements SchemeFactory { + public ThriftSerializedObjectStandardScheme getScheme() { + return new ThriftSerializedObjectStandardScheme(); + } + } + + private static class ThriftSerializedObjectStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ThriftSerializedObject struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // BITS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.bits = iprot.readBinary(); + struct.set_bits_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ThriftSerializedObject struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.name != null) { + oprot.writeFieldBegin(NAME_FIELD_DESC); + oprot.writeString(struct.name); + oprot.writeFieldEnd(); + } + if (struct.bits != null) { + oprot.writeFieldBegin(BITS_FIELD_DESC); + oprot.writeBinary(struct.bits); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ThriftSerializedObjectTupleSchemeFactory implements SchemeFactory { + public ThriftSerializedObjectTupleScheme getScheme() { + return new ThriftSerializedObjectTupleScheme(); + } + } + + private static class ThriftSerializedObjectTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ThriftSerializedObject struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.name); + oprot.writeBinary(struct.bits); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ThriftSerializedObject struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.name = iprot.readString(); + struct.set_name_isSet(true); + struct.bits = iprot.readBinary(); + struct.set_bits_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/TopologyAssignException.java b/jstorm-core/src/main/java/backtype/storm/generated/TopologyAssignException.java new file mode 100644 index 000000000..8c9587615 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/TopologyAssignException.java @@ -0,0 +1,389 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class TopologyAssignException extends TException implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyAssignException"); + + private static final org.apache.thrift.protocol.TField MSG_FIELD_DESC = new org.apache.thrift.protocol.TField("msg", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TopologyAssignExceptionStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TopologyAssignExceptionTupleSchemeFactory()); + } + + private String msg; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + MSG((short)1, "msg"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // MSG + return MSG; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.MSG, new org.apache.thrift.meta_data.FieldMetaData("msg", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyAssignException.class, metaDataMap); + } + + public TopologyAssignException() { + } + + public TopologyAssignException( + String msg) + { + this(); + this.msg = msg; + } + + /** + * Performs a deep copy on other. + */ + public TopologyAssignException(TopologyAssignException other) { + if (other.is_set_msg()) { + this.msg = other.msg; + } + } + + public TopologyAssignException deepCopy() { + return new TopologyAssignException(this); + } + + @Override + public void clear() { + this.msg = null; + } + + public String get_msg() { + return this.msg; + } + + public void set_msg(String msg) { + this.msg = msg; + } + + public void unset_msg() { + this.msg = null; + } + + /** Returns true if field msg is set (has been assigned a value) and false otherwise */ + public boolean is_set_msg() { + return this.msg != null; + } + + public void set_msg_isSet(boolean value) { + if (!value) { + this.msg = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case MSG: + if (value == null) { + unset_msg(); + } else { + set_msg((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case MSG: + return get_msg(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case MSG: + return is_set_msg(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TopologyAssignException) + return this.equals((TopologyAssignException)that); + return false; + } + + public boolean equals(TopologyAssignException that) { + if (that == null) + return false; + + boolean this_present_msg = true && this.is_set_msg(); + boolean that_present_msg = true && that.is_set_msg(); + if (this_present_msg || that_present_msg) { + if (!(this_present_msg && that_present_msg)) + return false; + if (!this.msg.equals(that.msg)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_msg = true && (is_set_msg()); + list.add(present_msg); + if (present_msg) + list.add(msg); + + return list.hashCode(); + } + + @Override + public int compareTo(TopologyAssignException other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_msg()).compareTo(other.is_set_msg()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_msg()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.msg, other.msg); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TopologyAssignException("); + boolean first = true; + + sb.append("msg:"); + if (this.msg == null) { + sb.append("null"); + } else { + sb.append(this.msg); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_msg()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'msg' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TopologyAssignExceptionStandardSchemeFactory implements SchemeFactory { + public TopologyAssignExceptionStandardScheme getScheme() { + return new TopologyAssignExceptionStandardScheme(); + } + } + + private static class TopologyAssignExceptionStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TopologyAssignException struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // MSG + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.msg = iprot.readString(); + struct.set_msg_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TopologyAssignException struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.msg != null) { + oprot.writeFieldBegin(MSG_FIELD_DESC); + oprot.writeString(struct.msg); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TopologyAssignExceptionTupleSchemeFactory implements SchemeFactory { + public TopologyAssignExceptionTupleScheme getScheme() { + return new TopologyAssignExceptionTupleScheme(); + } + } + + private static class TopologyAssignExceptionTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TopologyAssignException struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.msg); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TopologyAssignException struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.msg = iprot.readString(); + struct.set_msg_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/TopologyInfo.java b/jstorm-core/src/main/java/backtype/storm/generated/TopologyInfo.java new file mode 100644 index 000000000..1da8e98bf --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/TopologyInfo.java @@ -0,0 +1,810 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class TopologyInfo implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyInfo"); + + private static final org.apache.thrift.protocol.TField TOPOLOGY_FIELD_DESC = new org.apache.thrift.protocol.TField("topology", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField COMPONENTS_FIELD_DESC = new org.apache.thrift.protocol.TField("components", org.apache.thrift.protocol.TType.LIST, (short)2); + private static final org.apache.thrift.protocol.TField TASKS_FIELD_DESC = new org.apache.thrift.protocol.TField("tasks", org.apache.thrift.protocol.TType.LIST, (short)3); + private static final org.apache.thrift.protocol.TField METRICS_FIELD_DESC = new org.apache.thrift.protocol.TField("metrics", org.apache.thrift.protocol.TType.STRUCT, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TopologyInfoStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TopologyInfoTupleSchemeFactory()); + } + + private TopologySummary topology; // required + private List components; // required + private List tasks; // required + private TopologyMetric metrics; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TOPOLOGY((short)1, "topology"), + COMPONENTS((short)2, "components"), + TASKS((short)3, "tasks"), + METRICS((short)4, "metrics"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TOPOLOGY + return TOPOLOGY; + case 2: // COMPONENTS + return COMPONENTS; + case 3: // TASKS + return TASKS; + case 4: // METRICS + return METRICS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TOPOLOGY, new org.apache.thrift.meta_data.FieldMetaData("topology", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologySummary.class))); + tmpMap.put(_Fields.COMPONENTS, new org.apache.thrift.meta_data.FieldMetaData("components", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ComponentSummary.class)))); + tmpMap.put(_Fields.TASKS, new org.apache.thrift.meta_data.FieldMetaData("tasks", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TaskSummary.class)))); + tmpMap.put(_Fields.METRICS, new org.apache.thrift.meta_data.FieldMetaData("metrics", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TopologyMetric.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyInfo.class, metaDataMap); + } + + public TopologyInfo() { + } + + public TopologyInfo( + TopologySummary topology, + List components, + List tasks, + TopologyMetric metrics) + { + this(); + this.topology = topology; + this.components = components; + this.tasks = tasks; + this.metrics = metrics; + } + + /** + * Performs a deep copy on other. + */ + public TopologyInfo(TopologyInfo other) { + if (other.is_set_topology()) { + this.topology = new TopologySummary(other.topology); + } + if (other.is_set_components()) { + List __this__components = new ArrayList(other.components.size()); + for (ComponentSummary other_element : other.components) { + __this__components.add(new ComponentSummary(other_element)); + } + this.components = __this__components; + } + if (other.is_set_tasks()) { + List __this__tasks = new ArrayList(other.tasks.size()); + for (TaskSummary other_element : other.tasks) { + __this__tasks.add(new TaskSummary(other_element)); + } + this.tasks = __this__tasks; + } + if (other.is_set_metrics()) { + this.metrics = new TopologyMetric(other.metrics); + } + } + + public TopologyInfo deepCopy() { + return new TopologyInfo(this); + } + + @Override + public void clear() { + this.topology = null; + this.components = null; + this.tasks = null; + this.metrics = null; + } + + public TopologySummary get_topology() { + return this.topology; + } + + public void set_topology(TopologySummary topology) { + this.topology = topology; + } + + public void unset_topology() { + this.topology = null; + } + + /** Returns true if field topology is set (has been assigned a value) and false otherwise */ + public boolean is_set_topology() { + return this.topology != null; + } + + public void set_topology_isSet(boolean value) { + if (!value) { + this.topology = null; + } + } + + public int get_components_size() { + return (this.components == null) ? 0 : this.components.size(); + } + + public java.util.Iterator get_components_iterator() { + return (this.components == null) ? null : this.components.iterator(); + } + + public void add_to_components(ComponentSummary elem) { + if (this.components == null) { + this.components = new ArrayList(); + } + this.components.add(elem); + } + + public List get_components() { + return this.components; + } + + public void set_components(List components) { + this.components = components; + } + + public void unset_components() { + this.components = null; + } + + /** Returns true if field components is set (has been assigned a value) and false otherwise */ + public boolean is_set_components() { + return this.components != null; + } + + public void set_components_isSet(boolean value) { + if (!value) { + this.components = null; + } + } + + public int get_tasks_size() { + return (this.tasks == null) ? 0 : this.tasks.size(); + } + + public java.util.Iterator get_tasks_iterator() { + return (this.tasks == null) ? null : this.tasks.iterator(); + } + + public void add_to_tasks(TaskSummary elem) { + if (this.tasks == null) { + this.tasks = new ArrayList(); + } + this.tasks.add(elem); + } + + public List get_tasks() { + return this.tasks; + } + + public void set_tasks(List tasks) { + this.tasks = tasks; + } + + public void unset_tasks() { + this.tasks = null; + } + + /** Returns true if field tasks is set (has been assigned a value) and false otherwise */ + public boolean is_set_tasks() { + return this.tasks != null; + } + + public void set_tasks_isSet(boolean value) { + if (!value) { + this.tasks = null; + } + } + + public TopologyMetric get_metrics() { + return this.metrics; + } + + public void set_metrics(TopologyMetric metrics) { + this.metrics = metrics; + } + + public void unset_metrics() { + this.metrics = null; + } + + /** Returns true if field metrics is set (has been assigned a value) and false otherwise */ + public boolean is_set_metrics() { + return this.metrics != null; + } + + public void set_metrics_isSet(boolean value) { + if (!value) { + this.metrics = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TOPOLOGY: + if (value == null) { + unset_topology(); + } else { + set_topology((TopologySummary)value); + } + break; + + case COMPONENTS: + if (value == null) { + unset_components(); + } else { + set_components((List)value); + } + break; + + case TASKS: + if (value == null) { + unset_tasks(); + } else { + set_tasks((List)value); + } + break; + + case METRICS: + if (value == null) { + unset_metrics(); + } else { + set_metrics((TopologyMetric)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TOPOLOGY: + return get_topology(); + + case COMPONENTS: + return get_components(); + + case TASKS: + return get_tasks(); + + case METRICS: + return get_metrics(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TOPOLOGY: + return is_set_topology(); + case COMPONENTS: + return is_set_components(); + case TASKS: + return is_set_tasks(); + case METRICS: + return is_set_metrics(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TopologyInfo) + return this.equals((TopologyInfo)that); + return false; + } + + public boolean equals(TopologyInfo that) { + if (that == null) + return false; + + boolean this_present_topology = true && this.is_set_topology(); + boolean that_present_topology = true && that.is_set_topology(); + if (this_present_topology || that_present_topology) { + if (!(this_present_topology && that_present_topology)) + return false; + if (!this.topology.equals(that.topology)) + return false; + } + + boolean this_present_components = true && this.is_set_components(); + boolean that_present_components = true && that.is_set_components(); + if (this_present_components || that_present_components) { + if (!(this_present_components && that_present_components)) + return false; + if (!this.components.equals(that.components)) + return false; + } + + boolean this_present_tasks = true && this.is_set_tasks(); + boolean that_present_tasks = true && that.is_set_tasks(); + if (this_present_tasks || that_present_tasks) { + if (!(this_present_tasks && that_present_tasks)) + return false; + if (!this.tasks.equals(that.tasks)) + return false; + } + + boolean this_present_metrics = true && this.is_set_metrics(); + boolean that_present_metrics = true && that.is_set_metrics(); + if (this_present_metrics || that_present_metrics) { + if (!(this_present_metrics && that_present_metrics)) + return false; + if (!this.metrics.equals(that.metrics)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_topology = true && (is_set_topology()); + list.add(present_topology); + if (present_topology) + list.add(topology); + + boolean present_components = true && (is_set_components()); + list.add(present_components); + if (present_components) + list.add(components); + + boolean present_tasks = true && (is_set_tasks()); + list.add(present_tasks); + if (present_tasks) + list.add(tasks); + + boolean present_metrics = true && (is_set_metrics()); + list.add(present_metrics); + if (present_metrics) + list.add(metrics); + + return list.hashCode(); + } + + @Override + public int compareTo(TopologyInfo other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_topology()).compareTo(other.is_set_topology()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_topology()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology, other.topology); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_components()).compareTo(other.is_set_components()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_components()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.components, other.components); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_tasks()).compareTo(other.is_set_tasks()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_tasks()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tasks, other.tasks); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_metrics()).compareTo(other.is_set_metrics()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_metrics()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.metrics, other.metrics); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TopologyInfo("); + boolean first = true; + + sb.append("topology:"); + if (this.topology == null) { + sb.append("null"); + } else { + sb.append(this.topology); + } + first = false; + if (!first) sb.append(", "); + sb.append("components:"); + if (this.components == null) { + sb.append("null"); + } else { + sb.append(this.components); + } + first = false; + if (!first) sb.append(", "); + sb.append("tasks:"); + if (this.tasks == null) { + sb.append("null"); + } else { + sb.append(this.tasks); + } + first = false; + if (!first) sb.append(", "); + sb.append("metrics:"); + if (this.metrics == null) { + sb.append("null"); + } else { + sb.append(this.metrics); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_topology()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'topology' is unset! Struct:" + toString()); + } + + if (!is_set_components()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'components' is unset! Struct:" + toString()); + } + + if (!is_set_tasks()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'tasks' is unset! Struct:" + toString()); + } + + if (!is_set_metrics()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'metrics' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (topology != null) { + topology.validate(); + } + if (metrics != null) { + metrics.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TopologyInfoStandardSchemeFactory implements SchemeFactory { + public TopologyInfoStandardScheme getScheme() { + return new TopologyInfoStandardScheme(); + } + } + + private static class TopologyInfoStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TopologyInfo struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TOPOLOGY + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.topology = new TopologySummary(); + struct.topology.read(iprot); + struct.set_topology_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // COMPONENTS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list238 = iprot.readListBegin(); + struct.components = new ArrayList(_list238.size); + ComponentSummary _elem239; + for (int _i240 = 0; _i240 < _list238.size; ++_i240) + { + _elem239 = new ComponentSummary(); + _elem239.read(iprot); + struct.components.add(_elem239); + } + iprot.readListEnd(); + } + struct.set_components_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // TASKS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list241 = iprot.readListBegin(); + struct.tasks = new ArrayList(_list241.size); + TaskSummary _elem242; + for (int _i243 = 0; _i243 < _list241.size; ++_i243) + { + _elem242 = new TaskSummary(); + _elem242.read(iprot); + struct.tasks.add(_elem242); + } + iprot.readListEnd(); + } + struct.set_tasks_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // METRICS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.metrics = new TopologyMetric(); + struct.metrics.read(iprot); + struct.set_metrics_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TopologyInfo struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.topology != null) { + oprot.writeFieldBegin(TOPOLOGY_FIELD_DESC); + struct.topology.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.components != null) { + oprot.writeFieldBegin(COMPONENTS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.components.size())); + for (ComponentSummary _iter244 : struct.components) + { + _iter244.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.tasks != null) { + oprot.writeFieldBegin(TASKS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tasks.size())); + for (TaskSummary _iter245 : struct.tasks) + { + _iter245.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.metrics != null) { + oprot.writeFieldBegin(METRICS_FIELD_DESC); + struct.metrics.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TopologyInfoTupleSchemeFactory implements SchemeFactory { + public TopologyInfoTupleScheme getScheme() { + return new TopologyInfoTupleScheme(); + } + } + + private static class TopologyInfoTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TopologyInfo struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.topology.write(oprot); + { + oprot.writeI32(struct.components.size()); + for (ComponentSummary _iter246 : struct.components) + { + _iter246.write(oprot); + } + } + { + oprot.writeI32(struct.tasks.size()); + for (TaskSummary _iter247 : struct.tasks) + { + _iter247.write(oprot); + } + } + struct.metrics.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TopologyInfo struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.topology = new TopologySummary(); + struct.topology.read(iprot); + struct.set_topology_isSet(true); + { + org.apache.thrift.protocol.TList _list248 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.components = new ArrayList(_list248.size); + ComponentSummary _elem249; + for (int _i250 = 0; _i250 < _list248.size; ++_i250) + { + _elem249 = new ComponentSummary(); + _elem249.read(iprot); + struct.components.add(_elem249); + } + } + struct.set_components_isSet(true); + { + org.apache.thrift.protocol.TList _list251 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.tasks = new ArrayList(_list251.size); + TaskSummary _elem252; + for (int _i253 = 0; _i253 < _list251.size; ++_i253) + { + _elem252 = new TaskSummary(); + _elem252.read(iprot); + struct.tasks.add(_elem252); + } + } + struct.set_tasks_isSet(true); + struct.metrics = new TopologyMetric(); + struct.metrics.read(iprot); + struct.set_metrics_isSet(true); + } + } + +} + diff --git a/jstorm-client/src/main/java/backtype/storm/generated/TopologyInitialStatus.java b/jstorm-core/src/main/java/backtype/storm/generated/TopologyInitialStatus.java old mode 100644 new mode 100755 similarity index 82% rename from jstorm-client/src/main/java/backtype/storm/generated/TopologyInitialStatus.java rename to jstorm-core/src/main/java/backtype/storm/generated/TopologyInitialStatus.java index 2c97dd7f0..0d87ee183 --- a/jstorm-client/src/main/java/backtype/storm/generated/TopologyInitialStatus.java +++ b/jstorm-core/src/main/java/backtype/storm/generated/TopologyInitialStatus.java @@ -1,16 +1,17 @@ /** - * Autogenerated by Thrift Compiler (0.7.0) + * Autogenerated by Thrift Compiler (0.9.2) * * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated */ package backtype.storm.generated; import java.util.Map; import java.util.HashMap; -import org.apache.thrift7.TEnum; +import org.apache.thrift.TEnum; -public enum TopologyInitialStatus implements org.apache.thrift7.TEnum { +public enum TopologyInitialStatus implements org.apache.thrift.TEnum { ACTIVE(1), INACTIVE(2); diff --git a/jstorm-core/src/main/java/backtype/storm/generated/TopologyMetric.java b/jstorm-core/src/main/java/backtype/storm/generated/TopologyMetric.java new file mode 100644 index 000000000..6df50cfed --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/TopologyMetric.java @@ -0,0 +1,892 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class TopologyMetric implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologyMetric"); + + private static final org.apache.thrift.protocol.TField TOPOLOGY_METRIC_FIELD_DESC = new org.apache.thrift.protocol.TField("topologyMetric", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField COMPONENT_METRIC_FIELD_DESC = new org.apache.thrift.protocol.TField("componentMetric", org.apache.thrift.protocol.TType.MAP, (short)2); + private static final org.apache.thrift.protocol.TField WORKER_METRIC_FIELD_DESC = new org.apache.thrift.protocol.TField("workerMetric", org.apache.thrift.protocol.TType.MAP, (short)3); + private static final org.apache.thrift.protocol.TField TASK_METRIC_FIELD_DESC = new org.apache.thrift.protocol.TField("taskMetric", org.apache.thrift.protocol.TType.MAP, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TopologyMetricStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TopologyMetricTupleSchemeFactory()); + } + + private MetricInfo topologyMetric; // required + private Map componentMetric; // required + private Map workerMetric; // required + private Map taskMetric; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TOPOLOGY_METRIC((short)1, "topologyMetric"), + COMPONENT_METRIC((short)2, "componentMetric"), + WORKER_METRIC((short)3, "workerMetric"), + TASK_METRIC((short)4, "taskMetric"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TOPOLOGY_METRIC + return TOPOLOGY_METRIC; + case 2: // COMPONENT_METRIC + return COMPONENT_METRIC; + case 3: // WORKER_METRIC + return WORKER_METRIC; + case 4: // TASK_METRIC + return TASK_METRIC; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TOPOLOGY_METRIC, new org.apache.thrift.meta_data.FieldMetaData("topologyMetric", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, MetricInfo.class))); + tmpMap.put(_Fields.COMPONENT_METRIC, new org.apache.thrift.meta_data.FieldMetaData("componentMetric", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, MetricInfo.class)))); + tmpMap.put(_Fields.WORKER_METRIC, new org.apache.thrift.meta_data.FieldMetaData("workerMetric", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, MetricInfo.class)))); + tmpMap.put(_Fields.TASK_METRIC, new org.apache.thrift.meta_data.FieldMetaData("taskMetric", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, MetricInfo.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologyMetric.class, metaDataMap); + } + + public TopologyMetric() { + } + + public TopologyMetric( + MetricInfo topologyMetric, + Map componentMetric, + Map workerMetric, + Map taskMetric) + { + this(); + this.topologyMetric = topologyMetric; + this.componentMetric = componentMetric; + this.workerMetric = workerMetric; + this.taskMetric = taskMetric; + } + + /** + * Performs a deep copy on other. + */ + public TopologyMetric(TopologyMetric other) { + if (other.is_set_topologyMetric()) { + this.topologyMetric = new MetricInfo(other.topologyMetric); + } + if (other.is_set_componentMetric()) { + Map __this__componentMetric = new HashMap(other.componentMetric.size()); + for (Map.Entry other_element : other.componentMetric.entrySet()) { + + String other_element_key = other_element.getKey(); + MetricInfo other_element_value = other_element.getValue(); + + String __this__componentMetric_copy_key = other_element_key; + + MetricInfo __this__componentMetric_copy_value = new MetricInfo(other_element_value); + + __this__componentMetric.put(__this__componentMetric_copy_key, __this__componentMetric_copy_value); + } + this.componentMetric = __this__componentMetric; + } + if (other.is_set_workerMetric()) { + Map __this__workerMetric = new HashMap(other.workerMetric.size()); + for (Map.Entry other_element : other.workerMetric.entrySet()) { + + String other_element_key = other_element.getKey(); + MetricInfo other_element_value = other_element.getValue(); + + String __this__workerMetric_copy_key = other_element_key; + + MetricInfo __this__workerMetric_copy_value = new MetricInfo(other_element_value); + + __this__workerMetric.put(__this__workerMetric_copy_key, __this__workerMetric_copy_value); + } + this.workerMetric = __this__workerMetric; + } + if (other.is_set_taskMetric()) { + Map __this__taskMetric = new HashMap(other.taskMetric.size()); + for (Map.Entry other_element : other.taskMetric.entrySet()) { + + Integer other_element_key = other_element.getKey(); + MetricInfo other_element_value = other_element.getValue(); + + Integer __this__taskMetric_copy_key = other_element_key; + + MetricInfo __this__taskMetric_copy_value = new MetricInfo(other_element_value); + + __this__taskMetric.put(__this__taskMetric_copy_key, __this__taskMetric_copy_value); + } + this.taskMetric = __this__taskMetric; + } + } + + public TopologyMetric deepCopy() { + return new TopologyMetric(this); + } + + @Override + public void clear() { + this.topologyMetric = null; + this.componentMetric = null; + this.workerMetric = null; + this.taskMetric = null; + } + + public MetricInfo get_topologyMetric() { + return this.topologyMetric; + } + + public void set_topologyMetric(MetricInfo topologyMetric) { + this.topologyMetric = topologyMetric; + } + + public void unset_topologyMetric() { + this.topologyMetric = null; + } + + /** Returns true if field topologyMetric is set (has been assigned a value) and false otherwise */ + public boolean is_set_topologyMetric() { + return this.topologyMetric != null; + } + + public void set_topologyMetric_isSet(boolean value) { + if (!value) { + this.topologyMetric = null; + } + } + + public int get_componentMetric_size() { + return (this.componentMetric == null) ? 0 : this.componentMetric.size(); + } + + public void put_to_componentMetric(String key, MetricInfo val) { + if (this.componentMetric == null) { + this.componentMetric = new HashMap(); + } + this.componentMetric.put(key, val); + } + + public Map get_componentMetric() { + return this.componentMetric; + } + + public void set_componentMetric(Map componentMetric) { + this.componentMetric = componentMetric; + } + + public void unset_componentMetric() { + this.componentMetric = null; + } + + /** Returns true if field componentMetric is set (has been assigned a value) and false otherwise */ + public boolean is_set_componentMetric() { + return this.componentMetric != null; + } + + public void set_componentMetric_isSet(boolean value) { + if (!value) { + this.componentMetric = null; + } + } + + public int get_workerMetric_size() { + return (this.workerMetric == null) ? 0 : this.workerMetric.size(); + } + + public void put_to_workerMetric(String key, MetricInfo val) { + if (this.workerMetric == null) { + this.workerMetric = new HashMap(); + } + this.workerMetric.put(key, val); + } + + public Map get_workerMetric() { + return this.workerMetric; + } + + public void set_workerMetric(Map workerMetric) { + this.workerMetric = workerMetric; + } + + public void unset_workerMetric() { + this.workerMetric = null; + } + + /** Returns true if field workerMetric is set (has been assigned a value) and false otherwise */ + public boolean is_set_workerMetric() { + return this.workerMetric != null; + } + + public void set_workerMetric_isSet(boolean value) { + if (!value) { + this.workerMetric = null; + } + } + + public int get_taskMetric_size() { + return (this.taskMetric == null) ? 0 : this.taskMetric.size(); + } + + public void put_to_taskMetric(int key, MetricInfo val) { + if (this.taskMetric == null) { + this.taskMetric = new HashMap(); + } + this.taskMetric.put(key, val); + } + + public Map get_taskMetric() { + return this.taskMetric; + } + + public void set_taskMetric(Map taskMetric) { + this.taskMetric = taskMetric; + } + + public void unset_taskMetric() { + this.taskMetric = null; + } + + /** Returns true if field taskMetric is set (has been assigned a value) and false otherwise */ + public boolean is_set_taskMetric() { + return this.taskMetric != null; + } + + public void set_taskMetric_isSet(boolean value) { + if (!value) { + this.taskMetric = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TOPOLOGY_METRIC: + if (value == null) { + unset_topologyMetric(); + } else { + set_topologyMetric((MetricInfo)value); + } + break; + + case COMPONENT_METRIC: + if (value == null) { + unset_componentMetric(); + } else { + set_componentMetric((Map)value); + } + break; + + case WORKER_METRIC: + if (value == null) { + unset_workerMetric(); + } else { + set_workerMetric((Map)value); + } + break; + + case TASK_METRIC: + if (value == null) { + unset_taskMetric(); + } else { + set_taskMetric((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TOPOLOGY_METRIC: + return get_topologyMetric(); + + case COMPONENT_METRIC: + return get_componentMetric(); + + case WORKER_METRIC: + return get_workerMetric(); + + case TASK_METRIC: + return get_taskMetric(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TOPOLOGY_METRIC: + return is_set_topologyMetric(); + case COMPONENT_METRIC: + return is_set_componentMetric(); + case WORKER_METRIC: + return is_set_workerMetric(); + case TASK_METRIC: + return is_set_taskMetric(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TopologyMetric) + return this.equals((TopologyMetric)that); + return false; + } + + public boolean equals(TopologyMetric that) { + if (that == null) + return false; + + boolean this_present_topologyMetric = true && this.is_set_topologyMetric(); + boolean that_present_topologyMetric = true && that.is_set_topologyMetric(); + if (this_present_topologyMetric || that_present_topologyMetric) { + if (!(this_present_topologyMetric && that_present_topologyMetric)) + return false; + if (!this.topologyMetric.equals(that.topologyMetric)) + return false; + } + + boolean this_present_componentMetric = true && this.is_set_componentMetric(); + boolean that_present_componentMetric = true && that.is_set_componentMetric(); + if (this_present_componentMetric || that_present_componentMetric) { + if (!(this_present_componentMetric && that_present_componentMetric)) + return false; + if (!this.componentMetric.equals(that.componentMetric)) + return false; + } + + boolean this_present_workerMetric = true && this.is_set_workerMetric(); + boolean that_present_workerMetric = true && that.is_set_workerMetric(); + if (this_present_workerMetric || that_present_workerMetric) { + if (!(this_present_workerMetric && that_present_workerMetric)) + return false; + if (!this.workerMetric.equals(that.workerMetric)) + return false; + } + + boolean this_present_taskMetric = true && this.is_set_taskMetric(); + boolean that_present_taskMetric = true && that.is_set_taskMetric(); + if (this_present_taskMetric || that_present_taskMetric) { + if (!(this_present_taskMetric && that_present_taskMetric)) + return false; + if (!this.taskMetric.equals(that.taskMetric)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_topologyMetric = true && (is_set_topologyMetric()); + list.add(present_topologyMetric); + if (present_topologyMetric) + list.add(topologyMetric); + + boolean present_componentMetric = true && (is_set_componentMetric()); + list.add(present_componentMetric); + if (present_componentMetric) + list.add(componentMetric); + + boolean present_workerMetric = true && (is_set_workerMetric()); + list.add(present_workerMetric); + if (present_workerMetric) + list.add(workerMetric); + + boolean present_taskMetric = true && (is_set_taskMetric()); + list.add(present_taskMetric); + if (present_taskMetric) + list.add(taskMetric); + + return list.hashCode(); + } + + @Override + public int compareTo(TopologyMetric other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_topologyMetric()).compareTo(other.is_set_topologyMetric()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_topologyMetric()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topologyMetric, other.topologyMetric); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_componentMetric()).compareTo(other.is_set_componentMetric()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_componentMetric()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.componentMetric, other.componentMetric); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_workerMetric()).compareTo(other.is_set_workerMetric()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_workerMetric()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.workerMetric, other.workerMetric); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_taskMetric()).compareTo(other.is_set_taskMetric()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_taskMetric()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.taskMetric, other.taskMetric); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TopologyMetric("); + boolean first = true; + + sb.append("topologyMetric:"); + if (this.topologyMetric == null) { + sb.append("null"); + } else { + sb.append(this.topologyMetric); + } + first = false; + if (!first) sb.append(", "); + sb.append("componentMetric:"); + if (this.componentMetric == null) { + sb.append("null"); + } else { + sb.append(this.componentMetric); + } + first = false; + if (!first) sb.append(", "); + sb.append("workerMetric:"); + if (this.workerMetric == null) { + sb.append("null"); + } else { + sb.append(this.workerMetric); + } + first = false; + if (!first) sb.append(", "); + sb.append("taskMetric:"); + if (this.taskMetric == null) { + sb.append("null"); + } else { + sb.append(this.taskMetric); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_topologyMetric()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'topologyMetric' is unset! Struct:" + toString()); + } + + if (!is_set_componentMetric()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'componentMetric' is unset! Struct:" + toString()); + } + + if (!is_set_workerMetric()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'workerMetric' is unset! Struct:" + toString()); + } + + if (!is_set_taskMetric()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'taskMetric' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (topologyMetric != null) { + topologyMetric.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TopologyMetricStandardSchemeFactory implements SchemeFactory { + public TopologyMetricStandardScheme getScheme() { + return new TopologyMetricStandardScheme(); + } + } + + private static class TopologyMetricStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TopologyMetric struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TOPOLOGY_METRIC + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.topologyMetric = new MetricInfo(); + struct.topologyMetric.read(iprot); + struct.set_topologyMetric_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // COMPONENT_METRIC + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map166 = iprot.readMapBegin(); + struct.componentMetric = new HashMap(2*_map166.size); + String _key167; + MetricInfo _val168; + for (int _i169 = 0; _i169 < _map166.size; ++_i169) + { + _key167 = iprot.readString(); + _val168 = new MetricInfo(); + _val168.read(iprot); + struct.componentMetric.put(_key167, _val168); + } + iprot.readMapEnd(); + } + struct.set_componentMetric_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // WORKER_METRIC + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map170 = iprot.readMapBegin(); + struct.workerMetric = new HashMap(2*_map170.size); + String _key171; + MetricInfo _val172; + for (int _i173 = 0; _i173 < _map170.size; ++_i173) + { + _key171 = iprot.readString(); + _val172 = new MetricInfo(); + _val172.read(iprot); + struct.workerMetric.put(_key171, _val172); + } + iprot.readMapEnd(); + } + struct.set_workerMetric_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // TASK_METRIC + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map174 = iprot.readMapBegin(); + struct.taskMetric = new HashMap(2*_map174.size); + int _key175; + MetricInfo _val176; + for (int _i177 = 0; _i177 < _map174.size; ++_i177) + { + _key175 = iprot.readI32(); + _val176 = new MetricInfo(); + _val176.read(iprot); + struct.taskMetric.put(_key175, _val176); + } + iprot.readMapEnd(); + } + struct.set_taskMetric_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TopologyMetric struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.topologyMetric != null) { + oprot.writeFieldBegin(TOPOLOGY_METRIC_FIELD_DESC); + struct.topologyMetric.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.componentMetric != null) { + oprot.writeFieldBegin(COMPONENT_METRIC_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.componentMetric.size())); + for (Map.Entry _iter178 : struct.componentMetric.entrySet()) + { + oprot.writeString(_iter178.getKey()); + _iter178.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.workerMetric != null) { + oprot.writeFieldBegin(WORKER_METRIC_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.workerMetric.size())); + for (Map.Entry _iter179 : struct.workerMetric.entrySet()) + { + oprot.writeString(_iter179.getKey()); + _iter179.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.taskMetric != null) { + oprot.writeFieldBegin(TASK_METRIC_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, struct.taskMetric.size())); + for (Map.Entry _iter180 : struct.taskMetric.entrySet()) + { + oprot.writeI32(_iter180.getKey()); + _iter180.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TopologyMetricTupleSchemeFactory implements SchemeFactory { + public TopologyMetricTupleScheme getScheme() { + return new TopologyMetricTupleScheme(); + } + } + + private static class TopologyMetricTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TopologyMetric struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.topologyMetric.write(oprot); + { + oprot.writeI32(struct.componentMetric.size()); + for (Map.Entry _iter181 : struct.componentMetric.entrySet()) + { + oprot.writeString(_iter181.getKey()); + _iter181.getValue().write(oprot); + } + } + { + oprot.writeI32(struct.workerMetric.size()); + for (Map.Entry _iter182 : struct.workerMetric.entrySet()) + { + oprot.writeString(_iter182.getKey()); + _iter182.getValue().write(oprot); + } + } + { + oprot.writeI32(struct.taskMetric.size()); + for (Map.Entry _iter183 : struct.taskMetric.entrySet()) + { + oprot.writeI32(_iter183.getKey()); + _iter183.getValue().write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TopologyMetric struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.topologyMetric = new MetricInfo(); + struct.topologyMetric.read(iprot); + struct.set_topologyMetric_isSet(true); + { + org.apache.thrift.protocol.TMap _map184 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.componentMetric = new HashMap(2*_map184.size); + String _key185; + MetricInfo _val186; + for (int _i187 = 0; _i187 < _map184.size; ++_i187) + { + _key185 = iprot.readString(); + _val186 = new MetricInfo(); + _val186.read(iprot); + struct.componentMetric.put(_key185, _val186); + } + } + struct.set_componentMetric_isSet(true); + { + org.apache.thrift.protocol.TMap _map188 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.workerMetric = new HashMap(2*_map188.size); + String _key189; + MetricInfo _val190; + for (int _i191 = 0; _i191 < _map188.size; ++_i191) + { + _key189 = iprot.readString(); + _val190 = new MetricInfo(); + _val190.read(iprot); + struct.workerMetric.put(_key189, _val190); + } + } + struct.set_workerMetric_isSet(true); + { + org.apache.thrift.protocol.TMap _map192 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.taskMetric = new HashMap(2*_map192.size); + int _key193; + MetricInfo _val194; + for (int _i195 = 0; _i195 < _map192.size; ++_i195) + { + _key193 = iprot.readI32(); + _val194 = new MetricInfo(); + _val194.read(iprot); + struct.taskMetric.put(_key193, _val194); + } + } + struct.set_taskMetric_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/TopologySummary.java b/jstorm-core/src/main/java/backtype/storm/generated/TopologySummary.java new file mode 100644 index 000000000..febb0958d --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/TopologySummary.java @@ -0,0 +1,990 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class TopologySummary implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TopologySummary"); + + private static final org.apache.thrift.protocol.TField ID_FIELD_DESC = new org.apache.thrift.protocol.TField("id", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("name", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField UPTIME_SECS_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime_secs", org.apache.thrift.protocol.TType.I32, (short)4); + private static final org.apache.thrift.protocol.TField NUM_TASKS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_tasks", org.apache.thrift.protocol.TType.I32, (short)5); + private static final org.apache.thrift.protocol.TField NUM_WORKERS_FIELD_DESC = new org.apache.thrift.protocol.TField("num_workers", org.apache.thrift.protocol.TType.I32, (short)6); + private static final org.apache.thrift.protocol.TField ERROR_INFO_FIELD_DESC = new org.apache.thrift.protocol.TField("error_info", org.apache.thrift.protocol.TType.STRING, (short)7); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TopologySummaryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TopologySummaryTupleSchemeFactory()); + } + + private String id; // required + private String name; // required + private String status; // required + private int uptime_secs; // required + private int num_tasks; // required + private int num_workers; // required + private String error_info; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + ID((short)1, "id"), + NAME((short)2, "name"), + STATUS((short)3, "status"), + UPTIME_SECS((short)4, "uptime_secs"), + NUM_TASKS((short)5, "num_tasks"), + NUM_WORKERS((short)6, "num_workers"), + ERROR_INFO((short)7, "error_info"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // ID + return ID; + case 2: // NAME + return NAME; + case 3: // STATUS + return STATUS; + case 4: // UPTIME_SECS + return UPTIME_SECS; + case 5: // NUM_TASKS + return NUM_TASKS; + case 6: // NUM_WORKERS + return NUM_WORKERS; + case 7: // ERROR_INFO + return ERROR_INFO; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __UPTIME_SECS_ISSET_ID = 0; + private static final int __NUM_TASKS_ISSET_ID = 1; + private static final int __NUM_WORKERS_ISSET_ID = 2; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.ERROR_INFO}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.ID, new org.apache.thrift.meta_data.FieldMetaData("id", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.NAME, new org.apache.thrift.meta_data.FieldMetaData("name", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.UPTIME_SECS, new org.apache.thrift.meta_data.FieldMetaData("uptime_secs", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.NUM_TASKS, new org.apache.thrift.meta_data.FieldMetaData("num_tasks", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.NUM_WORKERS, new org.apache.thrift.meta_data.FieldMetaData("num_workers", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.ERROR_INFO, new org.apache.thrift.meta_data.FieldMetaData("error_info", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TopologySummary.class, metaDataMap); + } + + public TopologySummary() { + } + + public TopologySummary( + String id, + String name, + String status, + int uptime_secs, + int num_tasks, + int num_workers) + { + this(); + this.id = id; + this.name = name; + this.status = status; + this.uptime_secs = uptime_secs; + set_uptime_secs_isSet(true); + this.num_tasks = num_tasks; + set_num_tasks_isSet(true); + this.num_workers = num_workers; + set_num_workers_isSet(true); + } + + /** + * Performs a deep copy on other. + */ + public TopologySummary(TopologySummary other) { + __isset_bitfield = other.__isset_bitfield; + if (other.is_set_id()) { + this.id = other.id; + } + if (other.is_set_name()) { + this.name = other.name; + } + if (other.is_set_status()) { + this.status = other.status; + } + this.uptime_secs = other.uptime_secs; + this.num_tasks = other.num_tasks; + this.num_workers = other.num_workers; + if (other.is_set_error_info()) { + this.error_info = other.error_info; + } + } + + public TopologySummary deepCopy() { + return new TopologySummary(this); + } + + @Override + public void clear() { + this.id = null; + this.name = null; + this.status = null; + set_uptime_secs_isSet(false); + this.uptime_secs = 0; + set_num_tasks_isSet(false); + this.num_tasks = 0; + set_num_workers_isSet(false); + this.num_workers = 0; + this.error_info = null; + } + + public String get_id() { + return this.id; + } + + public void set_id(String id) { + this.id = id; + } + + public void unset_id() { + this.id = null; + } + + /** Returns true if field id is set (has been assigned a value) and false otherwise */ + public boolean is_set_id() { + return this.id != null; + } + + public void set_id_isSet(boolean value) { + if (!value) { + this.id = null; + } + } + + public String get_name() { + return this.name; + } + + public void set_name(String name) { + this.name = name; + } + + public void unset_name() { + this.name = null; + } + + /** Returns true if field name is set (has been assigned a value) and false otherwise */ + public boolean is_set_name() { + return this.name != null; + } + + public void set_name_isSet(boolean value) { + if (!value) { + this.name = null; + } + } + + public String get_status() { + return this.status; + } + + public void set_status(String status) { + this.status = status; + } + + public void unset_status() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean is_set_status() { + return this.status != null; + } + + public void set_status_isSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public int get_uptime_secs() { + return this.uptime_secs; + } + + public void set_uptime_secs(int uptime_secs) { + this.uptime_secs = uptime_secs; + set_uptime_secs_isSet(true); + } + + public void unset_uptime_secs() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID); + } + + /** Returns true if field uptime_secs is set (has been assigned a value) and false otherwise */ + public boolean is_set_uptime_secs() { + return EncodingUtils.testBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID); + } + + public void set_uptime_secs_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_SECS_ISSET_ID, value); + } + + public int get_num_tasks() { + return this.num_tasks; + } + + public void set_num_tasks(int num_tasks) { + this.num_tasks = num_tasks; + set_num_tasks_isSet(true); + } + + public void unset_num_tasks() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_TASKS_ISSET_ID); + } + + /** Returns true if field num_tasks is set (has been assigned a value) and false otherwise */ + public boolean is_set_num_tasks() { + return EncodingUtils.testBit(__isset_bitfield, __NUM_TASKS_ISSET_ID); + } + + public void set_num_tasks_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_TASKS_ISSET_ID, value); + } + + public int get_num_workers() { + return this.num_workers; + } + + public void set_num_workers(int num_workers) { + this.num_workers = num_workers; + set_num_workers_isSet(true); + } + + public void unset_num_workers() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID); + } + + /** Returns true if field num_workers is set (has been assigned a value) and false otherwise */ + public boolean is_set_num_workers() { + return EncodingUtils.testBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID); + } + + public void set_num_workers_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NUM_WORKERS_ISSET_ID, value); + } + + public String get_error_info() { + return this.error_info; + } + + public void set_error_info(String error_info) { + this.error_info = error_info; + } + + public void unset_error_info() { + this.error_info = null; + } + + /** Returns true if field error_info is set (has been assigned a value) and false otherwise */ + public boolean is_set_error_info() { + return this.error_info != null; + } + + public void set_error_info_isSet(boolean value) { + if (!value) { + this.error_info = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case ID: + if (value == null) { + unset_id(); + } else { + set_id((String)value); + } + break; + + case NAME: + if (value == null) { + unset_name(); + } else { + set_name((String)value); + } + break; + + case STATUS: + if (value == null) { + unset_status(); + } else { + set_status((String)value); + } + break; + + case UPTIME_SECS: + if (value == null) { + unset_uptime_secs(); + } else { + set_uptime_secs((Integer)value); + } + break; + + case NUM_TASKS: + if (value == null) { + unset_num_tasks(); + } else { + set_num_tasks((Integer)value); + } + break; + + case NUM_WORKERS: + if (value == null) { + unset_num_workers(); + } else { + set_num_workers((Integer)value); + } + break; + + case ERROR_INFO: + if (value == null) { + unset_error_info(); + } else { + set_error_info((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case ID: + return get_id(); + + case NAME: + return get_name(); + + case STATUS: + return get_status(); + + case UPTIME_SECS: + return Integer.valueOf(get_uptime_secs()); + + case NUM_TASKS: + return Integer.valueOf(get_num_tasks()); + + case NUM_WORKERS: + return Integer.valueOf(get_num_workers()); + + case ERROR_INFO: + return get_error_info(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case ID: + return is_set_id(); + case NAME: + return is_set_name(); + case STATUS: + return is_set_status(); + case UPTIME_SECS: + return is_set_uptime_secs(); + case NUM_TASKS: + return is_set_num_tasks(); + case NUM_WORKERS: + return is_set_num_workers(); + case ERROR_INFO: + return is_set_error_info(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TopologySummary) + return this.equals((TopologySummary)that); + return false; + } + + public boolean equals(TopologySummary that) { + if (that == null) + return false; + + boolean this_present_id = true && this.is_set_id(); + boolean that_present_id = true && that.is_set_id(); + if (this_present_id || that_present_id) { + if (!(this_present_id && that_present_id)) + return false; + if (!this.id.equals(that.id)) + return false; + } + + boolean this_present_name = true && this.is_set_name(); + boolean that_present_name = true && that.is_set_name(); + if (this_present_name || that_present_name) { + if (!(this_present_name && that_present_name)) + return false; + if (!this.name.equals(that.name)) + return false; + } + + boolean this_present_status = true && this.is_set_status(); + boolean that_present_status = true && that.is_set_status(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_uptime_secs = true; + boolean that_present_uptime_secs = true; + if (this_present_uptime_secs || that_present_uptime_secs) { + if (!(this_present_uptime_secs && that_present_uptime_secs)) + return false; + if (this.uptime_secs != that.uptime_secs) + return false; + } + + boolean this_present_num_tasks = true; + boolean that_present_num_tasks = true; + if (this_present_num_tasks || that_present_num_tasks) { + if (!(this_present_num_tasks && that_present_num_tasks)) + return false; + if (this.num_tasks != that.num_tasks) + return false; + } + + boolean this_present_num_workers = true; + boolean that_present_num_workers = true; + if (this_present_num_workers || that_present_num_workers) { + if (!(this_present_num_workers && that_present_num_workers)) + return false; + if (this.num_workers != that.num_workers) + return false; + } + + boolean this_present_error_info = true && this.is_set_error_info(); + boolean that_present_error_info = true && that.is_set_error_info(); + if (this_present_error_info || that_present_error_info) { + if (!(this_present_error_info && that_present_error_info)) + return false; + if (!this.error_info.equals(that.error_info)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_id = true && (is_set_id()); + list.add(present_id); + if (present_id) + list.add(id); + + boolean present_name = true && (is_set_name()); + list.add(present_name); + if (present_name) + list.add(name); + + boolean present_status = true && (is_set_status()); + list.add(present_status); + if (present_status) + list.add(status); + + boolean present_uptime_secs = true; + list.add(present_uptime_secs); + if (present_uptime_secs) + list.add(uptime_secs); + + boolean present_num_tasks = true; + list.add(present_num_tasks); + if (present_num_tasks) + list.add(num_tasks); + + boolean present_num_workers = true; + list.add(present_num_workers); + if (present_num_workers) + list.add(num_workers); + + boolean present_error_info = true && (is_set_error_info()); + list.add(present_error_info); + if (present_error_info) + list.add(error_info); + + return list.hashCode(); + } + + @Override + public int compareTo(TopologySummary other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_id()).compareTo(other.is_set_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.id, other.id); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_name()).compareTo(other.is_set_name()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_name()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.name, other.name); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_status()).compareTo(other.is_set_status()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_status()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(other.is_set_uptime_secs()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_uptime_secs()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime_secs, other.uptime_secs); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_num_tasks()).compareTo(other.is_set_num_tasks()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_num_tasks()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_tasks, other.num_tasks); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_num_workers()).compareTo(other.is_set_num_workers()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_num_workers()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.num_workers, other.num_workers); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_error_info()).compareTo(other.is_set_error_info()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_error_info()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.error_info, other.error_info); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TopologySummary("); + boolean first = true; + + sb.append("id:"); + if (this.id == null) { + sb.append("null"); + } else { + sb.append(this.id); + } + first = false; + if (!first) sb.append(", "); + sb.append("name:"); + if (this.name == null) { + sb.append("null"); + } else { + sb.append(this.name); + } + first = false; + if (!first) sb.append(", "); + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (!first) sb.append(", "); + sb.append("uptime_secs:"); + sb.append(this.uptime_secs); + first = false; + if (!first) sb.append(", "); + sb.append("num_tasks:"); + sb.append(this.num_tasks); + first = false; + if (!first) sb.append(", "); + sb.append("num_workers:"); + sb.append(this.num_workers); + first = false; + if (is_set_error_info()) { + if (!first) sb.append(", "); + sb.append("error_info:"); + if (this.error_info == null) { + sb.append("null"); + } else { + sb.append(this.error_info); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_id()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'id' is unset! Struct:" + toString()); + } + + if (!is_set_name()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'name' is unset! Struct:" + toString()); + } + + if (!is_set_status()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + if (!is_set_uptime_secs()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString()); + } + + if (!is_set_num_tasks()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'num_tasks' is unset! Struct:" + toString()); + } + + if (!is_set_num_workers()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'num_workers' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TopologySummaryStandardSchemeFactory implements SchemeFactory { + public TopologySummaryStandardScheme getScheme() { + return new TopologySummaryStandardScheme(); + } + } + + private static class TopologySummaryStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TopologySummary struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.id = iprot.readString(); + struct.set_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.name = iprot.readString(); + struct.set_name_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.status = iprot.readString(); + struct.set_status_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // UPTIME_SECS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.uptime_secs = iprot.readI32(); + struct.set_uptime_secs_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // NUM_TASKS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.num_tasks = iprot.readI32(); + struct.set_num_tasks_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 6: // NUM_WORKERS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.num_workers = iprot.readI32(); + struct.set_num_workers_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 7: // ERROR_INFO + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.error_info = iprot.readString(); + struct.set_error_info_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TopologySummary struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.id != null) { + oprot.writeFieldBegin(ID_FIELD_DESC); + oprot.writeString(struct.id); + oprot.writeFieldEnd(); + } + if (struct.name != null) { + oprot.writeFieldBegin(NAME_FIELD_DESC); + oprot.writeString(struct.name); + oprot.writeFieldEnd(); + } + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + oprot.writeString(struct.status); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC); + oprot.writeI32(struct.uptime_secs); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(NUM_TASKS_FIELD_DESC); + oprot.writeI32(struct.num_tasks); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC); + oprot.writeI32(struct.num_workers); + oprot.writeFieldEnd(); + if (struct.error_info != null) { + if (struct.is_set_error_info()) { + oprot.writeFieldBegin(ERROR_INFO_FIELD_DESC); + oprot.writeString(struct.error_info); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TopologySummaryTupleSchemeFactory implements SchemeFactory { + public TopologySummaryTupleScheme getScheme() { + return new TopologySummaryTupleScheme(); + } + } + + private static class TopologySummaryTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TopologySummary struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.id); + oprot.writeString(struct.name); + oprot.writeString(struct.status); + oprot.writeI32(struct.uptime_secs); + oprot.writeI32(struct.num_tasks); + oprot.writeI32(struct.num_workers); + BitSet optionals = new BitSet(); + if (struct.is_set_error_info()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.is_set_error_info()) { + oprot.writeString(struct.error_info); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TopologySummary struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.id = iprot.readString(); + struct.set_id_isSet(true); + struct.name = iprot.readString(); + struct.set_name_isSet(true); + struct.status = iprot.readString(); + struct.set_status_isSet(true); + struct.uptime_secs = iprot.readI32(); + struct.set_uptime_secs_isSet(true); + struct.num_tasks = iprot.readI32(); + struct.set_num_tasks_isSet(true); + struct.num_workers = iprot.readI32(); + struct.set_num_workers_isSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.error_info = iprot.readString(); + struct.set_error_info_isSet(true); + } + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/WorkerSummary.java b/jstorm-core/src/main/java/backtype/storm/generated/WorkerSummary.java new file mode 100644 index 000000000..9faaee713 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/WorkerSummary.java @@ -0,0 +1,738 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class WorkerSummary implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("WorkerSummary"); + + private static final org.apache.thrift.protocol.TField PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("port", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField UPTIME_FIELD_DESC = new org.apache.thrift.protocol.TField("uptime", org.apache.thrift.protocol.TType.I32, (short)2); + private static final org.apache.thrift.protocol.TField TOPOLOGY_FIELD_DESC = new org.apache.thrift.protocol.TField("topology", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField TASKS_FIELD_DESC = new org.apache.thrift.protocol.TField("tasks", org.apache.thrift.protocol.TType.LIST, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new WorkerSummaryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new WorkerSummaryTupleSchemeFactory()); + } + + private int port; // required + private int uptime; // required + private String topology; // required + private List tasks; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + PORT((short)1, "port"), + UPTIME((short)2, "uptime"), + TOPOLOGY((short)3, "topology"), + TASKS((short)4, "tasks"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // PORT + return PORT; + case 2: // UPTIME + return UPTIME; + case 3: // TOPOLOGY + return TOPOLOGY; + case 4: // TASKS + return TASKS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __PORT_ISSET_ID = 0; + private static final int __UPTIME_ISSET_ID = 1; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.PORT, new org.apache.thrift.meta_data.FieldMetaData("port", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.UPTIME, new org.apache.thrift.meta_data.FieldMetaData("uptime", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.TOPOLOGY, new org.apache.thrift.meta_data.FieldMetaData("topology", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TASKS, new org.apache.thrift.meta_data.FieldMetaData("tasks", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TaskComponent.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(WorkerSummary.class, metaDataMap); + } + + public WorkerSummary() { + } + + public WorkerSummary( + int port, + int uptime, + String topology, + List tasks) + { + this(); + this.port = port; + set_port_isSet(true); + this.uptime = uptime; + set_uptime_isSet(true); + this.topology = topology; + this.tasks = tasks; + } + + /** + * Performs a deep copy on other. + */ + public WorkerSummary(WorkerSummary other) { + __isset_bitfield = other.__isset_bitfield; + this.port = other.port; + this.uptime = other.uptime; + if (other.is_set_topology()) { + this.topology = other.topology; + } + if (other.is_set_tasks()) { + List __this__tasks = new ArrayList(other.tasks.size()); + for (TaskComponent other_element : other.tasks) { + __this__tasks.add(new TaskComponent(other_element)); + } + this.tasks = __this__tasks; + } + } + + public WorkerSummary deepCopy() { + return new WorkerSummary(this); + } + + @Override + public void clear() { + set_port_isSet(false); + this.port = 0; + set_uptime_isSet(false); + this.uptime = 0; + this.topology = null; + this.tasks = null; + } + + public int get_port() { + return this.port; + } + + public void set_port(int port) { + this.port = port; + set_port_isSet(true); + } + + public void unset_port() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PORT_ISSET_ID); + } + + /** Returns true if field port is set (has been assigned a value) and false otherwise */ + public boolean is_set_port() { + return EncodingUtils.testBit(__isset_bitfield, __PORT_ISSET_ID); + } + + public void set_port_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PORT_ISSET_ID, value); + } + + public int get_uptime() { + return this.uptime; + } + + public void set_uptime(int uptime) { + this.uptime = uptime; + set_uptime_isSet(true); + } + + public void unset_uptime() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __UPTIME_ISSET_ID); + } + + /** Returns true if field uptime is set (has been assigned a value) and false otherwise */ + public boolean is_set_uptime() { + return EncodingUtils.testBit(__isset_bitfield, __UPTIME_ISSET_ID); + } + + public void set_uptime_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __UPTIME_ISSET_ID, value); + } + + public String get_topology() { + return this.topology; + } + + public void set_topology(String topology) { + this.topology = topology; + } + + public void unset_topology() { + this.topology = null; + } + + /** Returns true if field topology is set (has been assigned a value) and false otherwise */ + public boolean is_set_topology() { + return this.topology != null; + } + + public void set_topology_isSet(boolean value) { + if (!value) { + this.topology = null; + } + } + + public int get_tasks_size() { + return (this.tasks == null) ? 0 : this.tasks.size(); + } + + public java.util.Iterator get_tasks_iterator() { + return (this.tasks == null) ? null : this.tasks.iterator(); + } + + public void add_to_tasks(TaskComponent elem) { + if (this.tasks == null) { + this.tasks = new ArrayList(); + } + this.tasks.add(elem); + } + + public List get_tasks() { + return this.tasks; + } + + public void set_tasks(List tasks) { + this.tasks = tasks; + } + + public void unset_tasks() { + this.tasks = null; + } + + /** Returns true if field tasks is set (has been assigned a value) and false otherwise */ + public boolean is_set_tasks() { + return this.tasks != null; + } + + public void set_tasks_isSet(boolean value) { + if (!value) { + this.tasks = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case PORT: + if (value == null) { + unset_port(); + } else { + set_port((Integer)value); + } + break; + + case UPTIME: + if (value == null) { + unset_uptime(); + } else { + set_uptime((Integer)value); + } + break; + + case TOPOLOGY: + if (value == null) { + unset_topology(); + } else { + set_topology((String)value); + } + break; + + case TASKS: + if (value == null) { + unset_tasks(); + } else { + set_tasks((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case PORT: + return Integer.valueOf(get_port()); + + case UPTIME: + return Integer.valueOf(get_uptime()); + + case TOPOLOGY: + return get_topology(); + + case TASKS: + return get_tasks(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case PORT: + return is_set_port(); + case UPTIME: + return is_set_uptime(); + case TOPOLOGY: + return is_set_topology(); + case TASKS: + return is_set_tasks(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof WorkerSummary) + return this.equals((WorkerSummary)that); + return false; + } + + public boolean equals(WorkerSummary that) { + if (that == null) + return false; + + boolean this_present_port = true; + boolean that_present_port = true; + if (this_present_port || that_present_port) { + if (!(this_present_port && that_present_port)) + return false; + if (this.port != that.port) + return false; + } + + boolean this_present_uptime = true; + boolean that_present_uptime = true; + if (this_present_uptime || that_present_uptime) { + if (!(this_present_uptime && that_present_uptime)) + return false; + if (this.uptime != that.uptime) + return false; + } + + boolean this_present_topology = true && this.is_set_topology(); + boolean that_present_topology = true && that.is_set_topology(); + if (this_present_topology || that_present_topology) { + if (!(this_present_topology && that_present_topology)) + return false; + if (!this.topology.equals(that.topology)) + return false; + } + + boolean this_present_tasks = true && this.is_set_tasks(); + boolean that_present_tasks = true && that.is_set_tasks(); + if (this_present_tasks || that_present_tasks) { + if (!(this_present_tasks && that_present_tasks)) + return false; + if (!this.tasks.equals(that.tasks)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_port = true; + list.add(present_port); + if (present_port) + list.add(port); + + boolean present_uptime = true; + list.add(present_uptime); + if (present_uptime) + list.add(uptime); + + boolean present_topology = true && (is_set_topology()); + list.add(present_topology); + if (present_topology) + list.add(topology); + + boolean present_tasks = true && (is_set_tasks()); + list.add(present_tasks); + if (present_tasks) + list.add(tasks); + + return list.hashCode(); + } + + @Override + public int compareTo(WorkerSummary other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_port()).compareTo(other.is_set_port()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_port()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.port, other.port); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_uptime()).compareTo(other.is_set_uptime()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_uptime()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uptime, other.uptime); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_topology()).compareTo(other.is_set_topology()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_topology()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology, other.topology); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_tasks()).compareTo(other.is_set_tasks()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_tasks()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tasks, other.tasks); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("WorkerSummary("); + boolean first = true; + + sb.append("port:"); + sb.append(this.port); + first = false; + if (!first) sb.append(", "); + sb.append("uptime:"); + sb.append(this.uptime); + first = false; + if (!first) sb.append(", "); + sb.append("topology:"); + if (this.topology == null) { + sb.append("null"); + } else { + sb.append(this.topology); + } + first = false; + if (!first) sb.append(", "); + sb.append("tasks:"); + if (this.tasks == null) { + sb.append("null"); + } else { + sb.append(this.tasks); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_port()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'port' is unset! Struct:" + toString()); + } + + if (!is_set_uptime()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'uptime' is unset! Struct:" + toString()); + } + + if (!is_set_topology()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'topology' is unset! Struct:" + toString()); + } + + if (!is_set_tasks()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'tasks' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class WorkerSummaryStandardSchemeFactory implements SchemeFactory { + public WorkerSummaryStandardScheme getScheme() { + return new WorkerSummaryStandardScheme(); + } + } + + private static class WorkerSummaryStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, WorkerSummary struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // PORT + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.port = iprot.readI32(); + struct.set_port_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // UPTIME + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.uptime = iprot.readI32(); + struct.set_uptime_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // TOPOLOGY + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.topology = iprot.readString(); + struct.set_topology_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // TASKS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list98 = iprot.readListBegin(); + struct.tasks = new ArrayList(_list98.size); + TaskComponent _elem99; + for (int _i100 = 0; _i100 < _list98.size; ++_i100) + { + _elem99 = new TaskComponent(); + _elem99.read(iprot); + struct.tasks.add(_elem99); + } + iprot.readListEnd(); + } + struct.set_tasks_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, WorkerSummary struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(PORT_FIELD_DESC); + oprot.writeI32(struct.port); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(UPTIME_FIELD_DESC); + oprot.writeI32(struct.uptime); + oprot.writeFieldEnd(); + if (struct.topology != null) { + oprot.writeFieldBegin(TOPOLOGY_FIELD_DESC); + oprot.writeString(struct.topology); + oprot.writeFieldEnd(); + } + if (struct.tasks != null) { + oprot.writeFieldBegin(TASKS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tasks.size())); + for (TaskComponent _iter101 : struct.tasks) + { + _iter101.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class WorkerSummaryTupleSchemeFactory implements SchemeFactory { + public WorkerSummaryTupleScheme getScheme() { + return new WorkerSummaryTupleScheme(); + } + } + + private static class WorkerSummaryTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, WorkerSummary struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI32(struct.port); + oprot.writeI32(struct.uptime); + oprot.writeString(struct.topology); + { + oprot.writeI32(struct.tasks.size()); + for (TaskComponent _iter102 : struct.tasks) + { + _iter102.write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, WorkerSummary struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.port = iprot.readI32(); + struct.set_port_isSet(true); + struct.uptime = iprot.readI32(); + struct.set_uptime_isSet(true); + struct.topology = iprot.readString(); + struct.set_topology_isSet(true); + { + org.apache.thrift.protocol.TList _list103 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.tasks = new ArrayList(_list103.size); + TaskComponent _elem104; + for (int _i105 = 0; _i105 < _list103.size; ++_i105) + { + _elem104 = new TaskComponent(); + _elem104.read(iprot); + struct.tasks.add(_elem104); + } + } + struct.set_tasks_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/generated/WorkerUploadMetrics.java b/jstorm-core/src/main/java/backtype/storm/generated/WorkerUploadMetrics.java new file mode 100644 index 000000000..934cfb36c --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/generated/WorkerUploadMetrics.java @@ -0,0 +1,967 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package backtype.storm.generated; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-27") +public class WorkerUploadMetrics implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("WorkerUploadMetrics"); + + private static final org.apache.thrift.protocol.TField TOPOLOGY_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("topology_id", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField SUPERVISOR_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("supervisor_id", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField PORT_FIELD_DESC = new org.apache.thrift.protocol.TField("port", org.apache.thrift.protocol.TType.I32, (short)3); + private static final org.apache.thrift.protocol.TField WORKER_METRIC_FIELD_DESC = new org.apache.thrift.protocol.TField("workerMetric", org.apache.thrift.protocol.TType.STRUCT, (short)4); + private static final org.apache.thrift.protocol.TField NETTY_METRIC_FIELD_DESC = new org.apache.thrift.protocol.TField("nettyMetric", org.apache.thrift.protocol.TType.STRUCT, (short)5); + private static final org.apache.thrift.protocol.TField TASK_METRIC_FIELD_DESC = new org.apache.thrift.protocol.TField("taskMetric", org.apache.thrift.protocol.TType.MAP, (short)6); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new WorkerUploadMetricsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new WorkerUploadMetricsTupleSchemeFactory()); + } + + private String topology_id; // required + private String supervisor_id; // required + private int port; // required + private MetricInfo workerMetric; // required + private NettyMetric nettyMetric; // required + private Map taskMetric; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TOPOLOGY_ID((short)1, "topology_id"), + SUPERVISOR_ID((short)2, "supervisor_id"), + PORT((short)3, "port"), + WORKER_METRIC((short)4, "workerMetric"), + NETTY_METRIC((short)5, "nettyMetric"), + TASK_METRIC((short)6, "taskMetric"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TOPOLOGY_ID + return TOPOLOGY_ID; + case 2: // SUPERVISOR_ID + return SUPERVISOR_ID; + case 3: // PORT + return PORT; + case 4: // WORKER_METRIC + return WORKER_METRIC; + case 5: // NETTY_METRIC + return NETTY_METRIC; + case 6: // TASK_METRIC + return TASK_METRIC; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __PORT_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TOPOLOGY_ID, new org.apache.thrift.meta_data.FieldMetaData("topology_id", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.SUPERVISOR_ID, new org.apache.thrift.meta_data.FieldMetaData("supervisor_id", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.PORT, new org.apache.thrift.meta_data.FieldMetaData("port", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.WORKER_METRIC, new org.apache.thrift.meta_data.FieldMetaData("workerMetric", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, MetricInfo.class))); + tmpMap.put(_Fields.NETTY_METRIC, new org.apache.thrift.meta_data.FieldMetaData("nettyMetric", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, NettyMetric.class))); + tmpMap.put(_Fields.TASK_METRIC, new org.apache.thrift.meta_data.FieldMetaData("taskMetric", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, MetricInfo.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(WorkerUploadMetrics.class, metaDataMap); + } + + public WorkerUploadMetrics() { + } + + public WorkerUploadMetrics( + String topology_id, + String supervisor_id, + int port, + MetricInfo workerMetric, + NettyMetric nettyMetric, + Map taskMetric) + { + this(); + this.topology_id = topology_id; + this.supervisor_id = supervisor_id; + this.port = port; + set_port_isSet(true); + this.workerMetric = workerMetric; + this.nettyMetric = nettyMetric; + this.taskMetric = taskMetric; + } + + /** + * Performs a deep copy on other. + */ + public WorkerUploadMetrics(WorkerUploadMetrics other) { + __isset_bitfield = other.__isset_bitfield; + if (other.is_set_topology_id()) { + this.topology_id = other.topology_id; + } + if (other.is_set_supervisor_id()) { + this.supervisor_id = other.supervisor_id; + } + this.port = other.port; + if (other.is_set_workerMetric()) { + this.workerMetric = new MetricInfo(other.workerMetric); + } + if (other.is_set_nettyMetric()) { + this.nettyMetric = new NettyMetric(other.nettyMetric); + } + if (other.is_set_taskMetric()) { + Map __this__taskMetric = new HashMap(other.taskMetric.size()); + for (Map.Entry other_element : other.taskMetric.entrySet()) { + + Integer other_element_key = other_element.getKey(); + MetricInfo other_element_value = other_element.getValue(); + + Integer __this__taskMetric_copy_key = other_element_key; + + MetricInfo __this__taskMetric_copy_value = new MetricInfo(other_element_value); + + __this__taskMetric.put(__this__taskMetric_copy_key, __this__taskMetric_copy_value); + } + this.taskMetric = __this__taskMetric; + } + } + + public WorkerUploadMetrics deepCopy() { + return new WorkerUploadMetrics(this); + } + + @Override + public void clear() { + this.topology_id = null; + this.supervisor_id = null; + set_port_isSet(false); + this.port = 0; + this.workerMetric = null; + this.nettyMetric = null; + this.taskMetric = null; + } + + public String get_topology_id() { + return this.topology_id; + } + + public void set_topology_id(String topology_id) { + this.topology_id = topology_id; + } + + public void unset_topology_id() { + this.topology_id = null; + } + + /** Returns true if field topology_id is set (has been assigned a value) and false otherwise */ + public boolean is_set_topology_id() { + return this.topology_id != null; + } + + public void set_topology_id_isSet(boolean value) { + if (!value) { + this.topology_id = null; + } + } + + public String get_supervisor_id() { + return this.supervisor_id; + } + + public void set_supervisor_id(String supervisor_id) { + this.supervisor_id = supervisor_id; + } + + public void unset_supervisor_id() { + this.supervisor_id = null; + } + + /** Returns true if field supervisor_id is set (has been assigned a value) and false otherwise */ + public boolean is_set_supervisor_id() { + return this.supervisor_id != null; + } + + public void set_supervisor_id_isSet(boolean value) { + if (!value) { + this.supervisor_id = null; + } + } + + public int get_port() { + return this.port; + } + + public void set_port(int port) { + this.port = port; + set_port_isSet(true); + } + + public void unset_port() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PORT_ISSET_ID); + } + + /** Returns true if field port is set (has been assigned a value) and false otherwise */ + public boolean is_set_port() { + return EncodingUtils.testBit(__isset_bitfield, __PORT_ISSET_ID); + } + + public void set_port_isSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PORT_ISSET_ID, value); + } + + public MetricInfo get_workerMetric() { + return this.workerMetric; + } + + public void set_workerMetric(MetricInfo workerMetric) { + this.workerMetric = workerMetric; + } + + public void unset_workerMetric() { + this.workerMetric = null; + } + + /** Returns true if field workerMetric is set (has been assigned a value) and false otherwise */ + public boolean is_set_workerMetric() { + return this.workerMetric != null; + } + + public void set_workerMetric_isSet(boolean value) { + if (!value) { + this.workerMetric = null; + } + } + + public NettyMetric get_nettyMetric() { + return this.nettyMetric; + } + + public void set_nettyMetric(NettyMetric nettyMetric) { + this.nettyMetric = nettyMetric; + } + + public void unset_nettyMetric() { + this.nettyMetric = null; + } + + /** Returns true if field nettyMetric is set (has been assigned a value) and false otherwise */ + public boolean is_set_nettyMetric() { + return this.nettyMetric != null; + } + + public void set_nettyMetric_isSet(boolean value) { + if (!value) { + this.nettyMetric = null; + } + } + + public int get_taskMetric_size() { + return (this.taskMetric == null) ? 0 : this.taskMetric.size(); + } + + public void put_to_taskMetric(int key, MetricInfo val) { + if (this.taskMetric == null) { + this.taskMetric = new HashMap(); + } + this.taskMetric.put(key, val); + } + + public Map get_taskMetric() { + return this.taskMetric; + } + + public void set_taskMetric(Map taskMetric) { + this.taskMetric = taskMetric; + } + + public void unset_taskMetric() { + this.taskMetric = null; + } + + /** Returns true if field taskMetric is set (has been assigned a value) and false otherwise */ + public boolean is_set_taskMetric() { + return this.taskMetric != null; + } + + public void set_taskMetric_isSet(boolean value) { + if (!value) { + this.taskMetric = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TOPOLOGY_ID: + if (value == null) { + unset_topology_id(); + } else { + set_topology_id((String)value); + } + break; + + case SUPERVISOR_ID: + if (value == null) { + unset_supervisor_id(); + } else { + set_supervisor_id((String)value); + } + break; + + case PORT: + if (value == null) { + unset_port(); + } else { + set_port((Integer)value); + } + break; + + case WORKER_METRIC: + if (value == null) { + unset_workerMetric(); + } else { + set_workerMetric((MetricInfo)value); + } + break; + + case NETTY_METRIC: + if (value == null) { + unset_nettyMetric(); + } else { + set_nettyMetric((NettyMetric)value); + } + break; + + case TASK_METRIC: + if (value == null) { + unset_taskMetric(); + } else { + set_taskMetric((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TOPOLOGY_ID: + return get_topology_id(); + + case SUPERVISOR_ID: + return get_supervisor_id(); + + case PORT: + return Integer.valueOf(get_port()); + + case WORKER_METRIC: + return get_workerMetric(); + + case NETTY_METRIC: + return get_nettyMetric(); + + case TASK_METRIC: + return get_taskMetric(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TOPOLOGY_ID: + return is_set_topology_id(); + case SUPERVISOR_ID: + return is_set_supervisor_id(); + case PORT: + return is_set_port(); + case WORKER_METRIC: + return is_set_workerMetric(); + case NETTY_METRIC: + return is_set_nettyMetric(); + case TASK_METRIC: + return is_set_taskMetric(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof WorkerUploadMetrics) + return this.equals((WorkerUploadMetrics)that); + return false; + } + + public boolean equals(WorkerUploadMetrics that) { + if (that == null) + return false; + + boolean this_present_topology_id = true && this.is_set_topology_id(); + boolean that_present_topology_id = true && that.is_set_topology_id(); + if (this_present_topology_id || that_present_topology_id) { + if (!(this_present_topology_id && that_present_topology_id)) + return false; + if (!this.topology_id.equals(that.topology_id)) + return false; + } + + boolean this_present_supervisor_id = true && this.is_set_supervisor_id(); + boolean that_present_supervisor_id = true && that.is_set_supervisor_id(); + if (this_present_supervisor_id || that_present_supervisor_id) { + if (!(this_present_supervisor_id && that_present_supervisor_id)) + return false; + if (!this.supervisor_id.equals(that.supervisor_id)) + return false; + } + + boolean this_present_port = true; + boolean that_present_port = true; + if (this_present_port || that_present_port) { + if (!(this_present_port && that_present_port)) + return false; + if (this.port != that.port) + return false; + } + + boolean this_present_workerMetric = true && this.is_set_workerMetric(); + boolean that_present_workerMetric = true && that.is_set_workerMetric(); + if (this_present_workerMetric || that_present_workerMetric) { + if (!(this_present_workerMetric && that_present_workerMetric)) + return false; + if (!this.workerMetric.equals(that.workerMetric)) + return false; + } + + boolean this_present_nettyMetric = true && this.is_set_nettyMetric(); + boolean that_present_nettyMetric = true && that.is_set_nettyMetric(); + if (this_present_nettyMetric || that_present_nettyMetric) { + if (!(this_present_nettyMetric && that_present_nettyMetric)) + return false; + if (!this.nettyMetric.equals(that.nettyMetric)) + return false; + } + + boolean this_present_taskMetric = true && this.is_set_taskMetric(); + boolean that_present_taskMetric = true && that.is_set_taskMetric(); + if (this_present_taskMetric || that_present_taskMetric) { + if (!(this_present_taskMetric && that_present_taskMetric)) + return false; + if (!this.taskMetric.equals(that.taskMetric)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_topology_id = true && (is_set_topology_id()); + list.add(present_topology_id); + if (present_topology_id) + list.add(topology_id); + + boolean present_supervisor_id = true && (is_set_supervisor_id()); + list.add(present_supervisor_id); + if (present_supervisor_id) + list.add(supervisor_id); + + boolean present_port = true; + list.add(present_port); + if (present_port) + list.add(port); + + boolean present_workerMetric = true && (is_set_workerMetric()); + list.add(present_workerMetric); + if (present_workerMetric) + list.add(workerMetric); + + boolean present_nettyMetric = true && (is_set_nettyMetric()); + list.add(present_nettyMetric); + if (present_nettyMetric) + list.add(nettyMetric); + + boolean present_taskMetric = true && (is_set_taskMetric()); + list.add(present_taskMetric); + if (present_taskMetric) + list.add(taskMetric); + + return list.hashCode(); + } + + @Override + public int compareTo(WorkerUploadMetrics other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(is_set_topology_id()).compareTo(other.is_set_topology_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_topology_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.topology_id, other.topology_id); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_supervisor_id()).compareTo(other.is_set_supervisor_id()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_supervisor_id()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.supervisor_id, other.supervisor_id); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_port()).compareTo(other.is_set_port()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_port()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.port, other.port); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_workerMetric()).compareTo(other.is_set_workerMetric()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_workerMetric()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.workerMetric, other.workerMetric); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_nettyMetric()).compareTo(other.is_set_nettyMetric()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_nettyMetric()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nettyMetric, other.nettyMetric); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(is_set_taskMetric()).compareTo(other.is_set_taskMetric()); + if (lastComparison != 0) { + return lastComparison; + } + if (is_set_taskMetric()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.taskMetric, other.taskMetric); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("WorkerUploadMetrics("); + boolean first = true; + + sb.append("topology_id:"); + if (this.topology_id == null) { + sb.append("null"); + } else { + sb.append(this.topology_id); + } + first = false; + if (!first) sb.append(", "); + sb.append("supervisor_id:"); + if (this.supervisor_id == null) { + sb.append("null"); + } else { + sb.append(this.supervisor_id); + } + first = false; + if (!first) sb.append(", "); + sb.append("port:"); + sb.append(this.port); + first = false; + if (!first) sb.append(", "); + sb.append("workerMetric:"); + if (this.workerMetric == null) { + sb.append("null"); + } else { + sb.append(this.workerMetric); + } + first = false; + if (!first) sb.append(", "); + sb.append("nettyMetric:"); + if (this.nettyMetric == null) { + sb.append("null"); + } else { + sb.append(this.nettyMetric); + } + first = false; + if (!first) sb.append(", "); + sb.append("taskMetric:"); + if (this.taskMetric == null) { + sb.append("null"); + } else { + sb.append(this.taskMetric); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!is_set_topology_id()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'topology_id' is unset! Struct:" + toString()); + } + + if (!is_set_supervisor_id()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'supervisor_id' is unset! Struct:" + toString()); + } + + if (!is_set_port()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'port' is unset! Struct:" + toString()); + } + + if (!is_set_workerMetric()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'workerMetric' is unset! Struct:" + toString()); + } + + if (!is_set_nettyMetric()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nettyMetric' is unset! Struct:" + toString()); + } + + if (!is_set_taskMetric()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'taskMetric' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (workerMetric != null) { + workerMetric.validate(); + } + if (nettyMetric != null) { + nettyMetric.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class WorkerUploadMetricsStandardSchemeFactory implements SchemeFactory { + public WorkerUploadMetricsStandardScheme getScheme() { + return new WorkerUploadMetricsStandardScheme(); + } + } + + private static class WorkerUploadMetricsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, WorkerUploadMetrics struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TOPOLOGY_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.topology_id = iprot.readString(); + struct.set_topology_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // SUPERVISOR_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.supervisor_id = iprot.readString(); + struct.set_supervisor_id_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // PORT + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.port = iprot.readI32(); + struct.set_port_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // WORKER_METRIC + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.workerMetric = new MetricInfo(); + struct.workerMetric.read(iprot); + struct.set_workerMetric_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // NETTY_METRIC + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.nettyMetric = new NettyMetric(); + struct.nettyMetric.read(iprot); + struct.set_nettyMetric_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 6: // TASK_METRIC + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map284 = iprot.readMapBegin(); + struct.taskMetric = new HashMap(2*_map284.size); + int _key285; + MetricInfo _val286; + for (int _i287 = 0; _i287 < _map284.size; ++_i287) + { + _key285 = iprot.readI32(); + _val286 = new MetricInfo(); + _val286.read(iprot); + struct.taskMetric.put(_key285, _val286); + } + iprot.readMapEnd(); + } + struct.set_taskMetric_isSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, WorkerUploadMetrics struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.topology_id != null) { + oprot.writeFieldBegin(TOPOLOGY_ID_FIELD_DESC); + oprot.writeString(struct.topology_id); + oprot.writeFieldEnd(); + } + if (struct.supervisor_id != null) { + oprot.writeFieldBegin(SUPERVISOR_ID_FIELD_DESC); + oprot.writeString(struct.supervisor_id); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(PORT_FIELD_DESC); + oprot.writeI32(struct.port); + oprot.writeFieldEnd(); + if (struct.workerMetric != null) { + oprot.writeFieldBegin(WORKER_METRIC_FIELD_DESC); + struct.workerMetric.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.nettyMetric != null) { + oprot.writeFieldBegin(NETTY_METRIC_FIELD_DESC); + struct.nettyMetric.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.taskMetric != null) { + oprot.writeFieldBegin(TASK_METRIC_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, struct.taskMetric.size())); + for (Map.Entry _iter288 : struct.taskMetric.entrySet()) + { + oprot.writeI32(_iter288.getKey()); + _iter288.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class WorkerUploadMetricsTupleSchemeFactory implements SchemeFactory { + public WorkerUploadMetricsTupleScheme getScheme() { + return new WorkerUploadMetricsTupleScheme(); + } + } + + private static class WorkerUploadMetricsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, WorkerUploadMetrics struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.topology_id); + oprot.writeString(struct.supervisor_id); + oprot.writeI32(struct.port); + struct.workerMetric.write(oprot); + struct.nettyMetric.write(oprot); + { + oprot.writeI32(struct.taskMetric.size()); + for (Map.Entry _iter289 : struct.taskMetric.entrySet()) + { + oprot.writeI32(_iter289.getKey()); + _iter289.getValue().write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, WorkerUploadMetrics struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.topology_id = iprot.readString(); + struct.set_topology_id_isSet(true); + struct.supervisor_id = iprot.readString(); + struct.set_supervisor_id_isSet(true); + struct.port = iprot.readI32(); + struct.set_port_isSet(true); + struct.workerMetric = new MetricInfo(); + struct.workerMetric.read(iprot); + struct.set_workerMetric_isSet(true); + struct.nettyMetric = new NettyMetric(); + struct.nettyMetric.read(iprot); + struct.set_nettyMetric_isSet(true); + { + org.apache.thrift.protocol.TMap _map290 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.taskMetric = new HashMap(2*_map290.size); + int _key291; + MetricInfo _val292; + for (int _i293 = 0; _i293 < _map290.size; ++_i293) + { + _key291 = iprot.readI32(); + _val292 = new MetricInfo(); + _val292.read(iprot); + struct.taskMetric.put(_key291, _val292); + } + } + struct.set_taskMetric_isSet(true); + } + } + +} + diff --git a/jstorm-core/src/main/java/backtype/storm/grouping/CustomStreamGrouping.java b/jstorm-core/src/main/java/backtype/storm/grouping/CustomStreamGrouping.java new file mode 100755 index 000000000..0e599f50e --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/grouping/CustomStreamGrouping.java @@ -0,0 +1,43 @@ +/** + * 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 backtype.storm.grouping; + +import backtype.storm.generated.GlobalStreamId; +import backtype.storm.task.WorkerTopologyContext; +import java.io.Serializable; +import java.util.List; + +public interface CustomStreamGrouping extends Serializable { + + /** + * Tells the stream grouping at runtime the tasks in the target bolt. + * This information should be used in chooseTasks to determine the target tasks. + * + * It also tells the grouping the metadata on the stream this grouping will be used on. + */ + void prepare(WorkerTopologyContext context, GlobalStreamId stream, List targetTasks); + + /** + * This function implements a custom stream grouping. It takes in as input + * the number of tasks in the target bolt in prepare and returns the + * tasks to send the tuples to. + * + * @param values the values to group on + */ + List chooseTasks(int taskId, List values); +} diff --git a/jstorm-core/src/main/java/backtype/storm/grouping/PartialKeyGrouping.java b/jstorm-core/src/main/java/backtype/storm/grouping/PartialKeyGrouping.java new file mode 100755 index 000000000..d1f534b3f --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/grouping/PartialKeyGrouping.java @@ -0,0 +1,82 @@ +/** + * 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 backtype.storm.grouping; + +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; + +import backtype.storm.generated.GlobalStreamId; +import backtype.storm.grouping.CustomStreamGrouping; +import backtype.storm.task.WorkerTopologyContext; +import backtype.storm.tuple.Fields; + +import com.google.common.hash.HashFunction; +import com.google.common.hash.Hashing; + +public class PartialKeyGrouping implements CustomStreamGrouping, Serializable { + private static final long serialVersionUID = -447379837314000353L; + private List targetTasks; + private long[] targetTaskStats; + private HashFunction h1 = Hashing.murmur3_128(13); + private HashFunction h2 = Hashing.murmur3_128(17); + private Fields fields = null; + private Fields outFields = null; + + public PartialKeyGrouping() { + //Empty + } + + public PartialKeyGrouping(Fields fields) { + this.fields = fields; + } + + @Override + public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List targetTasks) { + this.targetTasks = targetTasks; + targetTaskStats = new long[this.targetTasks.size()]; + if (this.fields != null) { + this.outFields = context.getComponentOutputFields(stream); + } + } + + @Override + public List chooseTasks(int taskId, List values) { + List boltIds = new ArrayList(1); + if (values.size() > 0) { + byte[] raw = null; + if (fields != null) { + List selectedFields = outFields.select(fields, values); + ByteBuffer out = ByteBuffer.allocate(selectedFields.size() * 4); + for (Object o: selectedFields) { + out.putInt(o.hashCode()); + } + raw = out.array(); + } else { + raw = values.get(0).toString().getBytes(); // assume key is the first field + } + int firstChoice = (int) (Math.abs(h1.hashBytes(raw).asLong()) % this.targetTasks.size()); + int secondChoice = (int) (Math.abs(h2.hashBytes(raw).asLong()) % this.targetTasks.size()); + int selected = targetTaskStats[firstChoice] > targetTaskStats[secondChoice] ? secondChoice : firstChoice; + boltIds.add(targetTasks.get(selected)); + targetTaskStats[selected]++; + } + return boltIds; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/hooks/BaseTaskHook.java b/jstorm-core/src/main/java/backtype/storm/hooks/BaseTaskHook.java new file mode 100755 index 000000000..12386d50f --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/hooks/BaseTaskHook.java @@ -0,0 +1,61 @@ +/** + * 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 backtype.storm.hooks; + +import backtype.storm.hooks.info.BoltAckInfo; +import backtype.storm.hooks.info.BoltExecuteInfo; +import backtype.storm.hooks.info.BoltFailInfo; +import backtype.storm.hooks.info.EmitInfo; +import backtype.storm.hooks.info.SpoutAckInfo; +import backtype.storm.hooks.info.SpoutFailInfo; +import backtype.storm.task.TopologyContext; +import java.util.Map; + +public class BaseTaskHook implements ITaskHook { + @Override + public void prepare(Map conf, TopologyContext context) { + } + + @Override + public void cleanup() { + } + + @Override + public void emit(EmitInfo info) { + } + + @Override + public void spoutAck(SpoutAckInfo info) { + } + + @Override + public void spoutFail(SpoutFailInfo info) { + } + + @Override + public void boltAck(BoltAckInfo info) { + } + + @Override + public void boltFail(BoltFailInfo info) { + } + + @Override + public void boltExecute(BoltExecuteInfo info) { + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/hooks/ITaskHook.java b/jstorm-core/src/main/java/backtype/storm/hooks/ITaskHook.java new file mode 100755 index 000000000..c2833ca46 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/hooks/ITaskHook.java @@ -0,0 +1,38 @@ +/** + * 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 backtype.storm.hooks; + +import backtype.storm.hooks.info.BoltAckInfo; +import backtype.storm.hooks.info.BoltExecuteInfo; +import backtype.storm.hooks.info.SpoutFailInfo; +import backtype.storm.hooks.info.SpoutAckInfo; +import backtype.storm.hooks.info.EmitInfo; +import backtype.storm.hooks.info.BoltFailInfo; +import backtype.storm.task.TopologyContext; +import java.util.Map; + +public interface ITaskHook { + void prepare(Map conf, TopologyContext context); + void cleanup(); + void emit(EmitInfo info); + void spoutAck(SpoutAckInfo info); + void spoutFail(SpoutFailInfo info); + void boltExecute(BoltExecuteInfo info); + void boltAck(BoltAckInfo info); + void boltFail(BoltFailInfo info); +} diff --git a/jstorm-core/src/main/java/backtype/storm/hooks/info/BoltAckInfo.java b/jstorm-core/src/main/java/backtype/storm/hooks/info/BoltAckInfo.java new file mode 100755 index 000000000..769a37cbd --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/hooks/info/BoltAckInfo.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.hooks.info; + +import backtype.storm.tuple.Tuple; + +public class BoltAckInfo { + public Tuple tuple; + public int ackingTaskId; + public Long processLatencyMs; // null if it wasn't sampled + + public BoltAckInfo(Tuple tuple, int ackingTaskId, Long processLatencyMs) { + this.tuple = tuple; + this.ackingTaskId = ackingTaskId; + this.processLatencyMs = processLatencyMs; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/hooks/info/BoltExecuteInfo.java b/jstorm-core/src/main/java/backtype/storm/hooks/info/BoltExecuteInfo.java new file mode 100755 index 000000000..52e2c709f --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/hooks/info/BoltExecuteInfo.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.hooks.info; + +import backtype.storm.tuple.Tuple; + +public class BoltExecuteInfo { + public Tuple tuple; + public int executingTaskId; + public Long executeLatencyMs; // null if it wasn't sampled + + public BoltExecuteInfo(Tuple tuple, int executingTaskId, Long executeLatencyMs) { + this.tuple = tuple; + this.executingTaskId = executingTaskId; + this.executeLatencyMs = executeLatencyMs; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/hooks/info/BoltFailInfo.java b/jstorm-core/src/main/java/backtype/storm/hooks/info/BoltFailInfo.java new file mode 100755 index 000000000..7dc930d48 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/hooks/info/BoltFailInfo.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.hooks.info; + +import backtype.storm.tuple.Tuple; + +public class BoltFailInfo { + public Tuple tuple; + public int failingTaskId; + public Long failLatencyMs; // null if it wasn't sampled + + public BoltFailInfo(Tuple tuple, int failingTaskId, Long failLatencyMs) { + this.tuple = tuple; + this.failingTaskId = failingTaskId; + this.failLatencyMs = failLatencyMs; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/hooks/info/EmitInfo.java b/jstorm-core/src/main/java/backtype/storm/hooks/info/EmitInfo.java new file mode 100755 index 000000000..59c01fad2 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/hooks/info/EmitInfo.java @@ -0,0 +1,35 @@ +/** + * 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 backtype.storm.hooks.info; + +import java.util.Collection; +import java.util.List; + +public class EmitInfo { + public List values; + public String stream; + public int taskId; + public Collection outTasks; + + public EmitInfo(List values, String stream, int taskId, Collection outTasks) { + this.values = values; + this.stream = stream; + this.taskId = taskId; + this.outTasks = outTasks; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/hooks/info/SpoutAckInfo.java b/jstorm-core/src/main/java/backtype/storm/hooks/info/SpoutAckInfo.java new file mode 100755 index 000000000..962f998e8 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/hooks/info/SpoutAckInfo.java @@ -0,0 +1,30 @@ +/** + * 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 backtype.storm.hooks.info; + +public class SpoutAckInfo { + public Object messageId; + public int spoutTaskId; + public Long completeLatencyMs; // null if it wasn't sampled + + public SpoutAckInfo(Object messageId, int spoutTaskId, Long completeLatencyMs) { + this.messageId = messageId; + this.spoutTaskId = spoutTaskId; + this.completeLatencyMs = completeLatencyMs; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/hooks/info/SpoutFailInfo.java b/jstorm-core/src/main/java/backtype/storm/hooks/info/SpoutFailInfo.java new file mode 100755 index 000000000..493d1e4d1 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/hooks/info/SpoutFailInfo.java @@ -0,0 +1,30 @@ +/** + * 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 backtype.storm.hooks.info; + +public class SpoutFailInfo { + public Object messageId; + public int spoutTaskId; + public Long failLatencyMs; // null if it wasn't sampled + + public SpoutFailInfo(Object messageId, int spoutTaskId, Long failLatencyMs) { + this.messageId = messageId; + this.spoutTaskId = spoutTaskId; + this.failLatencyMs = failLatencyMs; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/messaging/ConnectionWithStatus.java b/jstorm-core/src/main/java/backtype/storm/messaging/ConnectionWithStatus.java new file mode 100644 index 000000000..37981ca90 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/messaging/ConnectionWithStatus.java @@ -0,0 +1,49 @@ +/** + * 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 backtype.storm.messaging; + +public abstract class ConnectionWithStatus implements IConnection { + + public static enum Status { + + /** + * we are establishing a active connection with target host. The new data + * sending request can be buffered for future sending, or dropped(cases like + * there is no enough memory). It varies with difference IConnection + * implementations. + */ + Connecting, + + /** + * We have a alive connection channel, which can be used to transfer data. + */ + Ready, + + /** + * The connection channel is closed or being closed. We don't accept further + * data sending or receiving. All data sending request will be dropped. + */ + Closed + }; + + /** + * whether this connection is available to transfer data + */ + public abstract Status status(); + +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/messaging/IConnection.java b/jstorm-core/src/main/java/backtype/storm/messaging/IConnection.java new file mode 100644 index 000000000..24c404f66 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/messaging/IConnection.java @@ -0,0 +1,54 @@ +/** + * 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 backtype.storm.messaging; + +import java.util.List; + +import backtype.storm.utils.DisruptorQueue; + +public interface IConnection { + + /** + * (flags != 1) synchronously (flags==1) asynchronously + * + * @param flags + * @return + */ + public Object recv(Integer taskId, int flags); + + /** + * In the new design, receive flow is through registerQueue, then push + * message into queue + * + * @param recvQueu + */ + public void registerQueue(Integer taskId, DisruptorQueue recvQueu); + + public void enqueue(TaskMessage message); + + public void send(List messages); + + public void send(TaskMessage message); + + /** + * close this connection + */ + public void close(); + + public boolean isClosed(); +} diff --git a/jstorm-core/src/main/java/backtype/storm/messaging/IContext.java b/jstorm-core/src/main/java/backtype/storm/messaging/IContext.java new file mode 100644 index 000000000..261136604 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/messaging/IContext.java @@ -0,0 +1,68 @@ +/** + * 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 backtype.storm.messaging; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import backtype.storm.utils.DisruptorQueue; + +/** + * This interface needs to be implemented for messaging plugin. + * + * Messaging plugin is specified via Storm config parameter, + * storm.messaging.transport. + * + * A messaging plugin should have a default constructor and implements IContext + * interface. Upon construction, we will invoke IContext::prepare(storm_conf) to + * enable context to be configured according to storm configuration. + */ +public interface IContext { + /** + * This method is invoked at the startup of messaging plugin + * + * @param storm_conf storm configuration + */ + public void prepare(Map storm_conf); + + /** + * This method is invoked when a worker is unload a messaging plugin + */ + public void term(); + + /** + * This method establishes a server side connection + * + * @param topology_id topology ID + * @param port port # + * @param distribute true -- receive other worker's data + * @return server side connection + */ + public IConnection bind(String topology_id, int port, ConcurrentHashMap deserializedueue); + + /** + * This method establish a client side connection to a remote server + * + * @param topology_id topology ID + * @param host remote host + * @param port remote port + * @param distribute true -- send other worker data + * @return client side connection + */ + public IConnection connect(String topology_id, String host, int port); +}; diff --git a/jstorm-core/src/main/java/backtype/storm/messaging/TaskMessage.java b/jstorm-core/src/main/java/backtype/storm/messaging/TaskMessage.java new file mode 100755 index 000000000..fd95f5d20 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/messaging/TaskMessage.java @@ -0,0 +1,68 @@ +/** + * 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 backtype.storm.messaging; + +import java.nio.ByteBuffer; + +public class TaskMessage { + private int _task; + private byte[] _message; + + public TaskMessage(int task, byte[] message) { + _task = task; + _message = message; + } + + public int task() { + return _task; + } + + public byte[] message() { + return _message; + } + + public static boolean isEmpty(TaskMessage message) { + if (message == null) { + return true; + } else if (message.message() == null) { + return true; + } else if (message.message().length == 0) { + return true; + } + + return false; + } + + @Deprecated + public ByteBuffer serialize() { + ByteBuffer bb = ByteBuffer.allocate(_message.length + 2); + bb.putShort((short) _task); + bb.put(_message); + return bb; + } + + @Deprecated + public void deserialize(ByteBuffer packet) { + if (packet == null) + return; + _task = packet.getShort(); + _message = new byte[packet.limit() - 2]; + packet.get(_message); + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/messaging/TransportFactory.java b/jstorm-core/src/main/java/backtype/storm/messaging/TransportFactory.java new file mode 100755 index 000000000..4eddf4b0b --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/messaging/TransportFactory.java @@ -0,0 +1,68 @@ +/** + * 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 backtype.storm.messaging; + +import java.lang.reflect.Constructor; +import java.lang.reflect.Method; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; + +public class TransportFactory { + public static final Logger LOG = LoggerFactory.getLogger(TransportFactory.class); + + public static IContext makeContext(Map storm_conf) { + + // get factory class name + String transport_plugin_klassName = (String) storm_conf.get(Config.STORM_MESSAGING_TRANSPORT); + LOG.info("JStorm peer transport plugin:" + transport_plugin_klassName); + + IContext transport = null; + try { + // create a factory class + Class klass = Class.forName(transport_plugin_klassName); + // obtain a context object + // Object obj = klass.newInstance(); + Constructor constructor = klass.getDeclaredConstructor(); + constructor.setAccessible(true); + Object obj = constructor.newInstance(); + LOG.info("TransportFactory makeContext: new klass: " + obj); + if (obj instanceof IContext) { + // case 1: plugin is a IContext class + transport = (IContext) obj; + // initialize with storm configuration + transport.prepare(storm_conf); + LOG.info("TransportFactory makeContext: start prepare... " + storm_conf); + } else { + // case 2: Non-IContext plugin must have a + // makeContext(storm_conf) method that returns IContext object + Method method = klass.getMethod("makeContext", Map.class); + LOG.debug("object:" + obj + " method:" + method); + transport = (IContext) method.invoke(obj, storm_conf); + } + LOG.info("TransportFactory makeContext done..."); + } catch (Exception e) { + throw new RuntimeException("Fail to construct messaging plugin from plugin " + transport_plugin_klassName, e); + } + return transport; + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/metric/LoggingMetricsConsumer.java b/jstorm-core/src/main/java/backtype/storm/metric/LoggingMetricsConsumer.java new file mode 100755 index 000000000..c1c7c0a21 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/metric/LoggingMetricsConsumer.java @@ -0,0 +1,72 @@ +/** + * 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 backtype.storm.metric; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.Map; + +import backtype.storm.metric.api.IMetricsConsumer; +import backtype.storm.task.IErrorReporter; +import backtype.storm.task.TopologyContext; +import backtype.storm.utils.Utils; + +/* + * Listens for all metrics, dumps them to log + * + * To use, add this to your topology's configuration: + * conf.registerMetricsConsumer(backtype.storm.metrics.LoggingMetricsConsumer.class, 1); + * + * Or edit the storm.yaml config file: + * + * topology.metrics.consumer.register: + * - class: "backtype.storm.metrics.LoggingMetricsConsumer" + * parallelism.hint: 1 + * + */ +public class LoggingMetricsConsumer implements IMetricsConsumer { + public static final Logger LOG = LoggerFactory.getLogger(LoggingMetricsConsumer.class); + + @Override + public void prepare(Map stormConf, Object registrationArgument, TopologyContext context, IErrorReporter errorReporter) { } + + static private String padding = " "; + + @Override + public void handleDataPoints(TaskInfo taskInfo, Collection dataPoints) { + StringBuilder sb = new StringBuilder(); + String header = String.format("%d\t%15s:%-4d\t%3d:%-11s\t", + taskInfo.timestamp, + taskInfo.srcWorkerHost, taskInfo.srcWorkerPort, + taskInfo.srcTaskId, + taskInfo.srcComponentId); + sb.append(header); + for (DataPoint p : dataPoints) { + sb.delete(header.length(), sb.length()); + sb.append(p.name) + .append(padding).delete(header.length()+23,sb.length()).append("\t") + .append(p.value); + LOG.info(sb.toString()); + } + } + + @Override + public void cleanup() { } +} diff --git a/jstorm-core/src/main/java/backtype/storm/metric/MetricsConsumerBolt.java b/jstorm-core/src/main/java/backtype/storm/metric/MetricsConsumerBolt.java new file mode 100755 index 000000000..d8eb3bf5e --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/metric/MetricsConsumerBolt.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.metric; + +import backtype.storm.Config; +import backtype.storm.metric.api.IMetricsConsumer; +import backtype.storm.task.IBolt; +import backtype.storm.task.IErrorReporter; +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Tuple; +import java.util.Collection; +import java.util.Map; + +public class MetricsConsumerBolt implements IBolt { + IMetricsConsumer _metricsConsumer; + String _consumerClassName; + OutputCollector _collector; + Object _registrationArgument; + + public MetricsConsumerBolt(String consumerClassName, Object registrationArgument) { + _consumerClassName = consumerClassName; + _registrationArgument = registrationArgument; + } + + @Override + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + try { + _metricsConsumer = (IMetricsConsumer)Class.forName(_consumerClassName).newInstance(); + } catch (Exception e) { + throw new RuntimeException("Could not instantiate a class listed in config under section " + + Config.TOPOLOGY_METRICS_CONSUMER_REGISTER + " with fully qualified name " + _consumerClassName, e); + } + _metricsConsumer.prepare(stormConf, _registrationArgument, context, (IErrorReporter)collector); + _collector = collector; + } + + @Override + public void execute(Tuple input) { + _metricsConsumer.handleDataPoints((IMetricsConsumer.TaskInfo)input.getValue(0), (Collection)input.getValue(1)); + _collector.ack(input); + } + + @Override + public void cleanup() { + _metricsConsumer.cleanup(); + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/metric/SystemBolt.java b/jstorm-core/src/main/java/backtype/storm/metric/SystemBolt.java new file mode 100755 index 000000000..492bc2da7 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/metric/SystemBolt.java @@ -0,0 +1,155 @@ +/** + * 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 backtype.storm.metric; + +import backtype.storm.Config; +import backtype.storm.metric.api.AssignableMetric; +import backtype.storm.metric.api.IMetric; +import backtype.storm.task.IBolt; +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Tuple; +import clojure.lang.AFn; +import clojure.lang.IFn; +import clojure.lang.RT; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.management.*; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + + +// There is one task inside one executor for each worker of the topology. +// TaskID is always -1, therefore you can only send-unanchored tuples to co-located SystemBolt. +// This bolt was conceived to export worker stats via metrics api. +public class SystemBolt implements IBolt { + private static Logger LOG = LoggerFactory.getLogger(SystemBolt.class); + private static boolean _prepareWasCalled = false; + + private static class MemoryUsageMetric implements IMetric { + IFn _getUsage; + public MemoryUsageMetric(IFn getUsage) { + _getUsage = getUsage; + } + @Override + public Object getValueAndReset() { + MemoryUsage memUsage = (MemoryUsage)_getUsage.invoke(); + HashMap m = new HashMap(); + m.put("maxBytes", memUsage.getMax()); + m.put("committedBytes", memUsage.getCommitted()); + m.put("initBytes", memUsage.getInit()); + m.put("usedBytes", memUsage.getUsed()); + m.put("virtualFreeBytes", memUsage.getMax() - memUsage.getUsed()); + m.put("unusedBytes", memUsage.getCommitted() - memUsage.getUsed()); + return m; + } + } + + // canonically the metrics data exported is time bucketed when doing counts. + // convert the absolute values here into time buckets. + private static class GarbageCollectorMetric implements IMetric { + GarbageCollectorMXBean _gcBean; + Long _collectionCount; + Long _collectionTime; + public GarbageCollectorMetric(GarbageCollectorMXBean gcBean) { + _gcBean = gcBean; + } + @Override + public Object getValueAndReset() { + Long collectionCountP = _gcBean.getCollectionCount(); + Long collectionTimeP = _gcBean.getCollectionTime(); + + Map ret = null; + if(_collectionCount!=null && _collectionTime!=null) { + ret = new HashMap(); + ret.put("count", collectionCountP - _collectionCount); + ret.put("timeMs", collectionTimeP - _collectionTime); + } + + _collectionCount = collectionCountP; + _collectionTime = collectionTimeP; + return ret; + } + } + + @Override + public void prepare(final Map stormConf, TopologyContext context, OutputCollector collector) { + if(_prepareWasCalled && !"local".equals(stormConf.get(Config.STORM_CLUSTER_MODE))) { + throw new RuntimeException("A single worker should have 1 SystemBolt instance."); + } + _prepareWasCalled = true; + + int bucketSize = RT.intCast(stormConf.get(Config.TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS)); + + final RuntimeMXBean jvmRT = ManagementFactory.getRuntimeMXBean(); + + context.registerMetric("uptimeSecs", new IMetric() { + @Override + public Object getValueAndReset() { + return jvmRT.getUptime()/1000.0; + } + }, bucketSize); + + context.registerMetric("startTimeSecs", new IMetric() { + @Override + public Object getValueAndReset() { + return jvmRT.getStartTime()/1000.0; + } + }, bucketSize); + + context.registerMetric("newWorkerEvent", new IMetric() { + boolean doEvent = true; + + @Override + public Object getValueAndReset() { + if (doEvent) { + doEvent = false; + return 1; + } else return 0; + } + }, bucketSize); + + final MemoryMXBean jvmMemRT = ManagementFactory.getMemoryMXBean(); + + context.registerMetric("memory/heap", new MemoryUsageMetric(new AFn() { + public Object invoke() { + return jvmMemRT.getHeapMemoryUsage(); + } + }), bucketSize); + context.registerMetric("memory/nonHeap", new MemoryUsageMetric(new AFn() { + public Object invoke() { + return jvmMemRT.getNonHeapMemoryUsage(); + } + }), bucketSize); + + for(GarbageCollectorMXBean b : ManagementFactory.getGarbageCollectorMXBeans()) { + context.registerMetric("GC/" + b.getName().replaceAll("\\W", ""), new GarbageCollectorMetric(b), bucketSize); + } + } + + @Override + public void execute(Tuple input) { + throw new RuntimeException("Non-system tuples should never be sent to __system bolt."); + } + + @Override + public void cleanup() { + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/metric/api/AssignableMetric.java b/jstorm-core/src/main/java/backtype/storm/metric/api/AssignableMetric.java new file mode 100755 index 000000000..2fa87b075 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/metric/api/AssignableMetric.java @@ -0,0 +1,34 @@ +/** + * 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 backtype.storm.metric.api; + +public class AssignableMetric implements IMetric { + Object _value; + + public AssignableMetric(Object value) { + _value = value; + } + + public void setValue(Object value) { + _value = value; + } + + public Object getValueAndReset() { + return _value; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/metric/api/CombinedMetric.java b/jstorm-core/src/main/java/backtype/storm/metric/api/CombinedMetric.java new file mode 100755 index 000000000..5764a25e8 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/metric/api/CombinedMetric.java @@ -0,0 +1,38 @@ +/** + * 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 backtype.storm.metric.api; + +public class CombinedMetric implements IMetric { + private final ICombiner _combiner; + private Object _value; + + public CombinedMetric(ICombiner combiner) { + _combiner = combiner; + _value = _combiner.identity(); + } + + public void update(Object value) { + _value = _combiner.combine(_value, value); + } + + public Object getValueAndReset() { + Object ret = _value; + _value = _combiner.identity(); + return ret; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/metric/api/CountMetric.java b/jstorm-core/src/main/java/backtype/storm/metric/api/CountMetric.java new file mode 100755 index 000000000..dd048b8f4 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/metric/api/CountMetric.java @@ -0,0 +1,41 @@ +/** + * 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 backtype.storm.metric.api; + +import backtype.storm.metric.api.IMetric; + +public class CountMetric implements IMetric { + long _value = 0; + + public CountMetric() { + } + + public void incr() { + _value++; + } + + public void incrBy(long incrementBy) { + _value += incrementBy; + } + + public Object getValueAndReset() { + long ret = _value; + _value = 0; + return ret; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/metric/api/ICombiner.java b/jstorm-core/src/main/java/backtype/storm/metric/api/ICombiner.java new file mode 100755 index 000000000..04b3156f9 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/metric/api/ICombiner.java @@ -0,0 +1,23 @@ +/** + * 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 backtype.storm.metric.api; + +public interface ICombiner { + public T identity(); + public T combine(T a, T b); +} diff --git a/jstorm-core/src/main/java/backtype/storm/metric/api/IMetric.java b/jstorm-core/src/main/java/backtype/storm/metric/api/IMetric.java new file mode 100755 index 000000000..f6d0a3c17 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/metric/api/IMetric.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.metric.api; + +public interface IMetric { + public Object getValueAndReset(); +} diff --git a/jstorm-core/src/main/java/backtype/storm/metric/api/IMetricsConsumer.java b/jstorm-core/src/main/java/backtype/storm/metric/api/IMetricsConsumer.java new file mode 100755 index 000000000..14f1bf6ac --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/metric/api/IMetricsConsumer.java @@ -0,0 +1,60 @@ +/** + * 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 backtype.storm.metric.api; + +import backtype.storm.task.IErrorReporter; +import backtype.storm.task.TopologyContext; +import java.util.Collection; +import java.util.Map; + +public interface IMetricsConsumer { + public static class TaskInfo { + public TaskInfo() {} + public TaskInfo(String srcWorkerHost, int srcWorkerPort, String srcComponentId, int srcTaskId, long timestamp, int updateIntervalSecs) { + this.srcWorkerHost = srcWorkerHost; + this.srcWorkerPort = srcWorkerPort; + this.srcComponentId = srcComponentId; + this.srcTaskId = srcTaskId; + this.timestamp = timestamp; + this.updateIntervalSecs = updateIntervalSecs; + } + public String srcWorkerHost; + public int srcWorkerPort; + public String srcComponentId; + public int srcTaskId; + public long timestamp; + public int updateIntervalSecs; + } + public static class DataPoint { + public DataPoint() {} + public DataPoint(String name, Object value) { + this.name = name; + this.value = value; + } + @Override + public String toString() { + return "[" + name + " = " + value + "]"; + } + public String name; + public Object value; + } + + void prepare(Map stormConf, Object registrationArgument, TopologyContext context, IErrorReporter errorReporter); + void handleDataPoints(TaskInfo taskInfo, Collection dataPoints); + void cleanup(); +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/metric/api/IReducer.java b/jstorm-core/src/main/java/backtype/storm/metric/api/IReducer.java new file mode 100755 index 000000000..a58df3b65 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/metric/api/IReducer.java @@ -0,0 +1,24 @@ +/** + * 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 backtype.storm.metric.api; + +public interface IReducer { + T init(); + T reduce(T accumulator, Object input); + Object extractResult(T accumulator); +} diff --git a/jstorm-core/src/main/java/backtype/storm/metric/api/IStatefulObject.java b/jstorm-core/src/main/java/backtype/storm/metric/api/IStatefulObject.java new file mode 100755 index 000000000..459b9a94c --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/metric/api/IStatefulObject.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.metric.api; + +public interface IStatefulObject { + Object getState(); +} diff --git a/jstorm-core/src/main/java/backtype/storm/metric/api/MeanReducer.java b/jstorm-core/src/main/java/backtype/storm/metric/api/MeanReducer.java new file mode 100755 index 000000000..e25e26d2c --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/metric/api/MeanReducer.java @@ -0,0 +1,55 @@ +/** + * 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 backtype.storm.metric.api; + +import backtype.storm.metric.api.IReducer; + +class MeanReducerState { + public int count = 0; + public double sum = 0.0; +} + +public class MeanReducer implements IReducer { + public MeanReducerState init() { + return new MeanReducerState(); + } + + public MeanReducerState reduce(MeanReducerState acc, Object input) { + acc.count++; + if(input instanceof Double) { + acc.sum += (Double)input; + } else if(input instanceof Long) { + acc.sum += ((Long)input).doubleValue(); + } else if(input instanceof Integer) { + acc.sum += ((Integer)input).doubleValue(); + } else { + throw new RuntimeException( + "MeanReducer::reduce called with unsupported input type `" + input.getClass() + + "`. Supported types are Double, Long, Integer."); + } + return acc; + } + + public Object extractResult(MeanReducerState acc) { + if(acc.count > 0) { + return new Double(acc.sum / (double)acc.count); + } else { + return null; + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/metric/api/MultiCountMetric.java b/jstorm-core/src/main/java/backtype/storm/metric/api/MultiCountMetric.java new file mode 100755 index 000000000..c420a16ed --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/metric/api/MultiCountMetric.java @@ -0,0 +1,45 @@ +/** + * 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 backtype.storm.metric.api; + +import backtype.storm.metric.api.IMetric; +import java.util.HashMap; +import java.util.Map; + +public class MultiCountMetric implements IMetric { + Map _value = new HashMap(); + + public MultiCountMetric() { + } + + public CountMetric scope(String key) { + CountMetric val = _value.get(key); + if(val == null) { + _value.put(key, val = new CountMetric()); + } + return val; + } + + public Object getValueAndReset() { + Map ret = new HashMap(); + for(Map.Entry e : _value.entrySet()) { + ret.put(e.getKey(), e.getValue().getValueAndReset()); + } + return ret; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/metric/api/MultiReducedMetric.java b/jstorm-core/src/main/java/backtype/storm/metric/api/MultiReducedMetric.java new file mode 100755 index 000000000..530b168aa --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/metric/api/MultiReducedMetric.java @@ -0,0 +1,50 @@ +/** + * 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 backtype.storm.metric.api; + +import backtype.storm.metric.api.IMetric; +import java.util.HashMap; +import java.util.Map; + +public class MultiReducedMetric implements IMetric { + Map _value = new HashMap(); + IReducer _reducer; + + public MultiReducedMetric(IReducer reducer) { + _reducer = reducer; + } + + public ReducedMetric scope(String key) { + ReducedMetric val = _value.get(key); + if(val == null) { + _value.put(key, val = new ReducedMetric(_reducer)); + } + return val; + } + + public Object getValueAndReset() { + Map ret = new HashMap(); + for(Map.Entry e : _value.entrySet()) { + Object val = e.getValue().getValueAndReset(); + if(val != null) { + ret.put(e.getKey(), val); + } + } + return ret; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/metric/api/ReducedMetric.java b/jstorm-core/src/main/java/backtype/storm/metric/api/ReducedMetric.java new file mode 100755 index 000000000..2c00c0e31 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/metric/api/ReducedMetric.java @@ -0,0 +1,38 @@ +/** + * 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 backtype.storm.metric.api; + +public class ReducedMetric implements IMetric { + private final IReducer _reducer; + private Object _accumulator; + + public ReducedMetric(IReducer reducer) { + _reducer = reducer; + _accumulator = _reducer.init(); + } + + public void update(Object value) { + _accumulator = _reducer.reduce(_accumulator, value); + } + + public Object getValueAndReset() { + Object ret = _reducer.extractResult(_accumulator); + _accumulator = _reducer.init(); + return ret; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/metric/api/StateMetric.java b/jstorm-core/src/main/java/backtype/storm/metric/api/StateMetric.java new file mode 100755 index 000000000..264071218 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/metric/api/StateMetric.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.metric.api; + +public class StateMetric implements IMetric { + private IStatefulObject _obj; + + public StateMetric(IStatefulObject obj) { + _obj = obj; + } + + @Override + public Object getValueAndReset() { + return _obj.getState(); + } +} diff --git a/jstorm-client/src/main/java/backtype/storm/metric/api/rpc/AssignableShellMetric.java b/jstorm-core/src/main/java/backtype/storm/metric/api/rpc/AssignableShellMetric.java old mode 100644 new mode 100755 similarity index 100% rename from jstorm-client/src/main/java/backtype/storm/metric/api/rpc/AssignableShellMetric.java rename to jstorm-core/src/main/java/backtype/storm/metric/api/rpc/AssignableShellMetric.java diff --git a/jstorm-client/src/main/java/backtype/storm/metric/api/rpc/CombinedShellMetric.java b/jstorm-core/src/main/java/backtype/storm/metric/api/rpc/CombinedShellMetric.java old mode 100644 new mode 100755 similarity index 100% rename from jstorm-client/src/main/java/backtype/storm/metric/api/rpc/CombinedShellMetric.java rename to jstorm-core/src/main/java/backtype/storm/metric/api/rpc/CombinedShellMetric.java diff --git a/jstorm-client/src/main/java/backtype/storm/metric/api/rpc/CountShellMetric.java b/jstorm-core/src/main/java/backtype/storm/metric/api/rpc/CountShellMetric.java old mode 100644 new mode 100755 similarity index 100% rename from jstorm-client/src/main/java/backtype/storm/metric/api/rpc/CountShellMetric.java rename to jstorm-core/src/main/java/backtype/storm/metric/api/rpc/CountShellMetric.java diff --git a/jstorm-client/src/main/java/backtype/storm/metric/api/rpc/IShellMetric.java b/jstorm-core/src/main/java/backtype/storm/metric/api/rpc/IShellMetric.java old mode 100644 new mode 100755 similarity index 100% rename from jstorm-client/src/main/java/backtype/storm/metric/api/rpc/IShellMetric.java rename to jstorm-core/src/main/java/backtype/storm/metric/api/rpc/IShellMetric.java diff --git a/jstorm-client/src/main/java/backtype/storm/metric/api/rpc/ReducedShellMetric.java b/jstorm-core/src/main/java/backtype/storm/metric/api/rpc/ReducedShellMetric.java old mode 100644 new mode 100755 similarity index 100% rename from jstorm-client/src/main/java/backtype/storm/metric/api/rpc/ReducedShellMetric.java rename to jstorm-core/src/main/java/backtype/storm/metric/api/rpc/ReducedShellMetric.java diff --git a/jstorm-client/src/main/java/backtype/storm/multilang/BoltMsg.java b/jstorm-core/src/main/java/backtype/storm/multilang/BoltMsg.java old mode 100644 new mode 100755 similarity index 100% rename from jstorm-client/src/main/java/backtype/storm/multilang/BoltMsg.java rename to jstorm-core/src/main/java/backtype/storm/multilang/BoltMsg.java diff --git a/jstorm-client/src/main/java/backtype/storm/multilang/ISerializer.java b/jstorm-core/src/main/java/backtype/storm/multilang/ISerializer.java old mode 100644 new mode 100755 similarity index 100% rename from jstorm-client/src/main/java/backtype/storm/multilang/ISerializer.java rename to jstorm-core/src/main/java/backtype/storm/multilang/ISerializer.java diff --git a/jstorm-client/src/main/java/backtype/storm/multilang/JsonSerializer.java b/jstorm-core/src/main/java/backtype/storm/multilang/JsonSerializer.java old mode 100644 new mode 100755 similarity index 99% rename from jstorm-client/src/main/java/backtype/storm/multilang/JsonSerializer.java rename to jstorm-core/src/main/java/backtype/storm/multilang/JsonSerializer.java index 9fca312fd..0e2e1566c --- a/jstorm-client/src/main/java/backtype/storm/multilang/JsonSerializer.java +++ b/jstorm-core/src/main/java/backtype/storm/multilang/JsonSerializer.java @@ -130,7 +130,7 @@ public ShellMsg readShellMsg() throws IOException, NoOutputException { shellMsg.setTuple((List) msg.get("tuple")); - //List anchors = new ArrayList(); + List anchors = new ArrayList(); Object anchorObj = msg.get("anchors"); if (anchorObj != null) { if (anchorObj instanceof String) { diff --git a/jstorm-client/src/main/java/backtype/storm/multilang/NoOutputException.java b/jstorm-core/src/main/java/backtype/storm/multilang/NoOutputException.java old mode 100644 new mode 100755 similarity index 100% rename from jstorm-client/src/main/java/backtype/storm/multilang/NoOutputException.java rename to jstorm-core/src/main/java/backtype/storm/multilang/NoOutputException.java diff --git a/jstorm-client/src/main/java/backtype/storm/multilang/ShellMsg.java b/jstorm-core/src/main/java/backtype/storm/multilang/ShellMsg.java old mode 100644 new mode 100755 similarity index 100% rename from jstorm-client/src/main/java/backtype/storm/multilang/ShellMsg.java rename to jstorm-core/src/main/java/backtype/storm/multilang/ShellMsg.java diff --git a/jstorm-client/src/main/java/backtype/storm/multilang/SpoutMsg.java b/jstorm-core/src/main/java/backtype/storm/multilang/SpoutMsg.java old mode 100644 new mode 100755 similarity index 100% rename from jstorm-client/src/main/java/backtype/storm/multilang/SpoutMsg.java rename to jstorm-core/src/main/java/backtype/storm/multilang/SpoutMsg.java diff --git a/jstorm-core/src/main/java/backtype/storm/nimbus/DefaultTopologyValidator.java b/jstorm-core/src/main/java/backtype/storm/nimbus/DefaultTopologyValidator.java new file mode 100755 index 000000000..a68721535 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/nimbus/DefaultTopologyValidator.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.nimbus; + +import backtype.storm.generated.InvalidTopologyException; +import backtype.storm.generated.StormTopology; +import java.util.Map; + +public class DefaultTopologyValidator implements ITopologyValidator { + @Override + public void prepare(Map StormConf){ + } + @Override + public void validate(String topologyName, Map topologyConf, StormTopology topology) throws InvalidTopologyException { + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/nimbus/ITopologyValidator.java b/jstorm-core/src/main/java/backtype/storm/nimbus/ITopologyValidator.java new file mode 100755 index 000000000..99bd07b82 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/nimbus/ITopologyValidator.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.nimbus; + +import backtype.storm.generated.InvalidTopologyException; +import backtype.storm.generated.StormTopology; +import java.util.Map; + +public interface ITopologyValidator { + void prepare(Map StormConf); + void validate(String topologyName, Map topologyConf, StormTopology topology) + throws InvalidTopologyException; +} diff --git a/jstorm-core/src/main/java/backtype/storm/planner/CompoundSpout.java b/jstorm-core/src/main/java/backtype/storm/planner/CompoundSpout.java new file mode 100755 index 000000000..141b24bbb --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/planner/CompoundSpout.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.planner; + + +public class CompoundSpout + //implements ISpout +{ + +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/planner/CompoundTask.java b/jstorm-core/src/main/java/backtype/storm/planner/CompoundTask.java new file mode 100755 index 000000000..40a7f3735 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/planner/CompoundTask.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.planner; + + +public class CompoundTask +// implements IBolt +{ + +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/planner/TaskBundle.java b/jstorm-core/src/main/java/backtype/storm/planner/TaskBundle.java new file mode 100755 index 000000000..81c620962 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/planner/TaskBundle.java @@ -0,0 +1,33 @@ +/** + * 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 backtype.storm.planner; + +import backtype.storm.task.IBolt; +import java.io.Serializable; + + +public class TaskBundle implements Serializable { + public IBolt task; + public int componentId; + + public TaskBundle(IBolt task, int componentId) { + this.task = task; + this.componentId = componentId; + } + +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/scheduler/Cluster.java b/jstorm-core/src/main/java/backtype/storm/scheduler/Cluster.java new file mode 100755 index 000000000..e0c7cc729 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/scheduler/Cluster.java @@ -0,0 +1,448 @@ +/** + * 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 backtype.storm.scheduler; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class Cluster { + + /** + * key: supervisor id, value: supervisor details + */ + private Map supervisors; + /** + * key: topologyId, value: topology's current assignments. + */ + private Map assignments; + /** + * key topologyId, Value: scheduler's status. + */ + private Map status; + + /** + * a map from hostname to supervisor id. + */ + private Map> hostToId; + + private Set blackListedHosts = new HashSet(); + private INimbus inimbus; + + public Cluster(INimbus nimbus, Map supervisors, Map assignments){ + this.inimbus = nimbus; + this.supervisors = new HashMap(supervisors.size()); + this.supervisors.putAll(supervisors); + this.assignments = new HashMap(assignments.size()); + this.assignments.putAll(assignments); + this.status = new HashMap(); + this.hostToId = new HashMap>(); + for (String nodeId : supervisors.keySet()) { + SupervisorDetails supervisor = supervisors.get(nodeId); + String host = supervisor.getHost(); + if (!this.hostToId.containsKey(host)) { + this.hostToId.put(host, new ArrayList()); + } + this.hostToId.get(host).add(nodeId); + } + } + + public void setBlacklistedHosts(Set hosts) { + blackListedHosts = hosts; + } + + public Set getBlacklistedHosts() { + return blackListedHosts; + } + + public void blacklistHost(String host) { + // this is so it plays well with setting blackListedHosts to an immutable list + if(blackListedHosts==null) blackListedHosts = new HashSet(); + if(!(blackListedHosts instanceof HashSet)) + blackListedHosts = new HashSet(blackListedHosts); + blackListedHosts.add(host); + } + + public boolean isBlackListed(String supervisorId) { + return blackListedHosts != null && blackListedHosts.contains(getHost(supervisorId)); + } + + public boolean isBlacklistedHost(String host) { + return blackListedHosts != null && blackListedHosts.contains(host); + } + + public String getHost(String supervisorId) { + return inimbus.getHostName(supervisors, supervisorId); + } + + /** + * Gets all the topologies which needs scheduling. + * + * @param topologies + * @return + */ + public List needsSchedulingTopologies(Topologies topologies) { + List ret = new ArrayList(); + for (TopologyDetails topology : topologies.getTopologies()) { + if (needsScheduling(topology)) { + ret.add(topology); + } + } + + return ret; + } + + /** + * Does the topology need scheduling? + * + * A topology needs scheduling if one of the following conditions holds: + *
    + *
  • Although the topology is assigned slots, but is squeezed. i.e. the topology is assigned less slots than desired.
  • + *
  • There are unassigned executors in this topology
  • + *
+ */ + public boolean needsScheduling(TopologyDetails topology) { + int desiredNumWorkers = topology.getNumWorkers(); + int assignedNumWorkers = this.getAssignedNumWorkers(topology); + + if (desiredNumWorkers > assignedNumWorkers) { + return true; + } + + return this.getUnassignedExecutors(topology).size() > 0; + } + + /** + * Gets a executor -> component-id map which needs scheduling in this topology. + * + * @param topology + * @return + */ + public Map getNeedsSchedulingExecutorToComponents(TopologyDetails topology) { + Collection allExecutors = new HashSet(topology.getExecutors()); + + SchedulerAssignment assignment = this.assignments.get(topology.getId()); + if (assignment != null) { + Collection assignedExecutors = assignment.getExecutors(); + allExecutors.removeAll(assignedExecutors); + } + + return topology.selectExecutorToComponent(allExecutors); + } + + /** + * Gets a component-id -> executors map which needs scheduling in this topology. + * + * @param topology + * @return + */ + public Map> getNeedsSchedulingComponentToExecutors(TopologyDetails topology) { + Map executorToComponents = this.getNeedsSchedulingExecutorToComponents(topology); + Map> componentToExecutors = new HashMap>(); + for (ExecutorDetails executor : executorToComponents.keySet()) { + String component = executorToComponents.get(executor); + if (!componentToExecutors.containsKey(component)) { + componentToExecutors.put(component, new ArrayList()); + } + + componentToExecutors.get(component).add(executor); + } + + return componentToExecutors; + } + + + /** + * Get all the used ports of this supervisor. + * + * @param cluster + * @return + */ + public Set getUsedPorts(SupervisorDetails supervisor) { + Map assignments = this.getAssignments(); + Set usedPorts = new HashSet(); + + for (SchedulerAssignment assignment : assignments.values()) { + for (WorkerSlot slot : assignment.getExecutorToSlot().values()) { + if (slot.getNodeId().equals(supervisor.getId())) { + usedPorts.add(slot.getPort()); + } + } + } + + return usedPorts; + } + + /** + * Return the available ports of this supervisor. + * + * @param cluster + * @return + */ + public Set getAvailablePorts(SupervisorDetails supervisor) { + Set usedPorts = this.getUsedPorts(supervisor); + + Set ret = new HashSet(); + ret.addAll(getAssignablePorts(supervisor)); + ret.removeAll(usedPorts); + + return ret; + } + + public Set getAssignablePorts(SupervisorDetails supervisor) { + if(isBlackListed(supervisor.id)) return new HashSet(); + return supervisor.allPorts; + } + + /** + * Return all the available slots on this supervisor. + * + * @param cluster + * @return + */ + public List getAvailableSlots(SupervisorDetails supervisor) { + Set ports = this.getAvailablePorts(supervisor); + List slots = new ArrayList(ports.size()); + + for (Integer port : ports) { + slots.add(new WorkerSlot(supervisor.getId(), port)); + } + + return slots; + } + + public List getAssignableSlots(SupervisorDetails supervisor) { + Set ports = this.getAssignablePorts(supervisor); + List slots = new ArrayList(ports.size()); + + for (Integer port : ports) { + slots.add(new WorkerSlot(supervisor.getId(), port)); + } + + return slots; + } + + /** + * get the unassigned executors of the topology. + */ + public Collection getUnassignedExecutors(TopologyDetails topology) { + if (topology == null) { + return new ArrayList(0); + } + + Collection ret = new HashSet(topology.getExecutors()); + + SchedulerAssignment assignment = this.getAssignmentById(topology.getId()); + if (assignment != null) { + Set assignedExecutors = assignment.getExecutors(); + ret.removeAll(assignedExecutors); + } + + return ret; + } + + /** + * Gets the number of workers assigned to this topology. + * + * @param topology + * @return + */ + public int getAssignedNumWorkers(TopologyDetails topology) { + SchedulerAssignment assignment = this.getAssignmentById(topology.getId()); + if (topology == null || assignment == null) { + return 0; + } + + Set slots = new HashSet(); + slots.addAll(assignment.getExecutorToSlot().values()); + + return slots.size(); + } + + /** + * Assign the slot to the executors for this topology. + * + * @throws RuntimeException if the specified slot is already occupied. + */ + public void assign(WorkerSlot slot, String topologyId, Collection executors) { + if (this.isSlotOccupied(slot)) { + throw new RuntimeException("slot: [" + slot.getNodeId() + ", " + slot.getPort() + "] is already occupied."); + } + + SchedulerAssignmentImpl assignment = (SchedulerAssignmentImpl)this.getAssignmentById(topologyId); + if (assignment == null) { + assignment = new SchedulerAssignmentImpl(topologyId, new HashMap()); + this.assignments.put(topologyId, assignment); + } else { + for (ExecutorDetails executor : executors) { + if (assignment.isExecutorAssigned(executor)) { + throw new RuntimeException("the executor is already assigned, you should unassign it before assign it to another slot."); + } + } + } + + assignment.assign(slot, executors); + } + + /** + * Gets all the available slots in the cluster. + * + * @return + */ + public List getAvailableSlots() { + List slots = new ArrayList(); + for (SupervisorDetails supervisor : this.supervisors.values()) { + slots.addAll(this.getAvailableSlots(supervisor)); + } + + return slots; + } + + public List getAssignableSlots() { + List slots = new ArrayList(); + for (SupervisorDetails supervisor : this.supervisors.values()) { + slots.addAll(this.getAssignableSlots(supervisor)); + } + + return slots; + } + + /** + * Free the specified slot. + * + * @param slot + */ + public void freeSlot(WorkerSlot slot) { + // remove the slot from the existing assignments + for (SchedulerAssignmentImpl assignment : this.assignments.values()) { + if (assignment.isSlotOccupied(slot)) { + assignment.unassignBySlot(slot); + } + } + } + + /** + * free the slots. + * + * @param slots + */ + public void freeSlots(Collection slots) { + if(slots!=null) { + for (WorkerSlot slot : slots) { + this.freeSlot(slot); + } + } + } + + /** + * Checks the specified slot is occupied. + * + * @param slot the slot be to checked. + * @return + */ + public boolean isSlotOccupied(WorkerSlot slot) { + for (SchedulerAssignment assignment : this.assignments.values()) { + if (assignment.isSlotOccupied(slot)) { + return true; + } + } + + return false; + } + + /** + * get the current assignment for the topology. + */ + public SchedulerAssignment getAssignmentById(String topologyId) { + if (this.assignments.containsKey(topologyId)) { + return this.assignments.get(topologyId); + } + + return null; + } + + /** + * Get a specific supervisor with the nodeId + */ + public SupervisorDetails getSupervisorById(String nodeId) { + if (this.supervisors.containsKey(nodeId)) { + return this.supervisors.get(nodeId); + } + + return null; + } + + public Collection getUsedSlots() { + Set ret = new HashSet(); + for(SchedulerAssignmentImpl s: assignments.values()) { + ret.addAll(s.getExecutorToSlot().values()); + } + return ret; + } + + /** + * Get all the supervisors on the specified host. + * + * @param host hostname of the supervisor + * @return the SupervisorDetails object. + */ + public List getSupervisorsByHost(String host) { + List nodeIds = this.hostToId.get(host); + List ret = new ArrayList(); + + if (nodeIds != null) { + for (String nodeId : nodeIds) { + ret.add(this.getSupervisorById(nodeId)); + } + } + + return ret; + } + + /** + * Get all the assignments. + */ + public Map getAssignments() { + Map ret = new HashMap(this.assignments.size()); + + for (String topologyId : this.assignments.keySet()) { + ret.put(topologyId, this.assignments.get(topologyId)); + } + + return ret; + } + + /** + * Get all the supervisors. + */ + public Map getSupervisors() { + return this.supervisors; + } + + public void setStatus(String topologyId, String status) { + this.status.put(topologyId, status); + } + + public Map getStatusMap() { + return this.status; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/scheduler/ExecutorDetails.java b/jstorm-core/src/main/java/backtype/storm/scheduler/ExecutorDetails.java new file mode 100755 index 000000000..bcf4aca47 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/scheduler/ExecutorDetails.java @@ -0,0 +1,54 @@ +/** + * 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 backtype.storm.scheduler; + +public class ExecutorDetails { + int startTask; + int endTask; + + public ExecutorDetails(int startTask, int endTask){ + this.startTask = startTask; + this.endTask = endTask; + } + + public int getStartTask() { + return startTask; + } + + public int getEndTask() { + return endTask; + } + + public boolean equals(Object other) { + if (other == null || !(other instanceof ExecutorDetails)) { + return false; + } + + ExecutorDetails executor = (ExecutorDetails)other; + return (this.startTask == executor.startTask) && (this.endTask == executor.endTask); + } + + public int hashCode() { + return this.startTask + 13 * this.endTask; + } + + @Override + public String toString() { + return "[" + this.startTask + ", " + this.endTask + "]"; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/scheduler/INimbus.java b/jstorm-core/src/main/java/backtype/storm/scheduler/INimbus.java new file mode 100755 index 000000000..a0fb417b5 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/scheduler/INimbus.java @@ -0,0 +1,39 @@ +/** + * 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 backtype.storm.scheduler; + +import java.util.Collection; +import java.util.Map; +import java.util.Set; + +public interface INimbus { + void prepare(Map stormConf, String schedulerLocalDir); + /** + * Returns all slots that are available for the next round of scheduling. A slot is available for scheduling + * if it is free and can be assigned to, or if it is used and can be reassigned. + */ + Collection allSlotsAvailableForScheduling(Collection existingSupervisors, Topologies topologies, Set topologiesMissingAssignments); + + // this is called after the assignment is changed in ZK + void assignSlots(Topologies topologies, Map> newSlotsByTopologyId); + + // map from node id to supervisor details + String getHostName(Map existingSupervisors, String nodeId); + + IScheduler getForcedScheduler(); +} diff --git a/jstorm-core/src/main/java/backtype/storm/scheduler/IScheduler.java b/jstorm-core/src/main/java/backtype/storm/scheduler/IScheduler.java new file mode 100755 index 000000000..53958820e --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/scheduler/IScheduler.java @@ -0,0 +1,40 @@ +/** + * 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 backtype.storm.scheduler; + +import java.util.Map; + + +public interface IScheduler { + + void prepare(Map conf); + + /** + * Set assignments for the topologies which needs scheduling. The new assignments is available + * through cluster.getAssignments() + * + *@param topologies all the topologies in the cluster, some of them need schedule. Topologies object here + * only contain static information about topologies. Information like assignments, slots are all in + * the clusterobject. + *@param cluster the cluster these topologies are running in. cluster contains everything user + * need to develop a new scheduling logic. e.g. supervisors information, available slots, current + * assignments for all the topologies etc. User can set the new assignment for topologies using + * cluster.setAssignmentById + */ + void schedule(Topologies topologies, Cluster cluster); +} diff --git a/jstorm-core/src/main/java/backtype/storm/scheduler/ISupervisor.java b/jstorm-core/src/main/java/backtype/storm/scheduler/ISupervisor.java new file mode 100755 index 000000000..64e1595cc --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/scheduler/ISupervisor.java @@ -0,0 +1,45 @@ +/** + * 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 backtype.storm.scheduler; + +import java.util.Map; +import java.util.Collection; + + +public interface ISupervisor { + void prepare(Map stormConf, String schedulerLocalDir); + // for mesos, this is {hostname}-{topologyid} + /** + * The id used for writing metadata into ZK. + */ + String getSupervisorId(); + /** + * The id used in assignments. This combined with confirmAssigned decides what + * this supervisor is responsible for. The combination of this and getSupervisorId + * allows Nimbus to assign to a single machine and have multiple supervisors + * on that machine execute the assignment. This is important for achieving resource isolation. + */ + String getAssignmentId(); + Object getMetadata(); + + boolean confirmAssigned(int port); + // calls this before actually killing the worker locally... + // sends a "task finished" update + void killedWorker(int port); + void assigned(Collection ports); +} diff --git a/jstorm-core/src/main/java/backtype/storm/scheduler/SchedulerAssignment.java b/jstorm-core/src/main/java/backtype/storm/scheduler/SchedulerAssignment.java new file mode 100755 index 000000000..0212e48a2 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/scheduler/SchedulerAssignment.java @@ -0,0 +1,58 @@ +/** + * 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 backtype.storm.scheduler; + +import java.util.Map; +import java.util.Set; + +public interface SchedulerAssignment { + /** + * Does this slot occupied by this assignment? + * @param slot + * @return + */ + public boolean isSlotOccupied(WorkerSlot slot); + + /** + * is the executor assigned? + * + * @param executor + * @return + */ + public boolean isExecutorAssigned(ExecutorDetails executor); + + /** + * get the topology-id this assignment is for. + * @return + */ + public String getTopologyId(); + + /** + * get the executor -> slot map. + * @return + */ + public Map getExecutorToSlot(); + + /** + * Return the executors covered by this assignments + * @return + */ + public Set getExecutors(); + + public Set getSlots(); +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/scheduler/SchedulerAssignmentImpl.java b/jstorm-core/src/main/java/backtype/storm/scheduler/SchedulerAssignmentImpl.java new file mode 100755 index 000000000..08af4b704 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/scheduler/SchedulerAssignmentImpl.java @@ -0,0 +1,110 @@ +/** + * 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 backtype.storm.scheduler; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +//TODO: improve this by maintaining slot -> executors as well for more efficient operations +public class SchedulerAssignmentImpl implements SchedulerAssignment { + /** + * topology-id this assignment is for. + */ + String topologyId; + /** + * assignment detail, a mapping from executor to WorkerSlot + */ + Map executorToSlot; + + public SchedulerAssignmentImpl(String topologyId, Map executorToSlots) { + this.topologyId = topologyId; + this.executorToSlot = new HashMap(0); + if (executorToSlots != null) { + this.executorToSlot.putAll(executorToSlots); + } + } + + @Override + public Set getSlots() { + return new HashSet(executorToSlot.values()); + } + + /** + * Assign the slot to executors. + * @param slot + * @param executors + */ + public void assign(WorkerSlot slot, Collection executors) { + for (ExecutorDetails executor : executors) { + this.executorToSlot.put(executor, slot); + } + } + + /** + * Release the slot occupied by this assignment. + * @param slot + */ + public void unassignBySlot(WorkerSlot slot) { + List executors = new ArrayList(); + for (ExecutorDetails executor : this.executorToSlot.keySet()) { + WorkerSlot ws = this.executorToSlot.get(executor); + if (ws.equals(slot)) { + executors.add(executor); + } + } + + // remove + for (ExecutorDetails executor : executors) { + this.executorToSlot.remove(executor); + } + } + + /** + * Does this slot occupied by this assignment? + * @param slot + * @return + */ + public boolean isSlotOccupied(WorkerSlot slot) { + return this.executorToSlot.containsValue(slot); + } + + public boolean isExecutorAssigned(ExecutorDetails executor) { + return this.executorToSlot.containsKey(executor); + } + + public String getTopologyId() { + return this.topologyId; + } + + public Map getExecutorToSlot() { + return this.executorToSlot; + } + + /** + * Return the executors covered by this assignments + * @return + */ + public Set getExecutors() { + return this.executorToSlot.keySet(); + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/scheduler/SupervisorDetails.java b/jstorm-core/src/main/java/backtype/storm/scheduler/SupervisorDetails.java new file mode 100755 index 000000000..7497f26ff --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/scheduler/SupervisorDetails.java @@ -0,0 +1,89 @@ +/** + * 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 backtype.storm.scheduler; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Set; + +public class SupervisorDetails { + + String id; + /** + * hostname of this supervisor + */ + String host; + Object meta; + /** + * meta data configured for this supervisor + */ + Object schedulerMeta; + /** + * all the ports of the supervisor + */ + Set allPorts; + + public SupervisorDetails(String id, Object meta){ + this.id = id; + this.meta = meta; + allPorts = new HashSet(); + } + + public SupervisorDetails(String id, Object meta, Collection allPorts){ + this.id = id; + this.meta = meta; + setAllPorts(allPorts); + } + + public SupervisorDetails(String id, String host, Object schedulerMeta, Collection allPorts){ + this.id = id; + this.host = host; + this.schedulerMeta = schedulerMeta; + + setAllPorts(allPorts); + } + + private void setAllPorts(Collection allPorts) { + this.allPorts = new HashSet(); + if(allPorts!=null) { + for(Number n: allPorts) { + this.allPorts.add(n.intValue()); + } + } + } + + public String getId() { + return id; + } + + public String getHost() { + return host; + } + + public Object getMeta() { + return meta; + } + + public Set getAllPorts() { + return allPorts; + } + + public Object getSchedulerMeta() { + return this.schedulerMeta; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/scheduler/Topologies.java b/jstorm-core/src/main/java/backtype/storm/scheduler/Topologies.java new file mode 100755 index 000000000..70af1b41f --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/scheduler/Topologies.java @@ -0,0 +1,57 @@ +/** + * 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 backtype.storm.scheduler; + +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + +public class Topologies { + Map topologies; + Map nameToId; + + public Topologies(Map topologies) { + if(topologies==null) topologies = new HashMap(); + this.topologies = new HashMap(topologies.size()); + this.topologies.putAll(topologies); + this.nameToId = new HashMap(topologies.size()); + + for (String topologyId : topologies.keySet()) { + TopologyDetails topology = topologies.get(topologyId); + this.nameToId.put(topology.getName(), topologyId); + } + } + + public TopologyDetails getById(String topologyId) { + return this.topologies.get(topologyId); + } + + public TopologyDetails getByName(String topologyName) { + String topologyId = this.nameToId.get(topologyName); + + if (topologyId == null) { + return null; + } else { + return this.getById(topologyId); + } + } + + public Collection getTopologies() { + return this.topologies.values(); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/scheduler/TopologyDetails.java b/jstorm-core/src/main/java/backtype/storm/scheduler/TopologyDetails.java new file mode 100755 index 000000000..6daf4edae --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/scheduler/TopologyDetails.java @@ -0,0 +1,89 @@ +/** + * 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 backtype.storm.scheduler; + +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + +import backtype.storm.Config; +import backtype.storm.generated.StormTopology; + + +public class TopologyDetails { + String topologyId; + Map topologyConf; + StormTopology topology; + Map executorToComponent; + int numWorkers; + + public TopologyDetails(String topologyId, Map topologyConf, StormTopology topology, int numWorkers) { + this.topologyId = topologyId; + this.topologyConf = topologyConf; + this.topology = topology; + this.numWorkers = numWorkers; + } + + public TopologyDetails(String topologyId, Map topologyConf, StormTopology topology, int numWorkers, Map executorToComponents) { + this(topologyId, topologyConf, topology, numWorkers); + this.executorToComponent = new HashMap(0); + if (executorToComponents != null) { + this.executorToComponent.putAll(executorToComponents); + } + } + + public String getId() { + return topologyId; + } + + public String getName() { + return (String)this.topologyConf.get(Config.TOPOLOGY_NAME); + } + + public Map getConf() { + return topologyConf; + } + + public int getNumWorkers() { + return numWorkers; + } + + public StormTopology getTopology() { + return topology; + } + + public Map getExecutorToComponent() { + return this.executorToComponent; + } + + public Map selectExecutorToComponent(Collection executors) { + Map ret = new HashMap(executors.size()); + for (ExecutorDetails executor : executors) { + String compId = this.executorToComponent.get(executor); + if (compId != null) { + ret.put(executor, compId); + } + } + + return ret; + } + + public Collection getExecutors() { + return this.executorToComponent.keySet(); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/scheduler/WorkerSlot.java b/jstorm-core/src/main/java/backtype/storm/scheduler/WorkerSlot.java new file mode 100755 index 000000000..8331ad82d --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/scheduler/WorkerSlot.java @@ -0,0 +1,104 @@ +/** + * 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 backtype.storm.scheduler; + +import java.io.Serializable; + +public class WorkerSlot implements Comparable, Serializable { + + private static final long serialVersionUID = -4451854497340313268L; + String nodeId; + int port; + + public WorkerSlot(String nodeId, Number port) { + this.nodeId = nodeId; + this.port = port.intValue(); + } + + public WorkerSlot() { + + } + + public String getNodeId() { + return nodeId; + } + + public int getPort() { + return port; + } + + public void setNodeId(String nodeId) { + this.nodeId = nodeId; + } + + public void setPort(int port) { + this.port = port; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((nodeId == null) ? 0 : nodeId.hashCode()); + result = prime * result + port; + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + WorkerSlot other = (WorkerSlot) obj; + if (nodeId == null) { + if (other.nodeId != null) + return false; + } else if (!nodeId.equals(other.nodeId)) + return false; + if (port != other.port) + return false; + return true; + } + + @Override + public String toString() { + return this.nodeId + ":" + this.port; + } + + @Override + public int compareTo(WorkerSlot o) { + String otherNode = o.getNodeId(); + if (nodeId == null) { + if (otherNode != null) { + return -1; + } else { + return port - o.getPort(); + } + } else { + int ret = nodeId.compareTo(otherNode); + if (ret == 0) { + return port - o.getPort(); + } else { + return ret; + } + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/scheduler/multitenant/DefaultPool.java b/jstorm-core/src/main/java/backtype/storm/scheduler/multitenant/DefaultPool.java new file mode 100755 index 000000000..3053b5b8a --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/scheduler/multitenant/DefaultPool.java @@ -0,0 +1,219 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.scheduler.multitenant; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.scheduler.SchedulerAssignment; +import backtype.storm.scheduler.TopologyDetails; +import backtype.storm.scheduler.WorkerSlot; + +/** + * A pool of machines that anyone can use, but topologies are not isolated + */ +public class DefaultPool extends NodePool { + private static final Logger LOG = LoggerFactory.getLogger(DefaultPool.class); + private Set _nodes = new HashSet(); + private HashMap _tds = new HashMap(); + + @Override + public void addTopology(TopologyDetails td) { + String topId = td.getId(); + LOG.debug("Adding in Topology {}", topId); + _tds.put(topId, td); + SchedulerAssignment assignment = _cluster.getAssignmentById(topId); + if (assignment != null) { + for (WorkerSlot ws: assignment.getSlots()) { + Node n = _nodeIdToNode.get(ws.getNodeId()); + _nodes.add(n); + } + } + } + + @Override + public boolean canAdd(TopologyDetails td) { + return true; + } + + @Override + public Collection takeNodes(int nodesNeeded) { + HashSet ret = new HashSet(); + LinkedList sortedNodes = new LinkedList(_nodes); + Collections.sort(sortedNodes, Node.FREE_NODE_COMPARATOR_DEC); + for (Node n: sortedNodes) { + if (nodesNeeded <= ret.size()) { + break; + } + if (n.isAlive()) { + n.freeAllSlots(_cluster); + _nodes.remove(n); + ret.add(n); + } + } + return ret; + } + + @Override + public int nodesAvailable() { + int total = 0; + for (Node n: _nodes) { + if (n.isAlive()) total++; + } + return total; + } + + @Override + public int slotsAvailable() { + return Node.countTotalSlotsAlive(_nodes); + } + + @Override + public NodeAndSlotCounts getNodeAndSlotCountIfSlotsWereTaken(int slotsNeeded) { + int nodesFound = 0; + int slotsFound = 0; + LinkedList sortedNodes = new LinkedList(_nodes); + Collections.sort(sortedNodes, Node.FREE_NODE_COMPARATOR_DEC); + for (Node n: sortedNodes) { + if (slotsNeeded <= 0) { + break; + } + if (n.isAlive()) { + nodesFound++; + int totalSlotsFree = n.totalSlots(); + slotsFound += totalSlotsFree; + slotsNeeded -= totalSlotsFree; + } + } + return new NodeAndSlotCounts(nodesFound, slotsFound); + } + + @Override + public Collection takeNodesBySlots(int slotsNeeded) { + HashSet ret = new HashSet(); + LinkedList sortedNodes = new LinkedList(_nodes); + Collections.sort(sortedNodes, Node.FREE_NODE_COMPARATOR_DEC); + for (Node n: sortedNodes) { + if (slotsNeeded <= 0) { + break; + } + if (n.isAlive()) { + n.freeAllSlots(_cluster); + _nodes.remove(n); + ret.add(n); + slotsNeeded -= n.totalSlotsFree(); + } + } + return ret; + } + + @Override + public void scheduleAsNeeded(NodePool... lesserPools) { + for (TopologyDetails td : _tds.values()) { + String topId = td.getId(); + if (_cluster.needsScheduling(td)) { + LOG.debug("Scheduling topology {}",topId); + int totalTasks = td.getExecutors().size(); + int origRequest = td.getNumWorkers(); + int slotsRequested = Math.min(totalTasks, origRequest); + int slotsUsed = Node.countSlotsUsed(topId, _nodes); + int slotsFree = Node.countFreeSlotsAlive(_nodes); + //Check to see if we have enough slots before trying to get them + int slotsAvailable = 0; + if (slotsRequested > slotsFree) { + slotsAvailable = NodePool.slotsAvailable(lesserPools); + } + int slotsToUse = Math.min(slotsRequested - slotsUsed, slotsFree + slotsAvailable); + int executorsNotRunning = _cluster.getUnassignedExecutors(td).size(); + LOG.debug("Slots... requested {} used {} free {} available {} to be used {}, executors not running {}", + new Object[] {slotsRequested, slotsUsed, slotsFree, slotsAvailable, slotsToUse, executorsNotRunning}); + if (slotsToUse <= 0) { + if (executorsNotRunning > 0) { + _cluster.setStatus(topId,"Not fully scheduled (No free slots in default pool) "+executorsNotRunning+" executors not scheduled"); + } else { + if (slotsUsed < slotsRequested) { + _cluster.setStatus(topId,"Running with fewer slots than requested ("+slotsUsed+"/"+origRequest+")"); + } else { //slotsUsed < origRequest + _cluster.setStatus(topId,"Fully Scheduled (requested "+origRequest+" slots, but could only use "+slotsUsed+")"); + } + } + continue; + } + + int slotsNeeded = slotsToUse - slotsFree; + if (slotsNeeded > 0) { + _nodes.addAll(NodePool.takeNodesBySlot(slotsNeeded, lesserPools)); + } + + if (executorsNotRunning <= 0) { + //There are free slots that we can take advantage of now. + for (Node n: _nodes) { + n.freeTopology(topId, _cluster); + } + slotsFree = Node.countFreeSlotsAlive(_nodes); + slotsToUse = Math.min(slotsRequested, slotsFree); + } + + RoundRobinSlotScheduler slotSched = + new RoundRobinSlotScheduler(td, slotsToUse, _cluster); + + LinkedList nodes = new LinkedList(_nodes); + while (true) { + Node n = null; + do { + if (nodes.isEmpty()) { + throw new IllegalStateException("This should not happen, we" + + " messed up and did not get enough slots"); + } + n = nodes.peekFirst(); + if (n.totalSlotsFree() == 0) { + nodes.remove(); + n = null; + } + } while (n == null); + if (!slotSched.assignSlotTo(n)) { + break; + } + } + int afterSchedSlotsUsed = Node.countSlotsUsed(topId, _nodes); + if (afterSchedSlotsUsed < slotsRequested) { + _cluster.setStatus(topId,"Running with fewer slots than requested ("+afterSchedSlotsUsed+"/"+origRequest+")"); + } else if (afterSchedSlotsUsed < origRequest) { + _cluster.setStatus(topId,"Fully Scheduled (requested "+origRequest+" slots, but could only use "+afterSchedSlotsUsed+")"); + } else { + _cluster.setStatus(topId,"Fully Scheduled"); + } + } else { + _cluster.setStatus(topId,"Fully Scheduled"); + } + } + } + + @Override + public String toString() { + return "DefaultPool " + _nodes.size() + " nodes " + _tds.size() + " topologies"; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/scheduler/multitenant/FreePool.java b/jstorm-core/src/main/java/backtype/storm/scheduler/multitenant/FreePool.java new file mode 100755 index 000000000..c62589539 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/scheduler/multitenant/FreePool.java @@ -0,0 +1,125 @@ +/** + * 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 backtype.storm.scheduler.multitenant; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.scheduler.Cluster; +import backtype.storm.scheduler.TopologyDetails; + +/** + * All of the machines that currently have nothing assigned to them + */ +public class FreePool extends NodePool { + private static final Logger LOG = LoggerFactory.getLogger(FreePool.class); + private Set _nodes = new HashSet(); + private int _totalSlots = 0; + + @Override + public void init(Cluster cluster, Map nodeIdToNode) { + super.init(cluster, nodeIdToNode); + for (Node n: nodeIdToNode.values()) { + if(n.isTotallyFree() && n.isAlive()) { + _nodes.add(n); + _totalSlots += n.totalSlotsFree(); + } + } + LOG.debug("Found {} nodes with {} slots", _nodes.size(), _totalSlots); + } + + @Override + public void addTopology(TopologyDetails td) { + throw new IllegalArgumentException("The free pool cannot run any topologies"); + } + + @Override + public boolean canAdd(TopologyDetails td) { + // The free pool never has anything running + return false; + } + + @Override + public Collection takeNodes(int nodesNeeded) { + HashSet ret = new HashSet(); + Iterator it = _nodes.iterator(); + while (it.hasNext() && nodesNeeded > ret.size()) { + Node n = it.next(); + ret.add(n); + _totalSlots -= n.totalSlotsFree(); + it.remove(); + } + return ret; + } + + @Override + public int nodesAvailable() { + return _nodes.size(); + } + + @Override + public int slotsAvailable() { + return _totalSlots; + } + + @Override + public Collection takeNodesBySlots(int slotsNeeded) { + HashSet ret = new HashSet(); + Iterator it = _nodes.iterator(); + while (it.hasNext() && slotsNeeded > 0) { + Node n = it.next(); + ret.add(n); + _totalSlots -= n.totalSlotsFree(); + slotsNeeded -= n.totalSlotsFree(); + it.remove(); + } + return ret; + } + + @Override + public NodeAndSlotCounts getNodeAndSlotCountIfSlotsWereTaken(int slotsNeeded) { + int slotsFound = 0; + int nodesFound = 0; + Iterator it = _nodes.iterator(); + while (it.hasNext() && slotsNeeded > 0) { + Node n = it.next(); + nodesFound++; + int totalSlots = n.totalSlots(); + slotsFound += totalSlots; + slotsNeeded -= totalSlots; + } + return new NodeAndSlotCounts(nodesFound, slotsFound); + } + + @Override + public void scheduleAsNeeded(NodePool... lesserPools) { + //No topologies running so NOOP + } + + @Override + public String toString() { + return "FreePool of "+_nodes.size()+" nodes with "+_totalSlots+" slots"; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/scheduler/multitenant/IsolatedPool.java b/jstorm-core/src/main/java/backtype/storm/scheduler/multitenant/IsolatedPool.java new file mode 100755 index 000000000..dc7eded4f --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/scheduler/multitenant/IsolatedPool.java @@ -0,0 +1,346 @@ +/** + * 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 backtype.storm.scheduler.multitenant; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.scheduler.SchedulerAssignment; +import backtype.storm.scheduler.TopologyDetails; +import backtype.storm.scheduler.WorkerSlot; + +/** + * A pool of machines that can be used to run isolated topologies + */ +public class IsolatedPool extends NodePool { + private static final Logger LOG = LoggerFactory.getLogger(IsolatedPool.class); + private Map> _topologyIdToNodes = new HashMap>(); + private HashMap _tds = new HashMap(); + private HashSet _isolated = new HashSet(); + private int _maxNodes; + private int _usedNodes; + + public IsolatedPool(int maxNodes) { + _maxNodes = maxNodes; + _usedNodes = 0; + } + + @Override + public void addTopology(TopologyDetails td) { + String topId = td.getId(); + LOG.debug("Adding in Topology {}", topId); + SchedulerAssignment assignment = _cluster.getAssignmentById(topId); + Set assignedNodes = new HashSet(); + if (assignment != null) { + for (WorkerSlot ws: assignment.getSlots()) { + Node n = _nodeIdToNode.get(ws.getNodeId()); + assignedNodes.add(n); + } + } + _usedNodes += assignedNodes.size(); + _topologyIdToNodes.put(topId, assignedNodes); + _tds.put(topId, td); + if (td.getConf().get(Config.TOPOLOGY_ISOLATED_MACHINES) != null) { + _isolated.add(topId); + } + } + + @Override + public boolean canAdd(TopologyDetails td) { + //Only add topologies that are not sharing nodes with other topologies + String topId = td.getId(); + SchedulerAssignment assignment = _cluster.getAssignmentById(topId); + if (assignment != null) { + for (WorkerSlot ws: assignment.getSlots()) { + Node n = _nodeIdToNode.get(ws.getNodeId()); + if (n.getRunningTopologies().size() > 1) { + return false; + } + } + } + return true; + } + + @Override + public void scheduleAsNeeded(NodePool ... lesserPools) { + for (String topId : _topologyIdToNodes.keySet()) { + TopologyDetails td = _tds.get(topId); + if (_cluster.needsScheduling(td)) { + LOG.debug("Scheduling topology {}",topId); + Set allNodes = _topologyIdToNodes.get(topId); + Number nodesRequested = (Number) td.getConf().get(Config.TOPOLOGY_ISOLATED_MACHINES); + int slotsToUse = 0; + if (nodesRequested == null) { + slotsToUse = getNodesForNotIsolatedTop(td, allNodes, lesserPools); + } else { + slotsToUse = getNodesForIsolatedTop(td, allNodes, lesserPools, + nodesRequested.intValue()); + } + //No slots to schedule for some reason, so skip it. + if (slotsToUse <= 0) { + continue; + } + + RoundRobinSlotScheduler slotSched = + new RoundRobinSlotScheduler(td, slotsToUse, _cluster); + + LinkedList sortedNodes = new LinkedList(allNodes); + Collections.sort(sortedNodes, Node.FREE_NODE_COMPARATOR_DEC); + + LOG.debug("Nodes sorted by free space {}", sortedNodes); + while (true) { + Node n = sortedNodes.remove(); + if (!slotSched.assignSlotTo(n)) { + break; + } + int freeSlots = n.totalSlotsFree(); + for (int i = 0; i < sortedNodes.size(); i++) { + if (freeSlots >= sortedNodes.get(i).totalSlotsFree()) { + sortedNodes.add(i, n); + n = null; + break; + } + } + if (n != null) { + sortedNodes.add(n); + } + } + } + Set found = _topologyIdToNodes.get(topId); + int nc = found == null ? 0 : found.size(); + _cluster.setStatus(topId,"Scheduled Isolated on "+nc+" Nodes"); + } + } + + /** + * Get the nodes needed to schedule an isolated topology. + * @param td the topology to be scheduled + * @param allNodes the nodes already scheduled for this topology. + * This will be updated to include new nodes if needed. + * @param lesserPools node pools we can steal nodes from + * @return the number of additional slots that should be used for scheduling. + */ + private int getNodesForIsolatedTop(TopologyDetails td, Set allNodes, + NodePool[] lesserPools, int nodesRequested) { + String topId = td.getId(); + LOG.debug("Topology {} is isolated", topId); + int nodesFromUsAvailable = nodesAvailable(); + int nodesFromOthersAvailable = NodePool.nodesAvailable(lesserPools); + + int nodesUsed = _topologyIdToNodes.get(topId).size(); + int nodesNeeded = nodesRequested - nodesUsed; + LOG.debug("Nodes... requested {} used {} available from us {} " + + "avail from other {} needed {}", new Object[] {nodesRequested, + nodesUsed, nodesFromUsAvailable, nodesFromOthersAvailable, + nodesNeeded}); + if ((nodesNeeded - nodesFromUsAvailable) > (_maxNodes - _usedNodes)) { + _cluster.setStatus(topId,"Max Nodes("+_maxNodes+") for this user would be exceeded. " + + ((nodesNeeded - nodesFromUsAvailable) - (_maxNodes - _usedNodes)) + + " more nodes needed to run topology."); + return 0; + } + + //In order to avoid going over _maxNodes I may need to steal from + // myself even though other pools have free nodes. so figure out how + // much each group should provide + int nodesNeededFromOthers = Math.min(Math.min(_maxNodes - _usedNodes, + nodesFromOthersAvailable), nodesNeeded); + int nodesNeededFromUs = nodesNeeded - nodesNeededFromOthers; + LOG.debug("Nodes... needed from us {} needed from others {}", + nodesNeededFromUs, nodesNeededFromOthers); + + if (nodesNeededFromUs > nodesFromUsAvailable) { + _cluster.setStatus(topId, "Not Enough Nodes Available to Schedule Topology"); + return 0; + } + + //Get the nodes + Collection found = NodePool.takeNodes(nodesNeededFromOthers, lesserPools); + _usedNodes += found.size(); + allNodes.addAll(found); + Collection foundMore = takeNodes(nodesNeededFromUs); + _usedNodes += foundMore.size(); + allNodes.addAll(foundMore); + + int totalTasks = td.getExecutors().size(); + int origRequest = td.getNumWorkers(); + int slotsRequested = Math.min(totalTasks, origRequest); + int slotsUsed = Node.countSlotsUsed(allNodes); + int slotsFree = Node.countFreeSlotsAlive(allNodes); + int slotsToUse = Math.min(slotsRequested - slotsUsed, slotsFree); + if (slotsToUse <= 0) { + _cluster.setStatus(topId, "Node has partially crashed, if this situation persists rebalance the topology."); + } + return slotsToUse; + } + + /** + * Get the nodes needed to schedule a non-isolated topology. + * @param td the topology to be scheduled + * @param allNodes the nodes already scheduled for this topology. + * This will be updated to include new nodes if needed. + * @param lesserPools node pools we can steal nodes from + * @return the number of additional slots that should be used for scheduling. + */ + private int getNodesForNotIsolatedTop(TopologyDetails td, Set allNodes, + NodePool[] lesserPools) { + String topId = td.getId(); + LOG.debug("Topology {} is not isolated",topId); + int totalTasks = td.getExecutors().size(); + int origRequest = td.getNumWorkers(); + int slotsRequested = Math.min(totalTasks, origRequest); + int slotsUsed = Node.countSlotsUsed(topId, allNodes); + int slotsFree = Node.countFreeSlotsAlive(allNodes); + //Check to see if we have enough slots before trying to get them + int slotsAvailable = 0; + if (slotsRequested > slotsFree) { + slotsAvailable = NodePool.slotsAvailable(lesserPools); + } + int slotsToUse = Math.min(slotsRequested - slotsUsed, slotsFree + slotsAvailable); + LOG.debug("Slots... requested {} used {} free {} available {} to be used {}", + new Object[] {slotsRequested, slotsUsed, slotsFree, slotsAvailable, slotsToUse}); + if (slotsToUse <= 0) { + _cluster.setStatus(topId, "Not Enough Slots Available to Schedule Topology"); + return 0; + } + int slotsNeeded = slotsToUse - slotsFree; + int numNewNodes = NodePool.getNodeCountIfSlotsWereTaken(slotsNeeded, lesserPools); + LOG.debug("Nodes... new {} used {} max {}", + new Object[]{numNewNodes, _usedNodes, _maxNodes}); + if ((numNewNodes + _usedNodes) > _maxNodes) { + _cluster.setStatus(topId,"Max Nodes("+_maxNodes+") for this user would be exceeded. " + + (numNewNodes - (_maxNodes - _usedNodes)) + " more nodes needed to run topology."); + return 0; + } + + Collection found = NodePool.takeNodesBySlot(slotsNeeded, lesserPools); + _usedNodes += found.size(); + allNodes.addAll(found); + return slotsToUse; + } + + @Override + public Collection takeNodes(int nodesNeeded) { + LOG.debug("Taking {} from {}", nodesNeeded, this); + HashSet ret = new HashSet(); + for (Entry> entry: _topologyIdToNodes.entrySet()) { + if (!_isolated.contains(entry.getKey())) { + Iterator it = entry.getValue().iterator(); + while (it.hasNext()) { + if (nodesNeeded <= 0) { + return ret; + } + Node n = it.next(); + it.remove(); + n.freeAllSlots(_cluster); + ret.add(n); + nodesNeeded--; + _usedNodes--; + } + } + } + return ret; + } + + @Override + public int nodesAvailable() { + int total = 0; + for (Entry> entry: _topologyIdToNodes.entrySet()) { + if (!_isolated.contains(entry.getKey())) { + total += entry.getValue().size(); + } + } + return total; + } + + @Override + public int slotsAvailable() { + int total = 0; + for (Entry> entry: _topologyIdToNodes.entrySet()) { + if (!_isolated.contains(entry.getKey())) { + total += Node.countTotalSlotsAlive(entry.getValue()); + } + } + return total; + } + + @Override + public Collection takeNodesBySlots(int slotsNeeded) { + HashSet ret = new HashSet(); + for (Entry> entry: _topologyIdToNodes.entrySet()) { + if (!_isolated.contains(entry.getKey())) { + Iterator it = entry.getValue().iterator(); + while (it.hasNext()) { + Node n = it.next(); + if (n.isAlive()) { + it.remove(); + _usedNodes--; + n.freeAllSlots(_cluster); + ret.add(n); + slotsNeeded -= n.totalSlots(); + if (slotsNeeded <= 0) { + return ret; + } + } + } + } + } + return ret; + } + + @Override + public NodeAndSlotCounts getNodeAndSlotCountIfSlotsWereTaken(int slotsNeeded) { + int nodesFound = 0; + int slotsFound = 0; + for (Entry> entry: _topologyIdToNodes.entrySet()) { + if (!_isolated.contains(entry.getKey())) { + Iterator it = entry.getValue().iterator(); + while (it.hasNext()) { + Node n = it.next(); + if (n.isAlive()) { + nodesFound++; + int totalSlotsFree = n.totalSlots(); + slotsFound += totalSlotsFree; + slotsNeeded -= totalSlotsFree; + if (slotsNeeded <= 0) { + return new NodeAndSlotCounts(nodesFound, slotsFound); + } + } + } + } + } + return new NodeAndSlotCounts(nodesFound, slotsFound); + } + + @Override + public String toString() { + return "IsolatedPool... "; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/scheduler/multitenant/MultitenantScheduler.java b/jstorm-core/src/main/java/backtype/storm/scheduler/multitenant/MultitenantScheduler.java new file mode 100755 index 000000000..320b38893 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/scheduler/multitenant/MultitenantScheduler.java @@ -0,0 +1,98 @@ +/** + * 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 backtype.storm.scheduler.multitenant; + +import java.util.HashMap; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.scheduler.Cluster; +import backtype.storm.scheduler.IScheduler; +import backtype.storm.scheduler.Topologies; +import backtype.storm.scheduler.TopologyDetails; +import backtype.storm.utils.Utils; + +public class MultitenantScheduler implements IScheduler { + private static final Logger LOG = LoggerFactory.getLogger(MultitenantScheduler.class); + @SuppressWarnings("rawtypes") + private Map _conf; + + @Override + public void prepare(@SuppressWarnings("rawtypes") Map conf) { + _conf = conf; + } + + private Map getUserConf() { + Map ret = (Map)_conf.get(Config.MULTITENANT_SCHEDULER_USER_POOLS); + if (ret == null) { + ret = new HashMap(); + } else { + ret = new HashMap(ret); + } + + Map fromFile = Utils.findAndReadConfigFile("multitenant-scheduler.yaml", false); + Map tmp = (Map)fromFile.get(Config.MULTITENANT_SCHEDULER_USER_POOLS); + if (tmp != null) { + ret.putAll(tmp); + } + return ret; + } + + + @Override + public void schedule(Topologies topologies, Cluster cluster) { + LOG.debug("Rerunning scheduling..."); + Map nodeIdToNode = Node.getAllNodesFrom(cluster); + + Map userConf = getUserConf(); + + Map userPools = new HashMap(); + for (Map.Entry entry : userConf.entrySet()) { + userPools.put(entry.getKey(), new IsolatedPool(entry.getValue().intValue())); + } + DefaultPool defaultPool = new DefaultPool(); + FreePool freePool = new FreePool(); + + freePool.init(cluster, nodeIdToNode); + for (IsolatedPool pool : userPools.values()) { + pool.init(cluster, nodeIdToNode); + } + defaultPool.init(cluster, nodeIdToNode); + + for (TopologyDetails td: topologies.getTopologies()) { + String user = (String)td.getConf().get(Config.TOPOLOGY_SUBMITTER_USER); + LOG.debug("Found top {} run by user {}",td.getId(), user); + NodePool pool = userPools.get(user); + if (pool == null || !pool.canAdd(td)) { + pool = defaultPool; + } + pool.addTopology(td); + } + + //Now schedule all of the topologies that need to be scheduled + for (IsolatedPool pool : userPools.values()) { + pool.scheduleAsNeeded(freePool, defaultPool); + } + defaultPool.scheduleAsNeeded(freePool); + LOG.debug("Scheduling done..."); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/scheduler/multitenant/Node.java b/jstorm-core/src/main/java/backtype/storm/scheduler/multitenant/Node.java new file mode 100755 index 000000000..883c65fe6 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/scheduler/multitenant/Node.java @@ -0,0 +1,343 @@ +/** + * 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 backtype.storm.scheduler.multitenant; + +import java.util.Collection; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.Map.Entry; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.scheduler.Cluster; +import backtype.storm.scheduler.ExecutorDetails; +import backtype.storm.scheduler.SchedulerAssignment; +import backtype.storm.scheduler.SupervisorDetails; +import backtype.storm.scheduler.WorkerSlot; + +/** + * Represents a single node in the cluster. + */ +public class Node { + private static final Logger LOG = LoggerFactory.getLogger(Node.class); + private Map> _topIdToUsedSlots = new HashMap>(); + private Set _freeSlots = new HashSet(); + private final String _nodeId; + private boolean _isAlive; + + public Node(String nodeId, Set allPorts, boolean isAlive) { + _nodeId = nodeId; + _isAlive = isAlive; + if (_isAlive && allPorts != null) { + for (int port: allPorts) { + _freeSlots.add(new WorkerSlot(_nodeId, port)); + } + } + } + + public String getId() { + return _nodeId; + } + + public boolean isAlive() { + return _isAlive; + } + + /** + * @return a collection of the topology ids currently running on this node + */ + public Collection getRunningTopologies() { + return _topIdToUsedSlots.keySet(); + } + + public boolean isTotallyFree() { + return _topIdToUsedSlots.isEmpty(); + } + + public int totalSlotsFree() { + return _freeSlots.size(); + } + + public int totalSlotsUsed() { + int total = 0; + for (Set slots: _topIdToUsedSlots.values()) { + total += slots.size(); + } + return total; + } + + public int totalSlots() { + return totalSlotsFree() + totalSlotsUsed(); + } + + public int totalSlotsUsed(String topId) { + int total = 0; + Set slots = _topIdToUsedSlots.get(topId); + if (slots != null) { + total = slots.size(); + } + return total; + } + + private void validateSlot(WorkerSlot ws) { + if (!_nodeId.equals(ws.getNodeId())) { + throw new IllegalArgumentException( + "Trying to add a slot to the wrong node " + ws + + " is not a part of " + _nodeId); + } + } + + private void addOrphanedSlot(WorkerSlot ws) { + if (_isAlive) { + throw new IllegalArgumentException("Orphaned Slots " + + "only are allowed on dead nodes."); + } + validateSlot(ws); + if (_freeSlots.contains(ws)) { + return; + } + for (Set used: _topIdToUsedSlots.values()) { + if (used.contains(ws)) { + return; + } + } + _freeSlots.add(ws); + } + + boolean assignInternal(WorkerSlot ws, String topId, boolean dontThrow) { + validateSlot(ws); + if (!_freeSlots.remove(ws)) { + for (Entry> topologySetEntry : _topIdToUsedSlots.entrySet()) { + if (topologySetEntry.getValue().contains(ws)) { + if (dontThrow) { + LOG.warn("Worker slot [" + ws + "] can't be assigned to " + topId + + ". Its already assigned to " + topologySetEntry.getKey() + "."); + return true; + } + throw new IllegalStateException("Worker slot [" + ws + "] can't be assigned to " + + topId + ". Its already assigned to " + topologySetEntry.getKey() + "."); + } + } + LOG.warn("Adding Worker slot [" + ws + "] that was not reported in the supervisor heartbeats," + + " but the worker is already running for topology " + topId + "."); + } + Set usedSlots = _topIdToUsedSlots.get(topId); + if (usedSlots == null) { + usedSlots = new HashSet(); + _topIdToUsedSlots.put(topId, usedSlots); + } + usedSlots.add(ws); + return false; + } + + /** + * Free all slots on this node. This will update the Cluster too. + * @param cluster the cluster to be updated + */ + public void freeAllSlots(Cluster cluster) { + if (!_isAlive) { + LOG.warn("Freeing all slots on a dead node {} ",_nodeId); + } + for (Entry> entry : _topIdToUsedSlots.entrySet()) { + cluster.freeSlots(entry.getValue()); + if (_isAlive) { + _freeSlots.addAll(entry.getValue()); + } + } + _topIdToUsedSlots = new HashMap>(); + } + + /** + * Frees a single slot in this node + * @param ws the slot to free + * @param cluster the cluster to update + */ + public void free(WorkerSlot ws, Cluster cluster, boolean forceFree) { + if (_freeSlots.contains(ws)) return; + boolean wasFound = false; + for (Entry> entry : _topIdToUsedSlots.entrySet()) { + Set slots = entry.getValue(); + if (slots.remove(ws)) { + cluster.freeSlot(ws); + if (_isAlive) { + _freeSlots.add(ws); + } + wasFound = true; + } + } + if(!wasFound) + { + if(forceFree) + { + LOG.info("Forcefully freeing the " + ws); + cluster.freeSlot(ws); + _freeSlots.add(ws); + } else { + throw new IllegalArgumentException("Tried to free a slot that was not" + + " part of this node " + _nodeId); + } + } + } + + /** + * Frees all the slots for a topology. + * @param topId the topology to free slots for + * @param cluster the cluster to update + */ + public void freeTopology(String topId, Cluster cluster) { + Set slots = _topIdToUsedSlots.get(topId); + if (slots == null || slots.isEmpty()) return; + for (WorkerSlot ws : slots) { + cluster.freeSlot(ws); + if (_isAlive) { + _freeSlots.add(ws); + } + } + _topIdToUsedSlots.remove(topId); + } + + /** + * Assign a free slot on the node to the following topology and executors. + * This will update the cluster too. + * @param topId the topology to assign a free slot to. + * @param executors the executors to run in that slot. + * @param cluster the cluster to be updated + */ + public void assign(String topId, Collection executors, + Cluster cluster) { + if (!_isAlive) { + throw new IllegalStateException("Trying to adding to a dead node " + _nodeId); + } + if (_freeSlots.isEmpty()) { + throw new IllegalStateException("Trying to assign to a full node " + _nodeId); + } + if (executors.size() == 0) { + LOG.warn("Trying to assign nothing from " + topId + " to " + _nodeId + " (Ignored)"); + } else { + WorkerSlot slot = _freeSlots.iterator().next(); + cluster.assign(slot, topId, executors); + assignInternal(slot, topId, false); + } + } + + @Override + public boolean equals(Object other) { + if (other instanceof Node) { + return _nodeId.equals(((Node)other)._nodeId); + } + return false; + } + + @Override + public int hashCode() { + return _nodeId.hashCode(); + } + + @Override + public String toString() { + return "Node: " + _nodeId; + } + + public static int countSlotsUsed(String topId, Collection nodes) { + int total = 0; + for (Node n: nodes) { + total += n.totalSlotsUsed(topId); + } + return total; + } + + public static int countSlotsUsed(Collection nodes) { + int total = 0; + for (Node n: nodes) { + total += n.totalSlotsUsed(); + } + return total; + } + + public static int countFreeSlotsAlive(Collection nodes) { + int total = 0; + for (Node n: nodes) { + if (n.isAlive()) { + total += n.totalSlotsFree(); + } + } + return total; + } + + public static int countTotalSlotsAlive(Collection nodes) { + int total = 0; + for (Node n: nodes) { + if (n.isAlive()) { + total += n.totalSlots(); + } + } + return total; + } + + public static Map getAllNodesFrom(Cluster cluster) { + Map nodeIdToNode = new HashMap(); + for (SupervisorDetails sup : cluster.getSupervisors().values()) { + //Node ID and supervisor ID are the same. + String id = sup.getId(); + boolean isAlive = !cluster.isBlackListed(id); + LOG.debug("Found a {} Node {} {}", + new Object[] {isAlive? "living":"dead", id, sup.getAllPorts()}); + nodeIdToNode.put(id, new Node(id, sup.getAllPorts(), isAlive)); + } + + for (Entry entry : cluster.getAssignments().entrySet()) { + String topId = entry.getValue().getTopologyId(); + for (WorkerSlot ws: entry.getValue().getSlots()) { + String id = ws.getNodeId(); + Node node = nodeIdToNode.get(id); + if (node == null) { + LOG.debug("Found an assigned slot on a dead supervisor {}", ws); + node = new Node(id, null, false); + nodeIdToNode.put(id, node); + } + if (!node.isAlive()) { + //The supervisor on the node down so add an orphaned slot to hold the unsupervised worker + node.addOrphanedSlot(ws); + } + if (node.assignInternal(ws, topId, true)) { + LOG.warn("Bad scheduling state for topology [" + topId+ "], the slot " + + ws + " assigned to multiple workers, un-assigning everything..."); + node.free(ws, cluster, true); + } + } + } + + return nodeIdToNode; + } + + /** + * Used to sort a list of nodes so the node with the most free slots comes + * first. + */ + public static final Comparator FREE_NODE_COMPARATOR_DEC = new Comparator() { + @Override + public int compare(Node o1, Node o2) { + return o2.totalSlotsFree() - o1.totalSlotsFree(); + } + }; +} diff --git a/jstorm-core/src/main/java/backtype/storm/scheduler/multitenant/NodePool.java b/jstorm-core/src/main/java/backtype/storm/scheduler/multitenant/NodePool.java new file mode 100755 index 000000000..21d1577a4 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/scheduler/multitenant/NodePool.java @@ -0,0 +1,296 @@ +/** + * 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 backtype.storm.scheduler.multitenant; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.scheduler.Cluster; +import backtype.storm.scheduler.ExecutorDetails; +import backtype.storm.scheduler.SchedulerAssignment; +import backtype.storm.scheduler.TopologyDetails; +import backtype.storm.scheduler.WorkerSlot; + +/** + * A pool of nodes that can be used to run topologies. + */ +public abstract class NodePool { + protected Cluster _cluster; + protected Map _nodeIdToNode; + + public static class NodeAndSlotCounts { + public final int _nodes; + public final int _slots; + + public NodeAndSlotCounts(int nodes, int slots) { + _nodes = nodes; + _slots = slots; + } + } + + /** + * Place executors into slots in a round robin way, taking into account + * component spreading among different hosts. + */ + public static class RoundRobinSlotScheduler { + private Map> _nodeToComps; + private HashMap> _spreadToSchedule; + private LinkedList> _slots; + private Set _lastSlot; + private Cluster _cluster; + private String _topId; + + /** + * Create a new scheduler for a given topology + * @param td the topology to schedule + * @param slotsToUse the number of slots to use for the executors left to + * schedule. + * @param cluster the cluster to schedule this on. + */ + public RoundRobinSlotScheduler(TopologyDetails td, int slotsToUse, + Cluster cluster) { + _topId = td.getId(); + _cluster = cluster; + + Map execToComp = td.getExecutorToComponent(); + SchedulerAssignment assignment = _cluster.getAssignmentById(_topId); + _nodeToComps = new HashMap>(); + + if (assignment != null) { + Map execToSlot = assignment.getExecutorToSlot(); + + for (Entry entry: execToSlot.entrySet()) { + String nodeId = entry.getValue().getNodeId(); + Set comps = _nodeToComps.get(nodeId); + if (comps == null) { + comps = new HashSet(); + _nodeToComps.put(nodeId, comps); + } + comps.add(execToComp.get(entry.getKey())); + } + } + + _spreadToSchedule = new HashMap>(); + List spreadComps = (List)td.getConf().get(Config.TOPOLOGY_SPREAD_COMPONENTS); + if (spreadComps != null) { + for (String comp: spreadComps) { + _spreadToSchedule.put(comp, new ArrayList()); + } + } + + _slots = new LinkedList>(); + for (int i = 0; i < slotsToUse; i++) { + _slots.add(new HashSet()); + } + + int at = 0; + for (Entry> entry: _cluster.getNeedsSchedulingComponentToExecutors(td).entrySet()) { + LOG.debug("Scheduling for {}", entry.getKey()); + if (_spreadToSchedule.containsKey(entry.getKey())) { + LOG.debug("Saving {} for spread...",entry.getKey()); + _spreadToSchedule.get(entry.getKey()).addAll(entry.getValue()); + } else { + for (ExecutorDetails ed: entry.getValue()) { + LOG.debug("Assigning {} {} to slot {}", new Object[]{entry.getKey(), ed, at}); + _slots.get(at).add(ed); + at++; + if (at >= _slots.size()) { + at = 0; + } + } + } + } + _lastSlot = _slots.get(_slots.size() - 1); + } + + /** + * Assign a slot to the given node. + * @param n the node to assign a slot to. + * @return true if there are more slots to assign else false. + */ + public boolean assignSlotTo(Node n) { + if (_slots.isEmpty()) { + return false; + } + Set slot = _slots.pop(); + if (slot == _lastSlot) { + //The last slot fill it up + for (Entry> entry: _spreadToSchedule.entrySet()) { + if (entry.getValue().size() > 0) { + slot.addAll(entry.getValue()); + } + } + } else { + String nodeId = n.getId(); + Set nodeComps = _nodeToComps.get(nodeId); + if (nodeComps == null) { + nodeComps = new HashSet(); + _nodeToComps.put(nodeId, nodeComps); + } + for (Entry> entry: _spreadToSchedule.entrySet()) { + if (entry.getValue().size() > 0) { + String comp = entry.getKey(); + if (!nodeComps.contains(comp)) { + nodeComps.add(comp); + slot.add(entry.getValue().remove(0)); + } + } + } + } + n.assign(_topId, slot, _cluster); + return !_slots.isEmpty(); + } + } + + private static final Logger LOG = LoggerFactory.getLogger(NodePool.class); + /** + * Initialize the pool. + * @param cluster the cluster + * @param nodeIdToNode the mapping of node id to nodes + */ + public void init(Cluster cluster, Map nodeIdToNode) { + _cluster = cluster; + _nodeIdToNode = nodeIdToNode; + } + + /** + * Add a topology to the pool + * @param td the topology to add. + */ + public abstract void addTopology(TopologyDetails td); + + /** + * Check if this topology can be added to this pool + * @param td the topology + * @return true if it can else false + */ + public abstract boolean canAdd(TopologyDetails td); + + /** + * @return the number of nodes that are available to be taken + */ + public abstract int slotsAvailable(); + + /** + * Take nodes from this pool that can fulfill possibly up to the + * slotsNeeded + * @param slotsNeeded the number of slots that are needed. + * @return a Collection of nodes with the removed nodes in it. + * This may be empty, but should not be null. + */ + public abstract Collection takeNodesBySlots(int slotsNeeded); + + /** + * Get the number of nodes and slots this would provide to get the slots needed + * @param slots the number of slots needed + * @return the number of nodes and slots that would be returned. + */ + public abstract NodeAndSlotCounts getNodeAndSlotCountIfSlotsWereTaken(int slots); + + /** + * @return the number of nodes that are available to be taken + */ + public abstract int nodesAvailable(); + + /** + * Take up to nodesNeeded from this pool + * @param nodesNeeded the number of nodes that are needed. + * @return a Collection of nodes with the removed nodes in it. + * This may be empty, but should not be null. + */ + public abstract Collection takeNodes(int nodesNeeded); + + /** + * Reschedule any topologies as needed. + * @param lesserPools pools that may be used to steal nodes from. + */ + public abstract void scheduleAsNeeded(NodePool ... lesserPools); + + public static int slotsAvailable(NodePool[] pools) { + int slotsAvailable = 0; + for (NodePool pool: pools) { + slotsAvailable += pool.slotsAvailable(); + } + return slotsAvailable; + } + + public static int nodesAvailable(NodePool[] pools) { + int nodesAvailable = 0; + for (NodePool pool: pools) { + nodesAvailable += pool.nodesAvailable(); + } + return nodesAvailable; + } + + public static Collection takeNodesBySlot(int slotsNeeded,NodePool[] pools) { + LOG.debug("Trying to grab {} free slots from {}",slotsNeeded, pools); + HashSet ret = new HashSet(); + for (NodePool pool: pools) { + Collection got = pool.takeNodesBySlots(slotsNeeded); + ret.addAll(got); + slotsNeeded -= Node.countFreeSlotsAlive(got); + LOG.debug("Got {} nodes so far need {} more slots",ret.size(),slotsNeeded); + if (slotsNeeded <= 0) { + break; + } + } + return ret; + } + + public static Collection takeNodes(int nodesNeeded,NodePool[] pools) { + LOG.debug("Trying to grab {} free nodes from {}",nodesNeeded, pools); + HashSet ret = new HashSet(); + for (NodePool pool: pools) { + Collection got = pool.takeNodes(nodesNeeded); + ret.addAll(got); + nodesNeeded -= got.size(); + LOG.debug("Got {} nodes so far need {} more nodes", ret.size(), nodesNeeded); + if (nodesNeeded <= 0) { + break; + } + } + return ret; + } + + public static int getNodeCountIfSlotsWereTaken(int slots,NodePool[] pools) { + LOG.debug("How many nodes to get {} slots from {}",slots, pools); + int total = 0; + for (NodePool pool: pools) { + NodeAndSlotCounts ns = pool.getNodeAndSlotCountIfSlotsWereTaken(slots); + total += ns._nodes; + slots -= ns._slots; + LOG.debug("Found {} nodes so far {} more slots needed", total, slots); + if (slots <= 0) { + break; + } + } + return total; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/INimbusCredentialPlugin.java b/jstorm-core/src/main/java/backtype/storm/security/INimbusCredentialPlugin.java new file mode 100755 index 000000000..9670045eb --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/INimbusCredentialPlugin.java @@ -0,0 +1,47 @@ +/** + * 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 backtype.storm.security; + +import backtype.storm.daemon.Shutdownable; + +import java.util.Map; + +/** + * Nimbus auto credential plugin that will be called on nimbus host + * during submit topology option. User can specify a list of implementation using config key + * nimbus.autocredential.plugins.classes. + */ +public interface INimbusCredentialPlugin extends Shutdownable { + + /** + * this method will be called when nimbus initializes. + * @param conf + */ + void prepare(Map conf); + + /** + * Method that will be called on nimbus as part of submit topology. This plugin will be called + * at least once during the submit Topology action. It will be not be called during activate instead + * the credentials return by this method will be merged with the other credentials in the topology + * and stored in zookeeper. + * @param credentials credentials map where more credentials will be added. + * @param conf topology configuration + * @return + */ + void populateCredentials(Map credentials, Map conf); +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/AuthUtils.java b/jstorm-core/src/main/java/backtype/storm/security/auth/AuthUtils.java new file mode 100755 index 000000000..ac3fb53c3 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/AuthUtils.java @@ -0,0 +1,284 @@ +/** + * 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 backtype.storm.security.auth; + +import backtype.storm.Config; +import javax.security.auth.login.Configuration; +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.Subject; +import java.security.NoSuchAlgorithmException; +import java.security.URIParameter; + +import backtype.storm.security.INimbusCredentialPlugin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.URI; +import java.util.Collection; +import java.util.Set; +import java.util.HashSet; +import java.util.Map; +import java.util.concurrent.ExecutorService; + +public class AuthUtils { + private static final Logger LOG = LoggerFactory.getLogger(AuthUtils.class); + public static final String LOGIN_CONTEXT_SERVER = "StormServer"; + public static final String LOGIN_CONTEXT_CLIENT = "StormClient"; + public static final String SERVICE = "storm_thrift_server"; + + /** + * Construct a JAAS configuration object per storm configuration file + * @param storm_conf Storm configuration + * @return JAAS configuration object + */ + public static Configuration GetConfiguration(Map storm_conf) { + Configuration login_conf = null; + + //find login file configuration from Storm configuration + String loginConfigurationFile = (String)storm_conf.get("java.security.auth.login.config"); + if ((loginConfigurationFile != null) && (loginConfigurationFile.length()>0)) { + File config_file = new File(loginConfigurationFile); + if (! config_file.canRead()) { + throw new RuntimeException("File " + loginConfigurationFile + + " cannot be read."); + } + try { + URI config_uri = config_file.toURI(); + login_conf = Configuration.getInstance("JavaLoginConfig", new URIParameter(config_uri)); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + return login_conf; + } + + /** + * Construct a principal to local plugin + * @param conf storm configuration + * @return the plugin + */ + public static IPrincipalToLocal GetPrincipalToLocalPlugin(Map storm_conf) { + IPrincipalToLocal ptol = null; + try { + String ptol_klassName = (String) storm_conf.get(Config.STORM_PRINCIPAL_TO_LOCAL_PLUGIN); + Class klass = Class.forName(ptol_klassName); + ptol = (IPrincipalToLocal)klass.newInstance(); + ptol.prepare(storm_conf); + } catch (Exception e) { + throw new RuntimeException(e); + } + return ptol; + } + + /** + * Construct a group mapping service provider plugin + * @param conf storm configuration + * @return the plugin + */ + public static IGroupMappingServiceProvider GetGroupMappingServiceProviderPlugin(Map storm_conf) { + IGroupMappingServiceProvider gmsp = null; + try { + String gmsp_klassName = (String) storm_conf.get(Config.STORM_GROUP_MAPPING_SERVICE_PROVIDER_PLUGIN); + Class klass = Class.forName(gmsp_klassName); + gmsp = (IGroupMappingServiceProvider)klass.newInstance(); + gmsp.prepare(storm_conf); + } catch (Exception e) { + throw new RuntimeException(e); + } + return gmsp; + } + + /** + * Get all of the configured Credential Renwer Plugins. + * @param storm_conf the storm configuration to use. + * @return the configured credential renewers. + */ + public static Collection GetCredentialRenewers(Map conf) { + try { + Set ret = new HashSet(); + Collection clazzes = (Collection)conf.get(Config.NIMBUS_CREDENTIAL_RENEWERS); + if (clazzes != null) { + for (String clazz : clazzes) { + ICredentialsRenewer inst = (ICredentialsRenewer)Class.forName(clazz).newInstance(); + inst.prepare(conf); + ret.add(inst); + } + } + return ret; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * Get all the Nimbus Auto cred plugins. + * @param conf nimbus configuration to use. + * @return nimbus auto credential plugins. + */ + public static Collection getNimbusAutoCredPlugins(Map conf) { + try { + Set ret = new HashSet(); + Collection clazzes = (Collection)conf.get(Config.NIMBUS_AUTO_CRED_PLUGINS); + if (clazzes != null) { + for (String clazz : clazzes) { + INimbusCredentialPlugin inst = (INimbusCredentialPlugin)Class.forName(clazz).newInstance(); + inst.prepare(conf); + ret.add(inst); + } + } + return ret; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * Get all of the configured AutoCredential Plugins. + * @param storm_conf the storm configuration to use. + * @return the configured auto credentials. + */ + public static Collection GetAutoCredentials(Map storm_conf) { + try { + Set autos = new HashSet(); + Collection clazzes = (Collection)storm_conf.get(Config.TOPOLOGY_AUTO_CREDENTIALS); + if (clazzes != null) { + for (String clazz : clazzes) { + IAutoCredentials a = (IAutoCredentials)Class.forName(clazz).newInstance(); + a.prepare(storm_conf); + autos.add(a); + } + } + LOG.info("Got AutoCreds "+autos); + return autos; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * Populate a subject from credentials using the IAutoCredentials. + * @param subject the subject to populate or null if a new Subject should be created. + * @param autos the IAutoCredentials to call to populate the subject. + * @param credentials the credentials to pull from + * @return the populated subject. + */ + public static Subject populateSubject(Subject subject, Collection autos, Map credentials) { + try { + if (subject == null) { + subject = new Subject(); + } + for (IAutoCredentials autoCred : autos) { + autoCred.populateSubject(subject, credentials); + } + return subject; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * Update a subject from credentials using the IAutoCredentials. + * @param subject the subject to update + * @param autos the IAutoCredentials to call to update the subject. + * @param credentials the credentials to pull from + */ + public static void updateSubject(Subject subject, Collection autos, Map credentials) { + if (subject == null) { + throw new RuntimeException("The subject cannot be null when updating a subject with credentials"); + } + + try { + for (IAutoCredentials autoCred : autos) { + autoCred.updateSubject(subject, credentials); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * Construct a transport plugin per storm configuration + * @param conf storm configuration + * @return + */ + public static ITransportPlugin GetTransportPlugin(ThriftConnectionType type, Map storm_conf, Configuration login_conf) { + ITransportPlugin transportPlugin = null; + try { + String transport_plugin_klassName = type.getTransportPlugin(storm_conf); + Class klass = Class.forName(transport_plugin_klassName); + transportPlugin = (ITransportPlugin)klass.newInstance(); + transportPlugin.prepare(type, storm_conf, login_conf); + } catch(Exception e) { + throw new RuntimeException(e); + } + return transportPlugin; + } + + private static IHttpCredentialsPlugin GetHttpCredentialsPlugin(Map conf, + String klassName) { + IHttpCredentialsPlugin plugin = null; + try { + Class klass = Class.forName(klassName); + plugin = (IHttpCredentialsPlugin)klass.newInstance(); + plugin.prepare(conf); + } catch(Exception e) { + throw new RuntimeException(e); + } + return plugin; + } + + /** + * Construct an HttpServletRequest credential plugin specified by the UI + * storm configuration + * @param conf storm configuration + * @return the plugin + */ + public static IHttpCredentialsPlugin GetUiHttpCredentialsPlugin(Map conf) { + String klassName = (String)conf.get(Config.UI_HTTP_CREDS_PLUGIN); + return AuthUtils.GetHttpCredentialsPlugin(conf, klassName); + } + + /** + * Construct an HttpServletRequest credential plugin specified by the DRPC + * storm configuration + * @param conf storm configuration + * @return the plugin + */ + public static IHttpCredentialsPlugin GetDrpcHttpCredentialsPlugin(Map conf) { + String klassName = (String)conf.get(Config.DRPC_HTTP_CREDS_PLUGIN); + return AuthUtils.GetHttpCredentialsPlugin(conf, klassName); + } + + public static String get(Configuration configuration, String section, String key) throws IOException { + AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry(section); + if (configurationEntries == null) { + String errorMessage = "Could not find a '"+ section + "' entry in this configuration."; + throw new IOException(errorMessage); + } + + for(AppConfigurationEntry entry: configurationEntries) { + Object val = entry.getOptions().get(key); + if (val != null) + return (String)val; + } + return null; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/DefaultHttpCredentialsPlugin.java b/jstorm-core/src/main/java/backtype/storm/security/auth/DefaultHttpCredentialsPlugin.java new file mode 100755 index 000000000..e2469e5d9 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/DefaultHttpCredentialsPlugin.java @@ -0,0 +1,96 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth; + +import java.security.Principal; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import javax.security.auth.Subject; +import javax.servlet.http.HttpServletRequest; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.security.auth.ReqContext; + +public class DefaultHttpCredentialsPlugin implements IHttpCredentialsPlugin { + private static final Logger LOG = + LoggerFactory.getLogger(DefaultHttpCredentialsPlugin.class); + + /** + * No-op + * @param storm_conf Storm configuration + */ + @Override + public void prepare(Map storm_conf) { + // Do nothing. + } + + /** + * Gets the user name from the request principal. + * @param req the servlet request + * @return the authenticated user, or null if none is authenticated + */ + @Override + public String getUserName(HttpServletRequest req) { + Principal princ = null; + if (req != null && (princ = req.getUserPrincipal()) != null) { + String userName = princ.getName(); + if (userName != null && !userName.isEmpty()) { + LOG.debug("HTTP request had user ("+userName+")"); + return userName; + } + } + return null; + } + + /** + * Populates a given context with a new Subject derived from the + * credentials in a servlet request. + * @param context the context to be populated + * @param req the servlet request + * @return the context + */ + @Override + public ReqContext populateContext(ReqContext context, + HttpServletRequest req) { + String userName = getUserName(req); + + String doAsUser = req.getHeader("doAsUser"); + if(doAsUser == null) { + doAsUser = req.getParameter("doAsUser"); + } + + if(doAsUser != null) { + context.setRealPrincipal(new SingleUserPrincipal(userName)); + userName = doAsUser; + } + + Set principals = new HashSet(); + if(userName != null) { + Principal p = new SingleUserPrincipal(userName); + principals.add(p); + } + Subject s = new Subject(true, principals, new HashSet(), new HashSet()); + context.setSubject(s); + + return context; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/DefaultPrincipalToLocal.java b/jstorm-core/src/main/java/backtype/storm/security/auth/DefaultPrincipalToLocal.java new file mode 100755 index 000000000..729d744d4 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/DefaultPrincipalToLocal.java @@ -0,0 +1,43 @@ +/** + * 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 backtype.storm.security.auth; + +import java.util.Map; +import java.security.Principal; + +/** + * Storm can be configured to launch worker processed as a given user. + * Some transports need to map the Principal to a local user name. + */ +public class DefaultPrincipalToLocal implements IPrincipalToLocal { + /** + * Invoked once immediately after construction + * @param conf Storm configuration + */ + public void prepare(Map storm_conf) {} + + /** + * Convert a Principal to a local user name. + * @param principal the principal to convert + * @return The local user name. + */ + public String toLocal(Principal principal) { + return principal == null ? null : principal.getName(); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/IAuthorizer.java b/jstorm-core/src/main/java/backtype/storm/security/auth/IAuthorizer.java new file mode 100755 index 000000000..d592bb749 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/IAuthorizer.java @@ -0,0 +1,47 @@ +/** + * 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 backtype.storm.security.auth; + +import java.util.Map; + +/** + * Nimbus could be configured with an authorization plugin. + * If not specified, all requests are authorized. + * + * You could specify the authorization plugin via storm parameter. For example: + * storm -c nimbus.authorization.class=backtype.storm.security.auth.NoopAuthorizer ... + * + * You could also specify it via storm.yaml: + * nimbus.authorization.class: backtype.storm.security.auth.NoopAuthorizer + */ +public interface IAuthorizer { + /** + * Invoked once immediately after construction + * @param conf Storm configuration + */ + void prepare(Map storm_conf); + + /** + * permit() method is invoked for each incoming Thrift request. + * @param context request context includes info about + * @param operation operation name + * @param topology_storm configuration of targeted topology + * @return true if the request is authorized, false if reject + */ + public boolean permit(ReqContext context, String operation, Map topology_conf); +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/IAutoCredentials.java b/jstorm-core/src/main/java/backtype/storm/security/auth/IAutoCredentials.java new file mode 100755 index 000000000..b3886da8c --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/IAutoCredentials.java @@ -0,0 +1,55 @@ +/** + * 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 backtype.storm.security.auth; + +import java.util.Map; + +import javax.security.auth.Subject; + +/** + * Provides a way to automatically push credentials to a topology and to + * retreave them in the worker. + */ +public interface IAutoCredentials { + + public void prepare(Map conf); + + /** + * Called to populate the credentials on the client side. + * @param credentials the credentials to be populated. + */ + public void populateCredentials(Map credentials); + + /** + * Called to initially populate the subject on the worker side with credentials passed in. + * @param subject the subject to optionally put credentials in. + * @param credentials the credentials to be used. + */ + public void populateSubject(Subject subject, Map credentials); + + + /** + * Called to update the subject on the worker side when new credentials are recieved. + * This means that populateSubject has already been called on this subject. + * @param subject the subject to optionally put credentials in. + * @param credentials the credentials to be used. + */ + public void updateSubject(Subject subject, Map credentials); + +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/ICredentialsRenewer.java b/jstorm-core/src/main/java/backtype/storm/security/auth/ICredentialsRenewer.java new file mode 100755 index 000000000..3eaf6c4b3 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/ICredentialsRenewer.java @@ -0,0 +1,41 @@ +/** + * 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 backtype.storm.security.auth; + +import java.util.Collection; +import java.util.Map; + +/** + * Provides a way to renew credentials on behelf of a user. + */ +public interface ICredentialsRenewer { + + /** + * Called when initializing the service. + * @param conf the storm cluster configuration. + */ + public void prepare(Map conf); + + /** + * Renew any credentials that need to be renewed. (Update the credentials if needed) + * @param credentials the credentials that may have something to renew. + * @param topologyConf topology configuration. + */ + public void renew(Map credentials, Map topologyConf); +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/IGroupMappingServiceProvider.java b/jstorm-core/src/main/java/backtype/storm/security/auth/IGroupMappingServiceProvider.java new file mode 100755 index 000000000..5590b812a --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/IGroupMappingServiceProvider.java @@ -0,0 +1,42 @@ +/** + * 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 backtype.storm.security.auth; + +import java.io.IOException; +import java.util.Set; +import java.util.Map; + +public interface IGroupMappingServiceProvider { + + /** + * Invoked once immediately after construction + * @param storm_conf Storm configuration + */ + void prepare(Map storm_conf); + + /** + * Get all various group memberships of a given user. + * Returns EMPTY list in case of non-existing user + * @param user User's name + * @return group memberships of user + * @throws IOException + */ + public Set getGroups(String user) throws IOException; + +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/IHttpCredentialsPlugin.java b/jstorm-core/src/main/java/backtype/storm/security/auth/IHttpCredentialsPlugin.java new file mode 100755 index 000000000..a012ce418 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/IHttpCredentialsPlugin.java @@ -0,0 +1,50 @@ +/** + * 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 backtype.storm.security.auth; + +import java.util.Map; +import javax.servlet.http.HttpServletRequest; + +import backtype.storm.security.auth.ReqContext; + +/** + * Interface for handling credentials in an HttpServletRequest + */ +public interface IHttpCredentialsPlugin { + /** + * Invoked once immediately after construction + * @param storm_conf Storm configuration + */ + void prepare(Map storm_conf); + + /** + * Gets the user name from the request. + * @param req the servlet request + * @return the authenticated user, or null if none is authenticated. + */ + String getUserName(HttpServletRequest req); + + /** + * Populates a given context with credentials information from an HTTP + * request. + * @param req the servlet request + * @return the context + */ + ReqContext populateContext(ReqContext context, HttpServletRequest req); +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/IPrincipalToLocal.java b/jstorm-core/src/main/java/backtype/storm/security/auth/IPrincipalToLocal.java new file mode 100755 index 000000000..fca3d37e6 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/IPrincipalToLocal.java @@ -0,0 +1,41 @@ +/** + * 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 backtype.storm.security.auth; + +import java.util.Map; +import java.security.Principal; + +/** + * Storm can be configured to launch worker processed as a given user. + * Some transports need to map the Principal to a local user name. + */ +public interface IPrincipalToLocal { + /** + * Invoked once immediately after construction + * @param conf Storm configuration + */ + void prepare(Map storm_conf); + + /** + * Convert a Principal to a local user name. + * @param principal the principal to convert + * @return The local user name. + */ + public String toLocal(Principal principal); +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/ITransportPlugin.java b/jstorm-core/src/main/java/backtype/storm/security/auth/ITransportPlugin.java new file mode 100755 index 000000000..5ba25576a --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/ITransportPlugin.java @@ -0,0 +1,61 @@ +/** + * 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 backtype.storm.security.auth; + +import java.io.IOException; +import java.security.Principal; +import java.util.Map; +import java.util.concurrent.ExecutorService; + +import javax.security.auth.login.Configuration; + +import org.apache.thrift.TProcessor; +import org.apache.thrift.server.TServer; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; + +import backtype.storm.security.auth.ThriftConnectionType; + +/** + * Interface for Thrift Transport plugin + */ +public interface ITransportPlugin { + /** + * Invoked once immediately after construction + * @param type the type of connection this will process. + * @param storm_conf Storm configuration + * @param login_conf login configuration + */ + void prepare(ThriftConnectionType type, Map storm_conf, Configuration login_conf); + + /** + * Create a server associated with a given port, service handler, and purpose + * @param processor service handler + * @return server + */ + public TServer getServer(TProcessor processor) throws IOException, TTransportException; + + /** + * Connect to the specified server via framed transport + * @param transport The underlying Thrift transport. + * @param serverHost server host + * @param asUser the user as which the connection should be established, and all the subsequent actions should be executed. + * Only applicable when using secure storm cluster. A null/blank value here will just indicate to use the logged in user. + */ + public TTransport connect(TTransport transport, String serverHost, String asUser) throws IOException, TTransportException; +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/KerberosPrincipalToLocal.java b/jstorm-core/src/main/java/backtype/storm/security/auth/KerberosPrincipalToLocal.java new file mode 100755 index 000000000..35c778869 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/KerberosPrincipalToLocal.java @@ -0,0 +1,45 @@ +/** + * 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 backtype.storm.security.auth; + +import java.util.Map; +import java.security.Principal; + +/** + * Map a kerberos principal to a local user + */ +public class KerberosPrincipalToLocal implements IPrincipalToLocal { + + /** + * Invoked once immediately after construction + * @param conf Storm configuration + */ + public void prepare(Map storm_conf) {} + + /** + * Convert a Principal to a local user name. + * @param principal the principal to convert + * @return The local user name. + */ + public String toLocal(Principal principal) { + //This technically does not conform with rfc1964, but should work so + // long as you don't have any really odd names in your KDC. + return principal == null ? null : principal.getName().split("[/@]")[0]; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/ReqContext.java b/jstorm-core/src/main/java/backtype/storm/security/auth/ReqContext.java new file mode 100755 index 000000000..a252f85b8 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/ReqContext.java @@ -0,0 +1,142 @@ +/** + * 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 backtype.storm.security.auth; + +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.net.InetAddress; +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.security.AccessControlContext; +import java.security.AccessController; +import java.security.Principal; +import javax.security.auth.Subject; + +/** + * context request context includes info about + * (1) remote address, + * (2) remote subject and primary principal + * (3) request ID + */ +public class ReqContext { + private static final AtomicInteger uniqueId = new AtomicInteger(0); + private Subject _subject; + private InetAddress _remoteAddr; + private Integer _reqID; + private Map _storm_conf; + private Principal realPrincipal; + + private static final Logger LOG = LoggerFactory.getLogger(ReqContext.class); + + + /** + * Get a request context associated with current thread + * @return + */ + public static ReqContext context() { + return ctxt.get(); + } + + //each thread will have its own request context + private static final ThreadLocal < ReqContext > ctxt = + new ThreadLocal < ReqContext > () { + @Override + protected ReqContext initialValue() { + return new ReqContext(AccessController.getContext()); + } + }; + + //private constructor + @VisibleForTesting + public ReqContext(AccessControlContext acl_ctxt) { + _subject = Subject.getSubject(acl_ctxt); + _reqID = uniqueId.incrementAndGet(); + } + + //private constructor + @VisibleForTesting + public ReqContext(Subject sub) { + _subject = sub; + _reqID = uniqueId.incrementAndGet(); + } + + + /** + * client address + */ + public void setRemoteAddress(InetAddress addr) { + _remoteAddr = addr; + } + + public InetAddress remoteAddress() { + return _remoteAddr; + } + + /** + * Set remote subject explicitly + */ + public void setSubject(Subject subject) { + _subject = subject; + } + + /** + * Retrieve client subject associated with this request context + */ + public Subject subject() { + return _subject; + } + + /** + * The primary principal associated current subject + */ + public Principal principal() { + if (_subject == null) return null; + Set princs = _subject.getPrincipals(); + if (princs.size()==0) return null; + return (Principal) (princs.toArray()[0]); + } + + public void setRealPrincipal(Principal realPrincipal) { + this.realPrincipal = realPrincipal; + } + /** + * The real principal associated with the subject. + */ + public Principal realPrincipal() { + return this.realPrincipal; + } + + /** + * Returns true if this request is an impersonation request. + * @return + */ + public boolean isImpersonating() { + return this.realPrincipal != null; + } + + /** + * request ID of this request + */ + public Integer requestID() { + return _reqID; + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/SaslTransportPlugin.java b/jstorm-core/src/main/java/backtype/storm/security/auth/SaslTransportPlugin.java new file mode 100755 index 000000000..7208a17d8 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/SaslTransportPlugin.java @@ -0,0 +1,176 @@ +/** + * 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 backtype.storm.security.auth; + +import java.io.IOException; +import java.net.Socket; +import java.security.Principal; +import java.util.Map; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.TimeUnit; + +import javax.security.auth.Subject; +import javax.security.auth.login.Configuration; +import javax.security.sasl.SaslServer; + +import backtype.storm.utils.ExtendedThreadPoolExecutor; +import org.apache.thrift.TException; +import org.apache.thrift.TProcessor; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.protocol.TProtocol; +import org.apache.thrift.server.TServer; +import org.apache.thrift.server.TThreadPoolServer; +import org.apache.thrift.transport.TSaslServerTransport; +import org.apache.thrift.transport.TServerSocket; +import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; +import org.apache.thrift.transport.TTransportFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.security.auth.ThriftConnectionType; + +/** + * Base class for SASL authentication plugin. + */ +public abstract class SaslTransportPlugin implements ITransportPlugin { + protected ThriftConnectionType type; + protected Map storm_conf; + protected Configuration login_conf; + private static final Logger LOG = LoggerFactory.getLogger(SaslTransportPlugin.class); + + @Override + public void prepare(ThriftConnectionType type, Map storm_conf, Configuration login_conf) { + this.type = type; + this.storm_conf = storm_conf; + this.login_conf = login_conf; + } + + @Override + public TServer getServer(TProcessor processor) throws IOException, TTransportException { + int port = type.getPort(storm_conf); + TTransportFactory serverTransportFactory = getServerTransportFactory(); + TServerSocket serverTransport = new TServerSocket(port); + int numWorkerThreads = type.getNumThreads(storm_conf); + Integer queueSize = type.getQueueSize(storm_conf); + + TThreadPoolServer.Args server_args = new TThreadPoolServer.Args(serverTransport). + processor(new TUGIWrapProcessor(processor)). + minWorkerThreads(numWorkerThreads). + maxWorkerThreads(numWorkerThreads). + protocolFactory(new TBinaryProtocol.Factory(false, true)); + + if (serverTransportFactory != null) { + server_args.transportFactory(serverTransportFactory); + } + BlockingQueue workQueue = new SynchronousQueue(); + if (queueSize != null) { + workQueue = new ArrayBlockingQueue(queueSize); + } + ThreadPoolExecutor executorService = new ExtendedThreadPoolExecutor(numWorkerThreads, numWorkerThreads, + 60, TimeUnit.SECONDS, workQueue); + server_args.executorService(executorService); + return new TThreadPoolServer(server_args); + } + + /** + * All subclass must implement this method + * @return + * @throws IOException + */ + protected abstract TTransportFactory getServerTransportFactory() throws IOException; + + + /** + * Processor that pulls the SaslServer object out of the transport, and + * assumes the remote user's UGI before calling through to the original + * processor. + * + * This is used on the server side to set the UGI for each specific call. + */ + private class TUGIWrapProcessor implements TProcessor { + final TProcessor wrapped; + + TUGIWrapProcessor(TProcessor wrapped) { + this.wrapped = wrapped; + } + + public boolean process(final TProtocol inProt, final TProtocol outProt) throws TException { + //populating request context + ReqContext req_context = ReqContext.context(); + + TTransport trans = inProt.getTransport(); + //Sasl transport + TSaslServerTransport saslTrans = (TSaslServerTransport)trans; + //remote address + TSocket tsocket = (TSocket)saslTrans.getUnderlyingTransport(); + Socket socket = tsocket.getSocket(); + req_context.setRemoteAddress(socket.getInetAddress()); + + //remote subject + SaslServer saslServer = saslTrans.getSaslServer(); + String authId = saslServer.getAuthorizationID(); + Subject remoteUser = new Subject(); + remoteUser.getPrincipals().add(new User(authId)); + req_context.setSubject(remoteUser); + + //invoke service handler + return wrapped.process(inProt, outProt); + } + } + + public static class User implements Principal { + private final String name; + + public User(String name) { + this.name = name; + } + + /** + * Get the full name of the user. + */ + public String getName() { + return name; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (o == null || getClass() != o.getClass()) { + return false; + } else { + return (name.equals(((User) o).name)); + } + } + + @Override + public int hashCode() { + return name.hashCode(); + } + + @Override + public String toString() { + return name; + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/ShellBasedGroupsMapping.java b/jstorm-core/src/main/java/backtype/storm/security/auth/ShellBasedGroupsMapping.java new file mode 100755 index 000000000..62a4c7e74 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/ShellBasedGroupsMapping.java @@ -0,0 +1,94 @@ +/** + * 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 backtype.storm.security.auth; + +import java.io.IOException; +import java.util.Set; +import java.util.HashSet; +import java.util.Map; +import java.util.StringTokenizer; +import backtype.storm.Config; +import backtype.storm.utils.Utils; +import backtype.storm.utils.ShellUtils; +import backtype.storm.utils.TimeCacheMap; +import backtype.storm.utils.ShellUtils.ExitCodeException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class ShellBasedGroupsMapping implements + IGroupMappingServiceProvider { + + public static Logger LOG = LoggerFactory.getLogger(ShellBasedGroupsMapping.class); + public TimeCacheMap> cachedGroups; + + /** + * Invoked once immediately after construction + * @param storm_conf Storm configuration + */ + @Override + public void prepare(Map storm_conf) { + int timeout = Utils.getInt(storm_conf.get(Config.STORM_GROUP_MAPPING_SERVICE_CACHE_DURATION_SECS)); + cachedGroups = new TimeCacheMap>(timeout); + } + + /** + * Returns list of groups for a user + * + * @param user get groups for this user + * @return list of groups for a given user + */ + @Override + public Set getGroups(String user) throws IOException { + if(cachedGroups.containsKey(user)) { + return cachedGroups.get(user); + } + Set groups = getUnixGroups(user); + if(!groups.isEmpty()) + cachedGroups.put(user,groups); + return groups; + } + + /** + * Get the current user's group list from Unix by running the command 'groups' + * NOTE. For non-existing user it will return EMPTY list + * @param user user name + * @return the groups set that the user belongs to + * @throws IOException if encounter any error when running the command + */ + private static Set getUnixGroups(final String user) throws IOException { + String result = ""; + try { + result = ShellUtils.execCommand(ShellUtils.getGroupsForUserCommand(user)); + } catch (ExitCodeException e) { + // if we didn't get the group - just return empty list; + LOG.warn("got exception trying to get groups for user " + user, e); + return new HashSet(); + } + + StringTokenizer tokenizer = + new StringTokenizer(result, ShellUtils.TOKEN_SEPARATOR_REGEX); + Set groups = new HashSet(); + while (tokenizer.hasMoreTokens()) { + groups.add(tokenizer.nextToken()); + } + return groups; + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/SimpleTransportPlugin.java b/jstorm-core/src/main/java/backtype/storm/security/auth/SimpleTransportPlugin.java new file mode 100755 index 000000000..2abcdae89 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/SimpleTransportPlugin.java @@ -0,0 +1,163 @@ +/** + * 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 backtype.storm.security.auth; + +import java.io.IOException; +import java.security.Principal; +import java.net.InetAddress; +import java.net.Socket; +import java.net.UnknownHostException; +import java.util.Map; +import java.util.HashSet; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.TimeUnit; + +import javax.security.auth.login.Configuration; +import javax.security.auth.Subject; +import org.apache.thrift.TException; +import org.apache.thrift.TProcessor; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.protocol.TProtocol; +import org.apache.thrift.server.THsHaServer; +import org.apache.thrift.server.TServer; +import org.apache.thrift.transport.TFramedTransport; +import org.apache.thrift.transport.TMemoryInputTransport; +import org.apache.thrift.transport.TNonblockingServerSocket; +import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.security.auth.ThriftConnectionType; + +/** + * Simple transport for Thrift plugin. + * + * This plugin is designed to be backward compatible with existing Storm code. + */ +public class SimpleTransportPlugin implements ITransportPlugin { + protected ThriftConnectionType type; + protected Map storm_conf; + protected Configuration login_conf; + private static final Logger LOG = LoggerFactory.getLogger(SimpleTransportPlugin.class); + + @Override + public void prepare(ThriftConnectionType type, Map storm_conf, Configuration login_conf) { + this.type = type; + this.storm_conf = storm_conf; + this.login_conf = login_conf; + } + + @Override + public TServer getServer(TProcessor processor) throws IOException, TTransportException { + int port = type.getPort(storm_conf); + TNonblockingServerSocket serverTransport = new TNonblockingServerSocket(port); + int numWorkerThreads = type.getNumThreads(storm_conf); + int maxBufferSize = type.getMaxBufferSize(storm_conf); + Integer queueSize = type.getQueueSize(storm_conf); + + THsHaServer.Args server_args = new THsHaServer.Args(serverTransport). + processor(new SimpleWrapProcessor(processor)). + workerThreads(numWorkerThreads). + protocolFactory(new TBinaryProtocol.Factory(false, true, maxBufferSize, -1)); + + if (queueSize != null) { + server_args.executorService(new ThreadPoolExecutor(numWorkerThreads, numWorkerThreads, + 60, TimeUnit.SECONDS, new ArrayBlockingQueue(queueSize))); + } + + //construct THsHaServer + return new THsHaServer(server_args); + } + + /** + * Connect to the specified server via framed transport + * @param transport The underlying Thrift transport. + * @param serverHost unused. + * @param asUser unused. + */ + @Override + public TTransport connect(TTransport transport, String serverHost, String asUser) throws TTransportException { + int maxBufferSize = type.getMaxBufferSize(storm_conf); + //create a framed transport + TTransport conn = new TFramedTransport(transport, maxBufferSize); + + //connect + conn.open(); + LOG.debug("Simple client transport has been established"); + + return conn; + } + + /** + * @return the subject that will be used for all connections + */ + protected Subject getDefaultSubject() { + return null; + } + + /** + * Processor that populate simple transport info into ReqContext, and then invoke a service handler + */ + private class SimpleWrapProcessor implements TProcessor { + final TProcessor wrapped; + + SimpleWrapProcessor(TProcessor wrapped) { + this.wrapped = wrapped; + } + + public boolean process(final TProtocol inProt, final TProtocol outProt) throws TException { + //populating request context + ReqContext req_context = ReqContext.context(); + + TTransport trans = inProt.getTransport(); + if (trans instanceof TMemoryInputTransport) { + try { + req_context.setRemoteAddress(InetAddress.getLocalHost()); + } catch (UnknownHostException e) { + throw new RuntimeException(e); + } + } else if (trans instanceof TSocket) { + TSocket tsocket = (TSocket)trans; + //remote address + Socket socket = tsocket.getSocket(); + req_context.setRemoteAddress(socket.getInetAddress()); + } + + //anonymous user + Subject s = getDefaultSubject(); + if (s == null) { + final String user = (String)storm_conf.get("debug.simple.transport.user"); + if (user != null) { + HashSet principals = new HashSet(); + principals.add(new Principal() { + public String getName() { return user; } + public String toString() { return user; } + }); + s = new Subject(true, principals, new HashSet(), new HashSet()); + } + } + req_context.setSubject(s); + + //invoke service handler + return wrapped.process(inProt, outProt); + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/SingleUserPrincipal.java b/jstorm-core/src/main/java/backtype/storm/security/auth/SingleUserPrincipal.java new file mode 100755 index 000000000..6af17faf6 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/SingleUserPrincipal.java @@ -0,0 +1,56 @@ +/** + * 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 backtype.storm.security.auth; + +import java.security.Principal; + +/** + * A Principal that represents a user. + */ +public class SingleUserPrincipal implements Principal { + + private final String _userName; + + public SingleUserPrincipal(String userName) { + _userName = userName; + } + + @Override + public boolean equals(Object another) { + if (another instanceof SingleUserPrincipal) { + return _userName.equals(((SingleUserPrincipal)another)._userName); + } + return false; + } + + @Override + public String getName() { + return _userName; + } + + @Override + public String toString() { + return this.getName(); + } + + @Override + public int hashCode() { + return _userName.hashCode(); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/TBackoffConnect.java b/jstorm-core/src/main/java/backtype/storm/security/auth/TBackoffConnect.java new file mode 100755 index 000000000..f547868f8 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/TBackoffConnect.java @@ -0,0 +1,77 @@ +/** + * 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 backtype.storm.security.auth; + +import java.io.IOException; +import java.util.Random; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import backtype.storm.utils.StormBoundedExponentialBackoffRetry; + +public class TBackoffConnect { + private static final Logger LOG = LoggerFactory.getLogger(TBackoffConnect.class); + private int _completedRetries = 0; + private int _retryTimes; + private StormBoundedExponentialBackoffRetry waitGrabber; + + public TBackoffConnect(int retryTimes, int retryInterval, int retryIntervalCeiling) { + + _retryTimes = retryTimes; + waitGrabber = new StormBoundedExponentialBackoffRetry(retryInterval, + retryIntervalCeiling, + retryTimes); + } + + public TTransport doConnectWithRetry(ITransportPlugin transportPlugin, TTransport underlyingTransport, String host, String asUser) throws IOException { + boolean connected = false; + TTransport transportResult = null; + while(!connected) { + try { + transportResult = transportPlugin.connect(underlyingTransport, host, asUser); + connected = true; + } catch (TTransportException ex) { + retryNext(ex); + } + } + return transportResult; + } + + private void retryNext(TTransportException ex) { + if(!canRetry()) { + throw new RuntimeException(ex); + } + try { + int sleeptime = waitGrabber.getSleepTimeMs(_completedRetries, 0); + + LOG.debug("Failed to connect. Retrying... (" + Integer.toString( _completedRetries) + ") in " + Integer.toString(sleeptime) + "ms"); + + Thread.sleep(sleeptime); + } catch (InterruptedException e) { + LOG.info("Nimbus connection retry interrupted."); + } + + _completedRetries++; + } + + private boolean canRetry() { + return (_completedRetries < _retryTimes); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/ThriftClient.java b/jstorm-core/src/main/java/backtype/storm/security/auth/ThriftClient.java new file mode 100755 index 000000000..8d2136a2a --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/ThriftClient.java @@ -0,0 +1,204 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.security.auth; + +import java.io.IOException; +import java.security.InvalidParameterException; +import java.util.List; +import java.util.Map; + +import javax.security.auth.login.Configuration; + +import org.apache.commons.lang.StringUtils; +import org.apache.curator.framework.CuratorFramework; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.protocol.TProtocol; +import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransport; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.cluster.Cluster; + +import backtype.storm.Config; +import backtype.storm.utils.Utils; + +public class ThriftClient { + private static final Logger LOG = LoggerFactory.getLogger(ThriftClient.class); + private TTransport _transport; + protected TProtocol _protocol; + private String hostPort; + private String host; + private Integer port; + + private Map conf; + + private Integer timeout; + private ThriftConnectionType type; + private String asUser; + + public ThriftClient(Map conf, ThriftConnectionType type) throws Exception { + this(conf, type, null, null, null, null); + } + + @SuppressWarnings("unchecked") + public ThriftClient(Map conf, ThriftConnectionType type, Integer timeout) throws Exception { + this(conf, type, null, null, timeout, null); + } + + /** + * This is only for be compatible for Storm + * @param conf + * @param type + * @param host + */ + public ThriftClient(Map conf, ThriftConnectionType type, String host) { + this(conf, type, host, null, null, null); + } + + public ThriftClient(Map conf, ThriftConnectionType type, String host, Integer port, Integer timeout){ + this(conf, type, host, port, timeout, null); + } + + public ThriftClient(Map conf, ThriftConnectionType type, String host, Integer port, Integer timeout, String asUser) { + //create a socket with server + + this.timeout = timeout; + this.conf = conf; + this.type = type; + this.asUser = asUser; + + getMaster(conf, host, port); + reconnect(); + } + + + + public static String getMasterByZk(Map conf) throws Exception { + + + CuratorFramework zkobj = null; + String masterHost = null; + + try { + String root = String.valueOf(conf.get(Config.STORM_ZOOKEEPER_ROOT)); + String zkMasterDir = root + Cluster.MASTER_SUBTREE; + + zkobj = Utils.newCurator(conf, + (List) conf.get(Config.STORM_ZOOKEEPER_SERVERS), + conf.get(Config.STORM_ZOOKEEPER_PORT), + zkMasterDir); + zkobj.start(); + if (zkobj.checkExists().forPath("/") == null) { + throw new RuntimeException("No alive nimbus "); + } + + masterHost = new String(zkobj.getData().forPath("/")); + + LOG.info("masterHost:" + masterHost); + return masterHost; + } finally { + if (zkobj != null) { + zkobj.close(); + zkobj = null; + } + } + } + + public void getMaster(Map conf, String host, Integer port){ + if (StringUtils.isBlank(host) == false) { + this.host = host; + if (port == null) { + port = type.getPort(conf); + } + this.port = port; + this.hostPort = host + ":" + port; + }else { + try { + hostPort = ThriftClient.getMasterByZk(conf); + } catch (Exception e) { + // TODO Auto-generated catch block + throw new RuntimeException("Failed to get master from ZK.", e); + } + String[] host_port = hostPort.split(":"); + if (host_port.length != 2) { + throw new InvalidParameterException("Host format error: " + hostPort); + } + this.host = host_port[0]; + this.port = Integer.parseInt(host_port[1]); + } + + // create a socket with server + if (this.host == null) { + throw new IllegalArgumentException("host is not set"); + } + if (this.port == null || this.port <= 0) { + throw new IllegalArgumentException("invalid port: " + port); + } + } + + public synchronized TTransport transport() { + return _transport; + } + + public synchronized void reconnect() { + close(); + try { + TSocket socket = new TSocket(host, port); + if(timeout!=null) { + socket.setTimeout(timeout); + }else { + //@@@ Todo + // set the socket default Timeout as xxxx + } + + //locate login configuration + Configuration login_conf = AuthUtils.GetConfiguration(conf); + + //construct a transport plugin + ITransportPlugin transportPlugin = AuthUtils.GetTransportPlugin(type, conf, login_conf); + + final TTransport underlyingTransport = socket; + + //TODO get this from type instead of hardcoding to Nimbus. + //establish client-server transport via plugin + //do retries if the connect fails + TBackoffConnect connectionRetry + = new TBackoffConnect( + Utils.getInt(conf.get(Config.STORM_NIMBUS_RETRY_TIMES)), + Utils.getInt(conf.get(Config.STORM_NIMBUS_RETRY_INTERVAL)), + Utils.getInt(conf.get(Config.STORM_NIMBUS_RETRY_INTERVAL_CEILING))); + _transport = connectionRetry.doConnectWithRetry(transportPlugin, underlyingTransport, host, asUser); + } catch (IOException ex) { + throw new RuntimeException(ex); + } + _protocol = null; + if (_transport != null) { + _protocol = new TBinaryProtocol(_transport); + } + } + + public synchronized void close() { + if (_transport != null) { + _transport.close(); + _transport = null; + _protocol = null; + } + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/ThriftConnectionType.java b/jstorm-core/src/main/java/backtype/storm/security/auth/ThriftConnectionType.java new file mode 100755 index 000000000..f9be7ae5f --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/ThriftConnectionType.java @@ -0,0 +1,77 @@ +/** + * 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 backtype.storm.security.auth; + +import backtype.storm.utils.Utils; +import backtype.storm.Config; + +import java.util.Map; + +/** + * The purpose for which the Thrift server is created. + */ +public enum ThriftConnectionType { + NIMBUS(Config.NIMBUS_THRIFT_TRANSPORT_PLUGIN, Config.NIMBUS_THRIFT_PORT, null, + Config.NIMBUS_THRIFT_THREADS, Config.NIMBUS_THRIFT_MAX_BUFFER_SIZE), + DRPC(Config.DRPC_THRIFT_TRANSPORT_PLUGIN, Config.DRPC_PORT, Config.DRPC_QUEUE_SIZE, + Config.DRPC_WORKER_THREADS, Config.DRPC_MAX_BUFFER_SIZE), + DRPC_INVOCATIONS(Config.DRPC_INVOCATIONS_THRIFT_TRANSPORT_PLUGIN, Config.DRPC_INVOCATIONS_PORT, null, + Config.DRPC_INVOCATIONS_THREADS, Config.DRPC_MAX_BUFFER_SIZE); + + private final String _transConf; + private final String _portConf; + private final String _qConf; + private final String _threadsConf; + private final String _buffConf; + + ThriftConnectionType(String transConf, String portConf, String qConf, + String threadsConf, String buffConf) { + _transConf = transConf; + _portConf = portConf; + _qConf = qConf; + _threadsConf = threadsConf; + _buffConf = buffConf; + } + + public String getTransportPlugin(Map conf) { + String ret = (String)conf.get(_transConf); + if (ret == null) { + ret = (String)conf.get(Config.STORM_THRIFT_TRANSPORT_PLUGIN); + } + return ret; + } + + public int getPort(Map conf) { + return Utils.getInt(conf.get(_portConf)); + } + + public Integer getQueueSize(Map conf) { + if (_qConf == null) { + return null; + } + return (Integer)conf.get(_qConf); + } + + public int getNumThreads(Map conf) { + return Utils.getInt(conf.get(_threadsConf)); + } + + public int getMaxBufferSize(Map conf) { + return Utils.getInt(conf.get(_buffConf)); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/ThriftServer.java b/jstorm-core/src/main/java/backtype/storm/security/auth/ThriftServer.java new file mode 100755 index 000000000..64243cee6 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/ThriftServer.java @@ -0,0 +1,80 @@ +/** + * 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 backtype.storm.security.auth; + +import java.util.Map; + +import javax.security.auth.login.Configuration; + +import org.apache.thrift.TProcessor; +import org.apache.thrift.server.TServer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ThriftServer { + private static final Logger LOG = LoggerFactory.getLogger(ThriftServer.class); + private Map _storm_conf; //storm configuration + protected TProcessor _processor = null; + private final ThriftConnectionType _type; + private TServer _server = null; + private Configuration _login_conf; + + public ThriftServer(Map storm_conf, TProcessor processor, ThriftConnectionType type) { + _storm_conf = storm_conf; + _processor = processor; + _type = type; + + try { + //retrieve authentication configuration + _login_conf = AuthUtils.GetConfiguration(_storm_conf); + } catch (Exception x) { + LOG.error(x.getMessage(), x); + } + } + + public void stop() { + if (_server != null) + _server.stop(); + } + + /** + * Is ThriftServer listening to requests? + * @return + */ + public boolean isServing() { + if (_server == null) return false; + return _server.isServing(); + } + + public void serve() { + try { + //locate our thrift transport plugin + ITransportPlugin transportPlugin = AuthUtils.GetTransportPlugin(_type, _storm_conf, _login_conf); + + //server + _server = transportPlugin.getServer(_processor); + + //start accepting requests + _server.serve(); + } catch (Exception ex) { + LOG.error("ThriftServer is being stopped due to: " + ex, ex); + if (_server != null) _server.stop(); + Runtime.getRuntime().halt(1); //shutdown server process since we could not handle Thrift requests any more + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/DRPCAuthorizerBase.java b/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/DRPCAuthorizerBase.java new file mode 100755 index 000000000..8951edd68 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/DRPCAuthorizerBase.java @@ -0,0 +1,46 @@ +package backtype.storm.security.auth.authorizer; + +import java.util.Map; + +import backtype.storm.security.auth.IAuthorizer; +import backtype.storm.security.auth.ReqContext; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class DRPCAuthorizerBase implements IAuthorizer { + public static Logger LOG = LoggerFactory.getLogger(DRPCAuthorizerBase.class); + + /** + * A key name for the function requested to be executed by a user. + */ + public static final String FUNCTION_NAME = "function.name"; + + @Override + public abstract void prepare(Map conf); + + abstract protected boolean permitClientRequest(ReqContext context, String operation, Map params); + + abstract protected boolean permitInvocationRequest(ReqContext context, String operation, Map params); + + /** + * Authorizes request from to the DRPC server. + * @param context the client request context + * @param operation the operation requested by the DRPC server + * @param params a Map with any key-value entries of use to the authorization implementation + */ + @Override + public boolean permit(ReqContext context, String operation, Map params) { + if ("execute".equals(operation)) { + return permitClientRequest(context, operation, params); + } else if ("failRequest".equals(operation) || + "fetchRequest".equals(operation) || + "result".equals(operation)) { + return permitInvocationRequest(context, operation, params); + } + // Deny unsupported operations. + LOG.warn("Denying unsupported operation \""+operation+"\" from "+ + context.remoteAddress()); + return false; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer.java b/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer.java new file mode 100755 index 000000000..45eaea547 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer.java @@ -0,0 +1,157 @@ +package backtype.storm.security.auth.authorizer; + +import java.lang.reflect.Field; +import java.security.Principal; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import backtype.storm.Config; +import backtype.storm.security.auth.ReqContext; +import backtype.storm.security.auth.authorizer.DRPCAuthorizerBase; +import backtype.storm.security.auth.AuthUtils; +import backtype.storm.security.auth.IPrincipalToLocal; +import backtype.storm.utils.Utils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DRPCSimpleACLAuthorizer extends DRPCAuthorizerBase { + public static Logger LOG = + LoggerFactory.getLogger(DRPCSimpleACLAuthorizer.class); + + public static final String CLIENT_USERS_KEY = "client.users"; + public static final String INVOCATION_USER_KEY = "invocation.user"; + public static final String FUNCTION_KEY = "function.name"; + + protected String _aclFileName = ""; + protected IPrincipalToLocal _ptol; + protected boolean _permitWhenMissingFunctionEntry = false; + + protected class AclFunctionEntry { + final public Set clientUsers; + final public String invocationUser; + public AclFunctionEntry(Collection clientUsers, + String invocationUser) { + this.clientUsers = (clientUsers != null) ? + new HashSet(clientUsers) : new HashSet(); + this.invocationUser = invocationUser; + } + } + + private volatile Map _acl = null; + private volatile long _lastUpdate = 0; + + protected Map readAclFromConfig() { + //Thread safety is mostly around _acl. If _acl needs to be updated it is changed atomically + //More then one thread may be trying to update it at a time, but that is OK, because the + //change is atomic + long now = System.currentTimeMillis(); + if ((now - 5000) > _lastUpdate || _acl == null) { + Map acl = new HashMap(); + Map conf = Utils.findAndReadConfigFile(_aclFileName); + if (conf.containsKey(Config.DRPC_AUTHORIZER_ACL)) { + Map> confAcl = + (Map>) + conf.get(Config.DRPC_AUTHORIZER_ACL); + + for (String function : confAcl.keySet()) { + Map val = confAcl.get(function); + Collection clientUsers = + val.containsKey(CLIENT_USERS_KEY) ? + (Collection) val.get(CLIENT_USERS_KEY) : null; + String invocationUser = + val.containsKey(INVOCATION_USER_KEY) ? + (String) val.get(INVOCATION_USER_KEY) : null; + acl.put(function, + new AclFunctionEntry(clientUsers, invocationUser)); + } + } else if (!_permitWhenMissingFunctionEntry) { + LOG.warn("Requiring explicit ACL entries, but none given. " + + "Therefore, all operiations will be denied."); + } + _acl = acl; + _lastUpdate = System.currentTimeMillis(); + } + return _acl; + } + + @Override + public void prepare(Map conf) { + Boolean isStrict = + (Boolean) conf.get(Config.DRPC_AUTHORIZER_ACL_STRICT); + _permitWhenMissingFunctionEntry = + (isStrict != null && !isStrict) ? true : false; + _aclFileName = (String) conf.get(Config.DRPC_AUTHORIZER_ACL_FILENAME); + _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf); + } + + private String getUserFromContext(ReqContext context) { + if (context != null) { + Principal princ = context.principal(); + if (princ != null) { + return princ.getName(); + } + } + return null; + } + + private String getLocalUserFromContext(ReqContext context) { + if (context != null) { + return _ptol.toLocal(context.principal()); + } + return null; + } + + protected boolean permitClientOrInvocationRequest(ReqContext context, Map params, + String fieldName) { + Map acl = readAclFromConfig(); + String function = (String) params.get(FUNCTION_KEY); + if (function != null && ! function.isEmpty()) { + AclFunctionEntry entry = acl.get(function); + if (entry == null && _permitWhenMissingFunctionEntry) { + return true; + } + if (entry != null) { + Object value; + try { + Field field = AclFunctionEntry.class.getDeclaredField(fieldName); + value = field.get(entry); + } catch (Exception ex) { + LOG.warn("Caught Exception while accessing ACL", ex); + return false; + } + String principal = getUserFromContext(context); + String user = getLocalUserFromContext(context); + if (value == null) { + LOG.warn("Configuration for function '"+function+"' is "+ + "invalid: it should have both an invocation user "+ + "and a list of client users defined."); + } else if (value instanceof Set && + (((Set)value).contains(principal) || + ((Set)value).contains(user))) { + return true; + } else if (value instanceof String && + (value.equals(principal) || + value.equals(user))) { + return true; + } + } + } + return false; + } + + @Override + protected boolean permitClientRequest(ReqContext context, String operation, + Map params) { + return permitClientOrInvocationRequest(context, params, "clientUsers"); + } + + @Override + protected boolean permitInvocationRequest(ReqContext context, String operation, + Map params) { + return permitClientOrInvocationRequest(context, params, "invocationUser"); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/DenyAuthorizer.java b/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/DenyAuthorizer.java new file mode 100755 index 000000000..5e84b382f --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/DenyAuthorizer.java @@ -0,0 +1,57 @@ +/** + * 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 backtype.storm.security.auth.authorizer; + +import java.util.Map; + +import backtype.storm.Config; +import backtype.storm.security.auth.IAuthorizer; +import backtype.storm.security.auth.ReqContext; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An authorization implementation that denies everything, for testing purposes + */ +public class DenyAuthorizer implements IAuthorizer { + private static final Logger LOG = LoggerFactory.getLogger(DenyAuthorizer.class); + + /** + * Invoked once immediately after construction + * @param conf Storm configuration + */ + public void prepare(Map conf) { + } + + /** + * permit() method is invoked for each incoming Thrift request + * @param contrext request context + * @param operation operation name + * @param topology_storm configuration of targeted topology + * @return true if the request is authorized, false if reject + */ + public boolean permit(ReqContext context, String operation, Map topology_conf) { + LOG.info("[req "+ context.requestID()+ "] Access " + + " from: " + (context.remoteAddress() == null? "null" : context.remoteAddress().toString()) + + (context.principal() == null? "" : (" principal:"+ context.principal())) + +" op:"+operation + + (topology_conf == null? "" : (" topoology:"+topology_conf.get(Config.TOPOLOGY_NAME)))); + return false; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/ImpersonationAuthorizer.java b/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/ImpersonationAuthorizer.java new file mode 100755 index 000000000..d6431be56 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/ImpersonationAuthorizer.java @@ -0,0 +1,154 @@ +package backtype.storm.security.auth.authorizer; + +import backtype.storm.Config; +import backtype.storm.security.auth.*; +import com.google.common.collect.ImmutableSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.InetAddress; +import java.util.*; + + +public class ImpersonationAuthorizer implements IAuthorizer { + private static final Logger LOG = LoggerFactory.getLogger(ImpersonationAuthorizer.class); + protected static final String WILD_CARD = "*"; + + protected Map userImpersonationACL; + protected IPrincipalToLocal _ptol; + protected IGroupMappingServiceProvider _groupMappingProvider; + + @Override + public void prepare(Map conf) { + userImpersonationACL = new HashMap(); + + Map>> userToHostAndGroup = (Map>>) conf.get(Config.NIMBUS_IMPERSONATION_ACL); + + if (userToHostAndGroup != null) { + for (String user : userToHostAndGroup.keySet()) { + Set groups = ImmutableSet.copyOf(userToHostAndGroup.get(user).get("groups")); + Set hosts = ImmutableSet.copyOf(userToHostAndGroup.get(user).get("hosts")); + userImpersonationACL.put(user, new ImpersonationACL(user, groups, hosts)); + } + } + + _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf); + _groupMappingProvider = AuthUtils.GetGroupMappingServiceProviderPlugin(conf); + } + + @Override + public boolean permit(ReqContext context, String operation, Map topology_conf) { + if (!context.isImpersonating()) { + LOG.debug("Not an impersonation attempt."); + return true; + } + + String impersonatingPrincipal = context.realPrincipal().getName(); + String impersonatingUser = _ptol.toLocal(context.realPrincipal()); + String userBeingImpersonated = _ptol.toLocal(context.principal()); + InetAddress remoteAddress = context.remoteAddress(); + + LOG.info("user = {}, principal = {} is attmepting to impersonate user = {} for operation = {} from host = {}", + impersonatingUser, impersonatingPrincipal, userBeingImpersonated, operation, remoteAddress); + + /** + * no config is present for impersonating principal or user, do not permit impersonation. + */ + if (!userImpersonationACL.containsKey(impersonatingPrincipal) && !userImpersonationACL.containsKey(impersonatingUser)) { + LOG.info("user = {}, principal = {} is trying to impersonate user {}, but config {} does not have entry for impersonating user or principal." + + "Please see SECURITY.MD to learn how to configure users for impersonation." + , impersonatingUser, impersonatingPrincipal, userBeingImpersonated, Config.NIMBUS_IMPERSONATION_ACL); + return false; + } + + ImpersonationACL principalACL = userImpersonationACL.get(impersonatingPrincipal); + ImpersonationACL userACL = userImpersonationACL.get(impersonatingUser); + + Set authorizedHosts = new HashSet(); + Set authorizedGroups = new HashSet(); + + if (principalACL != null) { + authorizedHosts.addAll(principalACL.authorizedHosts); + authorizedGroups.addAll(principalACL.authorizedGroups); + } + + if (userACL != null) { + authorizedHosts.addAll(userACL.authorizedHosts); + authorizedGroups.addAll(userACL.authorizedGroups); + } + + LOG.debug("user = {}, principal = {} is allowed to impersonate groups = {} from hosts = {} ", + impersonatingUser, impersonatingPrincipal, authorizedGroups, authorizedHosts); + + if (!isAllowedToImpersonateFromHost(authorizedHosts, remoteAddress)) { + LOG.info("user = {}, principal = {} is not allowed to impersonate from host {} ", + impersonatingUser, impersonatingPrincipal, remoteAddress); + return false; + } + + if (!isAllowedToImpersonateUser(authorizedGroups, userBeingImpersonated)) { + LOG.info("user = {}, principal = {} is not allowed to impersonate any group that user {} is part of.", + impersonatingUser, impersonatingPrincipal, userBeingImpersonated); + return false; + } + + LOG.info("Allowing impersonation of user {} by user {}", userBeingImpersonated, impersonatingUser); + return true; + } + + private boolean isAllowedToImpersonateFromHost(Set authorizedHosts, InetAddress remoteAddress) { + return authorizedHosts.contains(WILD_CARD) || + authorizedHosts.contains(remoteAddress.getCanonicalHostName()) || + authorizedHosts.contains(remoteAddress.getHostName()) || + authorizedHosts.contains(remoteAddress.getHostAddress()); + } + + private boolean isAllowedToImpersonateUser(Set authorizedGroups, String userBeingImpersonated) { + if(authorizedGroups.contains(WILD_CARD)) { + return true; + } + + Set groups = null; + try { + groups = _groupMappingProvider.getGroups(userBeingImpersonated); + } catch (IOException e) { + throw new RuntimeException("failed to get groups for user " + userBeingImpersonated); + } + + if (groups == null || groups.isEmpty()) { + return false; + } + + for (String group : groups) { + if (authorizedGroups.contains(group)) { + return true; + } + } + + return false; + } + + protected class ImpersonationACL { + public String impersonatingUser; + //Groups this user is authorized to impersonate. + public Set authorizedGroups; + //Hosts this user is authorized to impersonate from. + public Set authorizedHosts; + + private ImpersonationACL(String impersonatingUser, Set authorizedGroups, Set authorizedHosts) { + this.impersonatingUser = impersonatingUser; + this.authorizedGroups = authorizedGroups; + this.authorizedHosts = authorizedHosts; + } + + @Override + public String toString() { + return "ImpersonationACL{" + + "impersonatingUser='" + impersonatingUser + '\'' + + ", authorizedGroups=" + authorizedGroups + + ", authorizedHosts=" + authorizedHosts + + '}'; + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/NoopAuthorizer.java b/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/NoopAuthorizer.java new file mode 100755 index 000000000..9af44d334 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/NoopAuthorizer.java @@ -0,0 +1,57 @@ +/** + * 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 backtype.storm.security.auth.authorizer; + +import java.util.Map; + +import backtype.storm.Config; +import backtype.storm.security.auth.IAuthorizer; +import backtype.storm.security.auth.ReqContext; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A no-op authorization implementation that illustrate info available for authorization decisions. + */ +public class NoopAuthorizer implements IAuthorizer { + private static final Logger LOG = LoggerFactory.getLogger(NoopAuthorizer.class); + + /** + * Invoked once immediately after construction + * @param conf Storm configuration + */ + public void prepare(Map conf) { + } + + /** + * permit() method is invoked for each incoming Thrift request + * @param context request context includes info about + * @param operation operation name + * @param topology_storm configuration of targeted topology + * @return true if the request is authorized, false if reject + */ + public boolean permit(ReqContext context, String operation, Map topology_conf) { + LOG.info("[req "+ context.requestID()+ "] Access " + + " from: " + (context.remoteAddress() == null? "null" : context.remoteAddress().toString()) + + (context.principal() == null? "" : (" principal:"+ context.principal())) + +" op:"+operation + + (topology_conf == null? "" : (" topoology:"+topology_conf.get(Config.TOPOLOGY_NAME)))); + return true; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java b/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java new file mode 100755 index 000000000..e50a587d3 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/SimpleACLAuthorizer.java @@ -0,0 +1,152 @@ +/** + * 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 backtype.storm.security.auth.authorizer; + +import java.util.Arrays; +import java.util.Map; +import java.util.Set; +import java.util.HashSet; +import java.util.Collection; +import java.io.IOException; + +import backtype.storm.Config; +import backtype.storm.security.auth.IAuthorizer; +import backtype.storm.security.auth.ReqContext; +import backtype.storm.security.auth.AuthUtils; +import backtype.storm.security.auth.IPrincipalToLocal; +import backtype.storm.security.auth.IGroupMappingServiceProvider; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An authorization implementation that simply checks if a user is allowed to perform specific + * operations. + */ +public class SimpleACLAuthorizer implements IAuthorizer { + private static final Logger LOG = LoggerFactory.getLogger(SimpleACLAuthorizer.class); + + protected Set _userCommands = new HashSet(Arrays.asList("submitTopology", "fileUpload", "getNimbusConf", "getClusterInfo")); + protected Set _supervisorCommands = new HashSet(Arrays.asList("fileDownload")); + protected Set _topoCommands = new HashSet(Arrays.asList("killTopology","rebalance","activate","deactivate","getTopologyConf","getTopology","getUserTopology","getTopologyInfo","uploadNewCredentials")); + + protected Set _admins; + protected Set _supervisors; + protected Set _nimbusUsers; + protected Set _nimbusGroups; + protected IPrincipalToLocal _ptol; + protected IGroupMappingServiceProvider _groupMappingProvider; + /** + * Invoked once immediately after construction + * @param conf Storm configuration + */ + @Override + public void prepare(Map conf) { + _admins = new HashSet(); + _supervisors = new HashSet(); + _nimbusUsers = new HashSet(); + _nimbusGroups = new HashSet(); + + if (conf.containsKey(Config.NIMBUS_ADMINS)) { + _admins.addAll((Collection)conf.get(Config.NIMBUS_ADMINS)); + } + if (conf.containsKey(Config.NIMBUS_SUPERVISOR_USERS)) { + _supervisors.addAll((Collection)conf.get(Config.NIMBUS_SUPERVISOR_USERS)); + } + if (conf.containsKey(Config.NIMBUS_USERS)) { + _nimbusUsers.addAll((Collection)conf.get(Config.NIMBUS_USERS)); + } + + if (conf.containsKey(Config.NIMBUS_GROUPS)) { + _nimbusGroups.addAll((Collection)conf.get(Config.NIMBUS_GROUPS)); + } + + _ptol = AuthUtils.GetPrincipalToLocalPlugin(conf); + _groupMappingProvider = AuthUtils.GetGroupMappingServiceProviderPlugin(conf); + } + + /** + * permit() method is invoked for each incoming Thrift request + * @param context request context includes info about + * @param operation operation name + * @param topology_conf configuration of targeted topology + * @return true if the request is authorized, false if reject + */ + @Override + public boolean permit(ReqContext context, String operation, Map topology_conf) { + LOG.info("[req " + context.requestID() + "] Access " + + " from: " + (context.remoteAddress() == null ? "null" : context.remoteAddress().toString()) + + (context.principal() == null ? "" : (" principal:" + context.principal())) + + " op:" + operation + + (topology_conf == null ? "" : (" topoology:" + topology_conf.get(Config.TOPOLOGY_NAME)))); + + String principal = context.principal().getName(); + String user = _ptol.toLocal(context.principal()); + Set userGroups = new HashSet(); + + if (_groupMappingProvider != null) { + try { + userGroups = _groupMappingProvider.getGroups(user); + } catch(IOException e) { + LOG.warn("Error while trying to fetch user groups",e); + } + } + + if (_admins.contains(principal) || _admins.contains(user)) { + return true; + } + + if (_supervisors.contains(principal) || _supervisors.contains(user)) { + return _supervisorCommands.contains(operation); + } + + if (_userCommands.contains(operation)) { + return _nimbusUsers.size() == 0 || _nimbusUsers.contains(user) || checkUserGroupAllowed(userGroups, _nimbusGroups); + } + + if (_topoCommands.contains(operation)) { + Set topoUsers = new HashSet(); + if (topology_conf.containsKey(Config.TOPOLOGY_USERS)) { + topoUsers.addAll((Collection)topology_conf.get(Config.TOPOLOGY_USERS)); + } + + if (topoUsers.contains(principal) || topoUsers.contains(user)) { + return true; + } + + Set topoGroups = new HashSet(); + if (topology_conf.containsKey(Config.TOPOLOGY_GROUPS) && topology_conf.get(Config.TOPOLOGY_GROUPS) != null) { + topoGroups.addAll((Collection)topology_conf.get(Config.TOPOLOGY_GROUPS)); + } + + if (checkUserGroupAllowed(userGroups, topoGroups)) return true; + } + return false; + } + + private Boolean checkUserGroupAllowed(Set userGroups, Set configuredGroups) { + if(userGroups.size() > 0 && configuredGroups.size() > 0) { + for (String tgroup : configuredGroups) { + if(userGroups.contains(tgroup)) + return true; + } + } + return false; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/SimpleWhitelistAuthorizer.java b/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/SimpleWhitelistAuthorizer.java new file mode 100755 index 000000000..55109f9da --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/authorizer/SimpleWhitelistAuthorizer.java @@ -0,0 +1,70 @@ +/** + * 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 backtype.storm.security.auth.authorizer; + +import java.util.Map; +import java.util.Set; +import java.util.HashSet; +import java.util.Collection; + +import backtype.storm.Config; +import backtype.storm.security.auth.IAuthorizer; +import backtype.storm.security.auth.ReqContext; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An authorization implementation that simply checks a whitelist of users that + * are allowed to use the cluster. + */ +public class SimpleWhitelistAuthorizer implements IAuthorizer { + private static final Logger LOG = LoggerFactory.getLogger(SimpleWhitelistAuthorizer.class); + public static String WHITELIST_USERS_CONF = "storm.auth.simple-white-list.users"; + protected Set users; + + /** + * Invoked once immediately after construction + * @param conf Storm configuration + */ + @Override + public void prepare(Map conf) { + users = new HashSet(); + if (conf.containsKey(WHITELIST_USERS_CONF)) { + users.addAll((Collection)conf.get(WHITELIST_USERS_CONF)); + } + } + + /** + * permit() method is invoked for each incoming Thrift request + * @param context request context includes info about + * @param operation operation name + * @param topology_storm configuration of targeted topology + * @return true if the request is authorized, false if reject + */ + @Override + public boolean permit(ReqContext context, String operation, Map topology_conf) { + LOG.info("[req "+ context.requestID()+ "] Access " + + " from: " + (context.remoteAddress() == null? "null" : context.remoteAddress().toString()) + + (context.principal() == null? "" : (" principal:"+ context.principal())) + +" op:"+operation + + (topology_conf == null? "" : (" topoology:"+topology_conf.get(Config.TOPOLOGY_NAME)))); + return context.principal() != null ? users.contains(context.principal().getName()) : false; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/digest/ClientCallbackHandler.java b/jstorm-core/src/main/java/backtype/storm/security/auth/digest/ClientCallbackHandler.java new file mode 100755 index 000000000..3caacaa12 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/digest/ClientCallbackHandler.java @@ -0,0 +1,110 @@ +/** + * 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 backtype.storm.security.auth.digest; + +import java.io.IOException; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.security.auth.AuthUtils; + +/** + * client side callback handler. + */ +public class ClientCallbackHandler implements CallbackHandler { + private static final String USERNAME = "username"; + private static final String PASSWORD = "password"; + private static final Logger LOG = LoggerFactory.getLogger(ClientCallbackHandler.class); + private String _username = null; + private String _password = null; + + /** + * Constructor based on a JAAS configuration + * + * For digest, you should have a pair of user name and password defined. + * + * @param configuration + * @throws IOException + */ + public ClientCallbackHandler(Configuration configuration) throws IOException { + if (configuration == null) return; + AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry(AuthUtils.LOGIN_CONTEXT_CLIENT); + if (configurationEntries == null) { + String errorMessage = "Could not find a '"+AuthUtils.LOGIN_CONTEXT_CLIENT + + "' entry in this configuration: Client cannot start."; + throw new IOException(errorMessage); + } + + _password = ""; + for(AppConfigurationEntry entry: configurationEntries) { + if (entry.getOptions().get(USERNAME) != null) { + _username = (String)entry.getOptions().get(USERNAME); + } + if (entry.getOptions().get(PASSWORD) != null) { + _password = (String)entry.getOptions().get(PASSWORD); + } + } + } + + /** + * This method is invoked by SASL for authentication challenges + * @param callbacks a collection of challenge callbacks + */ + public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { + for (Callback c : callbacks) { + if (c instanceof NameCallback) { + LOG.debug("name callback"); + NameCallback nc = (NameCallback) c; + nc.setName(_username); + } else if (c instanceof PasswordCallback) { + LOG.debug("password callback"); + PasswordCallback pc = (PasswordCallback)c; + if (_password != null) { + pc.setPassword(_password.toCharArray()); + } + } else if (c instanceof AuthorizeCallback) { + LOG.debug("authorization callback"); + AuthorizeCallback ac = (AuthorizeCallback) c; + String authid = ac.getAuthenticationID(); + String authzid = ac.getAuthorizationID(); + if (authid.equals(authzid)) { + ac.setAuthorized(true); + } else { + ac.setAuthorized(false); + } + if (ac.isAuthorized()) { + ac.setAuthorizedID(authzid); + } + } else if (c instanceof RealmCallback) { + RealmCallback rc = (RealmCallback) c; + ((RealmCallback) c).setText(rc.getDefaultText()); + } else { + throw new UnsupportedCallbackException(c); + } + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java b/jstorm-core/src/main/java/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java new file mode 100755 index 000000000..ad642d8b3 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/digest/DigestSaslTransportPlugin.java @@ -0,0 +1,70 @@ +/** + * 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 backtype.storm.security.auth.digest; + +import java.io.IOException; +import java.util.Map; + +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.login.Configuration; + +import org.apache.thrift.transport.TSaslClientTransport; +import org.apache.thrift.transport.TSaslServerTransport; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; +import org.apache.thrift.transport.TTransportFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.security.auth.AuthUtils; +import backtype.storm.security.auth.SaslTransportPlugin; + +public class DigestSaslTransportPlugin extends SaslTransportPlugin { + public static final String DIGEST = "DIGEST-MD5"; + private static final Logger LOG = LoggerFactory.getLogger(DigestSaslTransportPlugin.class); + + protected TTransportFactory getServerTransportFactory() throws IOException { + //create an authentication callback handler + CallbackHandler serer_callback_handler = new ServerCallbackHandler(login_conf); + + //create a transport factory that will invoke our auth callback for digest + TSaslServerTransport.Factory factory = new TSaslServerTransport.Factory(); + factory.addServerDefinition(DIGEST, AuthUtils.SERVICE, "localhost", null, serer_callback_handler); + + LOG.info("SASL DIGEST-MD5 transport factory will be used"); + return factory; + } + + @Override + public TTransport connect(TTransport transport, String serverHost, String asUser) throws TTransportException, IOException { + ClientCallbackHandler client_callback_handler = new ClientCallbackHandler(login_conf); + TSaslClientTransport wrapper_transport = new TSaslClientTransport(DIGEST, + null, + AuthUtils.SERVICE, + serverHost, + null, + client_callback_handler, + transport); + + wrapper_transport.open(); + LOG.debug("SASL DIGEST-MD5 client transport has been established"); + + return wrapper_transport; + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/digest/ServerCallbackHandler.java b/jstorm-core/src/main/java/backtype/storm/security/auth/digest/ServerCallbackHandler.java new file mode 100755 index 000000000..1788dab08 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/digest/ServerCallbackHandler.java @@ -0,0 +1,131 @@ +/** + * 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 backtype.storm.security.auth.digest; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import backtype.storm.security.auth.ReqContext; +import backtype.storm.security.auth.SaslTransportPlugin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.Subject; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.Configuration; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; + +import backtype.storm.security.auth.AuthUtils; + +/** + * SASL server side collback handler + */ +public class ServerCallbackHandler implements CallbackHandler { + private static final String USER_PREFIX = "user_"; + private static final Logger LOG = LoggerFactory.getLogger(ServerCallbackHandler.class); + private static final String SYSPROP_SUPER_PASSWORD = "storm.SASLAuthenticationProvider.superPassword"; + + private String userName; + private final Map credentials = new HashMap(); + + public ServerCallbackHandler(Configuration configuration) throws IOException { + if (configuration==null) return; + + AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry(AuthUtils.LOGIN_CONTEXT_SERVER); + if (configurationEntries == null) { + String errorMessage = "Could not find a '"+AuthUtils.LOGIN_CONTEXT_SERVER+"' entry in this configuration: Server cannot start."; + throw new IOException(errorMessage); + } + credentials.clear(); + for(AppConfigurationEntry entry: configurationEntries) { + Map options = entry.getOptions(); + // Populate DIGEST-MD5 user -> password map with JAAS configuration entries from the "Server" section. + // Usernames are distinguished from other options by prefixing the username with a "user_" prefix. + for(Map.Entry pair : options.entrySet()) { + String key = pair.getKey(); + if (key.startsWith(USER_PREFIX)) { + String userName = key.substring(USER_PREFIX.length()); + credentials.put(userName,(String)pair.getValue()); + } + } + } + } + + public void handle(Callback[] callbacks) throws UnsupportedCallbackException { + for (Callback callback : callbacks) { + if (callback instanceof NameCallback) { + handleNameCallback((NameCallback) callback); + } else if (callback instanceof PasswordCallback) { + handlePasswordCallback((PasswordCallback) callback); + } else if (callback instanceof RealmCallback) { + handleRealmCallback((RealmCallback) callback); + } else if (callback instanceof AuthorizeCallback) { + handleAuthorizeCallback((AuthorizeCallback) callback); + } + } + } + + private void handleNameCallback(NameCallback nc) { + LOG.debug("handleNameCallback"); + userName = nc.getDefaultName(); + nc.setName(nc.getDefaultName()); + } + + private void handlePasswordCallback(PasswordCallback pc) { + LOG.debug("handlePasswordCallback"); + if ("super".equals(this.userName) && System.getProperty(SYSPROP_SUPER_PASSWORD) != null) { + // superuser: use Java system property for password, if available. + pc.setPassword(System.getProperty(SYSPROP_SUPER_PASSWORD).toCharArray()); + } else if (credentials.containsKey(userName) ) { + pc.setPassword(credentials.get(userName).toCharArray()); + } else { + LOG.warn("No password found for user: " + userName); + } + } + + private void handleRealmCallback(RealmCallback rc) { + LOG.debug("handleRealmCallback: "+ rc.getDefaultText()); + rc.setText(rc.getDefaultText()); + } + + private void handleAuthorizeCallback(AuthorizeCallback ac) { + String authenticationID = ac.getAuthenticationID(); + LOG.info("Successfully authenticated client: authenticationID = " + authenticationID + " authorizationID = " + ac.getAuthorizationID()); + + //if authorizationId is not set, set it to authenticationId. + if(ac.getAuthorizationID() == null) { + ac.setAuthorizedID(authenticationID); + } + + //When authNid and authZid are not equal , authNId is attempting to impersonate authZid, We + //add the authNid as the real user in reqContext's subject which will be used during authorization. + if(!authenticationID.equals(ac.getAuthorizationID())) { + LOG.info("Impersonation attempt authenticationID = " + ac.getAuthenticationID() + " authorizationID = " + ac.getAuthorizationID()); + ReqContext.context().setRealPrincipal(new SaslTransportPlugin.User(ac.getAuthenticationID())); + } + + ac.setAuthorized(true); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/AutoTGT.java b/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/AutoTGT.java new file mode 100755 index 000000000..aed1c4f58 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/AutoTGT.java @@ -0,0 +1,281 @@ +/** + * 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 backtype.storm.security.auth.kerberos; + +import backtype.storm.security.auth.IAutoCredentials; +import backtype.storm.security.auth.ICredentialsRenewer; +import backtype.storm.security.auth.AuthUtils; + +import java.util.Date; +import java.util.Map; +import java.util.Set; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.lang.reflect.Method; +import java.lang.reflect.Constructor; +import java.security.Principal; +import java.util.concurrent.atomic.AtomicReference; +import java.util.Iterator; + +import javax.security.auth.kerberos.KerberosTicket; +import javax.security.auth.kerberos.KerberosPrincipal; +import javax.security.auth.login.Configuration; +import javax.security.auth.login.LoginException; +import javax.security.auth.login.LoginContext; +import javax.security.auth.DestroyFailedException; +import javax.security.auth.RefreshFailedException; +import javax.security.auth.Subject; +import javax.xml.bind.DatatypeConverter; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Automatically take a user's TGT, and push it, and renew it in Nimbus. + */ +public class AutoTGT implements IAutoCredentials, ICredentialsRenewer { + private static final Logger LOG = LoggerFactory.getLogger(AutoTGT.class); + private static final float TICKET_RENEW_WINDOW = 0.80f; + protected static AtomicReference kerbTicket = new AtomicReference(); + private Map conf; + + public void prepare(Map conf) { + this.conf = conf; + } + + private static KerberosTicket getTGT(Subject subject) { + Set tickets = subject.getPrivateCredentials(KerberosTicket.class); + for(KerberosTicket ticket: tickets) { + KerberosPrincipal server = ticket.getServer(); + if (server.getName().equals("krbtgt/" + server.getRealm() + "@" + server.getRealm())) { + tickets = null; + return ticket; + } + } + tickets = null; + return null; + } + + @Override + public void populateCredentials(Map credentials) { + //Log the user in and get the TGT + try { + Configuration login_conf = AuthUtils.GetConfiguration(conf); + ClientCallbackHandler client_callback_handler = new ClientCallbackHandler(login_conf); + + //login our user + Configuration.setConfiguration(login_conf); + LoginContext lc = new LoginContext(AuthUtils.LOGIN_CONTEXT_CLIENT, client_callback_handler); + try { + lc.login(); + final Subject subject = lc.getSubject(); + KerberosTicket tgt = getTGT(subject); + + if (tgt == null) { //error + throw new RuntimeException("Fail to verify user principal with section \"" + +AuthUtils.LOGIN_CONTEXT_CLIENT+"\" in login configuration file "+ login_conf); + } + + if (!tgt.isForwardable()) { + throw new RuntimeException("The TGT found is not forwardable"); + } + + if (!tgt.isRenewable()) { + throw new RuntimeException("The TGT found is not renewable"); + } + + LOG.info("Pushing TGT for "+tgt.getClient()+" to topology."); + saveTGT(tgt, credentials); + } finally { + lc.logout(); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public static void saveTGT(KerberosTicket tgt, Map credentials) { + try { + ByteArrayOutputStream bao = new ByteArrayOutputStream(); + ObjectOutputStream out = new ObjectOutputStream(bao); + out.writeObject(tgt); + out.flush(); + out.close(); + credentials.put("TGT", DatatypeConverter.printBase64Binary(bao.toByteArray())); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public static KerberosTicket getTGT(Map credentials) { + KerberosTicket ret = null; + if (credentials != null && credentials.containsKey("TGT")) { + try { + ByteArrayInputStream bin = new ByteArrayInputStream(DatatypeConverter.parseBase64Binary(credentials.get("TGT"))); + ObjectInputStream in = new ObjectInputStream(bin); + ret = (KerberosTicket)in.readObject(); + in.close(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + return ret; + } + + @Override + public void updateSubject(Subject subject, Map credentials) { + populateSubjectWithTGT(subject, credentials); + } + + @Override + public void populateSubject(Subject subject, Map credentials) { + populateSubjectWithTGT(subject, credentials); + loginHadoopUser(subject); + } + + private void populateSubjectWithTGT(Subject subject, Map credentials) { + KerberosTicket tgt = getTGT(credentials); + if (tgt != null) { + Set creds = subject.getPrivateCredentials(); + synchronized(creds) { + Iterator iterator = creds.iterator(); + while (iterator.hasNext()) { + Object o = iterator.next(); + if (o instanceof KerberosTicket) { + KerberosTicket t = (KerberosTicket)o; + iterator.remove(); + try { + t.destroy(); + } catch (DestroyFailedException e) { + LOG.warn("Failed to destory ticket ", e); + } + } + } + creds.add(tgt); + } + subject.getPrincipals().add(tgt.getClient()); + kerbTicket.set(tgt); + } else { + LOG.info("No TGT found in credentials"); + } + } + + /** + * Hadoop does not just go off of a TGT, it needs a bit more. This + * should fill in the rest. + * @param subject the subject that should have a TGT in it. + */ + private void loginHadoopUser(Subject subject) { + Class ugi = null; + try { + ugi = Class.forName("org.apache.hadoop.security.UserGroupInformation"); + } catch (ClassNotFoundException e) { + LOG.info("Hadoop was not found on the class path"); + return; + } + try { + Method isSecEnabled = ugi.getMethod("isSecurityEnabled"); + if (!((Boolean)isSecEnabled.invoke(null))) { + LOG.warn("Hadoop is on the classpath but not configured for " + + "security, if you want security you need to be sure that " + + "hadoop.security.authentication=kerberos in core-site.xml " + + "in your jar"); + return; + } + + try { + Method login = ugi.getMethod("loginUserFromSubject", Subject.class); + login.invoke(null, subject); + } catch (NoSuchMethodException me) { + //The version of Hadoop does not have the needed client changes. + // So don't look now, but do something really ugly to work around it. + // This is because we are reaching into the hidden bits of Hadoop security, and it works for now, but may stop at any point in time. + + //We are just trying to do the following + // Configuration conf = new Configuration(); + // HadoopKerberosName.setConfiguration(conf); + // subject.getPrincipals().add(new User(tgt.getClient().toString(), AuthenticationMethod.KERBEROS, null)); + String name = getTGT(subject).getClient().toString(); + + LOG.warn("The Hadoop client does not have loginUserFromSubject, Trying to hack around it. This may not work..."); + Class confClass = Class.forName("org.apache.hadoop.conf.Configuration"); + Constructor confCons = confClass.getConstructor(); + Object conf = confCons.newInstance(); + Class hknClass = Class.forName("org.apache.hadoop.security.HadoopKerberosName"); + Method hknSetConf = hknClass.getMethod("setConfiguration",confClass); + hknSetConf.invoke(null, conf); + + Class authMethodClass = Class.forName("org.apache.hadoop.security.UserGroupInformation$AuthenticationMethod"); + Object kerbAuthMethod = null; + for (Object authMethod : authMethodClass.getEnumConstants()) { + if ("KERBEROS".equals(authMethod.toString())) { + kerbAuthMethod = authMethod; + break; + } + } + + Class userClass = Class.forName("org.apache.hadoop.security.User"); + Constructor userCons = userClass.getConstructor(String.class, authMethodClass, LoginContext.class); + userCons.setAccessible(true); + Object user = userCons.newInstance(name, kerbAuthMethod, null); + subject.getPrincipals().add((Principal)user); + } + } catch (Exception e) { + LOG.warn("Something went wrong while trying to initialize Hadoop through reflection. This version of hadoop may not be compatible.", e); + } + } + + private long getRefreshTime(KerberosTicket tgt) { + long start = tgt.getStartTime().getTime(); + long end = tgt.getEndTime().getTime(); + return start + (long) ((end - start) * TICKET_RENEW_WINDOW); + } + + @Override + public void renew(Map credentials, Map topologyConf) { + KerberosTicket tgt = getTGT(credentials); + if (tgt != null) { + long refreshTime = getRefreshTime(tgt); + long now = System.currentTimeMillis(); + if (now >= refreshTime) { + try { + LOG.info("Renewing TGT for "+tgt.getClient()); + tgt.refresh(); + saveTGT(tgt, credentials); + } catch (RefreshFailedException e) { + LOG.warn("Failed to refresh TGT", e); + } + } + } + } + + public static void main(String[] args) throws Exception { + AutoTGT at = new AutoTGT(); + Map conf = new java.util.HashMap(); + conf.put("java.security.auth.login.config", args[0]); + at.prepare(conf); + Map creds = new java.util.HashMap(); + at.populateCredentials(creds); + Subject s = new Subject(); + at.populateSubject(s, creds); + LOG.info("Got a Subject "+s); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java b/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java new file mode 100755 index 000000000..807abe304 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModule.java @@ -0,0 +1,108 @@ +/** + * 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 backtype.storm.security.auth.kerberos; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.security.Principal; +import java.util.Map; +import javax.security.auth.Subject; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.kerberos.KerberosTicket; +import javax.security.auth.login.LoginException; +import javax.security.auth.spi.LoginModule; + + +/** + * Custom LoginModule to enable Auto Login based on cached ticket + */ +public class AutoTGTKrb5LoginModule implements LoginModule { + private static final Logger LOG = LoggerFactory.getLogger(AutoTGTKrb5LoginModule.class); + + // initial state + private Subject subject; + + protected KerberosTicket kerbTicket = null; + + public void initialize(Subject subject, + CallbackHandler callbackHandler, + Map sharedState, + Map options) { + + this.subject = subject; + } + + public boolean login() throws LoginException { + LOG.debug("Acquire TGT from Cache"); + getKerbTicketFromCache(); + if (kerbTicket != null) { + return true; + } else { + throw new LoginException("Authentication failed, the TGT not found."); + } + } + + protected void getKerbTicketFromCache() { + kerbTicket = AutoTGT.kerbTicket.get(); + } + + protected Principal getKerbTicketClient() { + if (kerbTicket != null) { + return kerbTicket.getClient(); + } + return null; + } + + public boolean commit() throws LoginException { + if (isSucceeded() == false) { + return false; + } + if (subject == null || subject.isReadOnly()) { + kerbTicket = null; + throw new LoginException("Authentication failed because the Subject is invalid."); + } + // Let us add the kerbClientPrinc and kerbTicket + subject.getPrivateCredentials().add(kerbTicket); + subject.getPrincipals().add(getKerbTicketClient()); + LOG.debug("Commit Succeeded."); + return true; + } + + public boolean abort() throws LoginException { + if (isSucceeded() == false) { + return false; + } else { + return logout(); + } + } + + public boolean logout() throws LoginException { + if (subject != null && !subject.isReadOnly() && kerbTicket != null) { + subject.getPrincipals().remove(kerbTicket.getClient()); + subject.getPrivateCredentials().remove(kerbTicket); + } + kerbTicket = null; + return true; + } + + private boolean isSucceeded() { + return kerbTicket != null; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModuleTest.java b/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModuleTest.java new file mode 100755 index 000000000..ba34fc9bf --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/AutoTGTKrb5LoginModuleTest.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package backtype.storm.security.auth.kerberos; + +import java.security.Principal; +import javax.security.auth.kerberos.KerberosTicket; + +/** + * Custom LoginModule extended for testing. + */ +public class AutoTGTKrb5LoginModuleTest extends AutoTGTKrb5LoginModule { + + public Principal client = null; + + public void setKerbTicket(KerberosTicket ticket) { + this.kerbTicket = ticket; + } + + @Override + protected void getKerbTicketFromCache() { + // Do nothing. + } + + @Override + protected Principal getKerbTicketClient() { + return this.client; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/ClientCallbackHandler.java b/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/ClientCallbackHandler.java new file mode 100755 index 000000000..d46aa8b5d --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/ClientCallbackHandler.java @@ -0,0 +1,104 @@ +/** + * 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 backtype.storm.security.auth.kerberos; + +import java.io.IOException; +import java.util.Map; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.security.auth.AuthUtils; + +/** + * SASL client side callback handler. + */ +public class ClientCallbackHandler implements CallbackHandler { + private static final Logger LOG = LoggerFactory.getLogger(ClientCallbackHandler.class); + + /** + * Constructor based on a JAAS configuration + * + * For digest, you should have a pair of user name and password defined in this figgure. + * + * @param configuration + * @throws IOException + */ + public ClientCallbackHandler(Configuration configuration) throws IOException { + if (configuration == null) return; + AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry(AuthUtils.LOGIN_CONTEXT_CLIENT); + if (configurationEntries == null) { + String errorMessage = "Could not find a '"+AuthUtils.LOGIN_CONTEXT_CLIENT + + "' entry in this configuration: Client cannot start."; + LOG.error(errorMessage); + throw new IOException(errorMessage); + } + } + + /** + * This method is invoked by SASL for authentication challenges + * @param callbacks a collection of challenge callbacks + */ + public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { + for (Callback c : callbacks) { + if (c instanceof NameCallback) { + LOG.debug("name callback"); + } else if (c instanceof PasswordCallback) { + LOG.debug("password callback"); + LOG.warn("Could not login: the client is being asked for a password, but the " + + " client code does not currently support obtaining a password from the user." + + " Make sure that the client is configured to use a ticket cache (using" + + " the JAAS configuration setting 'useTicketCache=true)' and restart the client. If" + + " you still get this message after that, the TGT in the ticket cache has expired and must" + + " be manually refreshed. To do so, first determine if you are using a password or a" + + " keytab. If the former, run kinit in a Unix shell in the environment of the user who" + + " is running this client using the command" + + " 'kinit ' (where is the name of the client's Kerberos principal)." + + " If the latter, do" + + " 'kinit -k -t ' (where is the name of the Kerberos principal, and" + + " is the location of the keytab file). After manually refreshing your cache," + + " restart this client. If you continue to see this message after manually refreshing" + + " your cache, ensure that your KDC host's clock is in sync with this host's clock."); + } else if (c instanceof AuthorizeCallback) { + LOG.debug("authorization callback"); + AuthorizeCallback ac = (AuthorizeCallback) c; + String authid = ac.getAuthenticationID(); + String authzid = ac.getAuthorizationID(); + if (authid.equals(authzid)) { + ac.setAuthorized(true); + } else { + ac.setAuthorized(false); + } + if (ac.isAuthorized()) { + ac.setAuthorizedID(authzid); + } + } else { + throw new UnsupportedCallbackException(c); + } + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java b/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java new file mode 100755 index 000000000..ecb0daf6a --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java @@ -0,0 +1,209 @@ +/** + * 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 backtype.storm.security.auth.kerberos; + +import java.io.IOException; +import java.security.Principal; +import java.security.PrivilegedActionException; +import java.security.PrivilegedExceptionAction; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import javax.security.auth.Subject; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.kerberos.KerberosTicket; +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.Configuration; +import javax.security.auth.login.LoginException; +import javax.security.sasl.Sasl; + +import org.apache.commons.lang.StringUtils; +import org.apache.thrift.transport.TSaslClientTransport; +import org.apache.thrift.transport.TSaslServerTransport; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; +import org.apache.thrift.transport.TTransportFactory; +import org.apache.zookeeper.Login; +import org.apache.zookeeper.server.auth.KerberosName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.security.auth.AuthUtils; +import backtype.storm.security.auth.SaslTransportPlugin; + +public class KerberosSaslTransportPlugin extends SaslTransportPlugin { + public static final String KERBEROS = "GSSAPI"; + private static final Logger LOG = LoggerFactory.getLogger(KerberosSaslTransportPlugin.class); + + public TTransportFactory getServerTransportFactory() throws IOException { + //create an authentication callback handler + CallbackHandler server_callback_handler = new ServerCallbackHandler(login_conf, storm_conf); + + //login our principal + Subject subject = null; + try { + //specify a configuration object to be used + Configuration.setConfiguration(login_conf); + //now login + Login login = new Login(AuthUtils.LOGIN_CONTEXT_SERVER, server_callback_handler); + subject = login.getSubject(); + } catch (LoginException ex) { + LOG.error("Server failed to login in principal:" + ex, ex); + throw new RuntimeException(ex); + } + + //check the credential of our principal + if (subject.getPrivateCredentials(KerberosTicket.class).isEmpty()) { + throw new RuntimeException("Fail to verify user principal with section \"" + +AuthUtils.LOGIN_CONTEXT_SERVER+"\" in login configuration file "+ login_conf); + } + + String principal = AuthUtils.get(login_conf, AuthUtils.LOGIN_CONTEXT_SERVER, "principal"); + LOG.debug("principal:"+principal); + KerberosName serviceKerberosName = new KerberosName(principal); + String serviceName = serviceKerberosName.getServiceName(); + String hostName = serviceKerberosName.getHostName(); + Map props = new TreeMap(); + props.put(Sasl.QOP, "auth"); + props.put(Sasl.SERVER_AUTH, "false"); + + //create a transport factory that will invoke our auth callback for digest + TSaslServerTransport.Factory factory = new TSaslServerTransport.Factory(); + factory.addServerDefinition(KERBEROS, serviceName, hostName, props, server_callback_handler); + + //create a wrap transport factory so that we could apply user credential during connections + TUGIAssumingTransportFactory wrapFactory = new TUGIAssumingTransportFactory(factory, subject); + + LOG.info("SASL GSSAPI transport factory will be used"); + return wrapFactory; + } + + @Override + public TTransport connect(TTransport transport, String serverHost, String asUser) throws TTransportException, IOException { + //create an authentication callback handler + ClientCallbackHandler client_callback_handler = new ClientCallbackHandler(login_conf); + + //login our user + Login login = null; + try { + //specify a configuration object to be used + Configuration.setConfiguration(login_conf); + //now login + login = new Login(AuthUtils.LOGIN_CONTEXT_CLIENT, client_callback_handler); + } catch (LoginException ex) { + LOG.error("Server failed to login in principal:" + ex, ex); + throw new RuntimeException(ex); + } + + final Subject subject = login.getSubject(); + if (subject.getPrivateCredentials(KerberosTicket.class).isEmpty()) { //error + throw new RuntimeException("Fail to verify user principal with section \"" + +AuthUtils.LOGIN_CONTEXT_CLIENT+"\" in login configuration file "+ login_conf); + } + + final String principal = StringUtils.isBlank(asUser) ? getPrincipal(subject) : asUser; + String serviceName = AuthUtils.get(login_conf, AuthUtils.LOGIN_CONTEXT_CLIENT, "serviceName"); + if (serviceName == null) { + serviceName = AuthUtils.SERVICE; + } + Map props = new TreeMap(); + props.put(Sasl.QOP, "auth"); + props.put(Sasl.SERVER_AUTH, "false"); + + LOG.debug("SASL GSSAPI client transport is being established"); + final TTransport sasalTransport = new TSaslClientTransport(KERBEROS, + principal, + serviceName, + serverHost, + props, + null, + transport); + + //open Sasl transport with the login credential + try { + Subject.doAs(subject, + new PrivilegedExceptionAction() { + public Void run() { + try { + LOG.debug("do as:"+ principal); + sasalTransport.open(); + } + catch (Exception e) { + LOG.error("Client failed to open SaslClientTransport to interact with a server during session initiation: " + e, e); + } + return null; + } + }); + } catch (PrivilegedActionException e) { + throw new RuntimeException(e); + } + + return sasalTransport; + } + + private String getPrincipal(Subject subject) { + Set principals = (Set)subject.getPrincipals(); + if (principals==null || principals.size()<1) { + LOG.info("No principal found in login subject"); + return null; + } + return ((Principal)(principals.toArray()[0])).getName(); + } + + /** A TransportFactory that wraps another one, but assumes a specified UGI + * before calling through. + * + * This is used on the server side to assume the server's Principal when accepting + * clients. + */ + static class TUGIAssumingTransportFactory extends TTransportFactory { + private final Subject subject; + private final TTransportFactory wrapped; + + public TUGIAssumingTransportFactory(TTransportFactory wrapped, Subject subject) { + this.wrapped = wrapped; + this.subject = subject; + + Set principals = (Set)subject.getPrincipals(); + if (principals.size()>0) + LOG.info("Service principal:"+ ((Principal)(principals.toArray()[0])).getName()); + } + + @Override + public TTransport getTransport(final TTransport trans) { + try { + return Subject.doAs(subject, + new PrivilegedExceptionAction() { + public TTransport run() { + try { + return wrapped.getTransport(trans); + } + catch (Exception e) { + LOG.error("Storm server failed to open transport to interact with a client during session initiation: " + e, e); + return null; + } + } + }); + } catch (PrivilegedActionException e) { + LOG.error("Storm server experienced a PrivilegedActionException exception while creating a transport using a JAAS principal context:" + e, e); + return null; + } + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/ServerCallbackHandler.java b/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/ServerCallbackHandler.java new file mode 100755 index 000000000..7b143f0d6 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/ServerCallbackHandler.java @@ -0,0 +1,94 @@ +/** + * 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 backtype.storm.security.auth.kerberos; + +import backtype.storm.security.auth.AuthUtils; +import backtype.storm.security.auth.ReqContext; +import backtype.storm.security.auth.SaslTransportPlugin; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.Subject; +import javax.security.auth.callback.*; +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.Configuration; +import javax.security.sasl.AuthorizeCallback; +import java.io.IOException; +import java.util.Map; + +/** + * SASL server side callback handler + */ +public class ServerCallbackHandler implements CallbackHandler { + private static final Logger LOG = LoggerFactory.getLogger(ServerCallbackHandler.class); + + private String userName; + + public ServerCallbackHandler(Configuration configuration, Map stormConf) throws IOException { + if (configuration==null) return; + + AppConfigurationEntry configurationEntries[] = configuration.getAppConfigurationEntry(AuthUtils.LOGIN_CONTEXT_SERVER); + if (configurationEntries == null) { + String errorMessage = "Could not find a '"+AuthUtils.LOGIN_CONTEXT_SERVER+"' entry in this configuration: Server cannot start."; + LOG.error(errorMessage); + throw new IOException(errorMessage); + } + + } + + public void handle(Callback[] callbacks) throws UnsupportedCallbackException { + for (Callback callback : callbacks) { + if (callback instanceof NameCallback) { + handleNameCallback((NameCallback) callback); + } else if (callback instanceof PasswordCallback) { + handlePasswordCallback((PasswordCallback) callback); + } else if (callback instanceof AuthorizeCallback) { + handleAuthorizeCallback((AuthorizeCallback) callback); + } + } + } + + private void handleNameCallback(NameCallback nc) { + LOG.debug("handleNameCallback"); + userName = nc.getDefaultName(); + nc.setName(nc.getDefaultName()); + } + + private void handlePasswordCallback(PasswordCallback pc) { + LOG.warn("No password found for user: " + userName); + } + + private void handleAuthorizeCallback(AuthorizeCallback ac) { + String authenticationID = ac.getAuthenticationID(); + LOG.info("Successfully authenticated client: authenticationID=" + authenticationID + " authorizationID= " + ac.getAuthorizationID()); + + //if authorizationId is not set, set it to authenticationId. + if(ac.getAuthorizationID() == null) { + ac.setAuthorizedID(authenticationID); + } + + //When authNid and authZid are not equal , authNId is attempting to impersonate authZid, We + //add the authNid as the real user in reqContext's subject which will be used during authorization. + if(!ac.getAuthenticationID().equals(ac.getAuthorizationID())) { + ReqContext.context().setRealPrincipal(new SaslTransportPlugin.User(ac.getAuthenticationID())); + } + + ac.setAuthorized(true); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/jaas_kerberos_cluster.conf b/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/jaas_kerberos_cluster.conf new file mode 100755 index 000000000..92a1399ac --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/jaas_kerberos_cluster.conf @@ -0,0 +1,31 @@ +/* +This is a sample JAAS configuration for Storm servers to handle Kerberos authentication +*/ + +/* + StormServer section should contains the info about server keytab file and server principal. + In Storm, we have 2 thrift servers: Nimbus and DRPC. These servers could be assigned with + different principals. +*/ +StormServer { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + keyTab="/etc/storm_server.keytab" + storeKey=true + useTicketCache=false + principal="storm_service/carcloth.corp.acme.com@STORM.CORP.ACME.COM"; +}; + +/* +StormClient section should contains the info about client keytab file and client principal. +For example, Supervisors are clients of Nimbus, and we should assign keytab/principal for supervisors. +*/ +StormClient { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + keyTab="/etc/storm_client.keytab" + storeKey=true + useTicketCache=false + serviceName="storm_service"; +}; + diff --git a/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/jaas_kerberos_launcher.conf b/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/jaas_kerberos_launcher.conf new file mode 100755 index 000000000..138e1f3f2 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/auth/kerberos/jaas_kerberos_launcher.conf @@ -0,0 +1,12 @@ +/* + This is a sample JAAS configuration for Storm topology launcher/submitter. + Since launcher machines are typically accessible by many folks, we + encourage you to leverage "kinit", instead of keytab. +*/ +StormClient { + com.sun.security.auth.module.Krb5LoginModule required + doNotPrompt=true + useTicketCache=true + serviceName="storm_service"; +}; + diff --git a/jstorm-core/src/main/java/backtype/storm/security/serialization/BlowfishTupleSerializer.java b/jstorm-core/src/main/java/backtype/storm/security/serialization/BlowfishTupleSerializer.java new file mode 100644 index 000000000..8e66cdf0e --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/security/serialization/BlowfishTupleSerializer.java @@ -0,0 +1,90 @@ +/** + * 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 backtype.storm.security.serialization; + +import java.util.Map; +import org.apache.commons.codec.binary.Hex; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.crypto.KeyGenerator; +import javax.crypto.SecretKey; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.serializers.BlowfishSerializer; + +import backtype.storm.serialization.types.ListDelegateSerializer; +import backtype.storm.utils.ListDelegate; +import backtype.storm.Config; + +/** + * Apply Blowfish encrption for tuple communication to bolts + */ +public class BlowfishTupleSerializer extends Serializer { + /** + * The secret key (if any) for data encryption by blowfish payload serialization factory (BlowfishSerializationFactory). + * You should use in via "storm -c topology.tuple.serializer.blowfish.key=YOURKEY -c topology.tuple.serializer=backtype.storm.security.serialization.BlowfishTupleSerializer jar ...". + */ + public static String SECRET_KEY = "topology.tuple.serializer.blowfish.key"; + private static final Logger LOG = LoggerFactory.getLogger(BlowfishTupleSerializer.class); + private BlowfishSerializer _serializer; + + public BlowfishTupleSerializer(Kryo kryo, Map storm_conf) { + String encryption_key = null; + try { + encryption_key = (String)storm_conf.get(SECRET_KEY); + LOG.debug("Blowfish serializer being constructed ..."); + if (encryption_key == null) { + throw new RuntimeException("Blowfish encryption key not specified"); + } + byte[] bytes = Hex.decodeHex(encryption_key.toCharArray()); + _serializer = new BlowfishSerializer(new ListDelegateSerializer(), bytes); + } catch (org.apache.commons.codec.DecoderException ex) { + throw new RuntimeException("Blowfish encryption key invalid", ex); + } + } + + @Override + public void write(Kryo kryo, Output output, ListDelegate object) { + _serializer.write(kryo, output, object); + } + + @Override + public ListDelegate read(Kryo kryo, Input input, Class type) { + return (ListDelegate)_serializer.read(kryo, input, type); + } + + /** + * Produce a blowfish key to be used in "Storm jar" command + */ + public static void main(String[] args) { + try{ + KeyGenerator kgen = KeyGenerator.getInstance("Blowfish"); + SecretKey skey = kgen.generateKey(); + byte[] raw = skey.getEncoded(); + String keyString = new String(Hex.encodeHex(raw)); + System.out.println("storm -c "+SECRET_KEY+"="+keyString+" -c "+Config.TOPOLOGY_TUPLE_SERIALIZER+"="+BlowfishTupleSerializer.class.getName() + " ..." ); + } catch (Exception ex) { + LOG.error(ex.getMessage()); + ex.printStackTrace(); + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/DefaultKryoFactory.java b/jstorm-core/src/main/java/backtype/storm/serialization/DefaultKryoFactory.java new file mode 100755 index 000000000..a055eb266 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/DefaultKryoFactory.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.serialization; + +import backtype.storm.Config; +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import java.util.Map; + + +public class DefaultKryoFactory implements IKryoFactory { + + public static class KryoSerializableDefault extends Kryo { + boolean _override = false; + + public void overrideDefault(boolean value) { + _override = value; + } + + @Override + public Serializer getDefaultSerializer(Class type) { + if(_override) { + return new SerializableSerializer(); + } else { + return super.getDefaultSerializer(type); + } + } + } + + @Override + public Kryo getKryo(Map conf) { + KryoSerializableDefault k = new KryoSerializableDefault(); + k.setRegistrationRequired(!((Boolean) conf.get(Config.TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION))); + k.setReferences(false); + return k; + } + + @Override + public void preRegister(Kryo k, Map conf) { + } + + public void postRegister(Kryo k, Map conf) { + ((KryoSerializableDefault)k).overrideDefault(true); + } + + @Override + public void postDecorate(Kryo k, Map conf) { + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/DefaultSerializationDelegate.java b/jstorm-core/src/main/java/backtype/storm/serialization/DefaultSerializationDelegate.java new file mode 100755 index 000000000..6d986af98 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/DefaultSerializationDelegate.java @@ -0,0 +1,58 @@ +/** + * 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 backtype.storm.serialization; + +import java.io.*; +import java.util.Map; + +@Deprecated +public class DefaultSerializationDelegate implements SerializationDelegate { + + @Override + public void prepare(Map stormConf) { + // No-op + } + + @Override + public byte[] serialize(Object object) { + try { + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + ObjectOutputStream oos = new ObjectOutputStream(bos); + oos.writeObject(object); + oos.close(); + return bos.toByteArray(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public T deserialize(byte[] bytes, Class clazz) { + try { + ByteArrayInputStream bis = new ByteArrayInputStream(bytes); + ObjectInputStream ois = new ObjectInputStream(bis); + Object ret = ois.readObject(); + ois.close(); + return (T)ret; + } catch(IOException ioe) { + throw new RuntimeException(ioe); + } catch(ClassNotFoundException e) { + throw new RuntimeException(e); + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/GzipBridgeSerializationDelegate.java b/jstorm-core/src/main/java/backtype/storm/serialization/GzipBridgeSerializationDelegate.java new file mode 100755 index 000000000..c8377c33e --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/GzipBridgeSerializationDelegate.java @@ -0,0 +1,65 @@ +/** + * 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 backtype.storm.serialization; + +import java.util.Map; +import java.util.zip.GZIPInputStream; + +/** + * Always writes gzip out, but tests incoming to see if it's gzipped. If it is, deserializes with gzip. If not, uses + * {@link backtype.storm.serialization.DefaultSerializationDelegate} to deserialize. Any logic needing to be enabled + * via {@link #prepare(java.util.Map)} is passed through to both delegates. + */ +@Deprecated +public class GzipBridgeSerializationDelegate implements SerializationDelegate { + + private DefaultSerializationDelegate defaultDelegate = new DefaultSerializationDelegate(); + private GzipSerializationDelegate gzipDelegate = new GzipSerializationDelegate(); + + @Override + public void prepare(Map stormConf) { + defaultDelegate.prepare(stormConf); + gzipDelegate.prepare(stormConf); + } + + @Override + public byte[] serialize(Object object) { + return gzipDelegate.serialize(object); + } + + @Override + public T deserialize(byte[] bytes, Class clazz) { + if (isGzipped(bytes)) { + return gzipDelegate.deserialize(bytes, clazz); + } else { + return defaultDelegate.deserialize(bytes,clazz); + } + } + + // Split up GZIP_MAGIC into readable bytes + private static final byte GZIP_MAGIC_FIRST_BYTE = (byte) GZIPInputStream.GZIP_MAGIC; + private static final byte GZIP_MAGIC_SECOND_BYTE = (byte) (GZIPInputStream.GZIP_MAGIC >> 8); + + /** + * Looks ahead to see if the GZIP magic constant is heading {@code bytes} + */ + private boolean isGzipped(byte[] bytes) { + return (bytes.length > 1) && (bytes[0] == GZIP_MAGIC_FIRST_BYTE) + && (bytes[1] == GZIP_MAGIC_SECOND_BYTE); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/GzipBridgeThriftSerializationDelegate.java b/jstorm-core/src/main/java/backtype/storm/serialization/GzipBridgeThriftSerializationDelegate.java new file mode 100755 index 000000000..e5e77c357 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/GzipBridgeThriftSerializationDelegate.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.serialization; + +import java.util.Map; +import java.util.zip.GZIPInputStream; + +/** + * Always writes gzip out, but tests incoming to see if it's gzipped. If it is, deserializes with gzip. If not, uses + * {@link backtype.storm.serialization.ThriftSerializationDelegate} to deserialize. Any logic needing to be enabled + * via {@link #prepare(java.util.Map)} is passed through to both delegates. + */ +public class GzipBridgeThriftSerializationDelegate implements SerializationDelegate { + + private ThriftSerializationDelegate defaultDelegate = new ThriftSerializationDelegate(); + private GzipThriftSerializationDelegate gzipDelegate = new GzipThriftSerializationDelegate(); + + @Override + public void prepare(Map stormConf) { + defaultDelegate.prepare(stormConf); + gzipDelegate.prepare(stormConf); + } + + @Override + public byte[] serialize(Object object) { + return gzipDelegate.serialize(object); + } + + @Override + public T deserialize(byte[] bytes, Class clazz) { + if (isGzipped(bytes)) { + return gzipDelegate.deserialize(bytes, clazz); + } else { + return defaultDelegate.deserialize(bytes,clazz); + } + } + + // Split up GZIP_MAGIC into readable bytes + private static final byte GZIP_MAGIC_FIRST_BYTE = (byte) GZIPInputStream.GZIP_MAGIC; + private static final byte GZIP_MAGIC_SECOND_BYTE = (byte) (GZIPInputStream.GZIP_MAGIC >> 8); + + /** + * Looks ahead to see if the GZIP magic constant is heading {@code bytes} + */ + private boolean isGzipped(byte[] bytes) { + return (bytes.length > 1) && (bytes[0] == GZIP_MAGIC_FIRST_BYTE) + && (bytes[1] == GZIP_MAGIC_SECOND_BYTE); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/GzipSerializationDelegate.java b/jstorm-core/src/main/java/backtype/storm/serialization/GzipSerializationDelegate.java new file mode 100755 index 000000000..3c8ee8b7c --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/GzipSerializationDelegate.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.serialization; + +import java.io.*; +import java.util.Map; +import java.util.zip.GZIPInputStream; +import java.util.zip.GZIPOutputStream; + +/** + * Note, this assumes it's deserializing a gzip byte stream, and will err if it encounters any other serialization. + */ +public class GzipSerializationDelegate implements SerializationDelegate { + + @Override + public void prepare(Map stormConf) { + // No-op + } + + @Override + public byte[] serialize(Object object) { + try { + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + GZIPOutputStream gos = new GZIPOutputStream(bos); + ObjectOutputStream oos = new ObjectOutputStream(gos); + oos.writeObject(object); + oos.close(); + return bos.toByteArray(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public T deserialize(byte[] bytes, Class clazz) { + try { + ByteArrayInputStream bis = new ByteArrayInputStream(bytes); + GZIPInputStream gis = new GZIPInputStream(bis); + ObjectInputStream ois = new ObjectInputStream(gis); + Object ret = ois.readObject(); + ois.close(); + return (T)ret; + } catch(IOException ioe) { + throw new RuntimeException(ioe); + } catch(ClassNotFoundException e) { + throw new RuntimeException(e); + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/GzipThriftSerializationDelegate.java b/jstorm-core/src/main/java/backtype/storm/serialization/GzipThriftSerializationDelegate.java new file mode 100755 index 000000000..933a1253f --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/GzipThriftSerializationDelegate.java @@ -0,0 +1,57 @@ +/** + * 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 backtype.storm.serialization; + +import java.io.IOException; +import java.util.Map; +import backtype.storm.utils.Utils; +import org.apache.thrift.TBase; +import org.apache.thrift.TDeserializer; +import org.apache.thrift.TException; +import org.apache.thrift.TSerializer; + +/** + * Note, this assumes it's deserializing a gzip byte stream, and will err if it encounters any other serialization. + */ +public class GzipThriftSerializationDelegate implements SerializationDelegate { + + @Override + public void prepare(Map stormConf) { + // No-op + } + + @Override + public byte[] serialize(Object object) { + try { + return Utils.gzip(new TSerializer().serialize((TBase) object)); + } catch (TException e) { + throw new RuntimeException(e); + } + } + + @Override + public T deserialize(byte[] bytes, Class clazz) { + try { + TBase instance = (TBase) clazz.newInstance(); + new TDeserializer().deserialize(instance, Utils.gunzip(bytes)); + return (T)instance; + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/IKryoDecorator.java b/jstorm-core/src/main/java/backtype/storm/serialization/IKryoDecorator.java new file mode 100755 index 000000000..b154a36ee --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/IKryoDecorator.java @@ -0,0 +1,23 @@ +/** + * 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 backtype.storm.serialization; +import com.esotericsoftware.kryo.Kryo; + +public interface IKryoDecorator { + void decorate(Kryo k); +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/IKryoFactory.java b/jstorm-core/src/main/java/backtype/storm/serialization/IKryoFactory.java new file mode 100755 index 000000000..60a847d89 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/IKryoFactory.java @@ -0,0 +1,40 @@ +/** + * 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 backtype.storm.serialization; + +import com.esotericsoftware.kryo.Kryo; +import java.util.Map; + +/** + * An interface that controls the Kryo instance used by Storm for serialization. + * The lifecycle is: + * + * 1. The Kryo instance is constructed using getKryo + * 2. Storm registers the default classes (e.g. arrays, lists, maps, etc.) + * 3. Storm calls preRegister hook + * 4. Storm registers all user-defined registrations through topology.kryo.register + * 5. Storm calls postRegister hook + * 6. Storm calls all user-defined decorators through topology.kryo.decorators + * 7. Storm calls postDecorate hook + */ +public interface IKryoFactory { + Kryo getKryo(Map conf); + void preRegister(Kryo k, Map conf); + void postRegister(Kryo k, Map conf); + void postDecorate(Kryo k, Map conf); +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/ITupleDeserializer.java b/jstorm-core/src/main/java/backtype/storm/serialization/ITupleDeserializer.java new file mode 100755 index 000000000..4e6865855 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/ITupleDeserializer.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.serialization; + +import backtype.storm.tuple.Tuple; +import java.io.IOException; + +public interface ITupleDeserializer { + Tuple deserialize(byte[] ser); +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/ITupleSerializer.java b/jstorm-core/src/main/java/backtype/storm/serialization/ITupleSerializer.java new file mode 100755 index 000000000..90ad93241 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/ITupleSerializer.java @@ -0,0 +1,26 @@ +/** + * 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 backtype.storm.serialization; + +import backtype.storm.tuple.Tuple; + + +public interface ITupleSerializer { + byte[] serialize(Tuple tuple); +// long crc32(Tuple tuple); +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/KryoTupleDeserializer.java b/jstorm-core/src/main/java/backtype/storm/serialization/KryoTupleDeserializer.java new file mode 100644 index 000000000..3496e683c --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/KryoTupleDeserializer.java @@ -0,0 +1,129 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.serialization; + +import java.util.List; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.task.GeneralTopologyContext; +import backtype.storm.tuple.BatchTuple; +import backtype.storm.tuple.MessageId; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.TupleImplExt; +import backtype.storm.utils.Utils; + +import com.esotericsoftware.kryo.io.Input; + +import java.io.IOException; +import java.net.URLClassLoader; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; + +public class KryoTupleDeserializer implements ITupleDeserializer { + private static final Logger LOG = LoggerFactory.getLogger(KryoTupleDeserializer.class); + + public static final boolean USE_RAW_PACKET = true; + + GeneralTopologyContext _context; + KryoValuesDeserializer _kryo; + SerializationFactory.IdDictionary _ids; + Input _kryoInput; + + public KryoTupleDeserializer(final Map conf, final GeneralTopologyContext context) { + _kryo = new KryoValuesDeserializer(conf); + _context = context; + _ids = new SerializationFactory.IdDictionary(context.getRawTopology()); + _kryoInput = new Input(1); + } + + public Tuple deserialize(byte[] ser) { + + int targetTaskId = 0; + int taskId = 0; + int streamId = 0; + String componentName = null; + String streamName = null; + MessageId id = null; + + try { + + _kryoInput.setBuffer(ser); + + targetTaskId = _kryoInput.readInt(); + taskId = _kryoInput.readInt(true); + streamId = _kryoInput.readInt(true); + componentName = _context.getComponentId(taskId); + streamName = _ids.getStreamName(componentName, streamId); + id = MessageId.deserialize(_kryoInput); + List values = _kryo.deserializeFrom(_kryoInput); + TupleImplExt tuple = new TupleImplExt(_context, values, taskId, streamName, id); + tuple.setTargetTaskId(targetTaskId); + return tuple; + } catch (Throwable e) { + StringBuilder sb = new StringBuilder(); + + sb.append("Deserialize error:"); + sb.append("targetTaskId:").append(targetTaskId); + sb.append(",taskId:").append(taskId); + sb.append(",streamId:").append(streamId); + sb.append(",componentName:").append(componentName); + sb.append(",streamName:").append(streamName); + sb.append(",MessageId").append(id); + + LOG.info(sb.toString(), e); + throw new RuntimeException(e); + } + } + + public BatchTuple deserializeBatch(byte[] ser) { + BatchTuple ret = new BatchTuple(); + + int offset = 0; + while(offset < ser.length) { + int tupleSize = Utils.readIntFromByteArray(ser, offset); + offset += 4; + + ByteBuffer buff = ByteBuffer.allocate(tupleSize); + buff.put(ser, offset, tupleSize); + ret.addToBatch(deserialize(buff.array())); + offset += tupleSize; + } + + return ret; + } + + /** + * just get target taskId + * + * @param ser + * @return + */ + public static int deserializeTaskId(byte[] ser) { + Input _kryoInput = new Input(1); + + _kryoInput.setBuffer(ser); + + int targetTaskId = _kryoInput.readInt(); + + return targetTaskId; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/KryoTupleSerializer.java b/jstorm-core/src/main/java/backtype/storm/serialization/KryoTupleSerializer.java new file mode 100644 index 000000000..1c53d5d7e --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/KryoTupleSerializer.java @@ -0,0 +1,112 @@ +/** + * 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 backtype.storm.serialization; + +import backtype.storm.task.GeneralTopologyContext; +import backtype.storm.tuple.BatchTuple; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.TupleExt; +import backtype.storm.utils.Utils; + +import com.esotericsoftware.kryo.io.Output; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Map; + +public class KryoTupleSerializer implements ITupleSerializer { + KryoValuesSerializer _kryo; + SerializationFactory.IdDictionary _ids; + Output _kryoOut; + + public KryoTupleSerializer(final Map conf, final GeneralTopologyContext context) { + _kryo = new KryoValuesSerializer(conf); + _kryoOut = new Output(2000, 2000000000); + _ids = new SerializationFactory.IdDictionary(context.getRawTopology()); + } + + /** + * @@@ in the furture, it will skill serialize 'targetTask' through check some flag + * @see backtype.storm.serialization.ITupleSerializer#serialize(int, backtype.storm.tuple.Tuple) + */ + public byte[] serialize(Tuple tuple) { + try { + + _kryoOut.clear(); + if (tuple instanceof TupleExt) { + _kryoOut.writeInt(((TupleExt) tuple).getTargetTaskId()); + } + + _kryoOut.writeInt(tuple.getSourceTask(), true); + _kryoOut.writeInt(_ids.getStreamId(tuple.getSourceComponent(), tuple.getSourceStreamId()), true); + tuple.getMessageId().serialize(_kryoOut); + _kryo.serializeInto(tuple.getValues(), _kryoOut); + return _kryoOut.toBytes(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public byte[] serializeBatch(BatchTuple batch) { + if (batch == null || batch.currBatchSize() == 0) + return null; + + byte[][] bytes = new byte[batch.currBatchSize()][]; + int i = 0, len = 0; + for (Tuple tuple : batch.getTuples()) { + /* byte structure: + * 1st tuple: length + tuple bytes + * 2nd tuple: length + tuple bytes + * ...... + */ + bytes[i] = serialize(tuple); + len += bytes[i].length; + // add length bytes (int) + len += 4; + i++; + } + + byte[] ret = new byte[len]; + int index = 0; + for (i = 0; i < bytes.length; i++) { + Utils.writeIntToByteArray(ret, index, bytes[i].length); + index += 4; + for (int j = 0; j < bytes[i].length; j++) { + ret[index++] = bytes[i][j]; + } + } + return ret; + } + + public static byte[] serialize(int targetTask) { + ByteBuffer buff = ByteBuffer.allocate((Integer.SIZE / 8)); + buff.putInt(targetTask); + byte[] rtn = buff.array(); + return rtn; + } + + // public long crc32(Tuple tuple) { + // try { + // CRC32OutputStream hasher = new CRC32OutputStream(); + // _kryo.serializeInto(tuple.getValues(), hasher); + // return hasher.getValue(); + // } catch (IOException e) { + // throw new RuntimeException(e); + // } + // } +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/KryoValuesDeserializer.java b/jstorm-core/src/main/java/backtype/storm/serialization/KryoValuesDeserializer.java new file mode 100755 index 000000000..209ae5392 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/KryoValuesDeserializer.java @@ -0,0 +1,51 @@ +/** + * 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 backtype.storm.serialization; + +import backtype.storm.utils.ListDelegate; +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class KryoValuesDeserializer { + Kryo _kryo; + Input _kryoInput; + + public KryoValuesDeserializer(Map conf) { + _kryo = SerializationFactory.getKryo(conf); + _kryoInput = new Input(1); + } + + public List deserializeFrom(Input input) { + ListDelegate delegate = (ListDelegate) _kryo.readObject(input, ListDelegate.class); + return delegate.getDelegate(); + } + + public List deserialize(byte[] ser) throws IOException { + _kryoInput.setBuffer(ser); + return deserializeFrom(_kryoInput); + } + + public Object deserializeObject(byte[] ser) throws IOException { + _kryoInput.setBuffer(ser); + return _kryo.readClassAndObject(_kryoInput); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/KryoValuesSerializer.java b/jstorm-core/src/main/java/backtype/storm/serialization/KryoValuesSerializer.java new file mode 100755 index 000000000..c4a2f71f5 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/KryoValuesSerializer.java @@ -0,0 +1,58 @@ +/** + * 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 backtype.storm.serialization; + +import backtype.storm.utils.ListDelegate; +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Output; +import java.io.IOException; +import java.util.List; +import java.util.Map; + +public class KryoValuesSerializer { + Kryo _kryo; + ListDelegate _delegate; + Output _kryoOut; + + public KryoValuesSerializer(Map conf) { + _kryo = SerializationFactory.getKryo(conf); + _delegate = new ListDelegate(); + _kryoOut = new Output(2000, 2000000000); + } + + public void serializeInto(List values, Output out) throws IOException { + // this ensures that list of values is always written the same way, regardless + // of whether it's a java collection or one of clojure's persistent collections + // (which have different serializers) + // Doing this lets us deserialize as ArrayList and avoid writing the class here + _delegate.setDelegate(values); + _kryo.writeObject(out, _delegate); + } + + public byte[] serialize(List values) throws IOException { + _kryoOut.clear(); + serializeInto(values, _kryoOut); + return _kryoOut.toBytes(); + } + + public byte[] serializeObject(Object obj) { + _kryoOut.clear(); + _kryo.writeClassAndObject(_kryoOut, obj); + return _kryoOut.toBytes(); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/SerializableSerializer.java b/jstorm-core/src/main/java/backtype/storm/serialization/SerializableSerializer.java new file mode 100755 index 000000000..376ad2a09 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/SerializableSerializer.java @@ -0,0 +1,62 @@ +/** + * 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 backtype.storm.serialization; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; + +import org.apache.commons.io.input.ClassLoaderObjectInputStream; + +public class SerializableSerializer extends Serializer { + + @Override + public void write(Kryo kryo, Output output, Object object) { + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + try { + ObjectOutputStream oos = new ObjectOutputStream(bos); + oos.writeObject(object); + oos.flush(); + } catch (IOException e) { + throw new RuntimeException(e); + } + byte[] ser = bos.toByteArray(); + output.writeInt(ser.length); + output.writeBytes(ser); + } + + @Override + public Object read(Kryo kryo, Input input, Class c) { + int len = input.readInt(); + byte[] ser = new byte[len]; + input.readBytes(ser); + ByteArrayInputStream bis = new ByteArrayInputStream(ser); + try { + ClassLoaderObjectInputStream ois = new ClassLoaderObjectInputStream(kryo.getClassLoader(), bis); + return ois.readObject(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/SerializationDelegate.java b/jstorm-core/src/main/java/backtype/storm/serialization/SerializationDelegate.java new file mode 100755 index 000000000..e0de7a376 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/SerializationDelegate.java @@ -0,0 +1,35 @@ +/** + * 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 backtype.storm.serialization; + +import java.util.Map; + +/** + * Allow {@link backtype.storm.utils.Utils} to delegate meta serialization. + */ +public interface SerializationDelegate { + + /** + * Lifecycle step that will be called after instantiating with nullary constructor. + */ + void prepare(Map stormConf); + + byte[] serialize(Object object); + + T deserialize(byte[] bytes, Class clazz); +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/SerializationFactory.java b/jstorm-core/src/main/java/backtype/storm/serialization/SerializationFactory.java new file mode 100755 index 000000000..ef859be99 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/SerializationFactory.java @@ -0,0 +1,226 @@ +/** + * 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 backtype.storm.serialization; + +import backtype.storm.Config; +import backtype.storm.generated.ComponentCommon; +import backtype.storm.generated.StormTopology; +import backtype.storm.serialization.types.ArrayListSerializer; +import backtype.storm.serialization.types.ListDelegateSerializer; +import backtype.storm.serialization.types.HashMapSerializer; +import backtype.storm.serialization.types.HashSetSerializer; +import backtype.storm.transactional.TransactionAttempt; +import backtype.storm.tuple.Values; +import backtype.storm.utils.ListDelegate; +import backtype.storm.utils.Utils; +import backtype.storm.utils.WorkerClassLoader; +import carbonite.JavaBridge; +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.serializers.DefaultSerializers.BigIntegerSerializer; +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SerializationFactory { + public static final Logger LOG = LoggerFactory.getLogger(SerializationFactory.class); + + public static Kryo getKryo(Map conf) { + IKryoFactory kryoFactory = (IKryoFactory) Utils.newInstance((String) conf.get(Config.TOPOLOGY_KRYO_FACTORY)); + Kryo k = kryoFactory.getKryo(conf); + if (WorkerClassLoader.getInstance() != null) + k.setClassLoader(WorkerClassLoader.getInstance()); + k.register(byte[].class); + + /* tuple payload serializer is specified via configuration */ + String payloadSerializerName = (String) conf.get(Config.TOPOLOGY_TUPLE_SERIALIZER); + try { + Class serializerClass = Class.forName(payloadSerializerName, true, k.getClassLoader()); + Serializer serializer = resolveSerializerInstance(k, ListDelegate.class, serializerClass, conf); + k.register(ListDelegate.class, serializer); + } catch (ClassNotFoundException ex) { + throw new RuntimeException(ex); + } + + k.register(ArrayList.class, new ArrayListSerializer()); + k.register(HashMap.class, new HashMapSerializer()); + k.register(HashSet.class, new HashSetSerializer()); + k.register(BigInteger.class, new BigIntegerSerializer()); + k.register(TransactionAttempt.class); + k.register(Values.class); + k.register(backtype.storm.metric.api.IMetricsConsumer.DataPoint.class); + k.register(backtype.storm.metric.api.IMetricsConsumer.TaskInfo.class); + try { + JavaBridge.registerPrimitives(k); + JavaBridge.registerCollections(k); + } catch (Exception e) { + throw new RuntimeException(e); + } + + Map registrations = normalizeKryoRegister(conf); + + kryoFactory.preRegister(k, conf); + + boolean skipMissing = (Boolean) conf.get(Config.TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS); + for (String klassName : registrations.keySet()) { + String serializerClassName = registrations.get(klassName); + try { + Class klass = Class.forName(klassName, true, k.getClassLoader()); + + Class serializerClass = null; + if (serializerClassName != null) + serializerClass = Class.forName(serializerClassName, true, k.getClassLoader()); + if (serializerClass == null) { + k.register(klass); + } else { + k.register(klass, resolveSerializerInstance(k, klass, serializerClass, conf)); + } + } catch (ClassNotFoundException e) { + if (skipMissing) { + LOG.info("Could not find serialization or class for " + serializerClassName + ". Skipping registration..."); + } else { + throw new RuntimeException(e); + } + } + } + + kryoFactory.postRegister(k, conf); + + if (conf.get(Config.TOPOLOGY_KRYO_DECORATORS) != null) { + for (String klassName : (List) conf.get(Config.TOPOLOGY_KRYO_DECORATORS)) { + try { + Class klass = Class.forName(klassName, true, k.getClassLoader()); + IKryoDecorator decorator = (IKryoDecorator) klass.newInstance(); + decorator.decorate(k); + } catch (ClassNotFoundException e) { + if (skipMissing) { + LOG.info("Could not find kryo decorator named " + klassName + ". Skipping registration..."); + } else { + throw new RuntimeException(e); + } + } catch (InstantiationException e) { + throw new RuntimeException(e); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + } + } + + kryoFactory.postDecorate(k, conf); + + return k; + } + + public static class IdDictionary { + Map> streamNametoId = new HashMap>(); + Map> streamIdToName = new HashMap>(); + + public IdDictionary(StormTopology topology) { + List componentNames = new ArrayList(topology.get_spouts().keySet()); + componentNames.addAll(topology.get_bolts().keySet()); + componentNames.addAll(topology.get_state_spouts().keySet()); + + for (String name : componentNames) { + ComponentCommon common = Utils.getComponentCommon(topology, name); + List streams = new ArrayList(common.get_streams().keySet()); + streamNametoId.put(name, idify(streams)); + streamIdToName.put(name, Utils.reverseMap(streamNametoId.get(name))); + } + } + + public int getStreamId(String component, String stream) { + return streamNametoId.get(component).get(stream); + } + + public String getStreamName(String component, int stream) { + return streamIdToName.get(component).get(stream); + } + + private static Map idify(List names) { + Collections.sort(names); + Map ret = new HashMap(); + int i = 1; + for (String name : names) { + ret.put(name, i); + i++; + } + return ret; + } + } + + private static Serializer resolveSerializerInstance(Kryo k, Class superClass, Class serializerClass, Map conf) { + try { + try { + return serializerClass.getConstructor(Kryo.class, Class.class, Map.class).newInstance(k, superClass, conf); + } catch (Exception ex1) { + try { + return serializerClass.getConstructor(Kryo.class, Class.class).newInstance(k, superClass); + } catch (Exception ex2) { + try { + return serializerClass.getConstructor(Kryo.class, Map.class).newInstance(k, conf); + } catch (Exception ex3) { + try { + return serializerClass.getConstructor(Kryo.class).newInstance(k); + } catch (Exception ex4) { + try { + return serializerClass.getConstructor(Class.class, Map.class).newInstance(superClass, conf); + } catch (Exception ex5) { + try { + return serializerClass.getConstructor(Class.class).newInstance(superClass); + } catch (Exception ex6) { + return serializerClass.newInstance(); + } + } + } + } + } + } + } catch (Exception ex) { + throw new IllegalArgumentException("Unable to create serializer \"" + serializerClass.getName() + "\" for class: " + superClass.getName(), ex); + } + } + + private static Map normalizeKryoRegister(Map conf) { + // TODO: de-duplicate this logic with the code in nimbus + Object res = conf.get(Config.TOPOLOGY_KRYO_REGISTER); + if (res == null) + return new TreeMap(); + Map ret = new HashMap(); + if (res instanceof Map) { + ret = (Map) res; + } else { + for (Object o : (List) res) { + if (o instanceof Map) { + ret.putAll((Map) o); + } else { + ret.put((String) o, null); + } + } + } + + // ensure always same order for registrations with TreeMap + return new TreeMap(ret); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/ThriftSerializationDelegate.java b/jstorm-core/src/main/java/backtype/storm/serialization/ThriftSerializationDelegate.java new file mode 100755 index 000000000..f5d03e4e2 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/ThriftSerializationDelegate.java @@ -0,0 +1,52 @@ +/** + * 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 backtype.storm.serialization; + +import org.apache.thrift.TBase; +import org.apache.thrift.TDeserializer; +import org.apache.thrift.TException; +import org.apache.thrift.TSerializer; + +import java.util.Map; + +public class ThriftSerializationDelegate implements SerializationDelegate { + + @Override + public void prepare(Map stormConf) { + } + + @Override + public byte[] serialize(Object object) { + try { + return new TSerializer().serialize((TBase) object); + } catch (TException e) { + throw new RuntimeException(e); + } + } + + @Override + public T deserialize(byte[] bytes, Class clazz) { + try { + TBase instance = (TBase) clazz.newInstance(); + new TDeserializer().deserialize(instance, bytes); + return (T)instance; + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/types/ArrayListSerializer.java b/jstorm-core/src/main/java/backtype/storm/serialization/types/ArrayListSerializer.java new file mode 100755 index 000000000..6b7e30889 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/types/ArrayListSerializer.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.serialization.types; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.serializers.CollectionSerializer; +import java.util.ArrayList; +import java.util.Collection; + + +public class ArrayListSerializer extends CollectionSerializer { + @Override + public Collection create(Kryo kryo, Input input, Class type) { + return new ArrayList(); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/types/HashMapSerializer.java b/jstorm-core/src/main/java/backtype/storm/serialization/types/HashMapSerializer.java new file mode 100755 index 000000000..662211bc5 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/types/HashMapSerializer.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.serialization.types; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.serializers.MapSerializer; +import java.util.HashMap; +import java.util.Map; + + +public class HashMapSerializer extends MapSerializer { + @Override + public Map create(Kryo kryo, Input input, Class type) { + return new HashMap(); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/types/HashSetSerializer.java b/jstorm-core/src/main/java/backtype/storm/serialization/types/HashSetSerializer.java new file mode 100755 index 000000000..77fc353b2 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/types/HashSetSerializer.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.serialization.types; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.serializers.CollectionSerializer; +import java.util.Collection; +import java.util.HashSet; + + +public class HashSetSerializer extends CollectionSerializer { + @Override + public Collection create(Kryo kryo, Input input, Class type) { + return new HashSet(); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/serialization/types/ListDelegateSerializer.java b/jstorm-core/src/main/java/backtype/storm/serialization/types/ListDelegateSerializer.java new file mode 100755 index 000000000..c71a19df9 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/serialization/types/ListDelegateSerializer.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.serialization.types; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.serializers.CollectionSerializer; +import backtype.storm.utils.ListDelegate; +import java.util.Collection; + + +public class ListDelegateSerializer extends CollectionSerializer { + @Override + public Collection create(Kryo kryo, Input input, Class type) { + return new ListDelegate(); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/spout/IMultiSchemableSpout.java b/jstorm-core/src/main/java/backtype/storm/spout/IMultiSchemableSpout.java new file mode 100755 index 000000000..5999fbb94 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/spout/IMultiSchemableSpout.java @@ -0,0 +1,23 @@ +/** + * 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 backtype.storm.spout; + +public interface IMultiSchemableSpout { + MultiScheme getScheme(); + void setScheme(MultiScheme scheme); +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/spout/ISchemableSpout.java b/jstorm-core/src/main/java/backtype/storm/spout/ISchemableSpout.java new file mode 100755 index 000000000..df455d961 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/spout/ISchemableSpout.java @@ -0,0 +1,24 @@ +/** + * 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 backtype.storm.spout; + + +public interface ISchemableSpout { + Scheme getScheme(); + void setScheme(Scheme scheme); +} diff --git a/jstorm-core/src/main/java/backtype/storm/spout/ISpout.java b/jstorm-core/src/main/java/backtype/storm/spout/ISpout.java new file mode 100755 index 000000000..c421feda6 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/spout/ISpout.java @@ -0,0 +1,105 @@ +/** + * 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 backtype.storm.spout; + +import backtype.storm.task.TopologyContext; +import java.util.Map; +import java.io.Serializable; + +/** + * ISpout is the core interface for implementing spouts. A Spout is responsible + * for feeding messages into the topology for processing. For every tuple emitted by + * a spout, Storm will track the (potentially very large) DAG of tuples generated + * based on a tuple emitted by the spout. When Storm detects that every tuple in + * that DAG has been successfully processed, it will send an ack message to the Spout. + * + *

If a tuple fails to be fully processed within the configured timeout for the + * topology (see {@link backtype.storm.Config}), Storm will send a fail message to the spout + * for the message.

+ * + *

When a Spout emits a tuple, it can tag the tuple with a message id. The message id + * can be any type. When Storm acks or fails a message, it will pass back to the + * spout the same message id to identify which tuple it's referring to. If the spout leaves out + * the message id, or sets it to null, then Storm will not track the message and the spout + * will not receive any ack or fail callbacks for the message.

+ * + *

Storm executes ack, fail, and nextTuple all on the same thread. This means that an implementor + * of an ISpout does not need to worry about concurrency issues between those methods. However, it + * also means that an implementor must ensure that nextTuple is non-blocking: otherwise + * the method could block acks and fails that are pending to be processed.

+ */ +public interface ISpout extends Serializable { + /** + * Called when a task for this component is initialized within a worker on the cluster. + * It provides the spout with the environment in which the spout executes. + * + *

This includes the:

+ * + * @param conf The Storm configuration for this spout. This is the configuration provided to the topology merged in with cluster configuration on this machine. + * @param context This object can be used to get information about this task's place within the topology, including the task id and component id of this task, input and output information, etc. + * @param collector The collector is used to emit tuples from this spout. Tuples can be emitted at any time, including the open and close methods. The collector is thread-safe and should be saved as an instance variable of this spout object. + */ + void open(Map conf, TopologyContext context, SpoutOutputCollector collector); + + /** + * Called when an ISpout is going to be shutdown. There is no guarentee that close + * will be called, because the supervisor kill -9's worker processes on the cluster. + * + *

The one context where close is guaranteed to be called is a topology is + * killed when running Storm in local mode.

+ */ + void close(); + + /** + * Called when a spout has been activated out of a deactivated mode. + * nextTuple will be called on this spout soon. A spout can become activated + * after having been deactivated when the topology is manipulated using the + * `storm` client. + */ + void activate(); + + /** + * Called when a spout has been deactivated. nextTuple will not be called while + * a spout is deactivated. The spout may or may not be reactivated in the future. + */ + void deactivate(); + + /** + * When this method is called, Storm is requesting that the Spout emit tuples to the + * output collector. This method should be non-blocking, so if the Spout has no tuples + * to emit, this method should return. nextTuple, ack, and fail are all called in a tight + * loop in a single thread in the spout task. When there are no tuples to emit, it is courteous + * to have nextTuple sleep for a short amount of time (like a single millisecond) + * so as not to waste too much CPU. + */ + void nextTuple(); + + /** + * Storm has determined that the tuple emitted by this spout with the msgId identifier + * has been fully processed. Typically, an implementation of this method will take that + * message off the queue and prevent it from being replayed. + */ + void ack(Object msgId); + + /** + * The tuple emitted by this spout with the msgId identifier has failed to be + * fully processed. Typically, an implementation of this method will put that + * message back on the queue to be replayed at a later time. + */ + void fail(Object msgId); +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/spout/ISpoutOutputCollector.java b/jstorm-core/src/main/java/backtype/storm/spout/ISpoutOutputCollector.java new file mode 100755 index 000000000..3cebe432e --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/spout/ISpoutOutputCollector.java @@ -0,0 +1,30 @@ +/** + * 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 backtype.storm.spout; + +import java.util.List; + +public interface ISpoutOutputCollector { + /** + Returns the task ids that received the tuples. + */ + List emit(String streamId, List tuple, Object messageId); + void emitDirect(int taskId, String streamId, List tuple, Object messageId); + void reportError(Throwable error); +} + diff --git a/jstorm-core/src/main/java/backtype/storm/spout/ISpoutWaitStrategy.java b/jstorm-core/src/main/java/backtype/storm/spout/ISpoutWaitStrategy.java new file mode 100755 index 000000000..d0bdfa8f8 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/spout/ISpoutWaitStrategy.java @@ -0,0 +1,34 @@ +/** + * 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 backtype.storm.spout; + +import java.util.Map; + +/** + * The strategy a spout needs to use when its waiting. Waiting is + * triggered in one of two conditions: + * + * 1. nextTuple emits no tuples + * 2. The spout has hit maxSpoutPending and can't emit any more tuples + * + * The default strategy sleeps for one millisecond. + */ +public interface ISpoutWaitStrategy { + void prepare(Map conf); + void emptyEmit(long streak); +} diff --git a/jstorm-core/src/main/java/backtype/storm/spout/MultiScheme.java b/jstorm-core/src/main/java/backtype/storm/spout/MultiScheme.java new file mode 100755 index 000000000..ca2ce91f2 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/spout/MultiScheme.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.spout; + +import java.util.List; +import java.io.Serializable; + +import backtype.storm.tuple.Fields; + +public interface MultiScheme extends Serializable { + public Iterable> deserialize(byte[] ser); + public Fields getOutputFields(); +} diff --git a/jstorm-core/src/main/java/backtype/storm/spout/NothingEmptyEmitStrategy.java b/jstorm-core/src/main/java/backtype/storm/spout/NothingEmptyEmitStrategy.java new file mode 100755 index 000000000..36bea947d --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/spout/NothingEmptyEmitStrategy.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.spout; + +import java.util.Map; + +public class NothingEmptyEmitStrategy implements ISpoutWaitStrategy { + @Override + public void emptyEmit(long streak) { + } + + @Override + public void prepare(Map conf) { + throw new UnsupportedOperationException("Not supported yet."); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/spout/RawMultiScheme.java b/jstorm-core/src/main/java/backtype/storm/spout/RawMultiScheme.java new file mode 100755 index 000000000..7f7397503 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/spout/RawMultiScheme.java @@ -0,0 +1,38 @@ +/** + * 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 backtype.storm.spout; + +import java.util.List; + +import backtype.storm.tuple.Fields; + + +import static backtype.storm.utils.Utils.tuple; +import static java.util.Arrays.asList; + +public class RawMultiScheme implements MultiScheme { + @Override + public Iterable> deserialize(byte[] ser) { + return asList(tuple(ser)); + } + + @Override + public Fields getOutputFields() { + return new Fields("bytes"); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/spout/RawScheme.java b/jstorm-core/src/main/java/backtype/storm/spout/RawScheme.java new file mode 100755 index 000000000..7e26770f2 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/spout/RawScheme.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.spout; + +import backtype.storm.tuple.Fields; +import java.util.List; +import static backtype.storm.utils.Utils.tuple; + +public class RawScheme implements Scheme { + public List deserialize(byte[] ser) { + return tuple(ser); + } + + public Fields getOutputFields() { + return new Fields("bytes"); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/spout/Scheme.java b/jstorm-core/src/main/java/backtype/storm/spout/Scheme.java new file mode 100755 index 000000000..ca6895427 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/spout/Scheme.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.spout; + +import backtype.storm.tuple.Fields; +import java.io.Serializable; +import java.util.List; + + +public interface Scheme extends Serializable { + public List deserialize(byte[] ser); + public Fields getOutputFields(); +} diff --git a/jstorm-core/src/main/java/backtype/storm/spout/SchemeAsMultiScheme.java b/jstorm-core/src/main/java/backtype/storm/spout/SchemeAsMultiScheme.java new file mode 100755 index 000000000..29f7fce6b --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/spout/SchemeAsMultiScheme.java @@ -0,0 +1,41 @@ +/** + * 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 backtype.storm.spout; + +import java.util.Arrays; +import java.util.List; + +import backtype.storm.tuple.Fields; + +public class SchemeAsMultiScheme implements MultiScheme { + public final Scheme scheme; + + public SchemeAsMultiScheme(Scheme scheme) { + this.scheme = scheme; + } + + @Override public Iterable> deserialize(final byte[] ser) { + List o = scheme.deserialize(ser); + if(o == null) return null; + else return Arrays.asList(o); + } + + @Override public Fields getOutputFields() { + return scheme.getOutputFields(); + } +} diff --git a/jstorm-client/src/main/java/backtype/storm/spout/ShellSpout.java b/jstorm-core/src/main/java/backtype/storm/spout/ShellSpout.java old mode 100644 new mode 100755 similarity index 94% rename from jstorm-client/src/main/java/backtype/storm/spout/ShellSpout.java rename to jstorm-core/src/main/java/backtype/storm/spout/ShellSpout.java index a8e18dc89..06c1647b1 --- a/jstorm-client/src/main/java/backtype/storm/spout/ShellSpout.java +++ b/jstorm-core/src/main/java/backtype/storm/spout/ShellSpout.java @@ -154,6 +154,8 @@ private void querySubprocess() { return; } else if (command.equals("log")) { handleLog(shellMsg); + } else if (command.equals("error")) { + handleError(shellMsg.getMsg()); } else if (command.equals("emit")) { String stream = shellMsg.getStream(); Long task = shellMsg.getTask(); @@ -206,6 +208,10 @@ private void handleLog(ShellMsg shellMsg) { } } + private void handleError(String msg) { + _collector.reportError(new Exception("Shell Process Exception: " + msg)); + } + @Override public void activate() { LOG.info("Start checking heartbeat..."); diff --git a/jstorm-core/src/main/java/backtype/storm/spout/SleepSpoutWaitStrategy.java b/jstorm-core/src/main/java/backtype/storm/spout/SleepSpoutWaitStrategy.java new file mode 100755 index 000000000..3ccf4e144 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/spout/SleepSpoutWaitStrategy.java @@ -0,0 +1,41 @@ +/** + * 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 backtype.storm.spout; + +import backtype.storm.Config; +import java.util.Map; + + +public class SleepSpoutWaitStrategy implements ISpoutWaitStrategy { + + long sleepMillis; + + @Override + public void prepare(Map conf) { + sleepMillis = ((Number) conf.get(Config.TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS)).longValue(); + } + + @Override + public void emptyEmit(long streak) { + try { + Thread.sleep(sleepMillis); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/spout/SpoutOutputCollector.java b/jstorm-core/src/main/java/backtype/storm/spout/SpoutOutputCollector.java new file mode 100755 index 000000000..7a3302697 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/spout/SpoutOutputCollector.java @@ -0,0 +1,134 @@ +/** + * 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 backtype.storm.spout; + +import backtype.storm.task.OutputCollector; +import backtype.storm.utils.Utils; +import java.util.List; + +/** + * This output collector exposes the API for emitting tuples from an {@link backtype.storm.topology.IRichSpout}. + * The main difference between this output collector and {@link OutputCollector} + * for {@link backtype.storm.topology.IRichBolt} is that spouts can tag messages with ids so that they can be + * acked or failed later on. This is the Spout portion of Storm's API to + * guarantee that each message is fully processed at least once. + */ +public class SpoutOutputCollector implements ISpoutOutputCollector { + ISpoutOutputCollector _delegate; + + public SpoutOutputCollector(ISpoutOutputCollector delegate) { + _delegate = delegate; + } + + /** + * Emits a new tuple to the specified output stream with the given message ID. + * When Storm detects that this tuple has been fully processed, or has failed + * to be fully processed, the spout will receive an ack or fail callback respectively + * with the messageId as long as the messageId was not null. If the messageId was null, + * Storm will not track the tuple and no callback will be received. The emitted values must be + * immutable. + * + * @return the list of task ids that this tuple was sent to + */ + public List emit(String streamId, List tuple, Object messageId) { + return _delegate.emit(streamId, tuple, messageId); + } + + /** + * Emits a new tuple to the default output stream with the given message ID. + * When Storm detects that this tuple has been fully processed, or has failed + * to be fully processed, the spout will receive an ack or fail callback respectively + * with the messageId as long as the messageId was not null. If the messageId was null, + * Storm will not track the tuple and no callback will be received. The emitted values must be + * immutable. + * + * @return the list of task ids that this tuple was sent to + */ + public List emit(List tuple, Object messageId) { + return emit(Utils.DEFAULT_STREAM_ID, tuple, messageId); + } + + /** + * Emits a tuple to the default output stream with a null message id. Storm will + * not track this message so ack and fail will never be called for this tuple. The + * emitted values must be immutable. + */ + public List emit(List tuple) { + return emit(tuple, null); + } + + /** + * Emits a tuple to the specified output stream with a null message id. Storm will + * not track this message so ack and fail will never be called for this tuple. The + * emitted values must be immutable. + */ + public List emit(String streamId, List tuple) { + return emit(streamId, tuple, null); + } + + /** + * Emits a tuple to the specified task on the specified output stream. This output + * stream must have been declared as a direct stream, and the specified task must + * use a direct grouping on this stream to receive the message. The emitted values must be + * immutable. + */ + public void emitDirect(int taskId, String streamId, List tuple, Object messageId) { + _delegate.emitDirect(taskId, streamId, tuple, messageId); + } + + /** + * Emits a tuple to the specified task on the default output stream. This output + * stream must have been declared as a direct stream, and the specified task must + * use a direct grouping on this stream to receive the message. The emitted values must be + * immutable. + */ + public void emitDirect(int taskId, List tuple, Object messageId) { + emitDirect(taskId, Utils.DEFAULT_STREAM_ID, tuple, messageId); + } + + /** + * Emits a tuple to the specified task on the specified output stream. This output + * stream must have been declared as a direct stream, and the specified task must + * use a direct grouping on this stream to receive the message. The emitted values must be + * immutable. + * + *

Because no message id is specified, Storm will not track this message + * so ack and fail will never be called for this tuple.

+ */ + public void emitDirect(int taskId, String streamId, List tuple) { + emitDirect(taskId, streamId, tuple, null); + } + + /** + * Emits a tuple to the specified task on the default output stream. This output + * stream must have been declared as a direct stream, and the specified task must + * use a direct grouping on this stream to receive the message. The emitted values must be + * immutable. + * + *

Because no message id is specified, Storm will not track this message + * so ack and fail will never be called for this tuple.

+ */ + public void emitDirect(int taskId, List tuple) { + emitDirect(taskId, tuple, null); + } + + @Override + public void reportError(Throwable error) { + _delegate.reportError(error); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/state/IStateSpout.java b/jstorm-core/src/main/java/backtype/storm/state/IStateSpout.java new file mode 100755 index 000000000..f4aa14f98 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/state/IStateSpout.java @@ -0,0 +1,29 @@ +/** + * 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 backtype.storm.state; + +import backtype.storm.task.TopologyContext; +import java.io.Serializable; +import java.util.Map; + +public interface IStateSpout extends Serializable { + void open(Map conf, TopologyContext context); + void close(); + void nextTuple(StateSpoutOutputCollector collector); + void synchronize(SynchronizeOutputCollector collector); +} diff --git a/jstorm-core/src/main/java/backtype/storm/state/IStateSpoutOutputCollector.java b/jstorm-core/src/main/java/backtype/storm/state/IStateSpoutOutputCollector.java new file mode 100755 index 000000000..e39401073 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/state/IStateSpoutOutputCollector.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.state; + +public interface IStateSpoutOutputCollector extends ISynchronizeOutputCollector { + void remove(int streamId, Object id); +} diff --git a/jstorm-core/src/main/java/backtype/storm/state/ISubscribedState.java b/jstorm-core/src/main/java/backtype/storm/state/ISubscribedState.java new file mode 100755 index 000000000..6eff72c34 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/state/ISubscribedState.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.state; + +import backtype.storm.tuple.Tuple; + +public interface ISubscribedState { + void set(Object id, Tuple tuple); + void remove(Object id); +} diff --git a/jstorm-core/src/main/java/backtype/storm/state/ISynchronizeOutputCollector.java b/jstorm-core/src/main/java/backtype/storm/state/ISynchronizeOutputCollector.java new file mode 100755 index 000000000..9c80a75d4 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/state/ISynchronizeOutputCollector.java @@ -0,0 +1,24 @@ +/** + * 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 backtype.storm.state; + +import java.util.List; + +public interface ISynchronizeOutputCollector { + void add(int streamId, Object id, List tuple); +} diff --git a/jstorm-core/src/main/java/backtype/storm/state/StateSpoutOutputCollector.java b/jstorm-core/src/main/java/backtype/storm/state/StateSpoutOutputCollector.java new file mode 100755 index 000000000..4bb10e074 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/state/StateSpoutOutputCollector.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.state; + + +public class StateSpoutOutputCollector extends SynchronizeOutputCollector implements IStateSpoutOutputCollector { + + @Override + public void remove(int streamId, Object id) { + throw new UnsupportedOperationException("Not supported yet."); + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/state/SynchronizeOutputCollector.java b/jstorm-core/src/main/java/backtype/storm/state/SynchronizeOutputCollector.java new file mode 100755 index 000000000..9fbba6eb9 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/state/SynchronizeOutputCollector.java @@ -0,0 +1,30 @@ +/** + * 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 backtype.storm.state; + +import java.util.List; + + +public class SynchronizeOutputCollector implements ISynchronizeOutputCollector { + + @Override + public void add(int streamId, Object id, List tuple) { + throw new UnsupportedOperationException("Not supported yet."); + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/task/GeneralTopologyContext.java b/jstorm-core/src/main/java/backtype/storm/task/GeneralTopologyContext.java new file mode 100644 index 000000000..7540500d1 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/task/GeneralTopologyContext.java @@ -0,0 +1,209 @@ +/** + * 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 backtype.storm.task; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.json.simple.JSONAware; + +import backtype.storm.Config; +import backtype.storm.Constants; +import backtype.storm.generated.ComponentCommon; +import backtype.storm.generated.GlobalStreamId; +import backtype.storm.generated.Grouping; +import backtype.storm.generated.StormTopology; +import backtype.storm.tuple.Fields; +import backtype.storm.utils.ThriftTopologyUtils; +import backtype.storm.utils.Utils; + +public class GeneralTopologyContext implements JSONAware { + private StormTopology _topology; + private Map _taskToComponent; + private Map> _componentToTasks; + private Map> _componentToStreamToFields; + private String _topologyId; + protected Map _stormConf; + + // pass in componentToSortedTasks for the case of running tons of tasks in single executor + public GeneralTopologyContext(StormTopology topology, Map stormConf, + Map taskToComponent, Map> componentToSortedTasks, + Map> componentToStreamToFields, String topologyId) { + _topology = topology; + _stormConf = stormConf; + _taskToComponent = taskToComponent; + _topologyId = topologyId; + _componentToTasks = componentToSortedTasks; + _componentToStreamToFields = componentToStreamToFields; + } + + /** + * Gets the unique id assigned to this topology. The id is the storm name with a unique nonce appended to it. + * + * @return the topology id + */ + public String getTopologyId() { + return _topologyId; + } + + /** + * Please use the getTopologId() instead. + * + * @return the topology id + */ + @Deprecated + public String getStormId() { + return _topologyId; + } + + /** + * Gets the Thrift object representing the topology. + * + * @return the Thrift definition representing the topology + */ + public StormTopology getRawTopology() { + return _topology; + } + + /** + * Gets the component id for the specified task id. The component id maps to a component id specified for a Spout or Bolt in the topology definition. + * + * @param taskId the task id + * @return the component id for the input task id + */ + public String getComponentId(int taskId) { + if (taskId == Constants.SYSTEM_TASK_ID) { + return Constants.SYSTEM_COMPONENT_ID; + } else { + return _taskToComponent.get(taskId); + } + } + + /** + * Gets the set of streams declared for the specified component. + */ + public Set getComponentStreams(String componentId) { + return getComponentCommon(componentId).get_streams().keySet(); + } + + /** + * Gets the task ids allocated for the given component id. The task ids are always returned in ascending order. + */ + public List getComponentTasks(String componentId) { + List ret = _componentToTasks.get(componentId); + if (ret == null) + return new ArrayList(); + else + return new ArrayList(ret); + } + + /** + * Gets the declared output fields for the specified component/stream. + */ + public Fields getComponentOutputFields(String componentId, String streamId) { + Fields ret = _componentToStreamToFields.get(componentId).get(streamId); + if (ret == null) { + throw new IllegalArgumentException("No output fields defined for component:stream " + componentId + ":" + streamId); + } + return ret; + } + + /** + * Gets the declared output fields for the specified global stream id. + */ + public Fields getComponentOutputFields(GlobalStreamId id) { + return getComponentOutputFields(id.get_componentId(), id.get_streamId()); + } + + /** + * Gets the declared inputs to the specified component. + * + * @return A map from subscribed component/stream to the grouping subscribed with. + */ + public Map getSources(String componentId) { + return getComponentCommon(componentId).get_inputs(); + } + + /** + * Gets information about who is consuming the outputs of the specified component, and how. + * + * @return Map from stream id to component id to the Grouping used. + */ + public Map> getTargets(String componentId) { + Map> ret = new HashMap>(); + for (String otherComponentId : getComponentIds()) { + Map inputs = getComponentCommon(otherComponentId).get_inputs(); + for (GlobalStreamId id : inputs.keySet()) { + if (id.get_componentId().equals(componentId)) { + Map curr = ret.get(id.get_streamId()); + if (curr == null) + curr = new HashMap(); + curr.put(otherComponentId, inputs.get(id)); + ret.put(id.get_streamId(), curr); + } + } + } + return ret; + } + + @Override + public String toJSONString() { + Map obj = new HashMap(); + obj.put("task->component", _taskToComponent); + // TODO: jsonify StormTopology + // at the minimum should send source info + return Utils.to_json(obj); + } + + /** + * Gets a map from task id to component id. + */ + public Map getTaskToComponent() { + return _taskToComponent; + } + + /** + * Gets a list of all component ids in this topology + */ + public Set getComponentIds() { + return ThriftTopologyUtils.getComponentIds(getRawTopology()); + } + + public ComponentCommon getComponentCommon(String componentId) { + return ThriftTopologyUtils.getComponentCommon(getRawTopology(), componentId); + } + + public int maxTopologyMessageTimeout() { + Integer max = Utils.getInt(_stormConf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS)); + for (String spout : getRawTopology().get_spouts().keySet()) { + ComponentCommon common = getComponentCommon(spout); + String jsonConf = common.get_json_conf(); + if (jsonConf != null) { + Map conf = (Map) Utils.from_json(jsonConf); + Object comp = conf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS); + if (comp != null) { + max = Math.max(Utils.getInt(comp), max); + } + } + } + return max; + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/task/IBolt.java b/jstorm-core/src/main/java/backtype/storm/task/IBolt.java new file mode 100755 index 000000000..48acddaf6 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/task/IBolt.java @@ -0,0 +1,84 @@ +/** + * 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 backtype.storm.task; + +import backtype.storm.tuple.Tuple; +import java.util.Map; +import java.io.Serializable; + +/** + * An IBolt represents a component that takes tuples as input and produces tuples + * as output. An IBolt can do everything from filtering to joining to functions + * to aggregations. It does not have to process a tuple immediately and may + * hold onto tuples to process later. + * + *

A bolt's lifecycle is as follows:

+ * + *

IBolt object created on client machine. The IBolt is serialized into the topology + * (using Java serialization) and submitted to the master machine of the cluster (Nimbus). + * Nimbus then launches workers which deserialize the object, call prepare on it, and then + * start processing tuples.

+ * + *

If you want to parameterize an IBolt, you should set the parameters through its + * constructor and save the parameterization state as instance variables (which will + * then get serialized and shipped to every task executing this bolt across the cluster).

+ * + *

When defining bolts in Java, you should use the IRichBolt interface which adds + * necessary methods for using the Java TopologyBuilder API.

+ */ +public interface IBolt extends Serializable { + /** + * Called when a task for this component is initialized within a worker on the cluster. + * It provides the bolt with the environment in which the bolt executes. + * + *

This includes the:

+ * + * @param stormConf The Storm configuration for this bolt. This is the configuration provided to the topology merged in with cluster configuration on this machine. + * @param context This object can be used to get information about this task's place within the topology, including the task id and component id of this task, input and output information, etc. + * @param collector The collector is used to emit tuples from this bolt. Tuples can be emitted at any time, including the prepare and cleanup methods. The collector is thread-safe and should be saved as an instance variable of this bolt object. + */ + void prepare(Map stormConf, TopologyContext context, OutputCollector collector); + + /** + * Process a single tuple of input. The Tuple object contains metadata on it + * about which component/stream/task it came from. The values of the Tuple can + * be accessed using Tuple#getValue. The IBolt does not have to process the Tuple + * immediately. It is perfectly fine to hang onto a tuple and process it later + * (for instance, to do an aggregation or join). + * + *

Tuples should be emitted using the OutputCollector provided through the prepare method. + * It is required that all input tuples are acked or failed at some point using the OutputCollector. + * Otherwise, Storm will be unable to determine when tuples coming off the spouts + * have been completed.

+ * + *

For the common case of acking an input tuple at the end of the execute method, + * see IBasicBolt which automates this.

+ * + * @param input The input tuple to be processed. + */ + void execute(Tuple input); + + /** + * Called when an IBolt is going to be shutdown. There is no guarentee that cleanup + * will be called, because the supervisor kill -9's worker processes on the cluster. + * + *

The one context where cleanup is guaranteed to be called is when a topology + * is killed when running Storm in local mode.

+ */ + void cleanup(); +} diff --git a/jstorm-core/src/main/java/backtype/storm/task/IErrorReporter.java b/jstorm-core/src/main/java/backtype/storm/task/IErrorReporter.java new file mode 100755 index 000000000..ad254504d --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/task/IErrorReporter.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.task; + +public interface IErrorReporter { + void reportError(Throwable error); +} diff --git a/jstorm-core/src/main/java/backtype/storm/task/IMetricsContext.java b/jstorm-core/src/main/java/backtype/storm/task/IMetricsContext.java new file mode 100755 index 000000000..a1d8bc216 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/task/IMetricsContext.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.task; + +import backtype.storm.metric.api.CombinedMetric; +import backtype.storm.metric.api.ICombiner; +import backtype.storm.metric.api.IMetric; +import backtype.storm.metric.api.IReducer; +import backtype.storm.metric.api.ReducedMetric; + + +public interface IMetricsContext { + T registerMetric(String name, T metric, int timeBucketSizeInSecs); + ReducedMetric registerMetric(String name, IReducer reducer, int timeBucketSizeInSecs); + CombinedMetric registerMetric(String name, ICombiner combiner, int timeBucketSizeInSecs); +} diff --git a/jstorm-core/src/main/java/backtype/storm/task/IOutputCollector.java b/jstorm-core/src/main/java/backtype/storm/task/IOutputCollector.java new file mode 100755 index 000000000..a62563a4a --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/task/IOutputCollector.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.task; + +import backtype.storm.tuple.Tuple; +import java.util.Collection; +import java.util.List; + +public interface IOutputCollector extends IErrorReporter { + /** + * Returns the task ids that received the tuples. + */ + List emit(String streamId, Collection anchors, List tuple); + void emitDirect(int taskId, String streamId, Collection anchors, List tuple); + void ack(Tuple input); + void fail(Tuple input); +} diff --git a/jstorm-core/src/main/java/backtype/storm/task/OutputCollector.java b/jstorm-core/src/main/java/backtype/storm/task/OutputCollector.java new file mode 100755 index 000000000..620d33de6 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/task/OutputCollector.java @@ -0,0 +1,225 @@ +/** + * 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 backtype.storm.task; + +import backtype.storm.tuple.Tuple; +import backtype.storm.utils.Utils; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; + +/** + * This output collector exposes the API for emitting tuples from an IRichBolt. + * This is the core API for emitting tuples. For a simpler API, and a more restricted + * form of stream processing, see IBasicBolt and BasicOutputCollector. + */ +public class OutputCollector implements IOutputCollector { + private IOutputCollector _delegate; + + + public OutputCollector(IOutputCollector delegate) { + _delegate = delegate; + } + + /** + * Emits a new tuple to a specific stream with a single anchor. The emitted values must be + * immutable. + * + * @param streamId the stream to emit to + * @param anchor the tuple to anchor to + * @param tuple the new output tuple from this bolt + * @return the list of task ids that this new tuple was sent to + */ + public List emit(String streamId, Tuple anchor, List tuple) { + return emit(streamId, Arrays.asList(anchor), tuple); + } + + /** + * Emits a new unanchored tuple to the specified stream. Because it's unanchored, + * if a failure happens downstream, this new tuple won't affect whether any + * spout tuples are considered failed or not. The emitted values must be + * immutable. + * + * @param streamId the stream to emit to + * @param tuple the new output tuple from this bolt + * @return the list of task ids that this new tuple was sent to + */ + public List emit(String streamId, List tuple) { + return emit(streamId, (List) null, tuple); + } + + /** + * Emits a new tuple to the default stream anchored on a group of input tuples. The emitted + * values must be immutable. + * + * @param anchors the tuples to anchor to + * @param tuple the new output tuple from this bolt + * @return the list of task ids that this new tuple was sent to + */ + public List emit(Collection anchors, List tuple) { + return emit(Utils.DEFAULT_STREAM_ID, anchors, tuple); + } + + + /** + * Emits a new tuple to the default stream anchored on a single tuple. The emitted values must be + * immutable. + * + * @param anchor the tuple to anchor to + * @param tuple the new output tuple from this bolt + * @return the list of task ids that this new tuple was sent to + */ + public List emit(Tuple anchor, List tuple) { + return emit(Utils.DEFAULT_STREAM_ID, anchor, tuple); + } + + /** + * Emits a new unanchored tuple to the default stream. Beacuse it's unanchored, + * if a failure happens downstream, this new tuple won't affect whether any + * spout tuples are considered failed or not. The emitted values must be + * immutable. + * + * @param tuple the new output tuple from this bolt + * @return the list of task ids that this new tuple was sent to + */ + public List emit(List tuple) { + return emit(Utils.DEFAULT_STREAM_ID, tuple); + } + + /** + * Emits a tuple directly to the specified task id on the specified stream. + * If the target bolt does not subscribe to this bolt using a direct grouping, + * the tuple will not be sent. If the specified output stream is not declared + * as direct, or the target bolt subscribes with a non-direct grouping, + * an error will occur at runtime. The emitted values must be + * immutable. + * + * @param taskId the taskId to send the new tuple to + * @param streamId the stream to send the tuple on. It must be declared as a direct stream in the topology definition. + * @param anchor the tuple to anchor to + * @param tuple the new output tuple from this bolt + */ + public void emitDirect(int taskId, String streamId, Tuple anchor, List tuple) { + emitDirect(taskId, streamId, Arrays.asList(anchor), tuple); + } + + /** + * Emits a tuple directly to the specified task id on the specified stream. + * If the target bolt does not subscribe to this bolt using a direct grouping, + * the tuple will not be sent. If the specified output stream is not declared + * as direct, or the target bolt subscribes with a non-direct grouping, + * an error will occur at runtime. Note that this method does not use anchors, + * so downstream failures won't affect the failure status of any spout tuples. + * The emitted values must be immutable. + * + * @param taskId the taskId to send the new tuple to + * @param streamId the stream to send the tuple on. It must be declared as a direct stream in the topology definition. + * @param tuple the new output tuple from this bolt + */ + public void emitDirect(int taskId, String streamId, List tuple) { + emitDirect(taskId, streamId, (List) null, tuple); + } + + /** + * Emits a tuple directly to the specified task id on the default stream. + * If the target bolt does not subscribe to this bolt using a direct grouping, + * the tuple will not be sent. If the specified output stream is not declared + * as direct, or the target bolt subscribes with a non-direct grouping, + * an error will occur at runtime. The emitted values must be + * immutable. + * + *

The default stream must be declared as direct in the topology definition. + * See OutputDeclarer#declare for how this is done when defining topologies + * in Java.

+ * + * @param taskId the taskId to send the new tuple to + * @param anchosr the tuples to anchor to + * @param tuple the new output tuple from this bolt + */ + public void emitDirect(int taskId, Collection anchors, List tuple) { + emitDirect(taskId, Utils.DEFAULT_STREAM_ID, anchors, tuple); + } + + /** + * Emits a tuple directly to the specified task id on the default stream. + * If the target bolt does not subscribe to this bolt using a direct grouping, + * the tuple will not be sent. If the specified output stream is not declared + * as direct, or the target bolt subscribes with a non-direct grouping, + * an error will occur at runtime. The emitted values must be + * immutable. + * + *

The default stream must be declared as direct in the topology definition. + * See OutputDeclarer#declare for how this is done when defining topologies + * in Java.

+ * + * @param taskId the taskId to send the new tuple to + * @param anchor the tuple to anchor to + * @param tuple the new output tuple from this bolt + */ + public void emitDirect(int taskId, Tuple anchor, List tuple) { + emitDirect(taskId, Utils.DEFAULT_STREAM_ID, anchor, tuple); + } + + + /** + * Emits a tuple directly to the specified task id on the default stream. + * If the target bolt does not subscribe to this bolt using a direct grouping, + * the tuple will not be sent. If the specified output stream is not declared + * as direct, or the target bolt subscribes with a non-direct grouping, + * an error will occur at runtime. The emitted values must be + * immutable. + * + *

The default stream must be declared as direct in the topology definition. + * See OutputDeclarer#declare for how this is done when defining topologies + * in Java.

+ * + *

Note that this method does not use anchors, so downstream failures won't + * affect the failure status of any spout tuples.

+ * + * @param taskId the taskId to send the new tuple to + * @param tuple the new output tuple from this bolt + */ + public void emitDirect(int taskId, List tuple) { + emitDirect(taskId, Utils.DEFAULT_STREAM_ID, tuple); + } + + @Override + public List emit(String streamId, Collection anchors, List tuple) { + return _delegate.emit(streamId, anchors, tuple); + } + + @Override + public void emitDirect(int taskId, String streamId, Collection anchors, List tuple) { + _delegate.emitDirect(taskId, streamId, anchors, tuple); + } + + @Override + public void ack(Tuple input) { + _delegate.ack(input); + } + + @Override + public void fail(Tuple input) { + _delegate.fail(input); + } + + @Override + public void reportError(Throwable error) { + _delegate.reportError(error); + } +} diff --git a/jstorm-client/src/main/java/backtype/storm/task/ShellBolt.java b/jstorm-core/src/main/java/backtype/storm/task/ShellBolt.java old mode 100644 new mode 100755 similarity index 95% rename from jstorm-client/src/main/java/backtype/storm/task/ShellBolt.java rename to jstorm-core/src/main/java/backtype/storm/task/ShellBolt.java index ba79f7d17..d9c8f03c4 --- a/jstorm-client/src/main/java/backtype/storm/task/ShellBolt.java +++ b/jstorm-core/src/main/java/backtype/storm/task/ShellBolt.java @@ -279,7 +279,10 @@ private long getLastHeartbeat() { private void die(Throwable exception) { String processInfo = _process.getProcessInfoString() + _process.getProcessTerminationInfoString(); _exception = new RuntimeException(processInfo, exception); - LOG.error("Halting process: ShellBolt died.", exception); + String message = String.format("Halting process: ShellBolt died. Command: %s, ProcessInfo %s", + Arrays.toString(_command), + processInfo); + LOG.error(message, exception); _collector.reportError(exception); if (_running || (exception instanceof Error)) { //don't exit if not running, unless it is an Error System.exit(11); diff --git a/jstorm-core/src/main/java/backtype/storm/task/TopologyContext.java b/jstorm-core/src/main/java/backtype/storm/task/TopologyContext.java new file mode 100755 index 000000000..812465115 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/task/TopologyContext.java @@ -0,0 +1,360 @@ +/** + * 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 backtype.storm.task; + +import backtype.storm.generated.GlobalStreamId; +import backtype.storm.generated.Grouping; +import backtype.storm.generated.StormTopology; +import backtype.storm.hooks.ITaskHook; +import backtype.storm.metric.api.IMetric; +import backtype.storm.metric.api.IReducer; +import backtype.storm.metric.api.ICombiner; +import backtype.storm.metric.api.ReducedMetric; +import backtype.storm.metric.api.CombinedMetric; +import backtype.storm.state.ISubscribedState; +import backtype.storm.tuple.Fields; +import backtype.storm.utils.Utils; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.commons.lang.NotImplementedException; +import org.json.simple.JSONValue; + +/** + * A TopologyContext is given to bolts and spouts in their "prepare" and "open" + * methods, respectively. This object provides information about the component's + * place within the topology, such as task ids, inputs and outputs, etc. + * + *

The TopologyContext is also used to declare ISubscribedState objects to + * synchronize state with StateSpouts this object is subscribed to.

+ */ +public class TopologyContext extends WorkerTopologyContext implements IMetricsContext { + private Integer _taskId; + private Map _taskData = new HashMap(); + private List _hooks = new ArrayList(); + private Map _executorData; + private Map>> _registeredMetrics; + private clojure.lang.Atom _openOrPrepareWasCalled; + + + public TopologyContext(StormTopology topology, Map stormConf, + Map taskToComponent, Map> componentToSortedTasks, + Map> componentToStreamToFields, + String stormId, String codeDir, String pidDir, Integer taskId, + Integer workerPort, List workerTasks, Map defaultResources, + Map userResources, Map executorData, Map registeredMetrics, + clojure.lang.Atom openOrPrepareWasCalled) { + super(topology, stormConf, taskToComponent, componentToSortedTasks, + componentToStreamToFields, stormId, codeDir, pidDir, + workerPort, workerTasks, defaultResources, userResources); + _taskId = taskId; + _executorData = executorData; + _registeredMetrics = registeredMetrics; + _openOrPrepareWasCalled = openOrPrepareWasCalled; + } + + /** + * All state from all subscribed state spouts streams will be synced with + * the provided object. + * + *

It is recommended that your ISubscribedState object is kept as an instance + * variable of this object. The recommended usage of this method is as follows:

+ * + *

+ * _myState = context.setAllSubscribedState(new MyState()); + *

+ * @param obj Provided ISubscribedState implementation + * @return Returns the ISubscribedState object provided + */ + public T setAllSubscribedState(T obj) { + //check that only subscribed to one component/stream for statespout + //setsubscribedstate appropriately + throw new NotImplementedException(); + } + + + /** + * Synchronizes the default stream from the specified state spout component + * id with the provided ISubscribedState object. + * + *

The recommended usage of this method is as follows:

+ *

+ * _myState = context.setSubscribedState(componentId, new MyState()); + *

+ * + * @param componentId the id of the StateSpout component to subscribe to + * @param obj Provided ISubscribedState implementation + * @return Returns the ISubscribedState object provided + */ + public T setSubscribedState(String componentId, T obj) { + return setSubscribedState(componentId, Utils.DEFAULT_STREAM_ID, obj); + } + + /** + * Synchronizes the specified stream from the specified state spout component + * id with the provided ISubscribedState object. + * + *

The recommended usage of this method is as follows:

+ *

+ * _myState = context.setSubscribedState(componentId, streamId, new MyState()); + *

+ * + * @param componentId the id of the StateSpout component to subscribe to + * @param streamId the stream to subscribe to + * @param obj Provided ISubscribedState implementation + * @return Returns the ISubscribedState object provided + */ + public T setSubscribedState(String componentId, String streamId, T obj) { + throw new NotImplementedException(); + } + + /** + * Gets the task id of this task. + * + * @return the task id + */ + public int getThisTaskId() { + return _taskId; + } + + /** + * Gets the component id for this task. The component id maps + * to a component id specified for a Spout or Bolt in the topology definition. + * @return + */ + public String getThisComponentId() { + return getComponentId(_taskId); + } + + /** + * Gets the declared output fields for the specified stream id for the + * component this task is a part of. + */ + public Fields getThisOutputFields(String streamId) { + return getComponentOutputFields(getThisComponentId(), streamId); + } + + /** + * Gets the declared output fields for the specified stream id for the + * component this task is a part of. + */ + public Map> getThisOutputFieldsForStreams() { + Map> streamToFields = new HashMap>(); + for (String stream : this.getThisStreams()) { + streamToFields.put(stream, this.getThisOutputFields(stream).toList()); + } + return streamToFields; + } + + /** + * Gets the set of streams declared for the component of this task. + */ + public Set getThisStreams() { + return getComponentStreams(getThisComponentId()); + } + + /** + * Gets the index of this task id in getComponentTasks(getThisComponentId()). + * An example use case for this method is determining which task + * accesses which resource in a distributed resource to ensure an even distribution. + */ + public int getThisTaskIndex() { + List tasks = new ArrayList(getComponentTasks(getThisComponentId())); + Collections.sort(tasks); + for(int i=0; i getThisSources() { + return getSources(getThisComponentId()); + } + + /** + * Gets information about who is consuming the outputs of this component, and how. + * + * @return Map from stream id to component id to the Grouping used. + */ + public Map> getThisTargets() { + return getTargets(getThisComponentId()); + } + + public void setTaskData(String name, Object data) { + _taskData.put(name, data); + } + + public Object getTaskData(String name) { + return _taskData.get(name); + } + + public void setExecutorData(String name, Object data) { + _executorData.put(name, data); + } + + public Object getExecutorData(String name) { + return _executorData.get(name); + } + + public void addTaskHook(ITaskHook hook) { + hook.prepare(_stormConf, this); + _hooks.add(hook); + } + + public Collection getHooks() { + return _hooks; + } + + private static Map groupingToJSONableMap(Grouping grouping) { + Map groupingMap = new HashMap(); + groupingMap.put("type", grouping.getSetField().toString()); + if (grouping.is_set_fields()) { + groupingMap.put("fields", grouping.get_fields()); + } + return groupingMap; + } + + @Override + public String toJSONString() { + Map obj = new HashMap(); + obj.put("task->component", this.getTaskToComponent()); + obj.put("taskid", this.getThisTaskId()); + obj.put("componentid", this.getThisComponentId()); + List streamList = new ArrayList(); + streamList.addAll(this.getThisStreams()); + obj.put("streams", streamList); + obj.put("stream->outputfields", this.getThisOutputFieldsForStreams()); + // Convert targets to a JSON serializable format + Map stringTargets = new HashMap(); + for (Map.Entry> entry : this.getThisTargets().entrySet()) { + Map stringTargetMap = new HashMap(); + for (Map.Entry innerEntry : entry.getValue().entrySet()) { + stringTargetMap.put(innerEntry.getKey(), groupingToJSONableMap(innerEntry.getValue())); + } + stringTargets.put(entry.getKey(), stringTargetMap); + } + obj.put("stream->target->grouping", stringTargets); + // Convert sources to a JSON serializable format + Map> stringSources = new HashMap>(); + for (Map.Entry entry : this.getThisSources().entrySet()) { + GlobalStreamId gid = entry.getKey(); + Map stringSourceMap = stringSources.get(gid.get_componentId()); + if (stringSourceMap == null) { + stringSourceMap = new HashMap(); + stringSources.put(gid.get_componentId(), stringSourceMap); + } + stringSourceMap.put(gid.get_streamId(), groupingToJSONableMap(entry.getValue())); + } + obj.put("source->stream->grouping", stringSources); + return JSONValue.toJSONString(obj); + } + + /* + * Register a IMetric instance. + * Storm will then call getValueAndReset on the metric every timeBucketSizeInSecs + * and the returned value is sent to all metrics consumers. + * You must call this during IBolt::prepare or ISpout::open. + * @return The IMetric argument unchanged. + */ + public T registerMetric(String name, T metric, int timeBucketSizeInSecs) { + if((Boolean)_openOrPrepareWasCalled.deref() == true) { + throw new RuntimeException("TopologyContext.registerMetric can only be called from within overridden " + + "IBolt::prepare() or ISpout::open() method."); + } + + if (metric == null) { + throw new IllegalArgumentException("Cannot register a null metric"); + } + + if (timeBucketSizeInSecs <= 0) { + throw new IllegalArgumentException("TopologyContext.registerMetric can only be called with timeBucketSizeInSecs " + + "greater than or equal to 1 second."); + } + + if (getRegisteredMetricByName(name) != null) { + throw new RuntimeException("The same metric name `" + name + "` was registered twice." ); + } + + Map m1 = _registeredMetrics; + if(!m1.containsKey(timeBucketSizeInSecs)) { + m1.put(timeBucketSizeInSecs, new HashMap()); + } + + Map m2 = (Map)m1.get(timeBucketSizeInSecs); + if(!m2.containsKey(_taskId)) { + m2.put(_taskId, new HashMap()); + } + + Map m3 = (Map)m2.get(_taskId); + if(m3.containsKey(name)) { + throw new RuntimeException("The same metric name `" + name + "` was registered twice." ); + } else { + m3.put(name, metric); + } + + return metric; + } + + /** + * Get component's metric from registered metrics by name. + * Notice: Normally, one component can only register one metric name once. + * But now registerMetric has a bug(https://issues.apache.org/jira/browse/STORM-254) + * cause the same metric name can register twice. + * So we just return the first metric we meet. + */ + public IMetric getRegisteredMetricByName(String name) { + IMetric metric = null; + + for (Map> taskIdToNameToMetric: _registeredMetrics.values()) { + Map nameToMetric = taskIdToNameToMetric.get(_taskId); + if (nameToMetric != null) { + metric = nameToMetric.get(name); + if (metric != null) { + //we just return the first metric we meet + break; + } + } + } + + return metric; + } + + /* + * Convinience method for registering ReducedMetric. + */ + public ReducedMetric registerMetric(String name, IReducer reducer, int timeBucketSizeInSecs) { + return registerMetric(name, new ReducedMetric(reducer), timeBucketSizeInSecs); + } + /* + * Convinience method for registering CombinedMetric. + */ + public CombinedMetric registerMetric(String name, ICombiner combiner, int timeBucketSizeInSecs) { + return registerMetric(name, new CombinedMetric(combiner), timeBucketSizeInSecs); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/task/WorkerTopologyContext.java b/jstorm-core/src/main/java/backtype/storm/task/WorkerTopologyContext.java new file mode 100755 index 000000000..de407ac73 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/task/WorkerTopologyContext.java @@ -0,0 +1,109 @@ +/** + * 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 backtype.storm.task; + +import backtype.storm.generated.StormTopology; +import backtype.storm.tuple.Fields; +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; + +public class WorkerTopologyContext extends GeneralTopologyContext { + public static final String SHARED_EXECUTOR = "executor"; + + private Integer _workerPort; + private List _workerTasks; + private String _codeDir; + private String _pidDir; + Map _userResources; + Map _defaultResources; + + public WorkerTopologyContext( + StormTopology topology, + Map stormConf, + Map taskToComponent, + Map> componentToSortedTasks, + Map> componentToStreamToFields, + String stormId, + String codeDir, + String pidDir, + Integer workerPort, + List workerTasks, + Map defaultResources, + Map userResources + ) { + super(topology, stormConf, taskToComponent, componentToSortedTasks, componentToStreamToFields, stormId); + _codeDir = codeDir; + _defaultResources = defaultResources; + _userResources = userResources; + try { + if(pidDir!=null) { + _pidDir = new File(pidDir).getCanonicalPath(); + } else { + _pidDir = null; + } + } catch (IOException e) { + throw new RuntimeException("Could not get canonical path for " + _pidDir, e); + } + _workerPort = workerPort; + _workerTasks = workerTasks; + } + + /** + * Gets all the task ids that are running in this worker process + * (including the task for this task). + */ + public List getThisWorkerTasks() { + return _workerTasks; + } + + public Integer getThisWorkerPort() { + return _workerPort; + } + + public void setThisWorkerTasks(List workerTasks) { + this._workerTasks = workerTasks; + } + /** + * Gets the location of the external resources for this worker on the + * local filesystem. These external resources typically include bolts implemented + * in other languages, such as Ruby or Python. + */ + public String getCodeDir() { + return _codeDir; + } + + /** + * If this task spawns any subprocesses, those subprocesses must immediately + * write their PID to this directory on the local filesystem to ensure that + * Storm properly destroys that process when the worker is shutdown. + */ + public String getPIDDir() { + return _pidDir; + } + + public Object getResource(String name) { + return _userResources.get(name); + } + + public ExecutorService getSharedExecutor() { + return (ExecutorService) _defaultResources.get(SHARED_EXECUTOR); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/AckFailDelegate.java b/jstorm-core/src/main/java/backtype/storm/testing/AckFailDelegate.java new file mode 100755 index 000000000..d65c8bd26 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/AckFailDelegate.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.testing; + +import java.io.Serializable; + +public interface AckFailDelegate extends Serializable { + public void ack(Object id); + public void fail(Object id); +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/AckFailMapTracker.java b/jstorm-core/src/main/java/backtype/storm/testing/AckFailMapTracker.java new file mode 100755 index 000000000..e16afd8cd --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/AckFailMapTracker.java @@ -0,0 +1,52 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.utils.RegisteredGlobalState; +import java.util.HashSet; +import java.util.Set; + +public class AckFailMapTracker implements AckFailDelegate { + + String _acked; + String _failed; + + public AckFailMapTracker() { + _acked = RegisteredGlobalState.registerState(new HashSet()); + _failed = RegisteredGlobalState.registerState(new HashSet()); + } + + public boolean isAcked(Object id) { + return ((Set)RegisteredGlobalState.getState(_acked)).contains(id); + } + + public boolean isFailed(Object id) { + return ((Set)RegisteredGlobalState.getState(_failed)).contains(id); + } + + @Override + public void ack(Object id) { + ((Set)RegisteredGlobalState.getState(_acked)).add(id); + } + + @Override + public void fail(Object id) { + ((Set)RegisteredGlobalState.getState(_failed)).add(id); + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/AckTracker.java b/jstorm-core/src/main/java/backtype/storm/testing/AckTracker.java new file mode 100755 index 000000000..ad8047543 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/AckTracker.java @@ -0,0 +1,52 @@ +/** + * 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 backtype.storm.testing; + +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; + +public class AckTracker implements AckFailDelegate { + private static Map acks = new ConcurrentHashMap(); + + private String _id; + + public AckTracker() { + _id = UUID.randomUUID().toString(); + acks.put(_id, new AtomicInteger(0)); + } + + @Override + public void ack(Object id) { + acks.get(_id).incrementAndGet(); + } + + @Override + public void fail(Object id) { + } + + public int getNumAcks() { + return acks.get(_id).intValue(); + } + + public void resetNumAcks() { + acks.get(_id).set(0); + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/BatchNumberList.java b/jstorm-core/src/main/java/backtype/storm/testing/BatchNumberList.java new file mode 100755 index 000000000..26f964a02 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/BatchNumberList.java @@ -0,0 +1,72 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.coordination.BatchOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseBatchBolt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class BatchNumberList extends BaseBatchBolt { + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("word", "list")); + } + + String _wordComponent; + + public BatchNumberList(String wordComponent) { + _wordComponent = wordComponent; + } + + String word = null; + List intSet = new ArrayList(); + BatchOutputCollector _collector; + + + @Override + public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, Object id) { + _collector = collector; + } + + @Override + public void execute(Tuple tuple) { + if(tuple.getSourceComponent().equals(_wordComponent)) { + this.word = tuple.getString(1); + } else { + intSet.add(tuple.getInteger(1)); + } + } + + @Override + public void finishBatch() { + if(word!=null) { + Collections.sort(intSet); + _collector.emit(new Values(word, intSet)); + } + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/BatchProcessWord.java b/jstorm-core/src/main/java/backtype/storm/testing/BatchProcessWord.java new file mode 100755 index 000000000..7f3eaf1f2 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/BatchProcessWord.java @@ -0,0 +1,39 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseBasicBolt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; + +public class BatchProcessWord extends BaseBasicBolt { + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("id", "size")); + } + + @Override + public void execute(Tuple input, BasicOutputCollector collector) { + collector.emit(new Values(input.getValue(0), input.getString(1).length())); + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/BatchRepeatA.java b/jstorm-core/src/main/java/backtype/storm/testing/BatchRepeatA.java new file mode 100755 index 000000000..107f2edfa --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/BatchRepeatA.java @@ -0,0 +1,48 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseBasicBolt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; + + +public class BatchRepeatA extends BaseBasicBolt { + + @Override + public void execute(Tuple input, BasicOutputCollector collector) { + Object id = input.getValue(0); + String word = input.getString(1); + for(int i=0; i getComponentConfiguration() { + return new HashMap(); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/CompleteTopologyParam.java b/jstorm-core/src/main/java/backtype/storm/testing/CompleteTopologyParam.java new file mode 100755 index 000000000..f3306cfa3 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/CompleteTopologyParam.java @@ -0,0 +1,87 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.Config; + +/** + * The param class for the Testing.completeTopology. + */ +public class CompleteTopologyParam { + /** + * The mocked spout sources + */ + private MockedSources mockedSources; + /** + * the config for the topology when it was submitted to the cluster + */ + private Config stormConf; + /** + * whether cleanup the state? + */ + private Boolean cleanupState; + /** + * the topology name you want to submit to the cluster + */ + private String topologyName; + + /** + * the timeout of topology you want to submit to the cluster + */ + private Integer timeoutMs; + + public MockedSources getMockedSources() { + return mockedSources; + } + + public void setMockedSources(MockedSources mockedSources) { + this.mockedSources = mockedSources; + } + + public Config getStormConf() { + return stormConf; + } + + public void setStormConf(Config stormConf) { + this.stormConf = stormConf; + } + + public Boolean getCleanupState() { + return cleanupState; + } + + public void setCleanupState(Boolean cleanupState) { + this.cleanupState = cleanupState; + } + + public String getTopologyName() { + return topologyName; + } + + public void setTopologyName(String topologyName) { + this.topologyName = topologyName; + } + + public Integer getTimeoutMs() { + return timeoutMs; + } + + public void setTimeoutMs(Integer timeoutMs) { + this.timeoutMs = timeoutMs; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/CountingBatchBolt.java b/jstorm-core/src/main/java/backtype/storm/testing/CountingBatchBolt.java new file mode 100755 index 000000000..882801c45 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/CountingBatchBolt.java @@ -0,0 +1,55 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseBatchBolt; +import backtype.storm.coordination.BatchOutputCollector; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; +import java.util.Map; + +public class CountingBatchBolt extends BaseBatchBolt { + BatchOutputCollector _collector; + Object _id; + int _count = 0; + + @Override + public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, Object id) { + _collector = collector; + _id = id; + } + + @Override + public void execute(Tuple tuple) { + _count++; + } + + @Override + public void finishBatch() { + _collector.emit(new Values(_id, _count)); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("tx", "count")); + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/CountingCommitBolt.java b/jstorm-core/src/main/java/backtype/storm/testing/CountingCommitBolt.java new file mode 100755 index 000000000..cb8f7e56a --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/CountingCommitBolt.java @@ -0,0 +1,57 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.coordination.BatchOutputCollector; +import backtype.storm.topology.base.BaseTransactionalBolt; +import backtype.storm.transactional.ICommitter; +import backtype.storm.transactional.TransactionAttempt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; +import java.util.Map; + +public class CountingCommitBolt extends BaseTransactionalBolt implements ICommitter { + BatchOutputCollector _collector; + TransactionAttempt _id; + int _count = 0; + + @Override + public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, TransactionAttempt id) { + _id = id; + _collector = collector; + } + + @Override + public void execute(Tuple tuple) { + _count++; + } + + @Override + public void finishBatch() { + _collector.emit(new Values(_id, _count)); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("tx", "count")); + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/FeederSpout.java b/jstorm-core/src/main/java/backtype/storm/testing/FeederSpout.java new file mode 100755 index 000000000..1ffb59417 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/FeederSpout.java @@ -0,0 +1,100 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.topology.OutputFieldsDeclarer; +import java.util.Map; +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.base.BaseRichSpout; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; +import backtype.storm.utils.InprocMessaging; +import java.util.HashMap; +import java.util.List; +import java.util.UUID; + + +public class FeederSpout extends BaseRichSpout { + private int _id; + private Fields _outFields; + private SpoutOutputCollector _collector; + private AckFailDelegate _ackFailDelegate; + + public FeederSpout(Fields outFields) { + _id = InprocMessaging.acquireNewPort(); + _outFields = outFields; + } + + public void setAckFailDelegate(AckFailDelegate d) { + _ackFailDelegate = d; + } + + public void feed(List tuple) { + feed(tuple, UUID.randomUUID().toString()); + } + + public void feed(List tuple, Object msgId) { + InprocMessaging.sendMessage(_id, new Values(tuple, msgId)); + } + + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + _collector = collector; + } + + public void close() { + + } + + public void nextTuple() { + List toEmit = (List) InprocMessaging.pollMessage(_id); + if(toEmit!=null) { + List tuple = (List) toEmit.get(0); + Object msgId = toEmit.get(1); + + _collector.emit(tuple, msgId); + } else { + try { + Thread.sleep(1); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + } + + public void ack(Object msgId) { + if(_ackFailDelegate!=null) { + _ackFailDelegate.ack(msgId); + } + } + + public void fail(Object msgId) { + if(_ackFailDelegate!=null) { + _ackFailDelegate.fail(msgId); + } + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(_outFields); + } + + @Override + public Map getComponentConfiguration() { + return new HashMap(); + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/testing/FixedTuple.java b/jstorm-core/src/main/java/backtype/storm/testing/FixedTuple.java new file mode 100755 index 000000000..ceb0d350a --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/FixedTuple.java @@ -0,0 +1,42 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.utils.Utils; +import java.io.Serializable; +import java.util.List; + +public class FixedTuple implements Serializable { + public String stream; + public List values; + + public FixedTuple(List values) { + this.stream = Utils.DEFAULT_STREAM_ID; + this.values = values; + } + + public FixedTuple(String stream, List values) { + this.stream = stream; + this.values = values; + } + + @Override + public String toString() { + return stream + ":" + "<" + values.toString() + ">"; + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/testing/FixedTupleSpout.java b/jstorm-core/src/main/java/backtype/storm/testing/FixedTupleSpout.java new file mode 100755 index 000000000..952780396 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/FixedTupleSpout.java @@ -0,0 +1,179 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.IRichSpout; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Fields; +import backtype.storm.utils.Utils; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import static backtype.storm.utils.Utils.get; + +public class FixedTupleSpout implements IRichSpout { + private static final Map acked = new HashMap(); + private static final Map failed = new HashMap(); + + public static int getNumAcked(String stormId) { + synchronized(acked) { + return get(acked, stormId, 0); + } + } + + public static int getNumFailed(String stormId) { + synchronized(failed) { + return get(failed, stormId, 0); + } + } + + public static void clear(String stormId) { + acked.remove(stormId); + failed.remove(stormId); + } + + private List _tuples; + private SpoutOutputCollector _collector; + + private TopologyContext _context; + private List _serveTuples; + private Map _pending; + + private String _id; + private String _fieldName; + + public FixedTupleSpout(List tuples) { + this(tuples, null); + } + + public FixedTupleSpout(List tuples, String fieldName) { + _id = UUID.randomUUID().toString(); + synchronized(acked) { + acked.put(_id, 0); + } + synchronized(failed) { + failed.put(_id, 0); + } + _tuples = new ArrayList(); + for(Object o: tuples) { + FixedTuple ft; + if(o instanceof FixedTuple) { + ft = (FixedTuple) o; + } else { + ft = new FixedTuple((List) o); + } + _tuples.add(ft); + } + _fieldName = fieldName; + } + + public List getSourceTuples() { + return _tuples; + } + + public int getCompleted() { + int ackedAmt; + int failedAmt; + + synchronized(acked) { + ackedAmt = acked.get(_id); + } + synchronized(failed) { + failedAmt = failed.get(_id); + } + return ackedAmt + failedAmt; + } + + public void cleanup() { + synchronized(acked) { + acked.remove(_id); + } + synchronized(failed) { + failed.remove(_id); + } + } + + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + _context = context; + List tasks = context.getComponentTasks(context.getThisComponentId()); + int startIndex; + for(startIndex=0; startIndex(); + _serveTuples = new ArrayList(); + for(int i=startIndex; i<_tuples.size(); i+=tasks.size()) { + _serveTuples.add(_tuples.get(i)); + } + } + + public void close() { + } + + public void nextTuple() { + if(_serveTuples.size()>0) { + FixedTuple ft = _serveTuples.remove(0); + String id = UUID.randomUUID().toString(); + _pending.put(id, ft); + _collector.emit(ft.stream, ft.values, id); + } else { + Utils.sleep(100); + } + } + + public void ack(Object msgId) { + synchronized(acked) { + int curr = get(acked, _id, 0); + acked.put(_id, curr+1); + } + } + + public void fail(Object msgId) { + synchronized(failed) { + int curr = get(failed, _id, 0); + failed.put(_id, curr+1); + } + } + + @Override + public void activate() { + } + + @Override + public void deactivate() { + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + if (_fieldName != null) { + declarer.declare(new Fields(_fieldName)); + } + } + + @Override + public Map getComponentConfiguration() { + return null; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/ForwardingMetricsConsumer.java b/jstorm-core/src/main/java/backtype/storm/testing/ForwardingMetricsConsumer.java new file mode 100755 index 000000000..010336e47 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/ForwardingMetricsConsumer.java @@ -0,0 +1,95 @@ +/** + * 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 backtype.storm.testing; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.Socket; +import java.util.Collection; +import java.util.Map; +import java.io.OutputStream; + +import backtype.storm.metric.api.IMetricsConsumer; +import backtype.storm.task.IErrorReporter; +import backtype.storm.task.TopologyContext; + +/* + * Listens for all metrics, dumps them as text to a configured host:port + * + * To use, add this to your topology's configuration: + * conf.registerMetricsConsumer(backtype.storm.testing.ForwardingMetricsConsumer.class, ":", 1); + * + * Or edit the storm.yaml config file: + * + * topology.metrics.consumer.register: + * - class: "backtype.storm.testing.ForwardingMetricsConsumer" + * - argument: "example.com:9999" + * parallelism.hint: 1 + * + */ +public class ForwardingMetricsConsumer implements IMetricsConsumer { + String host; + int port; + Socket socket; + OutputStream out; + + @Override + public void prepare(Map stormConf, Object registrationArgument, TopologyContext context, IErrorReporter errorReporter) { + String [] parts = ((String)registrationArgument).split(":",2); + host = parts[0]; + port = Integer.valueOf(parts[1]); + try { + socket = new Socket(host, port); + out = socket.getOutputStream(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public void handleDataPoints(TaskInfo taskInfo, Collection dataPoints) { + StringBuilder sb = new StringBuilder(); + String header = taskInfo.timestamp + "\t" + + taskInfo.srcWorkerHost + ":"+ taskInfo.srcWorkerPort + "\t"+ + taskInfo.srcTaskId + "\t" + taskInfo.srcComponentId + "\t"; + sb.append(header); + for (DataPoint p : dataPoints) { + sb.delete(header.length(), sb.length()); + sb.append(p.name) + .append("\t") + .append(p.value) + .append("\n"); + try { + out.write(sb.toString().getBytes()); + out.flush(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + @Override + public void cleanup() { + try { + socket.close(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/IdentityBolt.java b/jstorm-core/src/main/java/backtype/storm/testing/IdentityBolt.java new file mode 100755 index 000000000..dcad640c7 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/IdentityBolt.java @@ -0,0 +1,42 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseBasicBolt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; + +public class IdentityBolt extends BaseBasicBolt { + Fields _fields; + + public IdentityBolt(Fields fields) { + _fields = fields; + } + + @Override + public void execute(Tuple input, BasicOutputCollector collector) { + collector.emit(input.getValues()); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(_fields); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/KeyedCountingBatchBolt.java b/jstorm-core/src/main/java/backtype/storm/testing/KeyedCountingBatchBolt.java new file mode 100755 index 000000000..1c4d5b3a2 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/KeyedCountingBatchBolt.java @@ -0,0 +1,61 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseBatchBolt; +import backtype.storm.coordination.BatchOutputCollector; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; +import backtype.storm.utils.Utils; +import java.util.HashMap; +import java.util.Map; + +public class KeyedCountingBatchBolt extends BaseBatchBolt { + BatchOutputCollector _collector; + Object _id; + Map _counts = new HashMap(); + + @Override + public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, Object id) { + _collector = collector; + _id = id; + } + + @Override + public void execute(Tuple tuple) { + Object key = tuple.getValue(1); + int curr = Utils.get(_counts, key, 0); + _counts.put(key, curr + 1); + } + + @Override + public void finishBatch() { + for(Object key: _counts.keySet()) { + _collector.emit(new Values(_id, key, _counts.get(key))); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("tx", "key", "count")); + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/KeyedCountingCommitterBolt.java b/jstorm-core/src/main/java/backtype/storm/testing/KeyedCountingCommitterBolt.java new file mode 100755 index 000000000..813e5b21f --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/KeyedCountingCommitterBolt.java @@ -0,0 +1,24 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.transactional.ICommitter; + +public class KeyedCountingCommitterBolt extends KeyedCountingBatchBolt implements ICommitter { + +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/KeyedSummingBatchBolt.java b/jstorm-core/src/main/java/backtype/storm/testing/KeyedSummingBatchBolt.java new file mode 100755 index 000000000..887eb4ec7 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/KeyedSummingBatchBolt.java @@ -0,0 +1,61 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.coordination.BatchOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseBatchBolt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; +import backtype.storm.utils.Utils; +import clojure.lang.Numbers; +import java.util.HashMap; +import java.util.Map; + +public class KeyedSummingBatchBolt extends BaseBatchBolt { + BatchOutputCollector _collector; + Object _id; + Map _sums = new HashMap(); + + @Override + public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, Object id) { + _collector = collector; + _id = id; + } + + @Override + public void execute(Tuple tuple) { + Object key = tuple.getValue(1); + Number curr = Utils.get(_sums, key, 0); + _sums.put(key, Numbers.add(curr, tuple.getValue(2))); + } + + @Override + public void finishBatch() { + for(Object key: _sums.keySet()) { + _collector.emit(new Values(_id, key, _sums.get(key))); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("tx", "key", "sum")); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/MemoryTransactionalSpout.java b/jstorm-core/src/main/java/backtype/storm/testing/MemoryTransactionalSpout.java new file mode 100644 index 000000000..3b492e162 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/MemoryTransactionalSpout.java @@ -0,0 +1,181 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.Config; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.transactional.TransactionAttempt; +import backtype.storm.coordination.BatchOutputCollector; +import backtype.storm.transactional.partitioned.IPartitionedTransactionalSpout; +import backtype.storm.transactional.partitioned.IPartitionedTransactionalSpout.Emitter; +import backtype.storm.tuple.Fields; +import backtype.storm.utils.RegisteredGlobalState; +import backtype.storm.utils.Utils; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class MemoryTransactionalSpout implements IPartitionedTransactionalSpout { + public static String TX_FIELD = MemoryTransactionalSpout.class.getName() + "/id"; + + private String _id; + private String _finishedPartitionsId; + private int _takeAmt; + private Fields _outFields; + private Map>> _initialPartitions; + + public MemoryTransactionalSpout(Map>> partitions, Fields outFields, int takeAmt) { + _id = RegisteredGlobalState.registerState(partitions); + Map finished = Collections.synchronizedMap(new HashMap()); + _finishedPartitionsId = RegisteredGlobalState.registerState(finished); + _takeAmt = takeAmt; + _outFields = outFields; + _initialPartitions = partitions; + } + + public boolean isExhaustedTuples() { + Map statuses = getFinishedStatuses(); + for(Integer partition: getQueues().keySet()) { + if(!statuses.containsKey(partition) || !getFinishedStatuses().get(partition)) { + return false; + } + } + return true; + } + + class Coordinator implements IPartitionedTransactionalSpout.Coordinator { + + @Override + public int numPartitions() { + return getQueues().size(); + } + + @Override + public boolean isReady() { + return true; + } + + @Override + public void close() { + } + } + + class Emitter implements IPartitionedTransactionalSpout.Emitter { + + Integer _maxSpoutPending; + Map _emptyPartitions = new HashMap(); + + public Emitter(Map conf) { + Object c = conf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING); + if(c==null) _maxSpoutPending = 1; + else _maxSpoutPending = Utils.getInt(c); + } + + + @Override + public MemoryTransactionalSpoutMeta emitPartitionBatchNew(TransactionAttempt tx, BatchOutputCollector collector, int partition, MemoryTransactionalSpoutMeta lastPartitionMeta) { + int index; + if(lastPartitionMeta==null) { + index = 0; + } else { + index = lastPartitionMeta.index + lastPartitionMeta.amt; + } + List> queue = getQueues().get(partition); + int total = queue.size(); + int left = total - index; + int toTake = Math.min(left, _takeAmt); + + MemoryTransactionalSpoutMeta ret = new MemoryTransactionalSpoutMeta(index, toTake); + emitPartitionBatch(tx, collector, partition, ret); + if(toTake==0) { + // this is a pretty hacky way to determine when all the partitions have been committed + // wait until we've emitted max-spout-pending empty partitions for the partition + int curr = Utils.get(_emptyPartitions, partition, 0) + 1; + _emptyPartitions.put(partition, curr); + if(curr > _maxSpoutPending) { + Map finishedStatuses = getFinishedStatuses(); + // will be null in remote mode + if(finishedStatuses!=null) { + finishedStatuses.put(partition, true); + } + } + } + return ret; + } + + @Override + public void emitPartitionBatch(TransactionAttempt tx, BatchOutputCollector collector, int partition, MemoryTransactionalSpoutMeta partitionMeta) { + List> queue = getQueues().get(partition); + for(int i=partitionMeta.index; i < partitionMeta.index + partitionMeta.amt; i++) { + List toEmit = new ArrayList(queue.get(i)); + toEmit.add(0, tx); + collector.emit(toEmit); + } + } + + @Override + public void close() { + } + } + + @Override + public IPartitionedTransactionalSpout.Coordinator getCoordinator(Map conf, TopologyContext context) { + return new Coordinator(); + } + + @Override + public IPartitionedTransactionalSpout.Emitter getEmitter(Map conf, TopologyContext context) { + return new Emitter(conf); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + List toDeclare = new ArrayList(_outFields.toList()); + toDeclare.add(0, TX_FIELD); + declarer.declare(new Fields(toDeclare)); + } + + @Override + public Map getComponentConfiguration() { + Config conf = new Config(); + conf.registerSerialization(MemoryTransactionalSpoutMeta.class); + return conf; + } + + public void startup() { + getFinishedStatuses().clear(); + } + + public void cleanup() { + RegisteredGlobalState.clearState(_id); + RegisteredGlobalState.clearState(_finishedPartitionsId); + } + + private Map>> getQueues() { + Map>> ret = (Map>>) RegisteredGlobalState.getState(_id); + if(ret!=null) return ret; + else return _initialPartitions; + } + + private Map getFinishedStatuses() { + return (Map) RegisteredGlobalState.getState(_finishedPartitionsId); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/MemoryTransactionalSpoutMeta.java b/jstorm-core/src/main/java/backtype/storm/testing/MemoryTransactionalSpoutMeta.java new file mode 100755 index 000000000..29681fb65 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/MemoryTransactionalSpoutMeta.java @@ -0,0 +1,38 @@ +/** + * 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 backtype.storm.testing; + +public class MemoryTransactionalSpoutMeta { + int index; + int amt; + + // for kryo compatibility + public MemoryTransactionalSpoutMeta() { + + } + + public MemoryTransactionalSpoutMeta(int index, int amt) { + this.index = index; + this.amt = amt; + } + + @Override + public String toString() { + return "index: " + index + "; amt: " + amt; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/MkClusterParam.java b/jstorm-core/src/main/java/backtype/storm/testing/MkClusterParam.java new file mode 100755 index 000000000..cd677c8cd --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/MkClusterParam.java @@ -0,0 +1,57 @@ +/** + * 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 backtype.storm.testing; + +import java.util.Map; + +/** + * The param arg for Testing.withSimulatedTimeCluster and Testing.withTrackedCluster + */ +public class MkClusterParam { + /** + * count of supervisors for the cluster. + */ + private Integer supervisors; + /** + * count of port for each supervisor + */ + private Integer portsPerSupervisor; + /** + * cluster config + */ + private Map daemonConf; + + public Integer getSupervisors() { + return supervisors; + } + public void setSupervisors(Integer supervisors) { + this.supervisors = supervisors; + } + public Integer getPortsPerSupervisor() { + return portsPerSupervisor; + } + public void setPortsPerSupervisor(Integer portsPerSupervisor) { + this.portsPerSupervisor = portsPerSupervisor; + } + public Map getDaemonConf() { + return daemonConf; + } + public void setDaemonConf(Map daemonConf) { + this.daemonConf = daemonConf; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/MkTupleParam.java b/jstorm-core/src/main/java/backtype/storm/testing/MkTupleParam.java new file mode 100755 index 000000000..34a8c68f0 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/MkTupleParam.java @@ -0,0 +1,51 @@ +/** + * 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 backtype.storm.testing; + +import java.util.ArrayList; +import java.util.List; + +public class MkTupleParam { + private String stream; + private String component; + private List fields; + + public String getStream() { + return stream; + } + public void setStream(String stream) { + this.stream = stream; + } + + public String getComponent() { + return component; + } + public void setComponent(String component) { + this.component = component; + } + + public List getFields() { + return fields; + } + public void setFields(String... fields) { + this.fields = new ArrayList(); + for (int i = 0; i < fields.length; i++) { + this.fields.add(fields[i]); + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/MockedSources.java b/jstorm-core/src/main/java/backtype/storm/testing/MockedSources.java new file mode 100755 index 000000000..1fd6b852f --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/MockedSources.java @@ -0,0 +1,60 @@ +/** + * 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 backtype.storm.testing; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import backtype.storm.tuple.Values; +import backtype.storm.utils.Utils; + +public class MockedSources { + /** + * mocked spout sources for the [spout, stream] pair. + */ + private Map> data = new HashMap>(); + + /** + * add mock data for the spout. + * + * @param spoutId the spout to be mocked + * @param streamId the stream of the spout to be mocked + * @param objects the mocked data + */ + public void addMockData(String spoutId, String streamId, Values... valueses) { + if (!data.containsKey(spoutId)) { + data.put(spoutId, new ArrayList()); + } + + List tuples = data.get(spoutId); + for (int i = 0; i < valueses.length; i++) { + FixedTuple tuple = new FixedTuple(streamId, valueses[i]); + tuples.add(tuple); + } + } + + public void addMockData(String spoutId, Values... valueses) { + this.addMockData(spoutId, Utils.DEFAULT_STREAM_ID, valueses); + } + + public Map> getData() { + return this.data; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/NGrouping.java b/jstorm-core/src/main/java/backtype/storm/testing/NGrouping.java new file mode 100755 index 000000000..785ed9271 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/NGrouping.java @@ -0,0 +1,50 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.generated.GlobalStreamId; +import backtype.storm.grouping.CustomStreamGrouping; +import backtype.storm.task.WorkerTopologyContext; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +public class NGrouping implements CustomStreamGrouping { + int _n; + List _outTasks; + + public NGrouping(int n) { + _n = n; + } + + @Override + public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List targetTasks) { + targetTasks = new ArrayList(targetTasks); + Collections.sort(targetTasks); + _outTasks = new ArrayList(); + for(int i=0; i<_n; i++) { + _outTasks.add(targetTasks.get(i)); + } + } + + @Override + public List chooseTasks(int taskId, List values) { + return _outTasks; + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/NonRichBoltTracker.java b/jstorm-core/src/main/java/backtype/storm/testing/NonRichBoltTracker.java new file mode 100755 index 000000000..ccbb67f3e --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/NonRichBoltTracker.java @@ -0,0 +1,51 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.task.IBolt; +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Tuple; +import backtype.storm.utils.RegisteredGlobalState; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + + +public class NonRichBoltTracker implements IBolt { + IBolt _delegate; + String _trackId; + + public NonRichBoltTracker(IBolt delegate, String id) { + _delegate = delegate; + _trackId = id; + } + + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + _delegate.prepare(stormConf, context, collector); + } + + public void execute(Tuple input) { + _delegate.execute(input); + Map stats = (Map) RegisteredGlobalState.getState(_trackId); + ((AtomicInteger) stats.get("processed")).incrementAndGet(); + } + + public void cleanup() { + _delegate.cleanup(); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/OpaqueMemoryTransactionalSpout.java b/jstorm-core/src/main/java/backtype/storm/testing/OpaqueMemoryTransactionalSpout.java new file mode 100755 index 000000000..1ff01b9fd --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/OpaqueMemoryTransactionalSpout.java @@ -0,0 +1,186 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.Config; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.transactional.TransactionAttempt; +import backtype.storm.coordination.BatchOutputCollector; +import backtype.storm.transactional.partitioned.IOpaquePartitionedTransactionalSpout; +import backtype.storm.tuple.Fields; +import backtype.storm.utils.RegisteredGlobalState; +import backtype.storm.utils.Utils; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * This spout only works in local mode. + */ +public class OpaqueMemoryTransactionalSpout implements IOpaquePartitionedTransactionalSpout { + public static String TX_FIELD = MemoryTransactionalSpout.class.getName() + "/id"; + + private String _id; + private String _finishedPartitionsId; + private String _disabledId; + private int _takeAmt; + private Fields _outFields; + + public OpaqueMemoryTransactionalSpout(Map>> partitions, Fields outFields, int takeAmt) { + _id = RegisteredGlobalState.registerState(partitions); + + Map finished = Collections.synchronizedMap(new HashMap()); + _finishedPartitionsId = RegisteredGlobalState.registerState(finished); + + Map disabled = Collections.synchronizedMap(new HashMap()); + _disabledId = RegisteredGlobalState.registerState(disabled); + + _takeAmt = takeAmt; + _outFields = outFields; + } + + public void setDisabled(Integer partition, boolean disabled) { + getDisabledStatuses().put(partition, disabled); + } + + public boolean isExhaustedTuples() { + Map statuses = getFinishedStatuses(); + for(Integer partition: getQueues().keySet()) { + if(!statuses.containsKey(partition) || !getFinishedStatuses().get(partition)) { + return false; + } + } + return true; + } + + @Override + public IOpaquePartitionedTransactionalSpout.Emitter getEmitter(Map conf, TopologyContext context) { + return new Emitter(conf); + } + + @Override + public IOpaquePartitionedTransactionalSpout.Coordinator getCoordinator(Map conf, TopologyContext context) { + return new Coordinator(); + } + + class Coordinator implements IOpaquePartitionedTransactionalSpout.Coordinator { + @Override + public boolean isReady() { + return true; + } + + @Override + public void close() { + } + } + + class Emitter implements IOpaquePartitionedTransactionalSpout.Emitter { + + Integer _maxSpoutPending; + Map _emptyPartitions = new HashMap(); + + public Emitter(Map conf) { + Object c = conf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING); + if(c==null) _maxSpoutPending = 1; + else _maxSpoutPending = Utils.getInt(c); + } + + + @Override + public MemoryTransactionalSpoutMeta emitPartitionBatch(TransactionAttempt tx, BatchOutputCollector collector, int partition, MemoryTransactionalSpoutMeta lastPartitionMeta) { + if(!Boolean.FALSE.equals(getDisabledStatuses().get(partition))) { + int index; + if(lastPartitionMeta==null) { + index = 0; + } else { + index = lastPartitionMeta.index + lastPartitionMeta.amt; + } + List> queue = getQueues().get(partition); + int total = queue.size(); + int left = total - index; + int toTake = Math.min(left, _takeAmt); + + MemoryTransactionalSpoutMeta ret = new MemoryTransactionalSpoutMeta(index, toTake); + for(int i=ret.index; i < ret.index + ret.amt; i++) { + List toEmit = new ArrayList(queue.get(i)); + toEmit.add(0, tx); + collector.emit(toEmit); + } + if(toTake==0) { + // this is a pretty hacky way to determine when all the partitions have been committed + // wait until we've emitted max-spout-pending empty partitions for the partition + int curr = Utils.get(_emptyPartitions, partition, 0) + 1; + _emptyPartitions.put(partition, curr); + if(curr > _maxSpoutPending) { + getFinishedStatuses().put(partition, true); + } + } + return ret; + } else { + return null; + } + } + + @Override + public void close() { + } + + @Override + public int numPartitions() { + return getQueues().size(); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + List toDeclare = new ArrayList(_outFields.toList()); + toDeclare.add(0, TX_FIELD); + declarer.declare(new Fields(toDeclare)); + } + + @Override + public Map getComponentConfiguration() { + Config conf = new Config(); + conf.registerSerialization(MemoryTransactionalSpoutMeta.class); + return conf; + } + + public void startup() { + getFinishedStatuses().clear(); + } + + public void cleanup() { + RegisteredGlobalState.clearState(_id); + RegisteredGlobalState.clearState(_finishedPartitionsId); + } + + private Map>> getQueues() { + return (Map>>) RegisteredGlobalState.getState(_id); + } + + private Map getFinishedStatuses() { + return (Map) RegisteredGlobalState.getState(_finishedPartitionsId); + } + + private Map getDisabledStatuses() { + return (Map) RegisteredGlobalState.getState(_disabledId); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/PrepareBatchBolt.java b/jstorm-core/src/main/java/backtype/storm/testing/PrepareBatchBolt.java new file mode 100755 index 000000000..0bd983379 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/PrepareBatchBolt.java @@ -0,0 +1,52 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseBasicBolt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.utils.Utils; +import java.util.ArrayList; +import java.util.List; + + +public class PrepareBatchBolt extends BaseBasicBolt { + Fields _outFields; + + public PrepareBatchBolt(Fields outFields) { + _outFields = outFields; + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(_outFields); + } + + @Override + public void execute(Tuple input, BasicOutputCollector collector) { + long id = Utils.secureRandomLong(); + List toEmit = new ArrayList(); + toEmit.add(id); + toEmit.addAll(input.getValues()); + collector.emit(toEmit); + } + + +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/PythonShellMetricsBolt.java b/jstorm-core/src/main/java/backtype/storm/testing/PythonShellMetricsBolt.java new file mode 100755 index 000000000..4b85ce8c5 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/PythonShellMetricsBolt.java @@ -0,0 +1,49 @@ +/* + * 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 backtype.storm.testing; + +import java.util.Map; + +import backtype.storm.metric.api.rpc.CountShellMetric; +import backtype.storm.task.OutputCollector; +import backtype.storm.task.ShellBolt; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.IRichBolt; +import backtype.storm.topology.OutputFieldsDeclarer; + +public class PythonShellMetricsBolt extends ShellBolt implements IRichBolt { + private static final long serialVersionUID = 1999209252187463355L; + + public PythonShellMetricsBolt(String[] command) { + super(command); + } + + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + super.prepare(stormConf, context, collector); + + CountShellMetric cMetric = new CountShellMetric(); + context.registerMetric("my-custom-shell-metric", cMetric, 5); + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + } + + public Map getComponentConfiguration() { + return null; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/PythonShellMetricsSpout.java b/jstorm-core/src/main/java/backtype/storm/testing/PythonShellMetricsSpout.java new file mode 100755 index 000000000..3ccf93541 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/PythonShellMetricsSpout.java @@ -0,0 +1,52 @@ +/* + * 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 backtype.storm.testing; + +import java.util.Map; + +import backtype.storm.metric.api.rpc.CountShellMetric; +import backtype.storm.spout.ShellSpout; +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.IRichSpout; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Fields; + +public class PythonShellMetricsSpout extends ShellSpout implements IRichSpout { + private static final long serialVersionUID = 1999209252187463355L; + + public PythonShellMetricsSpout(String[] command) { + super(command); + } + + @Override + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + super.open(conf, context, collector); + + CountShellMetric cMetric = new CountShellMetric(); + context.registerMetric("my-custom-shellspout-metric", cMetric, 5); + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("field1")); + } + + public Map getComponentConfiguration() { + return null; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/SingleUserSimpleTransport.java b/jstorm-core/src/main/java/backtype/storm/testing/SingleUserSimpleTransport.java new file mode 100755 index 000000000..4d25ac747 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/SingleUserSimpleTransport.java @@ -0,0 +1,37 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.security.auth.SimpleTransportPlugin; +import javax.security.auth.Subject; +import java.security.Principal; +import java.util.HashSet; + + +public class SingleUserSimpleTransport extends SimpleTransportPlugin { + @Override + protected Subject getDefaultSubject() { + HashSet principals = new HashSet(); + principals.add(new Principal() { + public String getName() { return "user"; } + public String toString() { return "user"; } + }); + return new Subject(true, principals, new HashSet(), new HashSet()); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/SpoutTracker.java b/jstorm-core/src/main/java/backtype/storm/testing/SpoutTracker.java new file mode 100755 index 000000000..75ba2b824 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/SpoutTracker.java @@ -0,0 +1,105 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.spout.ISpoutOutputCollector; +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.IRichSpout; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseRichSpout; +import backtype.storm.utils.RegisteredGlobalState; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + + +public class SpoutTracker extends BaseRichSpout { + IRichSpout _delegate; + SpoutTrackOutputCollector _tracker; + String _trackId; + + + private class SpoutTrackOutputCollector implements ISpoutOutputCollector { + public int transferred = 0; + public int emitted = 0; + public SpoutOutputCollector _collector; + + public SpoutTrackOutputCollector(SpoutOutputCollector collector) { + _collector = collector; + } + + private void recordSpoutEmit() { + Map stats = (Map) RegisteredGlobalState.getState(_trackId); + ((AtomicInteger) stats.get("spout-emitted")).incrementAndGet(); + + } + + public List emit(String streamId, List tuple, Object messageId) { + List ret = _collector.emit(streamId, tuple, messageId); + recordSpoutEmit(); + return ret; + } + + public void emitDirect(int taskId, String streamId, List tuple, Object messageId) { + _collector.emitDirect(taskId, streamId, tuple, messageId); + recordSpoutEmit(); + } + + @Override + public void reportError(Throwable error) { + _collector.reportError(error); + } + } + + + public SpoutTracker(IRichSpout delegate, String trackId) { + _delegate = delegate; + _trackId = trackId; + } + + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + _tracker = new SpoutTrackOutputCollector(collector); + _delegate.open(conf, context, new SpoutOutputCollector(_tracker)); + } + + public void close() { + _delegate.close(); + } + + public void nextTuple() { + _delegate.nextTuple(); + } + + public void ack(Object msgId) { + _delegate.ack(msgId); + Map stats = (Map) RegisteredGlobalState.getState(_trackId); + ((AtomicInteger) stats.get("processed")).incrementAndGet(); + } + + public void fail(Object msgId) { + _delegate.fail(msgId); + Map stats = (Map) RegisteredGlobalState.getState(_trackId); + ((AtomicInteger) stats.get("processed")).incrementAndGet(); + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + _delegate.declareOutputFields(declarer); + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/TestAggregatesCounter.java b/jstorm-core/src/main/java/backtype/storm/testing/TestAggregatesCounter.java new file mode 100755 index 000000000..e8c0a61eb --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/TestAggregatesCounter.java @@ -0,0 +1,63 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.topology.base.BaseRichBolt; +import backtype.storm.task.OutputCollector; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Fields; +import java.util.Map; +import backtype.storm.task.TopologyContext; +import java.util.HashMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import static backtype.storm.utils.Utils.tuple; + + +public class TestAggregatesCounter extends BaseRichBolt { + public static Logger LOG = LoggerFactory.getLogger(TestWordCounter.class); + + Map _counts; + OutputCollector _collector; + + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + _collector = collector; + _counts = new HashMap(); + } + + public void execute(Tuple input) { + String word = (String) input.getValues().get(0); + int count = (Integer) input.getValues().get(1); + _counts.put(word, count); + int globalCount = 0; + for(String w: _counts.keySet()) { + globalCount+=_counts.get(w); + } + _collector.emit(tuple(globalCount)); + _collector.ack(input); + } + + public void cleanup() { + + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("agg-global")); + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/testing/TestConfBolt.java b/jstorm-core/src/main/java/backtype/storm/testing/TestConfBolt.java new file mode 100755 index 000000000..5790fb309 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/TestConfBolt.java @@ -0,0 +1,62 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseBasicBolt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; +import java.util.Map; + + +public class TestConfBolt extends BaseBasicBolt { + Map _componentConf; + Map _conf; + + public TestConfBolt() { + this(null); + } + + public TestConfBolt(Map componentConf) { + _componentConf = componentConf; + } + + @Override + public void prepare(Map conf, TopologyContext context) { + _conf = conf; + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("conf", "value")); + } + + @Override + public void execute(Tuple input, BasicOutputCollector collector) { + String name = input.getString(0); + collector.emit(new Values(name, _conf.get(name))); + } + + @Override + public Map getComponentConfiguration() { + return _componentConf; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/TestEventLogSpout.java b/jstorm-core/src/main/java/backtype/storm/testing/TestEventLogSpout.java new file mode 100755 index 000000000..1570aeb6b --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/TestEventLogSpout.java @@ -0,0 +1,139 @@ +/** + * 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 backtype.storm.testing; + +import static backtype.storm.utils.Utils.get; +import backtype.storm.topology.OutputFieldsDeclarer; + +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicLong; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.base.BaseRichSpout; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; + +public class TestEventLogSpout extends BaseRichSpout { + public static Logger LOG = LoggerFactory.getLogger(TestEventLogSpout.class); + + private static final Map acked = new HashMap(); + private static final Map failed = new HashMap(); + + private String uid; + private long totalCount; + + SpoutOutputCollector _collector; + private long eventId = 0; + private long myCount; + private int source; + + public static int getNumAcked(String stormId) { + synchronized(acked) { + return get(acked, stormId, 0); + } + } + + public static int getNumFailed(String stormId) { + synchronized(failed) { + return get(failed, stormId, 0); + } + } + + public TestEventLogSpout(long totalCount) { + this.uid = UUID.randomUUID().toString(); + + synchronized(acked) { + acked.put(uid, 0); + } + synchronized(failed) { + failed.put(uid, 0); + } + + this.totalCount = totalCount; + } + + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + _collector = collector; + this.source = context.getThisTaskId(); + long taskCount = context.getComponentTasks(context.getThisComponentId()).size(); + myCount = totalCount / taskCount; + } + + public void close() { + + } + + public void cleanup() { + synchronized(acked) { + acked.remove(uid); + } + synchronized(failed) { + failed.remove(uid); + } + } + + public boolean completed() { + + int ackedAmt; + int failedAmt; + + synchronized(acked) { + ackedAmt = acked.get(uid); + } + synchronized(failed) { + failedAmt = failed.get(uid); + } + int totalEmitted = ackedAmt + failedAmt; + + if (totalEmitted >= totalCount) { + return true; + } + return false; + } + + public void nextTuple() { + if (eventId < myCount) { + eventId++; + _collector.emit(new Values(source, eventId), eventId); + } + } + + public void ack(Object msgId) { + synchronized(acked) { + int curr = get(acked, uid, 0); + acked.put(uid, curr+1); + } + } + + public void fail(Object msgId) { + synchronized(failed) { + int curr = get(failed, uid, 0); + failed.put(uid, curr+1); + } + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("source", "eventId")); + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/testing/TestEventOrderCheckBolt.java b/jstorm-core/src/main/java/backtype/storm/testing/TestEventOrderCheckBolt.java new file mode 100755 index 000000000..1f8036217 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/TestEventOrderCheckBolt.java @@ -0,0 +1,76 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.topology.OutputFieldsDeclarer; + +import java.util.HashMap; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.base.BaseRichBolt; +import backtype.storm.topology.base.BaseRichSpout; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; + +public class TestEventOrderCheckBolt extends BaseRichBolt { + public static Logger LOG = LoggerFactory.getLogger(TestEventOrderCheckBolt.class); + + private int _count; + OutputCollector _collector; + Map recentEventId = new HashMap(); + + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + _collector = collector; + _count = 0; + } + + public void execute(Tuple input) { + Integer sourceId = input.getInteger(0); + Long eventId = input.getLong(1); + Long recentEvent = recentEventId.get(sourceId); + + if (null != recentEvent && eventId <= recentEvent) { + String error = "Error: event id is not in strict order! event source Id: " + + sourceId + ", last event Id: " + recentEvent + ", current event Id: " + eventId; + + _collector.emit(input, new Values(error)); + } + recentEventId.put(sourceId, eventId); + + _collector.ack(input); + } + + public void cleanup() { + + } + + public Fields getOutputFields() { + return new Fields("error"); + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("error")); + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/testing/TestGlobalCount.java b/jstorm-core/src/main/java/backtype/storm/testing/TestGlobalCount.java new file mode 100755 index 000000000..5ef464a5e --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/TestGlobalCount.java @@ -0,0 +1,60 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.topology.base.BaseRichBolt; +import backtype.storm.task.OutputCollector; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Fields; +import java.util.Map; +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Values; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class TestGlobalCount extends BaseRichBolt { + public static Logger LOG = LoggerFactory.getLogger(TestWordCounter.class); + + private int _count; + OutputCollector _collector; + + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + _collector = collector; + _count = 0; + } + + public void execute(Tuple input) { + _count++; + _collector.emit(input, new Values(_count)); + _collector.ack(input); + } + + public void cleanup() { + + } + + public Fields getOutputFields() { + return new Fields("global-count"); + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("global-count")); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/TestJob.java b/jstorm-core/src/main/java/backtype/storm/testing/TestJob.java new file mode 100755 index 000000000..d41c36a6c --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/TestJob.java @@ -0,0 +1,41 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.ILocalCluster; + +/** + * This is the core interface for the storm java testing, usually + * we put our java unit testing logic in the run method. A sample + * code will be: + * + * Testing.withSimulatedTimeLocalCluster(new TestJob() { + * public void run(Cluster cluster) { + * // your testing logic here. + * } + * }); + */ +public interface TestJob { + /** + * run the testing logic with the cluster. + * + * @param cluster the cluster which created by Testing.withSimulatedTimeLocalCluster + * and Testing.withTrackedCluster. + */ + public void run(ILocalCluster cluster) throws Exception; +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/TestKryoDecorator.java b/jstorm-core/src/main/java/backtype/storm/testing/TestKryoDecorator.java new file mode 100755 index 000000000..96addd0c3 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/TestKryoDecorator.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.testing; + +import backtype.storm.serialization.IKryoDecorator; +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; + +public class TestKryoDecorator implements IKryoDecorator { + + public void decorate(Kryo k) { + k.register(TestSerObject.class); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/TestPlannerBolt.java b/jstorm-core/src/main/java/backtype/storm/testing/TestPlannerBolt.java new file mode 100755 index 000000000..0d30b2650 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/TestPlannerBolt.java @@ -0,0 +1,45 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.task.OutputCollector; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Fields; +import java.util.Map; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.base.BaseRichBolt; + + +public class TestPlannerBolt extends BaseRichBolt { + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + + } + + public void execute(Tuple input) { + + } + + public Fields getOutputFields() { + return new Fields("field1", "field2"); + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(getOutputFields()); + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/testing/TestPlannerSpout.java b/jstorm-core/src/main/java/backtype/storm/testing/TestPlannerSpout.java new file mode 100755 index 000000000..f4c27c07f --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/TestPlannerSpout.java @@ -0,0 +1,85 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.Config; +import backtype.storm.topology.OutputFieldsDeclarer; +import java.util.Map; +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.base.BaseRichSpout; +import backtype.storm.tuple.Fields; +import backtype.storm.utils.Utils; +import java.util.HashMap; + + +public class TestPlannerSpout extends BaseRichSpout { + boolean _isDistributed; + Fields _outFields; + + public TestPlannerSpout(Fields outFields, boolean isDistributed) { + _isDistributed = isDistributed; + _outFields = outFields; + } + + public TestPlannerSpout(boolean isDistributed) { + this(new Fields("field1", "field2"), isDistributed); + } + + public TestPlannerSpout(Fields outFields) { + this(outFields, true); + } + + public Fields getOutputFields() { + return _outFields; + } + + + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + + } + + public void close() { + + } + + public void nextTuple() { + Utils.sleep(100); + } + + public void ack(Object msgId){ + + } + + public void fail(Object msgId){ + + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(getOutputFields()); + } + + @Override + public Map getComponentConfiguration() { + Map ret = new HashMap(); + if(!_isDistributed) { + ret.put(Config.TOPOLOGY_MAX_TASK_PARALLELISM, 1); + } + return ret; + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/testing/TestSerObject.java b/jstorm-core/src/main/java/backtype/storm/testing/TestSerObject.java new file mode 100755 index 000000000..1c7706f79 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/TestSerObject.java @@ -0,0 +1,56 @@ +/** + * 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 backtype.storm.testing; + +import java.io.Serializable; + +public class TestSerObject implements Serializable { + public int f1; + public int f2; + + public TestSerObject(int f1, int f2) { + this.f1 = f1; + this.f2 = f2; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + f1; + result = prime * result + f2; + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + TestSerObject other = (TestSerObject) obj; + if (f1 != other.f1) + return false; + if (f2 != other.f2) + return false; + return true; + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/TestWordCounter.java b/jstorm-core/src/main/java/backtype/storm/testing/TestWordCounter.java new file mode 100755 index 000000000..551b054f8 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/TestWordCounter.java @@ -0,0 +1,61 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.topology.base.BaseBasicBolt; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Fields; +import java.util.Map; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.BasicOutputCollector; +import java.util.HashMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import static backtype.storm.utils.Utils.tuple; + + +public class TestWordCounter extends BaseBasicBolt { + public static Logger LOG = LoggerFactory.getLogger(TestWordCounter.class); + + Map _counts; + + public void prepare(Map stormConf, TopologyContext context) { + _counts = new HashMap(); + } + + public void execute(Tuple input, BasicOutputCollector collector) { + String word = (String) input.getValues().get(0); + int count = 0; + if(_counts.containsKey(word)) { + count = _counts.get(word); + } + count++; + _counts.put(word, count); + collector.emit(tuple(word, count)); + } + + public void cleanup() { + + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("word", "count")); + } + +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/testing/TestWordSpout.java b/jstorm-core/src/main/java/backtype/storm/testing/TestWordSpout.java new file mode 100755 index 000000000..745bf71b6 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/TestWordSpout.java @@ -0,0 +1,86 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.Config; +import backtype.storm.topology.OutputFieldsDeclarer; +import java.util.Map; +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.base.BaseRichSpout; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; +import backtype.storm.utils.Utils; +import java.util.HashMap; +import java.util.Random; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class TestWordSpout extends BaseRichSpout { + public static Logger LOG = LoggerFactory.getLogger(TestWordSpout.class); + boolean _isDistributed; + SpoutOutputCollector _collector; + + public TestWordSpout() { + this(true); + } + + public TestWordSpout(boolean isDistributed) { + _isDistributed = isDistributed; + } + + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + _collector = collector; + } + + public void close() { + + } + + public void nextTuple() { + Utils.sleep(100); + final String[] words = new String[] {"nathan", "mike", "jackson", "golda", "bertels"}; + final Random rand = new Random(); + final String word = words[rand.nextInt(words.length)]; + _collector.emit(new Values(word)); + } + + public void ack(Object msgId) { + + } + + public void fail(Object msgId) { + + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("word")); + } + + @Override + public Map getComponentConfiguration() { + if(!_isDistributed) { + Map ret = new HashMap(); + ret.put(Config.TOPOLOGY_MAX_TASK_PARALLELISM, 1); + return ret; + } else { + return null; + } + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/testing/TrackedTopology.java b/jstorm-core/src/main/java/backtype/storm/testing/TrackedTopology.java new file mode 100755 index 000000000..f2691b7ad --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/TrackedTopology.java @@ -0,0 +1,34 @@ +/** + * 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 backtype.storm.testing; + +import java.util.HashMap; +import java.util.Map; + +import backtype.storm.generated.StormTopology; +import clojure.lang.Keyword; + +public class TrackedTopology extends HashMap{ + public TrackedTopology(Map map) { + super(map); + } + + public StormTopology getTopology() { + return (StormTopology)get(Keyword.intern("topology")); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/testing/TupleCaptureBolt.java b/jstorm-core/src/main/java/backtype/storm/testing/TupleCaptureBolt.java new file mode 100755 index 000000000..e16357654 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/testing/TupleCaptureBolt.java @@ -0,0 +1,83 @@ +/** + * 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 backtype.storm.testing; + +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.IRichBolt; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Tuple; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; + + +public class TupleCaptureBolt implements IRichBolt { + public static transient Map>> emitted_tuples = new HashMap>>(); + + private String _name; + private OutputCollector _collector; + + public TupleCaptureBolt() { + _name = UUID.randomUUID().toString(); + emitted_tuples.put(_name, new HashMap>()); + } + + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + _collector = collector; + } + + public void execute(Tuple input) { + String component = input.getSourceComponent(); + Map> captured = emitted_tuples.get(_name); + if(!captured.containsKey(component)) { + captured.put(component, new ArrayList()); + } + captured.get(component).add(new FixedTuple(input.getSourceStreamId(), input.getValues())); + _collector.ack(input); + } + + public Map> getResults() { + return emitted_tuples.get(_name); + } + + public void cleanup() { + } + + public Map> getAndRemoveResults() { + return emitted_tuples.remove(_name); + } + + public Map> getAndClearResults() { + Map> ret = new HashMap>(emitted_tuples.get(_name)); + emitted_tuples.get(_name).clear(); + return ret; + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + } + + @Override + public Map getComponentConfiguration() { + return null; + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/BaseConfigurationDeclarer.java b/jstorm-core/src/main/java/backtype/storm/topology/BaseConfigurationDeclarer.java new file mode 100755 index 000000000..0c6732451 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/BaseConfigurationDeclarer.java @@ -0,0 +1,54 @@ +/** + * 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 backtype.storm.topology; + +import backtype.storm.Config; +import java.util.HashMap; +import java.util.Map; + +public abstract class BaseConfigurationDeclarer implements ComponentConfigurationDeclarer { + @Override + public T addConfiguration(String config, Object value) { + Map configMap = new HashMap(); + configMap.put(config, value); + return addConfigurations(configMap); + } + + @Override + public T setDebug(boolean debug) { + return addConfiguration(Config.TOPOLOGY_DEBUG, debug); + } + + @Override + public T setMaxTaskParallelism(Number val) { + if(val!=null) val = val.intValue(); + return addConfiguration(Config.TOPOLOGY_MAX_TASK_PARALLELISM, val); + } + + @Override + public T setMaxSpoutPending(Number val) { + if(val!=null) val = val.intValue(); + return addConfiguration(Config.TOPOLOGY_MAX_SPOUT_PENDING, val); + } + + @Override + public T setNumTasks(Number val) { + if(val!=null) val = val.intValue(); + return addConfiguration(Config.TOPOLOGY_TASKS, val); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/BasicBoltExecutor.java b/jstorm-core/src/main/java/backtype/storm/topology/BasicBoltExecutor.java new file mode 100755 index 000000000..6c9cdc18c --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/BasicBoltExecutor.java @@ -0,0 +1,67 @@ +/** + * 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 backtype.storm.topology; + +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Tuple; +import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class BasicBoltExecutor implements IRichBolt { + public static Logger LOG = LoggerFactory.getLogger(BasicBoltExecutor.class); + + private IBasicBolt _bolt; + private transient BasicOutputCollector _collector; + + public BasicBoltExecutor(IBasicBolt bolt) { + _bolt = bolt; + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + _bolt.declareOutputFields(declarer); + } + + + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + _bolt.prepare(stormConf, context); + _collector = new BasicOutputCollector(collector); + } + + public void execute(Tuple input) { + _collector.setContext(input); + try { + _bolt.execute(input, _collector); + _collector.getOutputter().ack(input); + } catch(FailedException e) { + if(e instanceof ReportedFailedException) { + _collector.reportError(e); + } + _collector.getOutputter().fail(input); + } + } + + public void cleanup() { + _bolt.cleanup(); + } + + public Map getComponentConfiguration() { + return _bolt.getComponentConfiguration(); + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/topology/BasicOutputCollector.java b/jstorm-core/src/main/java/backtype/storm/topology/BasicOutputCollector.java new file mode 100755 index 000000000..be1c24213 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/BasicOutputCollector.java @@ -0,0 +1,62 @@ +/** + * 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 backtype.storm.topology; + +import backtype.storm.task.IOutputCollector; +import backtype.storm.task.OutputCollector; +import backtype.storm.tuple.Tuple; +import backtype.storm.utils.Utils; +import java.util.List; + + +public class BasicOutputCollector implements IBasicOutputCollector { + private OutputCollector out; + private Tuple inputTuple; + + public BasicOutputCollector(OutputCollector out) { + this.out = out; + } + + public List emit(String streamId, List tuple) { + return out.emit(streamId, inputTuple, tuple); + } + + public List emit(List tuple) { + return emit(Utils.DEFAULT_STREAM_ID, tuple); + } + + public void setContext(Tuple inputTuple) { + this.inputTuple = inputTuple; + } + + public void emitDirect(int taskId, String streamId, List tuple) { + out.emitDirect(taskId, streamId, inputTuple, tuple); + } + + public void emitDirect(int taskId, List tuple) { + emitDirect(taskId, Utils.DEFAULT_STREAM_ID, tuple); + } + + protected IOutputCollector getOutputter() { + return out; + } + + public void reportError(Throwable t) { + out.reportError(t); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/BoltDeclarer.java b/jstorm-core/src/main/java/backtype/storm/topology/BoltDeclarer.java new file mode 100755 index 000000000..0c4b200f0 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/BoltDeclarer.java @@ -0,0 +1,26 @@ +/** + * 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 backtype.storm.topology; + +/** + * BoltDeclarer includes grouping APIs for storm topology. + * @see Concepts -Stream groupings- + */ +public interface BoltDeclarer extends InputDeclarer, ComponentConfigurationDeclarer { + +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/ComponentConfigurationDeclarer.java b/jstorm-core/src/main/java/backtype/storm/topology/ComponentConfigurationDeclarer.java new file mode 100755 index 000000000..d05dda0d1 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/ComponentConfigurationDeclarer.java @@ -0,0 +1,29 @@ +/** + * 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 backtype.storm.topology; + +import java.util.Map; + +public interface ComponentConfigurationDeclarer { + T addConfigurations(Map conf); + T addConfiguration(String config, Object value); + T setDebug(boolean debug); + T setMaxTaskParallelism(Number val); + T setMaxSpoutPending(Number val); + T setNumTasks(Number val); +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/FailedException.java b/jstorm-core/src/main/java/backtype/storm/topology/FailedException.java new file mode 100755 index 000000000..e174b5a23 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/FailedException.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.topology; + +public class FailedException extends RuntimeException { + public FailedException() { + super(); + } + + public FailedException(String msg) { + super(msg); + } + + public FailedException(String msg, Throwable cause) { + super(msg, cause); + } + + public FailedException(Throwable cause) { + super(cause); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/IBasicBolt.java b/jstorm-core/src/main/java/backtype/storm/topology/IBasicBolt.java new file mode 100755 index 000000000..3b24f4e32 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/IBasicBolt.java @@ -0,0 +1,33 @@ +/** + * 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 backtype.storm.topology; + +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Tuple; +import java.util.Map; + +public interface IBasicBolt extends IComponent { + void prepare(Map stormConf, TopologyContext context); + /** + * Process the input tuple and optionally emit new tuples based on the input tuple. + * + * All acking is managed for you. Throw a FailedException if you want to fail the tuple. + */ + void execute(Tuple input, BasicOutputCollector collector); + void cleanup(); +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/topology/IBasicOutputCollector.java b/jstorm-core/src/main/java/backtype/storm/topology/IBasicOutputCollector.java new file mode 100755 index 000000000..92d60d2c0 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/IBasicOutputCollector.java @@ -0,0 +1,26 @@ +/** + * 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 backtype.storm.topology; + +import java.util.List; + +public interface IBasicOutputCollector { + List emit(String streamId, List tuple); + void emitDirect(int taskId, String streamId, List tuple); + void reportError(Throwable t); +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/IComponent.java b/jstorm-core/src/main/java/backtype/storm/topology/IComponent.java new file mode 100755 index 000000000..560c96f8e --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/IComponent.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.topology; + +import java.io.Serializable; +import java.util.Map; + +/** + * Common methods for all possible components in a topology. This interface is used + * when defining topologies using the Java API. + */ +public interface IComponent extends Serializable { + + /** + * Declare the output schema for all the streams of this topology. + * + * @param declarer this is used to declare output stream ids, output fields, and whether or not each output stream is a direct stream + */ + void declareOutputFields(OutputFieldsDeclarer declarer); + + /** + * Declare configuration specific to this component. Only a subset of the "topology.*" configs can + * be overridden. The component configuration can be further overridden when constructing the + * topology using {@link TopologyBuilder} + * + */ + Map getComponentConfiguration(); + +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/IConfig.java b/jstorm-core/src/main/java/backtype/storm/topology/IConfig.java new file mode 100644 index 000000000..3ce9da78b --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/IConfig.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.topology; + +import java.util.Map; + +/* + * This interface is used to notify the update of user configuration + * for bolt and spout + */ +public interface IConfig { + public void updateConf(Map conf); +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/IRichBolt.java b/jstorm-core/src/main/java/backtype/storm/topology/IRichBolt.java new file mode 100755 index 000000000..d35244e78 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/IRichBolt.java @@ -0,0 +1,29 @@ +/** + * 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 backtype.storm.topology; + +import backtype.storm.task.IBolt; + +/** + * When writing topologies using Java, {@link IRichBolt} and {@link IRichSpout} are the main interfaces + * to use to implement components of the topology. + * + */ +public interface IRichBolt extends IBolt, IComponent { + +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/IRichSpout.java b/jstorm-core/src/main/java/backtype/storm/topology/IRichSpout.java new file mode 100755 index 000000000..b08864103 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/IRichSpout.java @@ -0,0 +1,29 @@ +/** + * 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 backtype.storm.topology; + +import backtype.storm.spout.ISpout; + +/** + * When writing topologies using Java, {@link IRichBolt} and {@link IRichSpout} are the main interfaces + * to use to implement components of the topology. + * + */ +public interface IRichSpout extends ISpout, IComponent { + +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/IRichStateSpout.java b/jstorm-core/src/main/java/backtype/storm/topology/IRichStateSpout.java new file mode 100755 index 000000000..edcc0ff96 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/IRichStateSpout.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.topology; + +import backtype.storm.state.IStateSpout; + + +public interface IRichStateSpout extends IStateSpout, IComponent { + +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/InputDeclarer.java b/jstorm-core/src/main/java/backtype/storm/topology/InputDeclarer.java new file mode 100755 index 000000000..33540de0d --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/InputDeclarer.java @@ -0,0 +1,187 @@ +/** + * 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 backtype.storm.topology; + +import backtype.storm.generated.GlobalStreamId; +import backtype.storm.generated.Grouping; +import backtype.storm.grouping.CustomStreamGrouping; +import backtype.storm.tuple.Fields; + + +public interface InputDeclarer { + /** + * The stream is partitioned by the fields specified in the grouping. + * @param componentId + * @param fields + * @return + */ + public T fieldsGrouping(String componentId, Fields fields); + + /** + * The stream is partitioned by the fields specified in the grouping. + * @param componentId + * @param streamId + * @param fields + * @return + */ + public T fieldsGrouping(String componentId, String streamId, Fields fields); + + /** + * The entire stream goes to a single one of the bolt's tasks. + * Specifically, it goes to the task with the lowest id. + * @param componentId + * @return + */ + public T globalGrouping(String componentId); + + /** + * The entire stream goes to a single one of the bolt's tasks. + * Specifically, it goes to the task with the lowest id. + * @param componentId + * @param streamId + * @return + */ + public T globalGrouping(String componentId, String streamId); + + /** + * Tuples are randomly distributed across the bolt's tasks in a way such that + * each bolt is guaranteed to get an equal number of tuples. + * @param componentId + * @return + */ + public T shuffleGrouping(String componentId); + + /** + * Tuples are randomly distributed across the bolt's tasks in a way such that + * each bolt is guaranteed to get an equal number of tuples. + * @param componentId + * @param streamId + * @return + */ + public T shuffleGrouping(String componentId, String streamId); + + /** + * If the target bolt has one or more tasks in the same worker process, + * tuples will be shuffled to just those in-process tasks. + * Otherwise, this acts like a normal shuffle grouping. + * @param componentId + * @return + */ + public T localOrShuffleGrouping(String componentId); + + /** + * If the target bolt has one or more tasks in the same worker process, + * tuples will be shuffled to just those in-process tasks. + * Otherwise, this acts like a normal shuffle grouping. + * @param componentId + * @param streamId + * @return + */ + public T localOrShuffleGrouping(String componentId, String streamId); + + public T localFirstGrouping(String componentId); + + public T localFirstGrouping(String componentId, String streamId); + /** + * This grouping specifies that you don't care how the stream is grouped. + * @param componentId + * @return + */ + public T noneGrouping(String componentId); + + /** + * This grouping specifies that you don't care how the stream is grouped. + * @param componentId + * @param streamId + * @return + */ + public T noneGrouping(String componentId, String streamId); + + /** + * The stream is replicated across all the bolt's tasks. Use this grouping with care. + * @param componentId + * @return + */ + public T allGrouping(String componentId); + + /** + * The stream is replicated across all the bolt's tasks. Use this grouping with care. + * @param componentId + * @param streamId + * @return + */ + public T allGrouping(String componentId, String streamId); + + /** + * A stream grouped this way means that the producer of the tuple decides + * which task of the consumer will receive this tuple. + * @param componentId + * @return + */ + public T directGrouping(String componentId); + + /** + * A stream grouped this way means that the producer of the tuple decides + * which task of the consumer will receive this tuple. + * @param componentId + * @param streamId + * @return + */ + public T directGrouping(String componentId, String streamId); + + /** + * Tuples are passed to two hashing functions and each target task is + * decided based on the comparison of the state of candidate nodes. + * @see https://melmeric.files.wordpress.com/2014/11/the-power-of-both-choices-practical-load-balancing-for-distributed-stream-processing-engines.pdf + * @param componentId + * @param fields + * @return + */ + public T partialKeyGrouping(String componentId, Fields fields); + + /** + * Tuples are passed to two hashing functions and each target task is + * decided based on the comparison of the state of candidate nodes. + * @see https://melmeric.files.wordpress.com/2014/11/the-power-of-both-choices-practical-load-balancing-for-distributed-stream-processing-engines.pdf + * @param componentId + * @param streamId + * @param fields + * @return + */ + public T partialKeyGrouping(String componentId, String streamId, Fields fields); + + /** + * A custom stream grouping by implementing the CustomStreamGrouping interface. + * @param componentId + * @param grouping + * @return + */ + public T customGrouping(String componentId, CustomStreamGrouping grouping); + + /** + * A custom stream grouping by implementing the CustomStreamGrouping interface. + * @param componentId + * @param streamId + * @param grouping + * @return + */ + public T customGrouping(String componentId, String streamId, CustomStreamGrouping grouping); + + public T grouping(GlobalStreamId id, Grouping grouping); + +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/OutputFieldsDeclarer.java b/jstorm-core/src/main/java/backtype/storm/topology/OutputFieldsDeclarer.java new file mode 100755 index 000000000..2ac47949c --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/OutputFieldsDeclarer.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.topology; + +import backtype.storm.tuple.Fields; + + +public interface OutputFieldsDeclarer { + /** + * Uses default stream id. + */ + public void declare(Fields fields); + public void declare(boolean direct, Fields fields); + + public void declareStream(String streamId, Fields fields); + public void declareStream(String streamId, boolean direct, Fields fields); +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/OutputFieldsGetter.java b/jstorm-core/src/main/java/backtype/storm/topology/OutputFieldsGetter.java new file mode 100755 index 000000000..0e7fd5956 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/OutputFieldsGetter.java @@ -0,0 +1,53 @@ +/** + * 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 backtype.storm.topology; + +import backtype.storm.generated.StreamInfo; +import backtype.storm.tuple.Fields; +import backtype.storm.utils.Utils; +import java.util.HashMap; +import java.util.Map; + +public class OutputFieldsGetter implements OutputFieldsDeclarer { + private Map _fields = new HashMap(); + + public void declare(Fields fields) { + declare(false, fields); + } + + public void declare(boolean direct, Fields fields) { + declareStream(Utils.DEFAULT_STREAM_ID, direct, fields); + } + + public void declareStream(String streamId, Fields fields) { + declareStream(streamId, false, fields); + } + + public void declareStream(String streamId, boolean direct, Fields fields) { + if(_fields.containsKey(streamId)) { + throw new IllegalArgumentException("Fields for " + streamId + " already set"); + } + _fields.put(streamId, new StreamInfo(fields.toList(), direct)); + } + + + public Map getFieldsDeclaration() { + return _fields; + } + +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/topology/ReportedFailedException.java b/jstorm-core/src/main/java/backtype/storm/topology/ReportedFailedException.java new file mode 100755 index 000000000..4e4ebe491 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/ReportedFailedException.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.topology; + +public class ReportedFailedException extends FailedException { + public ReportedFailedException() { + super(); + } + + public ReportedFailedException(String msg) { + super(msg); + } + + public ReportedFailedException(String msg, Throwable cause) { + super(msg, cause); + } + + public ReportedFailedException(Throwable cause) { + super(cause); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/SpoutDeclarer.java b/jstorm-core/src/main/java/backtype/storm/topology/SpoutDeclarer.java new file mode 100755 index 000000000..c0d825456 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/SpoutDeclarer.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.topology; + +public interface SpoutDeclarer extends ComponentConfigurationDeclarer { + +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/TopologyBuilder.java b/jstorm-core/src/main/java/backtype/storm/topology/TopologyBuilder.java new file mode 100755 index 000000000..c04e4491b --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/TopologyBuilder.java @@ -0,0 +1,384 @@ +/** + * 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 backtype.storm.topology; + +import backtype.storm.Config; +import backtype.storm.generated.Bolt; +import backtype.storm.generated.ComponentCommon; +import backtype.storm.generated.ComponentObject; +import backtype.storm.generated.GlobalStreamId; +import backtype.storm.generated.Grouping; +import backtype.storm.generated.NullStruct; +import backtype.storm.generated.SpoutSpec; +import backtype.storm.generated.StateSpoutSpec; +import backtype.storm.generated.StormTopology; +import backtype.storm.grouping.CustomStreamGrouping; +import backtype.storm.grouping.PartialKeyGrouping; +import backtype.storm.tuple.Fields; +import backtype.storm.utils.Utils; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Map; +import org.json.simple.JSONValue; + +/** + * TopologyBuilder exposes the Java API for specifying a topology for Storm + * to execute. Topologies are Thrift structures in the end, but since the Thrift API + * is so verbose, TopologyBuilder greatly eases the process of creating topologies. + * The template for creating and submitting a topology looks something like: + * + *
+ * TopologyBuilder builder = new TopologyBuilder();
+ *
+ * builder.setSpout("1", new TestWordSpout(true), 5);
+ * builder.setSpout("2", new TestWordSpout(true), 3);
+ * builder.setBolt("3", new TestWordCounter(), 3)
+ *          .fieldsGrouping("1", new Fields("word"))
+ *          .fieldsGrouping("2", new Fields("word"));
+ * builder.setBolt("4", new TestGlobalCount())
+ *          .globalGrouping("1");
+ *
+ * Map conf = new HashMap();
+ * conf.put(Config.TOPOLOGY_WORKERS, 4);
+ * 
+ * StormSubmitter.submitTopology("mytopology", conf, builder.createTopology());
+ * 
+ * + * Running the exact same topology in local mode (in process), and configuring it to log all tuples + * emitted, looks like the following. Note that it lets the topology run for 10 seconds + * before shutting down the local cluster. + * + *
+ * TopologyBuilder builder = new TopologyBuilder();
+ *
+ * builder.setSpout("1", new TestWordSpout(true), 5);
+ * builder.setSpout("2", new TestWordSpout(true), 3);
+ * builder.setBolt("3", new TestWordCounter(), 3)
+ *          .fieldsGrouping("1", new Fields("word"))
+ *          .fieldsGrouping("2", new Fields("word"));
+ * builder.setBolt("4", new TestGlobalCount())
+ *          .globalGrouping("1");
+ *
+ * Map conf = new HashMap();
+ * conf.put(Config.TOPOLOGY_WORKERS, 4);
+ * conf.put(Config.TOPOLOGY_DEBUG, true);
+ *
+ * LocalCluster cluster = new LocalCluster();
+ * cluster.submitTopology("mytopology", conf, builder.createTopology());
+ * Utils.sleep(10000);
+ * cluster.shutdown();
+ * 
+ * + *

The pattern for TopologyBuilder is to map component ids to components using the setSpout + * and setBolt methods. Those methods return objects that are then used to declare + * the inputs for that component.

+ */ +public class TopologyBuilder { + private Map _bolts = new HashMap(); + private Map _spouts = new HashMap(); + private Map _commons = new HashMap(); + +// private Map> _inputs = new HashMap>(); + + private Map _stateSpouts = new HashMap(); + + + public StormTopology createTopology() { + Map boltSpecs = new HashMap(); + Map spoutSpecs = new HashMap(); + for(String boltId: _bolts.keySet()) { + IRichBolt bolt = _bolts.get(boltId); + ComponentCommon common = getComponentCommon(boltId, bolt); + boltSpecs.put(boltId, new Bolt(ComponentObject.serialized_java(Utils.javaSerialize(bolt)), common)); + } + for(String spoutId: _spouts.keySet()) { + IRichSpout spout = _spouts.get(spoutId); + ComponentCommon common = getComponentCommon(spoutId, spout); + spoutSpecs.put(spoutId, new SpoutSpec(ComponentObject.serialized_java(Utils.javaSerialize(spout)), common)); + + } + return new StormTopology(spoutSpecs, + boltSpecs, + new HashMap()); + } + + /** + * Define a new bolt in this topology with parallelism of just one thread. + * + * @param id the id of this component. This id is referenced by other components that want to consume this bolt's outputs. + * @param bolt the bolt + * @return use the returned object to declare the inputs to this component + */ + public BoltDeclarer setBolt(String id, IRichBolt bolt) { + return setBolt(id, bolt, null); + } + + /** + * Define a new bolt in this topology with the specified amount of parallelism. + * + * @param id the id of this component. This id is referenced by other components that want to consume this bolt's outputs. + * @param bolt the bolt + * @param parallelism_hint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process somewhere around the cluster. + * @return use the returned object to declare the inputs to this component + */ + public BoltDeclarer setBolt(String id, IRichBolt bolt, Number parallelism_hint) { + validateUnusedId(id); + initCommon(id, bolt, parallelism_hint); + _bolts.put(id, bolt); + return new BoltGetter(id); + } + + /** + * Define a new bolt in this topology. This defines a basic bolt, which is a + * simpler to use but more restricted kind of bolt. Basic bolts are intended + * for non-aggregation processing and automate the anchoring/acking process to + * achieve proper reliability in the topology. + * + * @param id the id of this component. This id is referenced by other components that want to consume this bolt's outputs. + * @param bolt the basic bolt + * @return use the returned object to declare the inputs to this component + */ + public BoltDeclarer setBolt(String id, IBasicBolt bolt) { + return setBolt(id, bolt, null); + } + + /** + * Define a new bolt in this topology. This defines a basic bolt, which is a + * simpler to use but more restricted kind of bolt. Basic bolts are intended + * for non-aggregation processing and automate the anchoring/acking process to + * achieve proper reliability in the topology. + * + * @param id the id of this component. This id is referenced by other components that want to consume this bolt's outputs. + * @param bolt the basic bolt + * @param parallelism_hint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process somwehere around the cluster. + * @return use the returned object to declare the inputs to this component + */ + public BoltDeclarer setBolt(String id, IBasicBolt bolt, Number parallelism_hint) { + return setBolt(id, new BasicBoltExecutor(bolt), parallelism_hint); + } + + /** + * Define a new spout in this topology. + * + * @param id the id of this component. This id is referenced by other components that want to consume this spout's outputs. + * @param spout the spout + */ + public SpoutDeclarer setSpout(String id, IRichSpout spout) { + return setSpout(id, spout, null); + } + + /** + * Define a new spout in this topology with the specified parallelism. If the spout declares + * itself as non-distributed, the parallelism_hint will be ignored and only one task + * will be allocated to this component. + * + * @param id the id of this component. This id is referenced by other components that want to consume this spout's outputs. + * @param parallelism_hint the number of tasks that should be assigned to execute this spout. Each task will run on a thread in a process somwehere around the cluster. + * @param spout the spout + */ + public SpoutDeclarer setSpout(String id, IRichSpout spout, Number parallelism_hint) { + validateUnusedId(id); + initCommon(id, spout, parallelism_hint); + _spouts.put(id, spout); + return new SpoutGetter(id); + } + + public void setStateSpout(String id, IRichStateSpout stateSpout) { + setStateSpout(id, stateSpout, null); + } + + public void setStateSpout(String id, IRichStateSpout stateSpout, Number parallelism_hint) { + validateUnusedId(id); + // TODO: finish + } + + + private void validateUnusedId(String id) { + if(_bolts.containsKey(id)) { + throw new IllegalArgumentException("Bolt has already been declared for id " + id); + } + if(_spouts.containsKey(id)) { + throw new IllegalArgumentException("Spout has already been declared for id " + id); + } + if(_stateSpouts.containsKey(id)) { + throw new IllegalArgumentException("State spout has already been declared for id " + id); + } + } + + private ComponentCommon getComponentCommon(String id, IComponent component) { + ComponentCommon ret = new ComponentCommon(_commons.get(id)); + + OutputFieldsGetter getter = new OutputFieldsGetter(); + component.declareOutputFields(getter); + ret.set_streams(getter.getFieldsDeclaration()); + return ret; + } + + private void initCommon(String id, IComponent component, Number parallelism) { + ComponentCommon common = new ComponentCommon(); + common.set_inputs(new HashMap()); + if(parallelism!=null) { + common.set_parallelism_hint(parallelism.intValue()); + }else { + common.set_parallelism_hint(1); + } + Map conf = component.getComponentConfiguration(); + if(conf!=null) common.set_json_conf(JSONValue.toJSONString(conf)); + _commons.put(id, common); + } + + protected class ConfigGetter extends BaseConfigurationDeclarer { + String _id; + + public ConfigGetter(String id) { + _id = id; + } + + @Override + public T addConfigurations(Map conf) { + if(conf!=null && conf.containsKey(Config.TOPOLOGY_KRYO_REGISTER)) { + throw new IllegalArgumentException("Cannot set serializations for a component using fluent API"); + } + String currConf = _commons.get(_id).get_json_conf(); + _commons.get(_id).set_json_conf(mergeIntoJson(parseJson(currConf), conf)); + return (T) this; + } + } + + protected class SpoutGetter extends ConfigGetter implements SpoutDeclarer { + public SpoutGetter(String id) { + super(id); + } + } + + protected class BoltGetter extends ConfigGetter implements BoltDeclarer { + private String _boltId; + + public BoltGetter(String boltId) { + super(boltId); + _boltId = boltId; + } + + public BoltDeclarer fieldsGrouping(String componentId, Fields fields) { + return fieldsGrouping(componentId, Utils.DEFAULT_STREAM_ID, fields); + } + + public BoltDeclarer fieldsGrouping(String componentId, String streamId, Fields fields) { + return grouping(componentId, streamId, Grouping.fields(fields.toList())); + } + + public BoltDeclarer globalGrouping(String componentId) { + return globalGrouping(componentId, Utils.DEFAULT_STREAM_ID); + } + + public BoltDeclarer globalGrouping(String componentId, String streamId) { + return grouping(componentId, streamId, Grouping.fields(new ArrayList())); + } + + public BoltDeclarer shuffleGrouping(String componentId) { + return shuffleGrouping(componentId, Utils.DEFAULT_STREAM_ID); + } + + public BoltDeclarer shuffleGrouping(String componentId, String streamId) { + return grouping(componentId, streamId, Grouping.shuffle(new NullStruct())); + } + + public BoltDeclarer localOrShuffleGrouping(String componentId) { + return localOrShuffleGrouping(componentId, Utils.DEFAULT_STREAM_ID); + } + + public BoltDeclarer localOrShuffleGrouping(String componentId, String streamId) { + return grouping(componentId, streamId, Grouping.local_or_shuffle(new NullStruct())); + } + + @Override + public BoltDeclarer localFirstGrouping(String componentId) { + return localFirstGrouping(componentId, Utils.DEFAULT_STREAM_ID); + } + + @Override + public BoltDeclarer localFirstGrouping(String componentId, String streamId) { + return grouping(componentId, streamId, Grouping.localFirst(new NullStruct())); + } + + public BoltDeclarer noneGrouping(String componentId) { + return noneGrouping(componentId, Utils.DEFAULT_STREAM_ID); + } + + public BoltDeclarer noneGrouping(String componentId, String streamId) { + return grouping(componentId, streamId, Grouping.none(new NullStruct())); + } + + public BoltDeclarer allGrouping(String componentId) { + return allGrouping(componentId, Utils.DEFAULT_STREAM_ID); + } + + public BoltDeclarer allGrouping(String componentId, String streamId) { + return grouping(componentId, streamId, Grouping.all(new NullStruct())); + } + + public BoltDeclarer directGrouping(String componentId) { + return directGrouping(componentId, Utils.DEFAULT_STREAM_ID); + } + + public BoltDeclarer directGrouping(String componentId, String streamId) { + return grouping(componentId, streamId, Grouping.direct(new NullStruct())); + } + + private BoltDeclarer grouping(String componentId, String streamId, Grouping grouping) { + _commons.get(_boltId).put_to_inputs(new GlobalStreamId(componentId, streamId), grouping); + return this; + } + + @Override + public BoltDeclarer partialKeyGrouping(String componentId, Fields fields) { + return customGrouping(componentId, new PartialKeyGrouping(fields)); + } + + @Override + public BoltDeclarer partialKeyGrouping(String componentId, String streamId, Fields fields) { + return customGrouping(componentId, streamId, new PartialKeyGrouping(fields)); + } + + @Override + public BoltDeclarer customGrouping(String componentId, CustomStreamGrouping grouping) { + return customGrouping(componentId, Utils.DEFAULT_STREAM_ID, grouping); + } + + @Override + public BoltDeclarer customGrouping(String componentId, String streamId, CustomStreamGrouping grouping) { + return grouping(componentId, streamId, Grouping.custom_serialized(Utils.javaSerialize(grouping))); + } + + @Override + public BoltDeclarer grouping(GlobalStreamId id, Grouping grouping) { + return grouping(id.get_componentId(), id.get_streamId(), grouping); + } + } + + private static Map parseJson(String json) { + if(json==null) return new HashMap(); + else return (Map) JSONValue.parse(json); + } + + private static String mergeIntoJson(Map into, Map newMap) { + Map res = new HashMap(into); + if(newMap!=null) res.putAll(newMap); + return JSONValue.toJSONString(res); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/base/BaseBasicBolt.java b/jstorm-core/src/main/java/backtype/storm/topology/base/BaseBasicBolt.java new file mode 100755 index 000000000..e585ee64d --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/base/BaseBasicBolt.java @@ -0,0 +1,33 @@ +/** + * 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 backtype.storm.topology.base; + +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.IBasicBolt; +import java.util.Map; + +public abstract class BaseBasicBolt extends BaseComponent implements IBasicBolt { + + @Override + public void prepare(Map stormConf, TopologyContext context) { + } + + @Override + public void cleanup() { + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/base/BaseBatchBolt.java b/jstorm-core/src/main/java/backtype/storm/topology/base/BaseBatchBolt.java new file mode 100755 index 000000000..320694108 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/base/BaseBatchBolt.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.topology.base; + +import backtype.storm.coordination.IBatchBolt; +import java.util.Map; + +public abstract class BaseBatchBolt extends BaseComponent implements IBatchBolt { + +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/base/BaseComponent.java b/jstorm-core/src/main/java/backtype/storm/topology/base/BaseComponent.java new file mode 100755 index 000000000..8afcdaaf2 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/base/BaseComponent.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.topology.base; + +import backtype.storm.topology.IComponent; +import java.util.Map; + +public abstract class BaseComponent implements IComponent { + @Override + public Map getComponentConfiguration() { + return null; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/base/BaseOpaquePartitionedTransactionalSpout.java b/jstorm-core/src/main/java/backtype/storm/topology/base/BaseOpaquePartitionedTransactionalSpout.java new file mode 100755 index 000000000..2d20a48aa --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/base/BaseOpaquePartitionedTransactionalSpout.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.topology.base; + +import backtype.storm.transactional.partitioned.IOpaquePartitionedTransactionalSpout; + + +public abstract class BaseOpaquePartitionedTransactionalSpout extends BaseComponent implements IOpaquePartitionedTransactionalSpout { + +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/base/BasePartitionedTransactionalSpout.java b/jstorm-core/src/main/java/backtype/storm/topology/base/BasePartitionedTransactionalSpout.java new file mode 100755 index 000000000..4bd344c82 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/base/BasePartitionedTransactionalSpout.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.topology.base; + +import backtype.storm.transactional.partitioned.IPartitionedTransactionalSpout; +import java.util.Map; + +public abstract class BasePartitionedTransactionalSpout extends BaseComponent implements IPartitionedTransactionalSpout { + +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/base/BaseRichBolt.java b/jstorm-core/src/main/java/backtype/storm/topology/base/BaseRichBolt.java new file mode 100755 index 000000000..266736e18 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/base/BaseRichBolt.java @@ -0,0 +1,26 @@ +/** + * 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 backtype.storm.topology.base; + +import backtype.storm.topology.IRichBolt; + +public abstract class BaseRichBolt extends BaseComponent implements IRichBolt { + @Override + public void cleanup() { + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/base/BaseRichSpout.java b/jstorm-core/src/main/java/backtype/storm/topology/base/BaseRichSpout.java new file mode 100755 index 000000000..37513b72d --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/base/BaseRichSpout.java @@ -0,0 +1,50 @@ +/** + * 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. + */ +/* + * To change this template, choose Tools | Templates + * and open the template in the editor. + */ +package backtype.storm.topology.base; + +import backtype.storm.topology.IRichSpout; + +/** + * + * @author nathan + */ +public abstract class BaseRichSpout extends BaseComponent implements IRichSpout { + @Override + public void close() { + } + + @Override + public void activate() { + } + + @Override + public void deactivate() { + } + + @Override + public void ack(Object msgId) { + } + + @Override + public void fail(Object msgId) { + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/base/BaseTransactionalBolt.java b/jstorm-core/src/main/java/backtype/storm/topology/base/BaseTransactionalBolt.java new file mode 100755 index 000000000..b6451e9ae --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/base/BaseTransactionalBolt.java @@ -0,0 +1,24 @@ +/** + * 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 backtype.storm.topology.base; + +import backtype.storm.transactional.TransactionAttempt; + +public abstract class BaseTransactionalBolt extends BaseBatchBolt { + +} diff --git a/jstorm-core/src/main/java/backtype/storm/topology/base/BaseTransactionalSpout.java b/jstorm-core/src/main/java/backtype/storm/topology/base/BaseTransactionalSpout.java new file mode 100755 index 000000000..704a95bb4 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/topology/base/BaseTransactionalSpout.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.topology.base; + +import backtype.storm.transactional.ITransactionalSpout; +import java.util.Map; + +public abstract class BaseTransactionalSpout extends BaseComponent implements ITransactionalSpout { + +} diff --git a/jstorm-core/src/main/java/backtype/storm/transactional/ICommitter.java b/jstorm-core/src/main/java/backtype/storm/transactional/ICommitter.java new file mode 100755 index 000000000..859bad242 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/transactional/ICommitter.java @@ -0,0 +1,26 @@ +/** + * 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 backtype.storm.transactional; + +/** + * This marks an IBatchBolt within a transactional topology as a committer. This causes the + * finishBatch method to be called in order of the transactions. + */ +public interface ICommitter { + +} diff --git a/jstorm-core/src/main/java/backtype/storm/transactional/ICommitterTransactionalSpout.java b/jstorm-core/src/main/java/backtype/storm/transactional/ICommitterTransactionalSpout.java new file mode 100755 index 000000000..5441ee22a --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/transactional/ICommitterTransactionalSpout.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.transactional; + +import backtype.storm.task.TopologyContext; +import java.util.Map; + + +public interface ICommitterTransactionalSpout extends ITransactionalSpout { + public interface Emitter extends ITransactionalSpout.Emitter { + void commit(TransactionAttempt attempt); + } + + @Override + public Emitter getEmitter(Map conf, TopologyContext context); +} diff --git a/jstorm-core/src/main/java/backtype/storm/transactional/ITransactionalSpout.java b/jstorm-core/src/main/java/backtype/storm/transactional/ITransactionalSpout.java new file mode 100755 index 000000000..320749372 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/transactional/ITransactionalSpout.java @@ -0,0 +1,94 @@ +/** + * 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 backtype.storm.transactional; + +import backtype.storm.coordination.BatchOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.IComponent; +import java.math.BigInteger; +import java.util.Map; + +public interface ITransactionalSpout extends IComponent { + public interface Coordinator { + /** + * Create metadata for this particular transaction id which has never + * been emitted before. The metadata should contain whatever is necessary + * to be able to replay the exact batch for the transaction at a later point. + * + * The metadata is stored in Zookeeper. + * + * Storm uses the Kryo serializations configured in the component configuration + * for this spout to serialize and deserialize the metadata. + * + * @param txid The id of the transaction. + * @param prevMetadata The metadata of the previous transaction + * @return the metadata for this new transaction + */ + X initializeTransaction(BigInteger txid, X prevMetadata); + + /** + * Returns true if its ok to emit start a new transaction, false otherwise (will skip this transaction). + * + * You should sleep here if you want a delay between asking for the next transaction (this will be called + * repeatedly in a loop). + */ + boolean isReady(); + + /** + * Release any resources from this coordinator. + */ + void close(); + } + + public interface Emitter { + /** + * Emit a batch for the specified transaction attempt and metadata for the transaction. The metadata + * was created by the Coordinator in the initializeTranaction method. This method must always emit + * the same batch of tuples across all tasks for the same transaction id. + * + * The first field of all emitted tuples must contain the provided TransactionAttempt. + * + */ + void emitBatch(TransactionAttempt tx, X coordinatorMeta, BatchOutputCollector collector); + + /** + * Any state for transactions prior to the provided transaction id can be safely cleaned up, so this + * method should clean up that state. + */ + void cleanupBefore(BigInteger txid); + + /** + * Release any resources held by this emitter. + */ + void close(); + } + + /** + * The coordinator for a TransactionalSpout runs in a single thread and indicates when batches + * of tuples should be emitted and when transactions should commit. The Coordinator that you provide + * in a TransactionalSpout provides metadata for each transaction so that the transactions can be replayed. + */ + Coordinator getCoordinator(Map conf, TopologyContext context); + + /** + * The emitter for a TransactionalSpout runs as many tasks across the cluster. Emitters are responsible for + * emitting batches of tuples for a transaction and must ensure that the same batch of tuples is always + * emitted for the same transaction id. + */ + Emitter getEmitter(Map conf, TopologyContext context); +} diff --git a/jstorm-core/src/main/java/backtype/storm/transactional/TransactionAttempt.java b/jstorm-core/src/main/java/backtype/storm/transactional/TransactionAttempt.java new file mode 100755 index 000000000..80bbb0ed4 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/transactional/TransactionAttempt.java @@ -0,0 +1,61 @@ +/** + * 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 backtype.storm.transactional; + +import java.math.BigInteger; + +public class TransactionAttempt { + BigInteger _txid; + long _attemptId; + + + // for kryo compatibility + public TransactionAttempt() { + + } + + public TransactionAttempt(BigInteger txid, long attemptId) { + _txid = txid; + _attemptId = attemptId; + } + + public BigInteger getTransactionId() { + return _txid; + } + + public long getAttemptId() { + return _attemptId; + } + + @Override + public int hashCode() { + return _txid.hashCode(); + } + + @Override + public boolean equals(Object o) { + if(!(o instanceof TransactionAttempt)) return false; + TransactionAttempt other = (TransactionAttempt) o; + return _txid.equals(other._txid) && _attemptId == other._attemptId; + } + + @Override + public String toString() { + return "" + _txid + ":" + _attemptId; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/transactional/TransactionalSpoutBatchExecutor.java b/jstorm-core/src/main/java/backtype/storm/transactional/TransactionalSpoutBatchExecutor.java new file mode 100755 index 000000000..53aacae88 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/transactional/TransactionalSpoutBatchExecutor.java @@ -0,0 +1,96 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.transactional; + +import backtype.storm.coordination.BatchOutputCollectorImpl; +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.FailedException; +import backtype.storm.topology.IRichBolt; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Tuple; +import java.math.BigInteger; +import java.util.Map; +import java.util.TreeMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TransactionalSpoutBatchExecutor implements IRichBolt { + public static Logger LOG = LoggerFactory.getLogger(TransactionalSpoutBatchExecutor.class); + + BatchOutputCollectorImpl _collector; + ITransactionalSpout _spout; + ITransactionalSpout.Emitter _emitter; + + TreeMap _activeTransactions = new TreeMap(); + + public TransactionalSpoutBatchExecutor(ITransactionalSpout spout) { + _spout = spout; + } + + @Override + public void prepare(Map conf, TopologyContext context, OutputCollector collector) { + _collector = new BatchOutputCollectorImpl(collector); + _emitter = _spout.getEmitter(conf, context); + } + + @Override + public void execute(Tuple input) { + TransactionAttempt attempt = (TransactionAttempt) input.getValue(0); + try { + if(input.getSourceStreamId().equals(TransactionalSpoutCoordinator.TRANSACTION_COMMIT_STREAM_ID)) { + if(attempt.equals(_activeTransactions.get(attempt.getTransactionId()))) { + ((ICommitterTransactionalSpout.Emitter) _emitter).commit(attempt); + _activeTransactions.remove(attempt.getTransactionId()); + _collector.ack(input); + } else { + _collector.fail(input); + } + } else { + _emitter.emitBatch(attempt, input.getValue(1), _collector); + _activeTransactions.put(attempt.getTransactionId(), attempt); + _collector.ack(input); + BigInteger committed = (BigInteger) input.getValue(2); + if(committed!=null) { + // valid to delete before what's been committed since + // those batches will never be accessed again + _activeTransactions.headMap(committed).clear(); + _emitter.cleanupBefore(committed); + } + } + } catch(FailedException e) { + LOG.warn("Failed to emit batch for transaction", e); + _collector.fail(input); + } + } + + @Override + public void cleanup() { + _emitter.close(); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + _spout.declareOutputFields(declarer); + } + + @Override + public Map getComponentConfiguration() { + return _spout.getComponentConfiguration(); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/transactional/TransactionalSpoutCoordinator.java b/jstorm-core/src/main/java/backtype/storm/transactional/TransactionalSpoutCoordinator.java new file mode 100755 index 000000000..f7ce53468 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/transactional/TransactionalSpoutCoordinator.java @@ -0,0 +1,217 @@ +/** + * 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 backtype.storm.transactional; + +import backtype.storm.Config; +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.FailedException; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseRichSpout; +import backtype.storm.transactional.state.RotatingTransactionalState; +import backtype.storm.transactional.state.TransactionalState; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; +import backtype.storm.utils.Utils; +import java.math.BigInteger; +import java.util.Map; +import java.util.TreeMap; +import java.util.Random; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TransactionalSpoutCoordinator extends BaseRichSpout { + public static final Logger LOG = LoggerFactory.getLogger(TransactionalSpoutCoordinator.class); + + public static final BigInteger INIT_TXID = BigInteger.ONE; + + + public static final String TRANSACTION_BATCH_STREAM_ID = TransactionalSpoutCoordinator.class.getName() + "/batch"; + public static final String TRANSACTION_COMMIT_STREAM_ID = TransactionalSpoutCoordinator.class.getName() + "/commit"; + + private static final String CURRENT_TX = "currtx"; + private static final String META_DIR = "meta"; + + private ITransactionalSpout _spout; + private ITransactionalSpout.Coordinator _coordinator; + private TransactionalState _state; + private RotatingTransactionalState _coordinatorState; + + TreeMap _activeTx = new TreeMap(); + + private SpoutOutputCollector _collector; + private Random _rand; + BigInteger _currTransaction; + int _maxTransactionActive; + StateInitializer _initializer; + + + public TransactionalSpoutCoordinator(ITransactionalSpout spout) { + _spout = spout; + } + + public ITransactionalSpout getSpout() { + return _spout; + } + + @Override + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + _rand = new Random(Utils.secureRandomLong()); + _state = TransactionalState.newCoordinatorState(conf, (String) conf.get(Config.TOPOLOGY_TRANSACTIONAL_ID), _spout.getComponentConfiguration()); + _coordinatorState = new RotatingTransactionalState(_state, META_DIR, true); + _collector = collector; + _coordinator = _spout.getCoordinator(conf, context); + _currTransaction = getStoredCurrTransaction(_state); + Object active = conf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING); + if(active==null) { + _maxTransactionActive = 1; + } else { + _maxTransactionActive = Utils.getInt(active); + } + _initializer = new StateInitializer(); + } + + @Override + public void close() { + _state.close(); + } + + @Override + public void nextTuple() { + sync(); + } + + @Override + public void ack(Object msgId) { + TransactionAttempt tx = (TransactionAttempt) msgId; + TransactionStatus status = _activeTx.get(tx.getTransactionId()); + if(status!=null && tx.equals(status.attempt)) { + if(status.status==AttemptStatus.PROCESSING) { + status.status = AttemptStatus.PROCESSED; + } else if(status.status==AttemptStatus.COMMITTING) { + _activeTx.remove(tx.getTransactionId()); + _coordinatorState.cleanupBefore(tx.getTransactionId()); + _currTransaction = nextTransactionId(tx.getTransactionId()); + _state.setData(CURRENT_TX, _currTransaction); + } + sync(); + } + } + + @Override + public void fail(Object msgId) { + TransactionAttempt tx = (TransactionAttempt) msgId; + TransactionStatus stored = _activeTx.remove(tx.getTransactionId()); + if(stored!=null && tx.equals(stored.attempt)) { + _activeTx.tailMap(tx.getTransactionId()).clear(); + sync(); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + // in partitioned example, in case an emitter task receives a later transaction than it's emitted so far, + // when it sees the earlier txid it should know to emit nothing + declarer.declareStream(TRANSACTION_BATCH_STREAM_ID, new Fields("tx", "tx-meta", "committed-txid")); + declarer.declareStream(TRANSACTION_COMMIT_STREAM_ID, new Fields("tx")); + } + + private void sync() { + // note that sometimes the tuples active may be less than max_spout_pending, e.g. + // max_spout_pending = 3 + // tx 1, 2, 3 active, tx 2 is acked. there won't be a commit for tx 2 (because tx 1 isn't committed yet), + // and there won't be a batch for tx 4 because there's max_spout_pending tx active + TransactionStatus maybeCommit = _activeTx.get(_currTransaction); + if(maybeCommit!=null && maybeCommit.status == AttemptStatus.PROCESSED) { + maybeCommit.status = AttemptStatus.COMMITTING; + _collector.emit(TRANSACTION_COMMIT_STREAM_ID, new Values(maybeCommit.attempt), maybeCommit.attempt); + } + + try { + if(_activeTx.size() < _maxTransactionActive) { + BigInteger curr = _currTransaction; + for(int i=0; i<_maxTransactionActive; i++) { + if((_coordinatorState.hasCache(curr) || _coordinator.isReady()) + && !_activeTx.containsKey(curr)) { + TransactionAttempt attempt = new TransactionAttempt(curr, _rand.nextLong()); + Object state = _coordinatorState.getState(curr, _initializer); + _activeTx.put(curr, new TransactionStatus(attempt)); + _collector.emit(TRANSACTION_BATCH_STREAM_ID, new Values(attempt, state, previousTransactionId(_currTransaction)), attempt); + } + curr = nextTransactionId(curr); + } + } + } catch(FailedException e) { + LOG.warn("Failed to get metadata for a transaction", e); + } + } + + @Override + public Map getComponentConfiguration() { + Config ret = new Config(); + ret.setMaxTaskParallelism(1); + return ret; + } + + private static enum AttemptStatus { + PROCESSING, + PROCESSED, + COMMITTING + } + + private static class TransactionStatus { + TransactionAttempt attempt; + AttemptStatus status; + + public TransactionStatus(TransactionAttempt attempt) { + this.attempt = attempt; + this.status = AttemptStatus.PROCESSING; + } + + @Override + public String toString() { + return attempt.toString() + " <" + status.toString() + ">"; + } + } + + + private BigInteger nextTransactionId(BigInteger id) { + return id.add(BigInteger.ONE); + } + + private BigInteger previousTransactionId(BigInteger id) { + if(id.equals(INIT_TXID)) { + return null; + } else { + return id.subtract(BigInteger.ONE); + } + } + + private BigInteger getStoredCurrTransaction(TransactionalState state) { + BigInteger ret = (BigInteger) state.getData(CURRENT_TX); + if(ret==null) return INIT_TXID; + else return ret; + } + + private class StateInitializer implements RotatingTransactionalState.StateInitializer { + @Override + public Object init(BigInteger txid, Object lastState) { + return _coordinator.initializeTransaction(txid, lastState); + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/transactional/TransactionalTopologyBuilder.java b/jstorm-core/src/main/java/backtype/storm/transactional/TransactionalTopologyBuilder.java new file mode 100755 index 000000000..98d116339 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/transactional/TransactionalTopologyBuilder.java @@ -0,0 +1,553 @@ +/** + * 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 backtype.storm.transactional; + +import backtype.storm.coordination.IBatchBolt; +import backtype.storm.coordination.BatchBoltExecutor; +import backtype.storm.Config; +import backtype.storm.Constants; +import backtype.storm.coordination.CoordinatedBolt; +import backtype.storm.coordination.CoordinatedBolt.IdStreamSpec; +import backtype.storm.coordination.CoordinatedBolt.SourceArgs; +import backtype.storm.generated.GlobalStreamId; +import backtype.storm.generated.Grouping; +import backtype.storm.generated.StormTopology; +import backtype.storm.grouping.CustomStreamGrouping; +import backtype.storm.grouping.PartialKeyGrouping; +import backtype.storm.topology.BaseConfigurationDeclarer; +import backtype.storm.topology.BasicBoltExecutor; +import backtype.storm.topology.BoltDeclarer; +import backtype.storm.topology.IBasicBolt; +import backtype.storm.topology.IRichBolt; +import backtype.storm.topology.InputDeclarer; +import backtype.storm.topology.SpoutDeclarer; +import backtype.storm.topology.TopologyBuilder; +import backtype.storm.transactional.partitioned.IOpaquePartitionedTransactionalSpout; +import backtype.storm.transactional.partitioned.IPartitionedTransactionalSpout; +import backtype.storm.transactional.partitioned.OpaquePartitionedTransactionalSpoutExecutor; +import backtype.storm.transactional.partitioned.PartitionedTransactionalSpoutExecutor; +import backtype.storm.tuple.Fields; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Trident subsumes the functionality provided by transactional topologies, so this + * class is deprecated. + * + */ +@Deprecated +public class TransactionalTopologyBuilder { + String _id; + String _spoutId; + ITransactionalSpout _spout; + Map _bolts = new HashMap(); + Integer _spoutParallelism; + List _spoutConfs = new ArrayList(); + + // id is used to store the state of this transactionalspout in zookeeper + // it would be very dangerous to have 2 topologies active with the same id in the same cluster + public TransactionalTopologyBuilder(String id, String spoutId, ITransactionalSpout spout, Number spoutParallelism) { + _id = id; + _spoutId = spoutId; + _spout = spout; + _spoutParallelism = (spoutParallelism == null) ? null : spoutParallelism.intValue(); + } + + public TransactionalTopologyBuilder(String id, String spoutId, ITransactionalSpout spout) { + this(id, spoutId, spout, null); + } + + public TransactionalTopologyBuilder(String id, String spoutId, IPartitionedTransactionalSpout spout, Number spoutParallelism) { + this(id, spoutId, new PartitionedTransactionalSpoutExecutor(spout), spoutParallelism); + } + + public TransactionalTopologyBuilder(String id, String spoutId, IPartitionedTransactionalSpout spout) { + this(id, spoutId, spout, null); + } + + public TransactionalTopologyBuilder(String id, String spoutId, IOpaquePartitionedTransactionalSpout spout, Number spoutParallelism) { + this(id, spoutId, new OpaquePartitionedTransactionalSpoutExecutor(spout), spoutParallelism); + } + + public TransactionalTopologyBuilder(String id, String spoutId, IOpaquePartitionedTransactionalSpout spout) { + this(id, spoutId, spout, null); + } + + public SpoutDeclarer getSpoutDeclarer() { + return new SpoutDeclarerImpl(); + } + + public BoltDeclarer setBolt(String id, IBatchBolt bolt) { + return setBolt(id, bolt, null); + } + + public BoltDeclarer setBolt(String id, IBatchBolt bolt, Number parallelism) { + return setBolt(id, new BatchBoltExecutor(bolt), parallelism, bolt instanceof ICommitter); + } + + public BoltDeclarer setCommitterBolt(String id, IBatchBolt bolt) { + return setCommitterBolt(id, bolt, null); + } + + public BoltDeclarer setCommitterBolt(String id, IBatchBolt bolt, Number parallelism) { + return setBolt(id, new BatchBoltExecutor(bolt), parallelism, true); + } + + public BoltDeclarer setBolt(String id, IBasicBolt bolt) { + return setBolt(id, bolt, null); + } + + public BoltDeclarer setBolt(String id, IBasicBolt bolt, Number parallelism) { + return setBolt(id, new BasicBoltExecutor(bolt), parallelism, false); + } + + private BoltDeclarer setBolt(String id, IRichBolt bolt, Number parallelism, boolean committer) { + Integer p = null; + if(parallelism!=null) p = parallelism.intValue(); + Component component = new Component(bolt, p, committer); + _bolts.put(id, component); + return new BoltDeclarerImpl(component); + } + + public TopologyBuilder buildTopologyBuilder() { + String coordinator = _spoutId + "/coordinator"; + TopologyBuilder builder = new TopologyBuilder(); + SpoutDeclarer declarer = builder.setSpout(coordinator, new TransactionalSpoutCoordinator(_spout)); + for(Map conf: _spoutConfs) { + declarer.addConfigurations(conf); + } + declarer.addConfiguration(Config.TOPOLOGY_TRANSACTIONAL_ID, _id); + + BoltDeclarer emitterDeclarer = + builder.setBolt(_spoutId, + new CoordinatedBolt(new TransactionalSpoutBatchExecutor(_spout), + null, + null), + _spoutParallelism) + .allGrouping(coordinator, TransactionalSpoutCoordinator.TRANSACTION_BATCH_STREAM_ID) + .addConfiguration(Config.TOPOLOGY_TRANSACTIONAL_ID, _id); + if(_spout instanceof ICommitterTransactionalSpout) { + emitterDeclarer.allGrouping(coordinator, TransactionalSpoutCoordinator.TRANSACTION_COMMIT_STREAM_ID); + } + for(String id: _bolts.keySet()) { + Component component = _bolts.get(id); + Map coordinatedArgs = new HashMap(); + for(String c: componentBoltSubscriptions(component)) { + coordinatedArgs.put(c, SourceArgs.all()); + } + + IdStreamSpec idSpec = null; + if(component.committer) { + idSpec = IdStreamSpec.makeDetectSpec(coordinator, TransactionalSpoutCoordinator.TRANSACTION_COMMIT_STREAM_ID); + } + BoltDeclarer input = builder.setBolt(id, + new CoordinatedBolt(component.bolt, + coordinatedArgs, + idSpec), + component.parallelism); + for(Map conf: component.componentConfs) { + input.addConfigurations(conf); + } + for(String c: componentBoltSubscriptions(component)) { + input.directGrouping(c, Constants.COORDINATED_STREAM_ID); + } + for(InputDeclaration d: component.declarations) { + d.declare(input); + } + if(component.committer) { + input.allGrouping(coordinator, TransactionalSpoutCoordinator.TRANSACTION_COMMIT_STREAM_ID); + } + } + return builder; + } + + public StormTopology buildTopology() { + return buildTopologyBuilder().createTopology(); + } + + private Set componentBoltSubscriptions(Component component) { + Set ret = new HashSet(); + for(InputDeclaration d: component.declarations) { + ret.add(d.getComponent()); + } + return ret; + } + + private static class Component { + public IRichBolt bolt; + public Integer parallelism; + public List declarations = new ArrayList(); + public List componentConfs = new ArrayList(); + public boolean committer; + + public Component(IRichBolt bolt, Integer parallelism, boolean committer) { + this.bolt = bolt; + this.parallelism = parallelism; + this.committer = committer; + } + } + + private static interface InputDeclaration { + void declare(InputDeclarer declarer); + String getComponent(); + } + + private class SpoutDeclarerImpl extends BaseConfigurationDeclarer implements SpoutDeclarer { + @Override + public SpoutDeclarer addConfigurations(Map conf) { + _spoutConfs.add(conf); + return this; + } + } + + private class BoltDeclarerImpl extends BaseConfigurationDeclarer implements BoltDeclarer { + Component _component; + + public BoltDeclarerImpl(Component component) { + _component = component; + } + + @Override + public BoltDeclarer fieldsGrouping(final String component, final Fields fields) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.fieldsGrouping(component, fields); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer fieldsGrouping(final String component, final String streamId, final Fields fields) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.fieldsGrouping(component, streamId, fields); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer globalGrouping(final String component) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.globalGrouping(component); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer globalGrouping(final String component, final String streamId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.globalGrouping(component, streamId); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer shuffleGrouping(final String component) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.shuffleGrouping(component); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer shuffleGrouping(final String component, final String streamId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.shuffleGrouping(component, streamId); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer localOrShuffleGrouping(final String component) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.localOrShuffleGrouping(component); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer localOrShuffleGrouping(final String component, final String streamId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.localOrShuffleGrouping(component, streamId); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer localFirstGrouping(final String component) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.localFirstGrouping(component); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer localFirstGrouping(final String component, final String streamId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.localFirstGrouping(component, streamId); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer noneGrouping(final String component) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.noneGrouping(component); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer noneGrouping(final String component, final String streamId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.noneGrouping(component, streamId); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer allGrouping(final String component) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.allGrouping(component); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer allGrouping(final String component, final String streamId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.allGrouping(component, streamId); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer directGrouping(final String component) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.directGrouping(component); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer directGrouping(final String component, final String streamId) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.directGrouping(component, streamId); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer partialKeyGrouping(String componentId, Fields fields) { + return customGrouping(componentId, new PartialKeyGrouping(fields)); + } + + @Override + public BoltDeclarer partialKeyGrouping(String componentId, String streamId, Fields fields) { + return customGrouping(componentId, streamId, new PartialKeyGrouping(fields)); + } + + @Override + public BoltDeclarer customGrouping(final String component, final CustomStreamGrouping grouping) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.customGrouping(component, grouping); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer customGrouping(final String component, final String streamId, final CustomStreamGrouping grouping) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.customGrouping(component, streamId, grouping); + } + + @Override + public String getComponent() { + return component; + } + }); + return this; + } + + @Override + public BoltDeclarer grouping(final GlobalStreamId stream, final Grouping grouping) { + addDeclaration(new InputDeclaration() { + @Override + public void declare(InputDeclarer declarer) { + declarer.grouping(stream, grouping); + } + + @Override + public String getComponent() { + return stream.get_componentId(); + } + }); + return this; + } + + private void addDeclaration(InputDeclaration declaration) { + _component.declarations.add(declaration); + } + + @Override + public BoltDeclarer addConfigurations(Map conf) { + _component.componentConfs.add(conf); + return this; + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/transactional/partitioned/IOpaquePartitionedTransactionalSpout.java b/jstorm-core/src/main/java/backtype/storm/transactional/partitioned/IOpaquePartitionedTransactionalSpout.java new file mode 100755 index 000000000..8d1f60b29 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/transactional/partitioned/IOpaquePartitionedTransactionalSpout.java @@ -0,0 +1,56 @@ +/** + * 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 backtype.storm.transactional.partitioned; + +import backtype.storm.coordination.BatchOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.IComponent; +import backtype.storm.transactional.TransactionAttempt; +import java.util.Map; + +/** + * This defines a transactional spout which does *not* necessarily + * replay the same batch every time it emits a batch for a transaction id. + */ +public interface IOpaquePartitionedTransactionalSpout extends IComponent { + public interface Coordinator { + /** + * Returns true if its ok to emit start a new transaction, false otherwise (will skip this transaction). + * + * You should sleep here if you want a delay between asking for the next transaction (this will be called + * repeatedly in a loop). + */ + boolean isReady(); + void close(); + } + + public interface Emitter { + /** + * Emit a batch of tuples for a partition/transaction. + * + * Return the metadata describing this batch that will be used as lastPartitionMeta + * for defining the parameters of the next batch. + */ + X emitPartitionBatch(TransactionAttempt tx, BatchOutputCollector collector, int partition, X lastPartitionMeta); + int numPartitions(); + void close(); + } + + Emitter getEmitter(Map conf, TopologyContext context); + Coordinator getCoordinator(Map conf, TopologyContext context); +} diff --git a/jstorm-core/src/main/java/backtype/storm/transactional/partitioned/IPartitionedTransactionalSpout.java b/jstorm-core/src/main/java/backtype/storm/transactional/partitioned/IPartitionedTransactionalSpout.java new file mode 100755 index 000000000..e42832884 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/transactional/partitioned/IPartitionedTransactionalSpout.java @@ -0,0 +1,69 @@ +/** + * 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 backtype.storm.transactional.partitioned; + +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.IComponent; +import backtype.storm.transactional.TransactionAttempt; +import backtype.storm.coordination.BatchOutputCollector; +import java.util.Map; + +/** + * This interface defines a transactional spout that reads its tuples from a partitioned set of + * brokers. It automates the storing of metadata for each partition to ensure that the same batch + * is always emitted for the same transaction id. The partition metadata is stored in Zookeeper. + */ +public interface IPartitionedTransactionalSpout extends IComponent { + public interface Coordinator { + /** + * Return the number of partitions currently in the source of data. The idea is + * is that if a new partition is added and a prior transaction is replayed, it doesn't + * emit tuples for the new partition because it knows how many partitions were in + * that transaction. + */ + int numPartitions(); + + /** + * Returns true if its ok to emit start a new transaction, false otherwise (will skip this transaction). + * + * You should sleep here if you want a delay between asking for the next transaction (this will be called + * repeatedly in a loop). + */ + boolean isReady(); + + void close(); + } + + public interface Emitter { + /** + * Emit a batch of tuples for a partition/transaction that's never been emitted before. + * Return the metadata that can be used to reconstruct this partition/batch in the future. + */ + X emitPartitionBatchNew(TransactionAttempt tx, BatchOutputCollector collector, int partition, X lastPartitionMeta); + + /** + * Emit a batch of tuples for a partition/transaction that has been emitted before, using + * the metadata created when it was first emitted. + */ + void emitPartitionBatch(TransactionAttempt tx, BatchOutputCollector collector, int partition, X partitionMeta); + void close(); + } + + Coordinator getCoordinator(Map conf, TopologyContext context); + Emitter getEmitter(Map conf, TopologyContext context); +} diff --git a/jstorm-core/src/main/java/backtype/storm/transactional/partitioned/OpaquePartitionedTransactionalSpoutExecutor.java b/jstorm-core/src/main/java/backtype/storm/transactional/partitioned/OpaquePartitionedTransactionalSpoutExecutor.java new file mode 100755 index 000000000..aabcb7acb --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/transactional/partitioned/OpaquePartitionedTransactionalSpoutExecutor.java @@ -0,0 +1,159 @@ +/** + * 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 backtype.storm.transactional.partitioned; + +import backtype.storm.Config; +import backtype.storm.coordination.BatchOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.transactional.ICommitterTransactionalSpout; +import backtype.storm.transactional.ITransactionalSpout; +import backtype.storm.transactional.TransactionAttempt; +import backtype.storm.transactional.state.RotatingTransactionalState; +import backtype.storm.transactional.state.TransactionalState; +import java.math.BigInteger; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.TreeMap; + + +public class OpaquePartitionedTransactionalSpoutExecutor implements ICommitterTransactionalSpout { + IOpaquePartitionedTransactionalSpout _spout; + + public class Coordinator implements ITransactionalSpout.Coordinator { + IOpaquePartitionedTransactionalSpout.Coordinator _coordinator; + + public Coordinator(Map conf, TopologyContext context) { + _coordinator = _spout.getCoordinator(conf, context); + } + + @Override + public Object initializeTransaction(BigInteger txid, Object prevMetadata) { + return null; + } + + @Override + public boolean isReady() { + return _coordinator.isReady(); + } + + @Override + public void close() { + _coordinator.close(); + } + } + + public class Emitter implements ICommitterTransactionalSpout.Emitter { + IOpaquePartitionedTransactionalSpout.Emitter _emitter; + TransactionalState _state; + TreeMap> _cachedMetas = new TreeMap>(); + Map _partitionStates = new HashMap(); + int _index; + int _numTasks; + + public Emitter(Map conf, TopologyContext context) { + _emitter = _spout.getEmitter(conf, context); + _index = context.getThisTaskIndex(); + _numTasks = context.getComponentTasks(context.getThisComponentId()).size(); + _state = TransactionalState.newUserState(conf, (String) conf.get(Config.TOPOLOGY_TRANSACTIONAL_ID), getComponentConfiguration()); + List existingPartitions = _state.list(""); + for(String p: existingPartitions) { + int partition = Integer.parseInt(p); + if((partition - _index) % _numTasks == 0) { + _partitionStates.put(partition, new RotatingTransactionalState(_state, p)); + } + } + } + + @Override + public void emitBatch(TransactionAttempt tx, Object coordinatorMeta, BatchOutputCollector collector) { + Map metas = new HashMap(); + _cachedMetas.put(tx.getTransactionId(), metas); + int partitions = _emitter.numPartitions(); + Entry> entry = _cachedMetas.lowerEntry(tx.getTransactionId()); + Map prevCached; + if(entry!=null) { + prevCached = entry.getValue(); + } else { + prevCached = new HashMap(); + } + + for(int i=_index; i < partitions; i+=_numTasks) { + RotatingTransactionalState state = _partitionStates.get(i); + if(state==null) { + state = new RotatingTransactionalState(_state, "" + i); + _partitionStates.put(i, state); + } + state.removeState(tx.getTransactionId()); + Object lastMeta = prevCached.get(i); + if(lastMeta==null) lastMeta = state.getLastState(); + Object meta = _emitter.emitPartitionBatch(tx, collector, i, lastMeta); + metas.put(i, meta); + } + } + + @Override + public void cleanupBefore(BigInteger txid) { + for(RotatingTransactionalState state: _partitionStates.values()) { + state.cleanupBefore(txid); + } + } + + @Override + public void commit(TransactionAttempt attempt) { + BigInteger txid = attempt.getTransactionId(); + Map metas = _cachedMetas.remove(txid); + for(Integer partition: metas.keySet()) { + Object meta = metas.get(partition); + _partitionStates.get(partition).overrideState(txid, meta); + } + } + + @Override + public void close() { + _emitter.close(); + } + } + + public OpaquePartitionedTransactionalSpoutExecutor(IOpaquePartitionedTransactionalSpout spout) { + _spout = spout; + } + + @Override + public ITransactionalSpout.Coordinator getCoordinator(Map conf, TopologyContext context) { + return new Coordinator(conf, context); + } + + @Override + public ICommitterTransactionalSpout.Emitter getEmitter(Map conf, TopologyContext context) { + return new Emitter(conf, context); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + _spout.declareOutputFields(declarer); + } + + @Override + public Map getComponentConfiguration() { + return _spout.getComponentConfiguration(); + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/transactional/partitioned/PartitionedTransactionalSpoutExecutor.java b/jstorm-core/src/main/java/backtype/storm/transactional/partitioned/PartitionedTransactionalSpoutExecutor.java new file mode 100644 index 000000000..479dda4c7 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/transactional/partitioned/PartitionedTransactionalSpoutExecutor.java @@ -0,0 +1,142 @@ +/** + * 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 backtype.storm.transactional.partitioned; + +import backtype.storm.Config; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.transactional.ITransactionalSpout; +import backtype.storm.transactional.TransactionAttempt; +import backtype.storm.coordination.BatchOutputCollector; +import backtype.storm.transactional.state.RotatingTransactionalState; +import backtype.storm.transactional.state.TransactionalState; +import java.math.BigInteger; +import java.util.HashMap; +import java.util.Map; + + +public class PartitionedTransactionalSpoutExecutor implements ITransactionalSpout { + IPartitionedTransactionalSpout _spout; + + public PartitionedTransactionalSpoutExecutor(IPartitionedTransactionalSpout spout) { + _spout = spout; + } + + public IPartitionedTransactionalSpout getPartitionedSpout() { + return _spout; + } + + class Coordinator implements ITransactionalSpout.Coordinator { + private IPartitionedTransactionalSpout.Coordinator _coordinator; + + public Coordinator(Map conf, TopologyContext context) { + _coordinator = _spout.getCoordinator(conf, context); + } + + @Override + public Integer initializeTransaction(BigInteger txid, Integer prevMetadata) { + return _coordinator.numPartitions(); + } + + @Override + public boolean isReady() { + return _coordinator.isReady(); + } + + @Override + public void close() { + _coordinator.close(); + } + } + + class Emitter implements ITransactionalSpout.Emitter { + private IPartitionedTransactionalSpout.Emitter _emitter; + private TransactionalState _state; + private Map _partitionStates = new HashMap(); + private int _index; + private int _numTasks; + + public Emitter(Map conf, TopologyContext context) { + _emitter = _spout.getEmitter(conf, context); + _state = TransactionalState.newUserState(conf, (String) conf.get(Config.TOPOLOGY_TRANSACTIONAL_ID), getComponentConfiguration()); + _index = context.getThisTaskIndex(); + _numTasks = context.getComponentTasks(context.getThisComponentId()).size(); + } + + @Override + public void emitBatch(final TransactionAttempt tx, final Integer partitions, + final BatchOutputCollector collector) { + for(int i=_index; i < partitions; i+=_numTasks) { + if(!_partitionStates.containsKey(i)) { + _partitionStates.put(i, new RotatingTransactionalState(_state, "" + i)); + } + RotatingTransactionalState state = _partitionStates.get(i); + final int partition = i; + Object meta = state.getStateOrCreate(tx.getTransactionId(), + new RotatingTransactionalState.StateInitializer() { + @Override + public Object init(BigInteger txid, Object lastState) { + return _emitter.emitPartitionBatchNew(tx, collector, partition, lastState); + } + }); + // it's null if one of: + // a) a later transaction batch was emitted before this, so we should skip this batch + // b) if didn't exist and was created (in which case the StateInitializer was invoked and + // it was emitted + if(meta!=null) { + _emitter.emitPartitionBatch(tx, collector, partition, meta); + } + } + + } + + @Override + public void cleanupBefore(BigInteger txid) { + for(RotatingTransactionalState state: _partitionStates.values()) { + state.cleanupBefore(txid); + } + } + + @Override + public void close() { + _state.close(); + _emitter.close(); + } + } + + @Override + public ITransactionalSpout.Coordinator getCoordinator(Map conf, TopologyContext context) { + return new Coordinator(conf, context); + } + + @Override + public ITransactionalSpout.Emitter getEmitter(Map conf, TopologyContext context) { + return new Emitter(conf, context); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + _spout.declareOutputFields(declarer); + } + + @Override + public Map getComponentConfiguration() { + return _spout.getComponentConfiguration(); + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/transactional/state/RotatingTransactionalState.java b/jstorm-core/src/main/java/backtype/storm/transactional/state/RotatingTransactionalState.java new file mode 100644 index 000000000..20c5cd3cc --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/transactional/state/RotatingTransactionalState.java @@ -0,0 +1,151 @@ +/** + * 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 backtype.storm.transactional.state; + +import backtype.storm.transactional.TransactionalSpoutCoordinator; + +import java.math.BigInteger; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; + +/** + * A map from txid to a value. Automatically deletes txids that have been committed. + */ +public class RotatingTransactionalState { + public static interface StateInitializer { + Object init(BigInteger txid, Object lastState); + } + + private TransactionalState _state; + private String _subdir; + private boolean _strictOrder; + + private TreeMap _curr = new TreeMap(); + + public RotatingTransactionalState(TransactionalState state, String subdir, boolean strictOrder) { + _state = state; + _subdir = subdir; + _strictOrder = strictOrder; + state.mkdir(subdir); + sync(); + } + + public RotatingTransactionalState(TransactionalState state, String subdir) { + this(state, subdir, false); + } + + public Object getLastState() { + if(_curr.isEmpty()) return null; + else return _curr.lastEntry().getValue(); + } + + public void overrideState(BigInteger txid, Object state) { + _state.setData(txPath(txid), state); + _curr.put(txid, state); + } + + public void removeState(BigInteger txid) { + if(_curr.containsKey(txid)) { + _curr.remove(txid); + _state.delete(txPath(txid)); + } + } + + public Object getState(BigInteger txid, StateInitializer init) { + if(!_curr.containsKey(txid)) { + SortedMap prevMap = _curr.headMap(txid); + SortedMap afterMap = _curr.tailMap(txid); + + BigInteger prev = null; + if(!prevMap.isEmpty()) prev = prevMap.lastKey(); + + if (_strictOrder) { + if (prev == null && !txid.equals(TransactionalSpoutCoordinator.INIT_TXID)) { + throw new IllegalStateException("Trying to initialize transaction for which there should be a previous state"); + } + if (prev != null && !prev.equals(txid.subtract(BigInteger.ONE))) { + throw new IllegalStateException("Expecting previous txid state to be the previous transaction"); + } + if (!afterMap.isEmpty()) { + throw new IllegalStateException("Expecting tx state to be initialized in strict order but there are txids after that have state"); + } + } + + Object data; + if (afterMap.isEmpty()) { + Object prevData; + if (prev != null) { + prevData = _curr.get(prev); + } else { + prevData = null; + } + data = init.init(txid, prevData); + } else { + data = null; + } + _curr.put(txid, data); + _state.setData(txPath(txid), data); + } + return _curr.get(txid); + } + + public boolean hasCache(BigInteger txid) { + return _curr.containsKey(txid); + } + + /** + * Returns null if it was created, the value otherwise. + */ + public Object getStateOrCreate(BigInteger txid, StateInitializer init) { + if (_curr.containsKey(txid)) { + return _curr.get(txid); + } else { + getState(txid, init); + return null; + } + } + + public void cleanupBefore(BigInteger txid) { + Set toDelete = new HashSet(); + toDelete.addAll(_curr.headMap(txid).keySet()); + for (BigInteger tx : toDelete) { + _curr.remove(tx); + _state.delete(txPath(tx)); + } + } + + private void sync() { + List txids = _state.list(_subdir); + for(String txid_s: txids) { + Object data = _state.getData(txPath(txid_s)); + _curr.put(new BigInteger(txid_s), data); + } + } + + private String txPath(BigInteger tx) { + return txPath(tx.toString()); + } + + private String txPath(String tx) { + return _subdir + "/" + tx; + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/transactional/state/TestTransactionalState.java b/jstorm-core/src/main/java/backtype/storm/transactional/state/TestTransactionalState.java new file mode 100755 index 000000000..3d4a46323 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/transactional/state/TestTransactionalState.java @@ -0,0 +1,47 @@ +/** + * 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 backtype.storm.transactional.state; + +import java.util.List; +import java.util.Map; + +import backtype.storm.utils.ZookeeperAuthInfo; +import org.apache.curator.framework.CuratorFramework; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.data.ACL; + +/** + * Facilitates testing of non-public methods in the parent class. + */ +public class TestTransactionalState extends TransactionalState { + + /** + * Matching constructor in absence of a default constructor in the parent + * class. + */ + protected TestTransactionalState(Map conf, String id, Map componentConf, String subroot) { + super(conf, id, componentConf, subroot); + } + + public static void createNode(CuratorFramework curator, + String rootDir, byte[] data, List acls, CreateMode mode) + throws Exception { + TransactionalState.createNode(curator, rootDir, data, acls, mode); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/transactional/state/TransactionalState.java b/jstorm-core/src/main/java/backtype/storm/transactional/state/TransactionalState.java new file mode 100755 index 000000000..5afcd0a2e --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/transactional/state/TransactionalState.java @@ -0,0 +1,173 @@ +/** + * 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 backtype.storm.transactional.state; + +import backtype.storm.Config; +import backtype.storm.serialization.KryoValuesDeserializer; +import backtype.storm.serialization.KryoValuesSerializer; +import backtype.storm.utils.Utils; +import backtype.storm.utils.ZookeeperAuthInfo; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.api.ProtectACLCreateModePathAndBytesable; +import org.apache.curator.framework.api.PathAndBytesable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.data.Id; + +public class TransactionalState { + CuratorFramework _curator; + KryoValuesSerializer _ser; + KryoValuesDeserializer _des; + List _zkAcls = null; + + public static TransactionalState newUserState(Map conf, String id, Map componentConf) { + return new TransactionalState(conf, id, componentConf, "user"); + } + + public static TransactionalState newCoordinatorState(Map conf, String id, Map componentConf) { + return new TransactionalState(conf, id, componentConf, "coordinator"); + } + + protected TransactionalState(Map conf, String id, Map componentConf, String subroot) { + try { + conf = new HashMap(conf); + // ensure that the serialization registrations are consistent with the declarations in this spout + if(componentConf!=null) { + conf.put(Config.TOPOLOGY_KRYO_REGISTER, + componentConf + .get(Config.TOPOLOGY_KRYO_REGISTER)); + } + String transactionalRoot = (String)conf.get(Config.TRANSACTIONAL_ZOOKEEPER_ROOT); + String rootDir = transactionalRoot + "/" + id + "/" + subroot; + List servers = (List) getWithBackup(conf, Config.TRANSACTIONAL_ZOOKEEPER_SERVERS, Config.STORM_ZOOKEEPER_SERVERS); + Object port = getWithBackup(conf, Config.TRANSACTIONAL_ZOOKEEPER_PORT, Config.STORM_ZOOKEEPER_PORT); + ZookeeperAuthInfo auth = new ZookeeperAuthInfo(conf); + CuratorFramework initter = Utils.newCuratorStarted(conf, servers, port, auth); + _zkAcls = Utils.getWorkerACL(conf); + try { + TransactionalState.createNode(initter, transactionalRoot, null, null, null); + } catch (KeeperException.NodeExistsException e) { + } + try { + TransactionalState.createNode(initter, rootDir, null, _zkAcls, null); + } catch (KeeperException.NodeExistsException e) { + } + initter.close(); + + _curator = Utils.newCuratorStarted(conf, servers, port, rootDir, auth); + _ser = new KryoValuesSerializer(conf); + _des = new KryoValuesDeserializer(conf); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + protected static String forPath(PathAndBytesable builder, + String path, byte[] data) throws Exception { + return (data == null) + ? builder.forPath(path) + : builder.forPath(path, data); + } + + protected static void createNode(CuratorFramework curator, String path, + byte[] data, List acls, CreateMode mode) throws Exception { + ProtectACLCreateModePathAndBytesable builder = + curator.create().creatingParentsIfNeeded(); + + if (acls == null) { + if (mode == null ) { + TransactionalState.forPath(builder, path, data); + } else { + TransactionalState.forPath(builder.withMode(mode), path, data); + } + return; + } + + TransactionalState.forPath(builder.withACL(acls), path, data); + } + + public void setData(String path, Object obj) { + path = "/" + path; + byte[] ser = _ser.serializeObject(obj); + try { + if(_curator.checkExists().forPath(path)!=null) { + _curator.setData().forPath(path, ser); + } else { + TransactionalState.createNode(_curator, path, ser, _zkAcls, + CreateMode.PERSISTENT); + } + } catch(Exception e) { + throw new RuntimeException(e); + } + } + + public void delete(String path) { + path = "/" + path; + try { + _curator.delete().forPath(path); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public List list(String path) { + path = "/" + path; + try { + if(_curator.checkExists().forPath(path)==null) { + return new ArrayList(); + } else { + return _curator.getChildren().forPath(path); + } + } catch(Exception e) { + throw new RuntimeException(e); + } + } + + public void mkdir(String path) { + setData(path, 7); + } + + public Object getData(String path) { + path = "/" + path; + try { + if(_curator.checkExists().forPath(path)!=null) { + return _des.deserializeObject(_curator.getData().forPath(path)); + } else { + return null; + } + } catch(Exception e) { + throw new RuntimeException(e); + } + } + + public void close() { + _curator.close(); + } + + private Object getWithBackup(Map amap, Object primary, Object backup) { + Object ret = amap.get(primary); + if(ret==null) return amap.get(backup); + return ret; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/tuple/BatchTuple.java b/jstorm-core/src/main/java/backtype/storm/tuple/BatchTuple.java new file mode 100644 index 000000000..47df545db --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/tuple/BatchTuple.java @@ -0,0 +1,77 @@ +/** + * 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 backtype.storm.tuple; + +import java.util.ArrayList; +import java.util.List; + + +public class BatchTuple { + private int targetTaskId; + + private List batch; + private int batchSize; + + public BatchTuple() { + + } + + public BatchTuple(int targetTaskId, int batchSize) { + this.targetTaskId = targetTaskId; + this.batchSize = batchSize; + } + + public void addToBatch(Tuple tuple) { + if (batch == null) { + batch = new ArrayList(); + } + batch.add(tuple); + } + + public boolean isBatchFull() { + boolean ret = false; + if (batch != null && batch.size() >= batchSize) + ret = true; + + return ret; + } + + public List getTuples() { + return batch; + } + + public void resetBatch() { + batch = new ArrayList(); + } + + public int currBatchSize() { + return batch == null ? 0 : batch.size(); + } + + public void setTargetTaskId(int taskId) { + this.targetTaskId = taskId; + } + + public int getTargetTaskId() { + return targetTaskId; + } + + public void setBatchSize(int batchSize) { + this.batchSize = batchSize; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/tuple/Fields.java b/jstorm-core/src/main/java/backtype/storm/tuple/Fields.java new file mode 100644 index 000000000..9805ba6bc --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/tuple/Fields.java @@ -0,0 +1,100 @@ +/** + * 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 backtype.storm.tuple; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.io.Serializable; + +public class Fields implements Iterable, Serializable { + private List _fields; + private Map _index = new HashMap(); + + public Fields(String... fields) { + this(Arrays.asList(fields)); + } + + public Fields(List fields) { + _fields = new ArrayList(fields.size()); + for (String field : fields) { + if (_fields.contains(field)) + throw new IllegalArgumentException( + String.format("duplicate field '%s'", field) + ); + _fields.add(field); + } + index(); + } + + public List select(Fields selector, List tuple) { + List ret = new ArrayList(selector.size()); + for(String s: selector) { + ret.add(tuple.get(_index.get(s))); + } + return ret; + } + + public List toList() { + return new ArrayList(_fields); + } + + public int size() { + return _fields.size(); + } + + public String get(int index) { + return _fields.get(index); + } + + public Iterator iterator() { + return _fields.iterator(); + } + + /** + * Returns the position of the specified field. + */ + public int fieldIndex(String field) { + Integer ret = _index.get(field); + if(ret==null) { + throw new IllegalArgumentException(field + " does not exist"); + } + return ret; + } + + /** + * Returns true if this contains the specified name of the field. + */ + public boolean contains(String field) { + return _index.containsKey(field); + } + + private void index() { + for(int i=0; i<_fields.size(); i++) { + _index.put(_fields.get(i), i); + } + } + + @Override + public String toString() { + return _fields.toString(); + } +} diff --git a/jstorm-client/src/main/java/backtype/storm/tuple/ITuple.java b/jstorm-core/src/main/java/backtype/storm/tuple/ITuple.java old mode 100644 new mode 100755 similarity index 78% rename from jstorm-client/src/main/java/backtype/storm/tuple/ITuple.java rename to jstorm-core/src/main/java/backtype/storm/tuple/ITuple.java index b00279d7f..c85848d7f --- a/jstorm-client/src/main/java/backtype/storm/tuple/ITuple.java +++ b/jstorm-core/src/main/java/backtype/storm/tuple/ITuple.java @@ -1,3 +1,20 @@ +/** + * 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 backtype.storm.tuple; import java.util.List; diff --git a/jstorm-core/src/main/java/backtype/storm/tuple/MessageId.java b/jstorm-core/src/main/java/backtype/storm/tuple/MessageId.java new file mode 100755 index 000000000..688946d55 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/tuple/MessageId.java @@ -0,0 +1,103 @@ +/** + * 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 backtype.storm.tuple; + +import backtype.storm.utils.Utils; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Random; +import java.util.Set; + +public class MessageId { + private Map _anchorsToIds; + + @Deprecated + public static long generateId() { + return Utils.secureRandomLong(); + } + + public static long generateId(Random rand) { + return rand.nextLong(); + } + + public static MessageId makeUnanchored() { + return makeId(new HashMap()); + } + + public static MessageId makeId(Map anchorsToIds) { + return new MessageId(anchorsToIds); + } + + public static MessageId makeRootId(long id, long val) { + Map anchorsToIds = new HashMap(); + anchorsToIds.put(id, val); + return new MessageId(anchorsToIds); + } + + protected MessageId(Map anchorsToIds) { + _anchorsToIds = anchorsToIds; + } + + public Map getAnchorsToIds() { + return _anchorsToIds; + } + + public Set getAnchors() { + return _anchorsToIds.keySet(); + } + + @Override + public int hashCode() { + return _anchorsToIds.hashCode(); + } + + @Override + public boolean equals(Object other) { + if(other instanceof MessageId) { + return _anchorsToIds.equals(((MessageId) other)._anchorsToIds); + } else { + return false; + } + } + + @Override + public String toString() { + return _anchorsToIds.toString(); + } + + public void serialize(Output out) throws IOException { + out.writeInt(_anchorsToIds.size(), true); + for(Entry anchorToId: _anchorsToIds.entrySet()) { + out.writeLong(anchorToId.getKey()); + out.writeLong(anchorToId.getValue()); + } + } + + public static MessageId deserialize(Input in) throws IOException { + int numAnchors = in.readInt(true); + Map anchorsToIds = new HashMap(); + for(int i=0; i values; + private int taskId; + private String streamId; + private GeneralTopologyContext context; + private MessageId id; + private IPersistentMap _meta = null; + + public TupleImpl(GeneralTopologyContext context, List values, int taskId, String streamId, MessageId id) { + this.values = values; + this.taskId = taskId; + this.streamId = streamId; + this.id = id; + this.context = context; + + String componentId = context.getComponentId(taskId); + Fields schema = context.getComponentOutputFields(componentId, streamId); + if(values.size()!=schema.size()) { + throw new IllegalArgumentException( + "Tuple created with wrong number of fields. " + + "Expected " + schema.size() + " fields but got " + + values.size() + " fields"); + } + } + + public TupleImpl(GeneralTopologyContext context, List values, int taskId, String streamId) { + this(context, values, taskId, streamId, MessageId.makeUnanchored()); + } + + Long _processSampleStartTime = null; + Long _executeSampleStartTime = null; + + public void setProcessSampleStartTime(long ms) { + _processSampleStartTime = ms; + } + + public Long getProcessSampleStartTime() { + return _processSampleStartTime; + } + + public void setExecuteSampleStartTime(long ms) { + _executeSampleStartTime = ms; + } + + public Long getExecuteSampleStartTime() { + return _executeSampleStartTime; + } + + long _outAckVal = 0; + + public void updateAckVal(long val) { + _outAckVal = _outAckVal ^ val; + } + + public long getAckVal() { + return _outAckVal; + } + + public int size() { + return values.size(); + } + + public int fieldIndex(String field) { + return getFields().fieldIndex(field); + } + + public boolean contains(String field) { + return getFields().contains(field); + } + + public Object getValue(int i) { + return values.get(i); + } + + public String getString(int i) { + return (String) values.get(i); + } + + public Integer getInteger(int i) { + return (Integer) values.get(i); + } + + public Long getLong(int i) { + return (Long) values.get(i); + } + + public Boolean getBoolean(int i) { + return (Boolean) values.get(i); + } + + public Short getShort(int i) { + return (Short) values.get(i); + } + + public Byte getByte(int i) { + return (Byte) values.get(i); + } + + public Double getDouble(int i) { + return (Double) values.get(i); + } + + public Float getFloat(int i) { + return (Float) values.get(i); + } + + public byte[] getBinary(int i) { + return (byte[]) values.get(i); + } + + + public Object getValueByField(String field) { + return values.get(fieldIndex(field)); + } + + public String getStringByField(String field) { + return (String) values.get(fieldIndex(field)); + } + + public Integer getIntegerByField(String field) { + return (Integer) values.get(fieldIndex(field)); + } + + public Long getLongByField(String field) { + return (Long) values.get(fieldIndex(field)); + } + + public Boolean getBooleanByField(String field) { + return (Boolean) values.get(fieldIndex(field)); + } + + public Short getShortByField(String field) { + return (Short) values.get(fieldIndex(field)); + } + + public Byte getByteByField(String field) { + return (Byte) values.get(fieldIndex(field)); + } + + public Double getDoubleByField(String field) { + return (Double) values.get(fieldIndex(field)); + } + + public Float getFloatByField(String field) { + return (Float) values.get(fieldIndex(field)); + } + + public byte[] getBinaryByField(String field) { + return (byte[]) values.get(fieldIndex(field)); + } + + public List getValues() { + return values; + } + + public Fields getFields() { + return context.getComponentOutputFields(getSourceComponent(), getSourceStreamId()); + } + + public List select(Fields selector) { + return getFields().select(selector, values); + } + + public GlobalStreamId getSourceGlobalStreamid() { + return new GlobalStreamId(getSourceComponent(), streamId); + } + + public String getSourceComponent() { + return context.getComponentId(taskId); + } + + public int getSourceTask() { + return taskId; + } + + public String getSourceStreamId() { + return streamId; + } + + public MessageId getMessageId() { + return id; + } + + @Override + public String toString() { + return "source: " + getSourceComponent() + ":" + taskId + ", stream: " + streamId + ", id: "+ id.toString() + ", " + values.toString(); + } + + @Override + public boolean equals(Object other) { + return this == other; + } + + @Override + public int hashCode() { + return System.identityHashCode(this); + } + + private final Keyword makeKeyword(String name) { + return Keyword.intern(Symbol.create(name)); + } + + /* ILookup */ + @Override + public Object valAt(Object o) { + try { + if(o instanceof Keyword) { + return getValueByField(((Keyword) o).getName()); + } else if(o instanceof String) { + return getValueByField((String) o); + } + } catch(IllegalArgumentException e) { + } + return null; + } + + /* Seqable */ + public ISeq seq() { + if(values.size() > 0) { + return new Seq(getFields().toList(), values, 0); + } + return null; + } + + static class Seq extends ASeq implements Counted { + final List fields; + final List values; + final int i; + + Seq(List fields, List values, int i) { + this.fields = fields; + this.values = values; + assert i >= 0; + this.i = i; + } + + public Seq(IPersistentMap meta, List fields, List values, int i) { + super(meta); + this.fields= fields; + this.values = values; + assert i >= 0; + this.i = i; + } + + public Object first() { + return new MapEntry(fields.get(i), values.get(i)); + } + + public ISeq next() { + if(i+1 < fields.size()) { + return new Seq(fields, values, i+1); + } + return null; + } + + public int count() { + assert fields.size() -i >= 0 : "index out of bounds"; + // i being the position in the fields of this seq, the remainder of the seq is the size + return fields.size() -i; + } + + public Obj withMeta(IPersistentMap meta) { + return new Seq(meta, fields, values, i); + } + } + + /* Indexed */ + public Object nth(int i) { + if(i < values.size()) { + return values.get(i); + } else { + return null; + } + } + + public Object nth(int i, Object notfound) { + Object ret = nth(i); + if(ret==null) ret = notfound; + return ret; + } + + /* Counted */ + public int count() { + return values.size(); + } + + /* IMeta */ + public IPersistentMap meta() { + if(_meta==null) { + _meta = new PersistentArrayMap( new Object[] { + makeKeyword("stream"), getSourceStreamId(), + makeKeyword("component"), getSourceComponent(), + makeKeyword("task"), getSourceTask()}); + } + return _meta; + } + + private PersistentArrayMap toMap() { + Object array[] = new Object[values.size()*2]; + List fields = getFields().toList(); + for(int i=0; i < values.size(); i++) { + array[i*2] = fields.get(i); + array[(i*2)+1] = values.get(i); + } + return new PersistentArrayMap(array); + } + + public IPersistentMap getMap() { + if(_map==null) { + setMap(toMap()); + } + return _map; + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/tuple/TupleImplExt.java b/jstorm-core/src/main/java/backtype/storm/tuple/TupleImplExt.java new file mode 100755 index 000000000..2e966a0c7 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/tuple/TupleImplExt.java @@ -0,0 +1,46 @@ +/** + * 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 backtype.storm.tuple; + +import java.util.List; + +import backtype.storm.task.GeneralTopologyContext; + +public class TupleImplExt extends TupleImpl implements TupleExt { + + protected int targetTaskId; + + public TupleImplExt(GeneralTopologyContext context, List values, int taskId, String streamId) { + super(context, values, taskId, streamId); + } + + public TupleImplExt(GeneralTopologyContext context, List values, int taskId, String streamId, MessageId id) { + super(context, values, taskId, streamId, id); + } + + @Override + public int getTargetTaskId() { + return targetTaskId; + } + + @Override + public void setTargetTaskId(int targetTaskId) { + this.targetTaskId = targetTaskId; + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/tuple/Values.java b/jstorm-core/src/main/java/backtype/storm/tuple/Values.java new file mode 100755 index 000000000..41bbc716f --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/tuple/Values.java @@ -0,0 +1,37 @@ +/** + * 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 backtype.storm.tuple; + +import java.util.ArrayList; + +/** + * A convenience class for making tuple values using new Values("field1", 2, 3) + * syntax. + */ +public class Values extends ArrayList{ + public Values() { + + } + + public Values(Object... vals) { + super(vals.length); + for(Object o: vals) { + add(o); + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/ui/InvalidRequestException.java b/jstorm-core/src/main/java/backtype/storm/ui/InvalidRequestException.java new file mode 100755 index 000000000..35d2105f5 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/ui/InvalidRequestException.java @@ -0,0 +1,37 @@ +/* + * 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 backtype.storm.ui; + +public class InvalidRequestException extends Exception { + + public InvalidRequestException() { + super(); + } + + public InvalidRequestException(String msg) { + super(msg); + } + + public InvalidRequestException(String msg, Throwable cause) { + super(msg, cause); + } + + public InvalidRequestException(Throwable cause) { + super(cause); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/BufferFileInputStream.java b/jstorm-core/src/main/java/backtype/storm/utils/BufferFileInputStream.java new file mode 100755 index 000000000..1311d6d7a --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/BufferFileInputStream.java @@ -0,0 +1,54 @@ +/** + * 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 backtype.storm.utils; + +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Arrays; + + +public class BufferFileInputStream { + byte[] buffer; + FileInputStream stream; + + public BufferFileInputStream(String file, int bufferSize) throws FileNotFoundException { + stream = new FileInputStream(file); + buffer = new byte[bufferSize]; + } + + public BufferFileInputStream(String file) throws FileNotFoundException { + this(file, 15*1024); + } + + public byte[] read() throws IOException { + int length = stream.read(buffer); + if(length==-1) { + close(); + return new byte[0]; + } else if(length==buffer.length) { + return buffer; + } else { + return Arrays.copyOf(buffer, length); + } + } + + public void close() throws IOException { + stream.close(); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/CRC32OutputStream.java b/jstorm-core/src/main/java/backtype/storm/utils/CRC32OutputStream.java new file mode 100755 index 000000000..7d5ce731a --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/CRC32OutputStream.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.utils; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.zip.CRC32; + +public class CRC32OutputStream extends OutputStream { + private CRC32 hasher; + + public CRC32OutputStream() { + hasher = new CRC32(); + } + + public long getValue() { + return hasher.getValue(); + } + + @Override + public void write(int i) throws IOException { + hasher.update(i); + } + + @Override + public void write(byte[] bytes, int start, int end) throws IOException { + hasher.update(bytes, start, end); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/ClojureTimerTask.java b/jstorm-core/src/main/java/backtype/storm/utils/ClojureTimerTask.java new file mode 100755 index 000000000..ca9b010f6 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/ClojureTimerTask.java @@ -0,0 +1,35 @@ +/** + * 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 backtype.storm.utils; + +import clojure.lang.IFn; +import java.util.TimerTask; + +public class ClojureTimerTask extends TimerTask { + IFn _afn; + + public ClojureTimerTask(IFn afn) { + super(); + _afn = afn; + } + + @Override + public void run() { + _afn.run(); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/Container.java b/jstorm-core/src/main/java/backtype/storm/utils/Container.java new file mode 100755 index 000000000..d4edcdfde --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/Container.java @@ -0,0 +1,24 @@ +/** + * 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 backtype.storm.utils; + +import java.io.Serializable; + +public class Container implements Serializable { + public Object object; +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/utils/DRPCClient.java b/jstorm-core/src/main/java/backtype/storm/utils/DRPCClient.java new file mode 100755 index 000000000..b2a2a7d85 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/DRPCClient.java @@ -0,0 +1,65 @@ +/** + * 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 backtype.storm.utils; + +import backtype.storm.Config; +import backtype.storm.generated.DRPCExecutionException; +import backtype.storm.generated.DistributedRPC; +import backtype.storm.generated.AuthorizationException; +import org.apache.thrift.TException; +import org.apache.thrift.transport.TTransport; +import backtype.storm.security.auth.ThriftClient; +import backtype.storm.security.auth.ThriftConnectionType; +import org.apache.thrift.transport.TTransportException; + +import java.util.Map; + +public class DRPCClient extends ThriftClient implements DistributedRPC.Iface { + private TTransport conn; + private DistributedRPC.Client client; + private String host; + private int port; + private Integer timeout; + + public DRPCClient(Map conf, String host, int port) throws TTransportException { + this(conf, host, port, null); + } + + public DRPCClient(Map conf, String host, int port, Integer timeout) throws TTransportException { + super(conf, ThriftConnectionType.DRPC, host, port, timeout, null); + this.host = host; + this.port = port; + this.client = new DistributedRPC.Client(_protocol); + } + + public String getHost() { + return host; + } + + public int getPort() { + return port; + } + + public String execute(String func, String args) throws TException, DRPCExecutionException, AuthorizationException { + return client.execute(func, args); + } + + public DistributedRPC.Client getClient() { + return client; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/DisruptorQueue.java b/jstorm-core/src/main/java/backtype/storm/utils/DisruptorQueue.java new file mode 100755 index 000000000..94768e698 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/DisruptorQueue.java @@ -0,0 +1,80 @@ +/** + * 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 backtype.storm.utils; + +import backtype.storm.metric.api.IStatefulObject; + +import com.lmax.disruptor.EventHandler; +import com.lmax.disruptor.InsufficientCapacityException; +import com.lmax.disruptor.WaitStrategy; +import com.lmax.disruptor.dsl.ProducerType; + +/** + * + * A single consumer queue that uses the LMAX Disruptor. They key to the performance is the ability to catch up to the producer by processing tuples in batches. + */ +public abstract class DisruptorQueue implements IStatefulObject { + public static void setUseSleep(boolean useSleep) { + DisruptorQueueImpl.setUseSleep(useSleep); + } + + private static boolean CAPACITY_LIMITED = false; + + public static void setLimited(boolean limited) { + CAPACITY_LIMITED = limited; + } + + public static DisruptorQueue mkInstance(String queueName, ProducerType producerType, int bufferSize, WaitStrategy wait) { + if (CAPACITY_LIMITED == true) { + return new DisruptorQueueImpl(queueName, producerType, bufferSize, wait); + } else { + return new DisruptorWrapBlockingQueue(queueName, producerType, bufferSize, wait); + } + } + + public abstract String getName(); + + public abstract void haltWithInterrupt(); + + public abstract Object poll(); + + public abstract Object take(); + + public abstract void consumeBatch(EventHandler handler); + + public abstract void consumeBatchWhenAvailable(EventHandler handler); + + public abstract void publish(Object obj); + + public abstract void publish(Object obj, boolean block) throws InsufficientCapacityException; + + public abstract void consumerStarted(); + + public abstract void clear(); + + public abstract long population(); + + public abstract long capacity(); + + public abstract long writePos(); + + public abstract long readPos(); + + public abstract float pctFull(); + +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/DisruptorQueueImpl.java b/jstorm-core/src/main/java/backtype/storm/utils/DisruptorQueueImpl.java new file mode 100644 index 000000000..58d831324 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/DisruptorQueueImpl.java @@ -0,0 +1,306 @@ +/** + * 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 backtype.storm.utils; + +import java.util.HashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.utils.disruptor.AbstractSequencerExt; +import backtype.storm.utils.disruptor.RingBuffer; + +import com.lmax.disruptor.AlertException; +import com.lmax.disruptor.EventFactory; +import com.lmax.disruptor.EventHandler; +import com.lmax.disruptor.InsufficientCapacityException; +import com.lmax.disruptor.Sequence; +import com.lmax.disruptor.SequenceBarrier; +import com.lmax.disruptor.TimeoutException; +import com.lmax.disruptor.WaitStrategy; +import com.lmax.disruptor.dsl.ProducerType; + +/** + * + * A single consumer queue that uses the LMAX Disruptor. They key to the performance is the ability to catch up to the producer by processing tuples in batches. + */ +public class DisruptorQueueImpl extends DisruptorQueue { + private static final Logger LOG = LoggerFactory.getLogger(DisruptorQueueImpl.class); + static boolean useSleep = true; + + public static void setUseSleep(boolean useSleep) { + AbstractSequencerExt.setWaitSleep(useSleep); + } + + private static final Object FLUSH_CACHE = new Object(); + private static final Object INTERRUPT = new Object(); + private static final String PREFIX = "disruptor-"; + + private final String _queueName; + private final RingBuffer _buffer; + private final Sequence _consumer; + private final SequenceBarrier _barrier; + + // TODO: consider having a threadlocal cache of this variable to speed up + // reads? + volatile boolean consumerStartedFlag = false; + + private final HashMap state = new HashMap(4); + private final ConcurrentLinkedQueue _cache = new ConcurrentLinkedQueue(); + private final ReentrantReadWriteLock cacheLock = new ReentrantReadWriteLock(); + private final Lock readLock = cacheLock.readLock(); + private final Lock writeLock = cacheLock.writeLock(); + + public DisruptorQueueImpl(String queueName, ProducerType producerType, int bufferSize, WaitStrategy wait) { + this._queueName = PREFIX + queueName; + _buffer = RingBuffer.create(producerType, new ObjectEventFactory(), bufferSize, wait); + _consumer = new Sequence(); + _barrier = _buffer.newBarrier(); + _buffer.addGatingSequences(_consumer); + if (producerType == ProducerType.SINGLE) { + consumerStartedFlag = true; + } else { + // make sure we flush the pending messages in cache first + if (bufferSize < 2) { + throw new RuntimeException("QueueSize must >= 2"); + } + try { + publishDirect(FLUSH_CACHE, true); + } catch (InsufficientCapacityException e) { + throw new RuntimeException("This code should be unreachable!", e); + } + } + } + + public String getName() { + return _queueName; + } + + public void consumeBatch(EventHandler handler) { + consumeBatchToCursor(_barrier.getCursor(), handler); + } + + public void haltWithInterrupt() { + publish(INTERRUPT); + } + + public Object poll() { + // @@@ + // should use _cache.isEmpty, but it is slow + // I will change the logic later + if (consumerStartedFlag == false) { + return _cache.poll(); + } + + final long nextSequence = _consumer.get() + 1; + if (nextSequence <= _barrier.getCursor()) { + MutableObject mo = _buffer.get(nextSequence); + _consumer.set(nextSequence); + Object ret = mo.o; + mo.setObject(null); + return ret; + } + return null; + } + + public Object take() { + // @@@ + // should use _cache.isEmpty, but it is slow + // I will change the logic later + if (consumerStartedFlag == false) { + return _cache.poll(); + } + + final long nextSequence = _consumer.get() + 1; + // final long availableSequence; + try { + _barrier.waitFor(nextSequence); + } catch (AlertException e) { + LOG.error(e.getMessage(), e); + throw new RuntimeException(e); + } catch (InterruptedException e) { + LOG.error("InterruptedException " + e.getCause()); + // throw new RuntimeException(e); + return null; + } catch (TimeoutException e) { + //LOG.error(e.getCause(), e); + return null; + } + MutableObject mo = _buffer.get(nextSequence); + _consumer.set(nextSequence); + Object ret = mo.o; + mo.setObject(null); + return ret; + } + + public void consumeBatchWhenAvailable(EventHandler handler) { + try { + final long nextSequence = _consumer.get() + 1; + final long availableSequence = _barrier.waitFor(nextSequence); + if (availableSequence >= nextSequence) { + consumeBatchToCursor(availableSequence, handler); + } + } catch (AlertException e) { + LOG.error(e.getMessage(), e); + throw new RuntimeException(e); + } catch (InterruptedException e) { + LOG.error("InterruptedException " + e.getCause()); + return; + } catch (TimeoutException e) { + //LOG.error(e.getCause(), e); + return; + } + } + + public void consumeBatchToCursor(long cursor, EventHandler handler) { + for (long curr = _consumer.get() + 1; curr <= cursor; curr++) { + try { + MutableObject mo = _buffer.get(curr); + Object o = mo.o; + mo.setObject(null); + if (o == FLUSH_CACHE) { + Object c = null; + while (true) { + c = _cache.poll(); + if (c == null) + break; + else + handler.onEvent(c, curr, true); + } + } else if (o == INTERRUPT) { + throw new InterruptedException("Disruptor processing interrupted"); + } else { + handler.onEvent(o, curr, curr == cursor); + } + } catch (InterruptedException e) { + // throw new RuntimeException(e); + LOG.error(e.getMessage(), e); + return; + } catch (Exception e) { + LOG.error(e.getMessage(), e); + throw new RuntimeException(e); + } + } + // TODO: only set this if the consumer cursor has changed? + _consumer.set(cursor); + } + + /* + * Caches until consumerStarted is called, upon which the cache is flushed to the consumer + */ + public void publish(Object obj) { + try { + publish(obj, true); + } catch (InsufficientCapacityException ex) { + throw new RuntimeException("This code should be unreachable!"); + } + } + + public void tryPublish(Object obj) throws InsufficientCapacityException { + publish(obj, false); + } + + public void publish(Object obj, boolean block) throws InsufficientCapacityException { + + boolean publishNow = consumerStartedFlag; + + if (!publishNow) { + readLock.lock(); + try { + publishNow = consumerStartedFlag; + if (!publishNow) { + _cache.add(obj); + } + } finally { + readLock.unlock(); + } + } + + if (publishNow) { + publishDirect(obj, block); + } + } + + protected void publishDirect(Object obj, boolean block) throws InsufficientCapacityException { + final long id; + if (block) { + id = _buffer.next(); + } else { + id = _buffer.tryNext(1); + } + final MutableObject m = _buffer.get(id); + m.setObject(obj); + _buffer.publish(id); + } + + public void consumerStarted() { + + writeLock.lock(); + consumerStartedFlag = true; + + writeLock.unlock(); + } + + public void clear() { + while (population() != 0L) { + poll(); + } + } + + public long population() { + return (writePos() - readPos()); + } + + public long capacity() { + return _buffer.getBufferSize(); + } + + public long writePos() { + return _buffer.getCursor(); + } + + public long readPos() { + return _consumer.get(); + } + + public float pctFull() { + return (1.0F * population() / capacity()); + } + + @Override + public Object getState() { + // get readPos then writePos so it's never an under-estimate + long rp = readPos(); + long wp = writePos(); + state.put("capacity", capacity()); + state.put("population", wp - rp); + state.put("write_pos", wp); + state.put("read_pos", rp); + return state; + } + + public static class ObjectEventFactory implements EventFactory { + @Override + public MutableObject newInstance() { + return new MutableObject(); + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/DisruptorWrapBlockingQueue.java b/jstorm-core/src/main/java/backtype/storm/utils/DisruptorWrapBlockingQueue.java new file mode 100755 index 000000000..af5618b1a --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/DisruptorWrapBlockingQueue.java @@ -0,0 +1,200 @@ +/** + * 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 backtype.storm.utils; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.LinkedBlockingDeque; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.lmax.disruptor.EventFactory; +import com.lmax.disruptor.EventHandler; +import com.lmax.disruptor.InsufficientCapacityException; +import com.lmax.disruptor.WaitStrategy; +import com.lmax.disruptor.dsl.ProducerType; + +/** + * + * A single consumer queue that uses the LMAX Disruptor. They key to the performance is the ability to catch up to the producer by processing tuples in batches. + */ +public class DisruptorWrapBlockingQueue extends DisruptorQueue { + private static final Logger LOG = LoggerFactory.getLogger(DisruptorWrapBlockingQueue.class); + + private static final long QUEUE_CAPACITY = 512; + private LinkedBlockingDeque queue; + + private String queueName; + + public DisruptorWrapBlockingQueue(String queueName, ProducerType producerType, int bufferSize, WaitStrategy wait) { + this.queueName = queueName; + queue = new LinkedBlockingDeque(); + } + + public String getName() { + return queueName; + } + + // poll method + public void consumeBatch(EventHandler handler) { + consumeBatchToCursor(0, handler); + } + + public void haltWithInterrupt() { + } + + public Object poll() { + return queue.poll(); + } + + public Object take() { + try { + return queue.take(); + } catch (InterruptedException e) { + return null; + } + } + + public void drainQueue(Object object, EventHandler handler) { + while (object != null) { + try { + handler.onEvent(object, 0, false); + object = queue.poll(); + } catch (InterruptedException e) { + LOG.warn("Occur interrupt error, " + object); + break; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + public void consumeBatchWhenAvailable(EventHandler handler) { + Object object = queue.poll(); + if (object == null) { + try { + object = queue.take(); + } catch (InterruptedException e) { + LOG.warn("Occur interrupt error, " + object); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + drainQueue(object, handler); + + } + + public void consumeBatchToCursor(long cursor, EventHandler handler) { + Object object = queue.poll(); + drainQueue(object, handler); + } + + /* + * Caches until consumerStarted is called, upon which the cache is flushed to the consumer + */ + public void publish(Object obj) { + boolean isSuccess = queue.offer(obj); + while (isSuccess == false) { + try { + Thread.sleep(1); + } catch (InterruptedException e) { + } + isSuccess = queue.offer(obj); + } + + } + + public void tryPublish(Object obj) throws InsufficientCapacityException { + boolean isSuccess = queue.offer(obj); + if (isSuccess == false) { + throw InsufficientCapacityException.INSTANCE; + } + + } + + public void publish(Object obj, boolean block) throws InsufficientCapacityException { + if (block == true) { + publish(obj); + } else { + tryPublish(obj); + } + } + + public void consumerStarted() { + } + + private void flushCache() { + } + + public void clear() { + queue.clear(); + } + + public long population() { + return queue.size(); + } + + public long capacity() { + long used = queue.size(); + if (used < QUEUE_CAPACITY) { + return QUEUE_CAPACITY; + } else { + return used; + } + } + + public long writePos() { + return 0; + } + + public long readPos() { + return queue.size(); + } + + public float pctFull() { + long used = queue.size(); + if (used < QUEUE_CAPACITY) { + return (1.0F * used / QUEUE_CAPACITY); + } else { + return 1.0f; + } + } + + @Override + public Object getState() { + Map state = new HashMap(); + // get readPos then writePos so it's never an under-estimate + long rp = readPos(); + long wp = writePos(); + state.put("capacity", capacity()); + state.put("population", wp - rp); + state.put("write_pos", wp); + state.put("read_pos", rp); + return state; + } + + public static class ObjectEventFactory implements EventFactory { + @Override + public MutableObject newInstance() { + return new MutableObject(); + } + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/ExtendedThreadPoolExecutor.java b/jstorm-core/src/main/java/backtype/storm/utils/ExtendedThreadPoolExecutor.java new file mode 100755 index 000000000..4614366ad --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/ExtendedThreadPoolExecutor.java @@ -0,0 +1,67 @@ +package backtype.storm.utils; + +/** + * 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. + */ + +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CancellationException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.RejectedExecutionHandler; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +public class ExtendedThreadPoolExecutor extends ThreadPoolExecutor{ + + public ExtendedThreadPoolExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime, TimeUnit unit, BlockingQueue workQueue) { + super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue); + } + + public ExtendedThreadPoolExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime, TimeUnit unit, BlockingQueue workQueue, ThreadFactory threadFactory) { + super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory); + } + + public ExtendedThreadPoolExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime, TimeUnit unit, BlockingQueue workQueue, RejectedExecutionHandler handler) { + super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, handler); + } + + public ExtendedThreadPoolExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime, TimeUnit unit, BlockingQueue workQueue, ThreadFactory threadFactory, RejectedExecutionHandler handler) { + super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, handler); + } + + @Override + protected void afterExecute(Runnable r, Throwable t) { + super.afterExecute(r, t); + if (t == null && r instanceof Future) { + try { + Object result = ((Future) r).get(); + } catch (CancellationException ce) { + t = ce; + } catch (ExecutionException ee) { + t = ee.getCause(); + } catch (InterruptedException ie) { + // If future got interrupted exception, we want to interrupt parent thread itself. + Thread.currentThread().interrupt(); + } + } + if (t != null) { + Utils.handleUncaughtException(t); + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/IndifferentAccessMap.java b/jstorm-core/src/main/java/backtype/storm/utils/IndifferentAccessMap.java new file mode 100755 index 000000000..c0190ccec --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/IndifferentAccessMap.java @@ -0,0 +1,177 @@ +/** + * 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 backtype.storm.utils; + + +import clojure.lang.ILookup; +import clojure.lang.ISeq; +import clojure.lang.AFn; +import clojure.lang.IPersistentMap; +import clojure.lang.PersistentArrayMap; +import clojure.lang.IMapEntry; +import clojure.lang.IPersistentCollection; +import clojure.lang.Keyword; +import java.util.Iterator; +import java.util.Map; +import java.util.Collection; +import java.util.Set; + +public class IndifferentAccessMap extends AFn implements ILookup, IPersistentMap, Map { + + protected IPersistentMap _map; + + protected IndifferentAccessMap() { + } + + public IndifferentAccessMap(IPersistentMap map) { + setMap(map); + } + + public IPersistentMap getMap() { + return _map; + } + + public IPersistentMap setMap(IPersistentMap map) { + _map = map; + return _map; + } + + public int size() { + return ((Map) getMap()).size(); + } + + public int count() { + return size(); + } + + public ISeq seq() { + return getMap().seq(); + } + + @Override + public Object valAt(Object o) { + if(o instanceof Keyword) { + return valAt(((Keyword) o).getName()); + } + return getMap().valAt(o); + } + + @Override + public Object valAt(Object o, Object def) { + Object ret = valAt(o); + if(ret==null) ret = def; + return ret; + } + + /* IFn */ + @Override + public Object invoke(Object o) { + return valAt(o); + } + + @Override + public Object invoke(Object o, Object notfound) { + return valAt(o, notfound); + } + + /* IPersistentMap */ + /* Naive implementation, but it might be good enough */ + public IPersistentMap assoc(Object k, Object v) { + if(k instanceof Keyword) return assoc(((Keyword) k).getName(), v); + + return new IndifferentAccessMap(getMap().assoc(k, v)); + } + + public IPersistentMap assocEx(Object k, Object v) { + if(k instanceof Keyword) return assocEx(((Keyword) k).getName(), v); + + return new IndifferentAccessMap(getMap().assocEx(k, v)); + } + + public IPersistentMap without(Object k) { + if(k instanceof Keyword) return without(((Keyword) k).getName()); + + return new IndifferentAccessMap(getMap().without(k)); + } + + public boolean containsKey(Object k) { + if(k instanceof Keyword) return containsKey(((Keyword) k).getName()); + return getMap().containsKey(k); + } + + public IMapEntry entryAt(Object k) { + if(k instanceof Keyword) return entryAt(((Keyword) k).getName()); + + return getMap().entryAt(k); + } + + public IPersistentCollection cons(Object o) { + return getMap().cons(o); + } + + public IPersistentCollection empty() { + return new IndifferentAccessMap(PersistentArrayMap.EMPTY); + } + + public boolean equiv(Object o) { + return getMap().equiv(o); + } + + public Iterator iterator() { + return getMap().iterator(); + } + + /* Map */ + public boolean containsValue(Object v) { + return ((Map) getMap()).containsValue(v); + } + + public Set entrySet() { + return ((Map) getMap()).entrySet(); + } + + public Object get(Object k) { + return valAt(k); + } + + public boolean isEmpty() { + return ((Map) getMap()).isEmpty(); + } + + public Set keySet() { + return ((Map) getMap()).keySet(); + } + + public Collection values() { + return ((Map) getMap()).values(); + } + + /* Not implemented */ + public void clear() { + throw new UnsupportedOperationException(); + } + public Object put(Object k, Object v) { + throw new UnsupportedOperationException(); + } + public void putAll(Map m) { + throw new UnsupportedOperationException(); + } + public Object remove(Object k) { + throw new UnsupportedOperationException(); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/InprocMessaging.java b/jstorm-core/src/main/java/backtype/storm/utils/InprocMessaging.java new file mode 100755 index 000000000..b20c7755b --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/InprocMessaging.java @@ -0,0 +1,59 @@ +/** + * 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 backtype.storm.utils; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.LinkedBlockingQueue; + +public class InprocMessaging { + private static Map> _queues = new HashMap>(); + private static final Object _lock = new Object(); + private static int port = 1; + + public static int acquireNewPort() { + int ret; + synchronized(_lock) { + ret = port; + port++; + } + return ret; + } + + public static void sendMessage(int port, Object msg) { + getQueue(port).add(msg); + } + + public static Object takeMessage(int port) throws InterruptedException { + return getQueue(port).take(); + } + + public static Object pollMessage(int port) { + return getQueue(port).poll(); + } + + private static LinkedBlockingQueue getQueue(int port) { + synchronized(_lock) { + if(!_queues.containsKey(port)) { + _queues.put(port, new LinkedBlockingQueue()); + } + return _queues.get(port); + } + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/KeyedRoundRobinQueue.java b/jstorm-core/src/main/java/backtype/storm/utils/KeyedRoundRobinQueue.java new file mode 100755 index 000000000..3cb455d4e --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/KeyedRoundRobinQueue.java @@ -0,0 +1,68 @@ +/** + * 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 backtype.storm.utils; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.concurrent.Semaphore; + +public class KeyedRoundRobinQueue { + private final Object _lock = new Object(); + private Semaphore _size = new Semaphore(0); + private Map> _queues = new HashMap>(); + private List _keyOrder = new ArrayList(); + private int _currIndex = 0; + + public void add(Object key, V val) { + synchronized(_lock) { + Queue queue = _queues.get(key); + if(queue==null) { + queue = new LinkedList(); + _queues.put(key, queue); + _keyOrder.add(key); + } + queue.add(val); + } + _size.release(); + } + + public V take() throws InterruptedException { + _size.acquire(); + synchronized(_lock) { + Object key = _keyOrder.get(_currIndex); + Queue queue = _queues.get(key); + V ret = queue.remove(); + if(queue.isEmpty()) { + _keyOrder.remove(_currIndex); + _queues.remove(key); + if(_keyOrder.size()==0) { + _currIndex = 0; + } else { + _currIndex = _currIndex % _keyOrder.size(); + } + } else { + _currIndex = (_currIndex + 1) % _keyOrder.size(); + } + return ret; + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/ListDelegate.java b/jstorm-core/src/main/java/backtype/storm/utils/ListDelegate.java new file mode 100755 index 000000000..1e091f082 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/ListDelegate.java @@ -0,0 +1,156 @@ +/** + * 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 backtype.storm.utils; + +import java.util.Collection; +import java.util.Iterator; +import java.util.ArrayList; +import java.util.List; +import java.util.ListIterator; + +public class ListDelegate implements List { + private List _delegate; + + public ListDelegate() { + _delegate = new ArrayList(); + } + + public void setDelegate(List delegate) { + _delegate = delegate; + } + + public List getDelegate() { + return _delegate; + } + + @Override + public int size() { + return _delegate.size(); + } + + @Override + public boolean isEmpty() { + return _delegate.isEmpty(); + } + + @Override + public boolean contains(Object o) { + return _delegate.contains(o); + } + + @Override + public Iterator iterator() { + return _delegate.iterator(); + } + + @Override + public Object[] toArray() { + return _delegate.toArray(); + } + + @Override + public T[] toArray(T[] ts) { + return _delegate.toArray(ts); + } + + @Override + public boolean add(Object e) { + return _delegate.add(e); + } + + @Override + public boolean remove(Object o) { + return _delegate.remove(o); + } + + @Override + public boolean containsAll(Collection clctn) { + return _delegate.containsAll(clctn); + } + + @Override + public boolean addAll(Collection clctn) { + return _delegate.addAll(clctn); + } + + @Override + public boolean addAll(int i, Collection clctn) { + return _delegate.addAll(i, clctn); + } + + @Override + public boolean removeAll(Collection clctn) { + return _delegate.removeAll(clctn); + } + + @Override + public boolean retainAll(Collection clctn) { + return _delegate.retainAll(clctn); + } + + @Override + public void clear() { + _delegate.clear(); + } + + @Override + public Object get(int i) { + return _delegate.get(i); + } + + @Override + public Object set(int i, Object e) { + return _delegate.set(i, e); + } + + @Override + public void add(int i, Object e) { + _delegate.add(i, e); + } + + @Override + public Object remove(int i) { + return _delegate.remove(i); + } + + @Override + public int indexOf(Object o) { + return _delegate.indexOf(o); + } + + @Override + public int lastIndexOf(Object o) { + return _delegate.lastIndexOf(o); + } + + @Override + public ListIterator listIterator() { + return _delegate.listIterator(); + } + + @Override + public ListIterator listIterator(int i) { + return _delegate.listIterator(i); + } + + @Override + public List subList(int i, int i1) { + return _delegate.subList(i, i1); + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/LocalState.java b/jstorm-core/src/main/java/backtype/storm/utils/LocalState.java new file mode 100755 index 000000000..0d8292fcf --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/LocalState.java @@ -0,0 +1,99 @@ +/** + * 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 backtype.storm.utils; + +import org.apache.commons.io.FileUtils; + +import java.io.File; +import java.util.Map; +import java.util.HashMap; +import java.io.IOException; + +/** + * A simple, durable, atomic K/V database. *Very inefficient*, should only be + * used for occasional reads/writes. Every read/write hits disk. + * + * @@@ + * Right now, This class hasn't upgrade to storm's LocalState + * It is need define every type in thrift, it is too complicated to do + */ +public class LocalState { + private VersionedStore _vs; + + public LocalState(String backingDir) throws IOException { + _vs = new VersionedStore(backingDir); + } + + public synchronized Map snapshot() throws IOException { + int attempts = 0; + while (true) { + String latestPath = _vs.mostRecentVersionPath(); + if (latestPath == null) + return new HashMap(); + try { + return (Map) Utils.javaDeserialize(FileUtils + .readFileToByteArray(new File(latestPath))); + } catch (IOException e) { + attempts++; + if (attempts >= 10) { + throw e; + } + } + } + } + + public Object get(Object key) throws IOException { + return snapshot().get(key); + } + + public synchronized void put(Object key, Object val) throws IOException { + put(key, val, true); + } + + public synchronized void put(Object key, Object val, boolean cleanup) + throws IOException { + Map curr = snapshot(); + curr.put(key, val); + persist(curr, cleanup); + } + + public synchronized void remove(Object key) throws IOException { + remove(key, true); + } + + public synchronized void remove(Object key, boolean cleanup) + throws IOException { + Map curr = snapshot(); + curr.remove(key); + persist(curr, cleanup); + } + + public synchronized void cleanup(int keepVersions) throws IOException { + _vs.cleanup(keepVersions); + } + + private void persist(Map val, boolean cleanup) + throws IOException { + byte[] toWrite = Utils.serialize(val); + String newPath = _vs.createVersion(); + FileUtils.writeByteArrayToFile(new File(newPath), toWrite); + _vs.succeedVersion(newPath); + if (cleanup) + _vs.cleanup(4); + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/utils/Monitor.java b/jstorm-core/src/main/java/backtype/storm/utils/Monitor.java new file mode 100755 index 000000000..eb57e9917 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/Monitor.java @@ -0,0 +1,268 @@ +/** + * 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 backtype.storm.utils; + +import backtype.storm.generated.*; + +import java.util.HashSet; +import java.util.List; +import java.util.Map; + +/** + * Deprecated in JStorm + * @author zhongyan.feng + * + */ +@Deprecated +public class Monitor { + private static final String WATCH_TRANSFERRED = "transferred"; + private static final String WATCH_EMITTED = "emitted"; + + private int _interval = 4; + private String _topology; + private String _component; + private String _stream; + private String _watch; + + private static class MetricsState { + private long lastTime = 0; + private long lastStatted = 0; + + private MetricsState(long lastTime, long lastStatted) { + this.lastTime = lastTime; + this.lastStatted = lastStatted; + } + + public long getLastStatted() { + return lastStatted; + } + + public void setLastStatted(long lastStatted) { + this.lastStatted = lastStatted; + } + + public long getLastTime() { + return lastTime; + } + + public void setLastTime(long lastTime) { + this.lastTime = lastTime; + } + } + + private static class Poller { + private long startTime = 0; + private long pollMs = 0; + + private Poller(long startTime, long pollMs) { + this.startTime = startTime; + this.pollMs = pollMs; + } + + public long nextPoll() throws InterruptedException { + long now = System.currentTimeMillis(); + long cycle = (now - startTime) / pollMs; + long wakeupTime = startTime + (pollMs * (cycle + 1)); + long sleepTime = wakeupTime - now; + if (sleepTime > 0) { + Thread.sleep(sleepTime); + } + now = System.currentTimeMillis(); + return now; + } + + public long getStartTime() { + return startTime; + } + + public void setStartTime(long startTime) { + this.startTime = startTime; + } + + public long getPollMs() { + return pollMs; + } + + public void setPollMs(long pollMs) { + this.pollMs = pollMs; + } + } + + /** + * @@@ Don't be compatible with Storm + * + * Here skip the logic + * @param client + * @param topology + * @return + * @throws Exception + */ + private HashSet getComponents(Nimbus.Client client, String topology) throws Exception{ + HashSet components = new HashSet(); + ClusterSummary clusterSummary = client.getClusterInfo(); + TopologySummary topologySummary = null; + for (TopologySummary ts: clusterSummary.get_topologies()) { + if (topology.equals(ts.get_name())) { + topologySummary = ts; + break; + } + } + if (topologySummary == null) { + throw new IllegalArgumentException("topology: " + topology + " not found"); + } else { + String id = topologySummary.get_id(); +// GetInfoOptions getInfoOpts = new GetInfoOptions(); +// getInfoOpts.set_num_err_choice(NumErrorsChoice.NONE); +// TopologyInfo info = client.getTopologyInfoWithOpts(id, getInfoOpts); +// for (ExecutorSummary es: info.get_executors()) { +// components.add(es.get_component_id()); +// } + } + return components; + } + + public void metrics(Nimbus.Client client) throws Exception { + if (_interval <= 0) { + throw new IllegalArgumentException("poll interval must be positive"); + } + + if (_topology == null || _topology.isEmpty()) { + throw new IllegalArgumentException("topology name must be something"); + } + + if (_component == null || _component.isEmpty()) { + HashSet components = getComponents(client, _topology); + System.out.println("Available components for " + _topology + " :"); + System.out.println("------------------"); + for (String comp : components) { + System.out.println(comp); + } + System.out.println("------------------"); + System.out.println("Please use -m to specify one component"); + return; + } + + if (_stream == null || _stream.isEmpty()) { + throw new IllegalArgumentException("stream name must be something"); + } + + if ( !WATCH_TRANSFERRED.equals(_watch) && !WATCH_EMITTED.equals(_watch)) { + throw new IllegalArgumentException("watch item must either be transferred or emitted"); + } + System.out.println("topology\tcomponent\tparallelism\tstream\ttime-diff ms\t" + _watch + "\tthroughput (Kt/s)"); + + long pollMs = _interval * 1000; + long now = System.currentTimeMillis(); + MetricsState state = new MetricsState(now, 0); + Poller poller = new Poller(now, pollMs); + + do { + metrics(client, now, state); + try { + now = poller.nextPoll(); + } catch (InterruptedException e) { + e.printStackTrace(); + break; + } + } while (true); + } + + public void metrics(Nimbus.Client client, long now, MetricsState state) throws Exception { + long totalStatted = 0; + + int componentParallelism = 0; + boolean streamFound = false; + ClusterSummary clusterSummary = client.getClusterInfo(); + TopologySummary topologySummary = null; + for (TopologySummary ts: clusterSummary.get_topologies()) { + if (_topology.equals(ts.get_name())) { + topologySummary = ts; + break; + } + } + if (topologySummary == null) { + throw new IllegalArgumentException("topology: " + _topology + " not found"); + } else { +// String id = topologySummary.get_id(); +// GetInfoOptions getInfoOpts = new GetInfoOptions(); +// getInfoOpts.set_num_err_choice(NumErrorsChoice.NONE); +// TopologyInfo info = client.getTopologyInfoWithOpts(id, getInfoOpts); +// for (ExecutorSummary es: info.get_executors()) { +// if (_component.equals(es.get_component_id())) { +// componentParallelism ++; +// ExecutorStats stats = es.get_stats(); +// if (stats != null) { +// Map> statted = +// WATCH_EMITTED.equals(_watch) ? stats.get_emitted() : stats.get_transferred(); +// if ( statted != null) { +// Map e2 = statted.get(":all-time"); +// if (e2 != null) { +// Long stream = e2.get(_stream); +// if (stream != null){ +// streamFound = true; +// totalStatted += stream; +// } +// } +// } +// } +// } +// } + } + + if (componentParallelism <= 0) { + HashSet components = getComponents(client, _topology); + System.out.println("Available components for " + _topology + " :"); + System.out.println("------------------"); + for (String comp : components) { + System.out.println(comp); + } + System.out.println("------------------"); + throw new IllegalArgumentException("component: " + _component + " not found"); + } + + if (!streamFound) { + throw new IllegalArgumentException("stream: " + _stream + " not found"); + } + long timeDelta = now - state.getLastTime(); + long stattedDelta = totalStatted - state.getLastStatted(); + state.setLastTime(now); + state.setLastStatted(totalStatted); + double throughput = (stattedDelta == 0 || timeDelta == 0) ? 0.0 : ((double)stattedDelta/(double)timeDelta); + System.out.println(_topology+"\t"+_component+"\t"+componentParallelism+"\t"+_stream+"\t"+timeDelta+"\t"+stattedDelta+"\t"+throughput); + } + + public void set_interval(int _interval) { + this._interval = _interval; + } + + public void set_topology(String _topology) { + this._topology = _topology; + } + + public void set_component(String _component) { + this._component = _component; + } + + public void set_stream(String _stream) { + this._stream = _stream; + } + + public void set_watch(String _watch) { + this._watch = _watch; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/MutableInt.java b/jstorm-core/src/main/java/backtype/storm/utils/MutableInt.java new file mode 100755 index 000000000..326ade02e --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/MutableInt.java @@ -0,0 +1,43 @@ +/** + * 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 backtype.storm.utils; + +public class MutableInt { + int val; + + public MutableInt(int val) { + this.val = val; + } + + public void set(int val) { + this.val = val; + } + + public int get() { + return val; + } + + public int increment() { + return increment(1); + } + + public int increment(int amt) { + val+=amt; + return val; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/MutableLong.java b/jstorm-core/src/main/java/backtype/storm/utils/MutableLong.java new file mode 100755 index 000000000..a744c1cee --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/MutableLong.java @@ -0,0 +1,43 @@ +/** + * 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 backtype.storm.utils; + +public class MutableLong { + long val; + + public MutableLong(long val) { + this.val = val; + } + + public void set(long val) { + this.val = val; + } + + public long get() { + return val; + } + + public long increment() { + return increment(1); + } + + public long increment(long amt) { + val+=amt; + return val; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/MutableObject.java b/jstorm-core/src/main/java/backtype/storm/utils/MutableObject.java new file mode 100755 index 000000000..d5cb7dbf7 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/MutableObject.java @@ -0,0 +1,38 @@ +/** + * 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 backtype.storm.utils; + +public class MutableObject { + Object o = null; + + public MutableObject() { + + } + + public MutableObject(Object o) { + this.o = o; + } + + public void setObject(Object o) { + this.o = o; + } + + public Object getObject() { + return o; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/NimbusClient.java b/jstorm-core/src/main/java/backtype/storm/utils/NimbusClient.java new file mode 100755 index 000000000..5829b678a --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/NimbusClient.java @@ -0,0 +1,107 @@ +/** + * 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 backtype.storm.utils; + +import java.util.Map; + +import org.apache.thrift.TException; +import org.apache.thrift.transport.TTransportException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.generated.Nimbus; +import backtype.storm.security.auth.ThriftClient; +import backtype.storm.security.auth.ThriftConnectionType; + +public class NimbusClient extends ThriftClient { + private static final Logger LOG = LoggerFactory.getLogger(NimbusClient.class); + + private Nimbus.Client _client; + private static String clientVersion = Utils.getVersion(); + + @SuppressWarnings("unchecked") + public static NimbusClient getConfiguredClient(Map conf) { + return getConfiguredClient(conf, null); + } + + @SuppressWarnings("unchecked") + public static NimbusClient getConfiguredClient(Map conf, Integer timeout) { + return getConfiguredClientAs(conf, timeout, null); + } + + public static NimbusClient getConfiguredClientAs(Map conf, String asUser) { + return getConfiguredClientAs(conf, null, asUser); + } + + public static void checkVersion(NimbusClient client) { + String serverVersion; + try { + serverVersion = client.getClient().getVersion(); + } catch (TException e) { + // TODO Auto-generated catch block + LOG.warn("Failed to get nimbus version "); + return ; + } + if (!clientVersion.equals(serverVersion)) { + LOG.warn("Your client version: " + clientVersion + " but nimbus version: " + serverVersion); + } + } + + public static NimbusClient getConfiguredClientAs(Map conf, Integer timeout, String asUser) { + try { + if(conf.containsKey(Config.STORM_DO_AS_USER)) { + if(asUser != null && !asUser.isEmpty()) { + LOG.warn("You have specified a doAsUser as param {} and a doAsParam as config, config will take precedence." + , asUser, conf.get(Config.STORM_DO_AS_USER)); + } + asUser = (String) conf.get(Config.STORM_DO_AS_USER); + } + + NimbusClient client = new NimbusClient(conf, null, null, timeout, asUser); + checkVersion(client); + return client; + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + public NimbusClient(Map conf, String host, int port) throws TTransportException { + this(conf, host, port, null); + } + + public NimbusClient(Map conf, String host, int port, Integer timeout) throws TTransportException { + super(conf, ThriftConnectionType.NIMBUS, host, port, timeout, null); + _client = new Nimbus.Client(_protocol); + } + + public NimbusClient(Map conf, String host, Integer port, Integer timeout, String asUser) throws TTransportException { + super(conf, ThriftConnectionType.NIMBUS, host, port, timeout, asUser); + _client = new Nimbus.Client(_protocol); + } + + public NimbusClient(Map conf, String host) throws TTransportException { + super(conf, ThriftConnectionType.NIMBUS, host, null, null, null); + _client = new Nimbus.Client(_protocol); + } + + public Nimbus.Client getClient() { + return _client; + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/RegisteredGlobalState.java b/jstorm-core/src/main/java/backtype/storm/utils/RegisteredGlobalState.java new file mode 100755 index 000000000..48053fcdd --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/RegisteredGlobalState.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.utils; + +import java.util.HashMap; +import java.util.UUID; + +/** + * This class is used as part of testing Storm. It is used to keep track of "global metrics" + * in an atomic way. For example, it is used for doing fine-grained detection of when a + * local Storm cluster is idle by tracking the number of transferred tuples vs the number of processed + * tuples. + */ +public class RegisteredGlobalState { + private static HashMap _states = new HashMap(); + private static final Object _lock = new Object(); + + public static Object globalLock() { + return _lock; + } + + public static String registerState(Object init) { + synchronized(_lock) { + String id = UUID.randomUUID().toString(); + _states.put(id, init); + return id; + } + } + + public static void setState(String id, Object init) { + synchronized(_lock) { + _states.put(id, init); + } + } + + public static Object getState(String id) { + synchronized(_lock) { + Object ret = _states.get(id); + //System.out.println("State: " + ret.toString()); + return ret; + } + } + + public static void clearState(String id) { + synchronized(_lock) { + _states.remove(id); + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/RotatingMap.java b/jstorm-core/src/main/java/backtype/storm/utils/RotatingMap.java new file mode 100644 index 000000000..2ed0e33f9 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/RotatingMap.java @@ -0,0 +1,125 @@ +/** + * 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 backtype.storm.utils; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.Map; +import java.util.Map.Entry; + +/** + * Expires keys that have not been updated in the configured number of seconds. + * The algorithm used will take between expirationSecs and + * expirationSecs * (1 + 1 / (numBuckets-1)) to actually expire the message. + * + * get, put, remove, containsKey, and size take O(numBuckets) time to run. + * + * The advantage of this design is that the expiration thread only locks the object + * for O(1) time, meaning the object is essentially always available for gets/puts. + */ +@Deprecated +public class RotatingMap { + //this default ensures things expire at most 50% past the expiration time + private static final int DEFAULT_NUM_BUCKETS = 3; + + public static interface ExpiredCallback { + public void expire(K key, V val); + } + + private LinkedList> _buckets; + + private ExpiredCallback _callback; + + public RotatingMap(int numBuckets, ExpiredCallback callback) { + if(numBuckets<2) { + throw new IllegalArgumentException("numBuckets must be >= 2"); + } + _buckets = new LinkedList>(); + for(int i=0; i()); + } + + _callback = callback; + } + + public RotatingMap(ExpiredCallback callback) { + this(DEFAULT_NUM_BUCKETS, callback); + } + + public RotatingMap(int numBuckets) { + this(numBuckets, null); + } + + public Map rotate() { + Map dead = _buckets.removeLast(); + _buckets.addFirst(new HashMap()); + if(_callback!=null) { + for(Entry entry: dead.entrySet()) { + _callback.expire(entry.getKey(), entry.getValue()); + } + } + return dead; + } + + public boolean containsKey(K key) { + for(HashMap bucket: _buckets) { + if(bucket.containsKey(key)) { + return true; + } + } + return false; + } + + public V get(K key) { + for(HashMap bucket: _buckets) { + if(bucket.containsKey(key)) { + return bucket.get(key); + } + } + return null; + } + + public void put(K key, V value) { + Iterator> it = _buckets.iterator(); + HashMap bucket = it.next(); + bucket.put(key, value); + while(it.hasNext()) { + bucket = it.next(); + bucket.remove(key); + } + } + + + public Object remove(K key) { + for(HashMap bucket: _buckets) { + if(bucket.containsKey(key)) { + return bucket.remove(key); + } + } + return null; + } + + public int size() { + int size = 0; + for(HashMap bucket: _buckets) { + size+=bucket.size(); + } + return size; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/ServiceRegistry.java b/jstorm-core/src/main/java/backtype/storm/utils/ServiceRegistry.java new file mode 100755 index 000000000..724bc3e59 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/ServiceRegistry.java @@ -0,0 +1,47 @@ +/** + * 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 backtype.storm.utils; + +import java.util.HashMap; +import java.util.UUID; + +// this class should be combined with RegisteredGlobalState +public class ServiceRegistry { + private static HashMap _services = new HashMap(); + private static final Object _lock = new Object(); + + public static String registerService(Object service) { + synchronized(_lock) { + String id = UUID.randomUUID().toString(); + _services.put(id, service); + return id; + } + } + + public static Object getService(String id) { + synchronized(_lock) { + return _services.get(id); + } + } + + public static void unregisterService(String id) { + synchronized(_lock) { + _services.remove(id); + } + } +} diff --git a/jstorm-client/src/main/java/backtype/storm/utils/ShellProcess.java b/jstorm-core/src/main/java/backtype/storm/utils/ShellProcess.java similarity index 87% rename from jstorm-client/src/main/java/backtype/storm/utils/ShellProcess.java rename to jstorm-core/src/main/java/backtype/storm/utils/ShellProcess.java index a847976b3..78f47d64b 100644 --- a/jstorm-client/src/main/java/backtype/storm/utils/ShellProcess.java +++ b/jstorm-core/src/main/java/backtype/storm/utils/ShellProcess.java @@ -31,11 +31,11 @@ import java.util.List; import java.util.Map; -import org.apache.commons.io.IOUtils; -import org.apache.log4j.Logger; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ShellProcess implements Serializable { - public static Logger LOG = Logger.getLogger(ShellProcess.class); + public static Logger LOG = LoggerFactory.getLogger(ShellProcess.class); public static Logger ShellLogger; private Process _subprocess; private InputStream processErrorStream; @@ -52,7 +52,7 @@ public Number launch(Map conf, TopologyContext context) { ProcessBuilder builder = new ProcessBuilder(command); builder.directory(new File(context.getCodeDir())); - ShellLogger = Logger.getLogger(context.getThisComponentId()); + ShellLogger = LoggerFactory.getLogger(context.getThisComponentId()); this.componentName = context.getThisComponentId(); this.serializer = getSerializer(conf); @@ -135,7 +135,14 @@ public void logErrorStream() { public String getErrorsString() { if (processErrorStream != null) { try { - return IOUtils.toString(processErrorStream); + StringBuilder sb = new StringBuilder(); + while (processErrorStream.available() > 0) { + int bufferSize = processErrorStream.available(); + byte[] errorReadingBuffer = new byte[bufferSize]; + processErrorStream.read(errorReadingBuffer, 0, bufferSize); + sb.append(new String(errorReadingBuffer)); + } + return sb.toString(); } catch (IOException e) { return "(Unable to capture error stream)"; } @@ -179,4 +186,4 @@ public String getProcessInfoString() { public String getProcessTerminationInfoString() { return String.format(" exitCode:%s, errorString:%s ", getExitCode(), getErrorsString()); } -} \ No newline at end of file +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/ShellUtils.java b/jstorm-core/src/main/java/backtype/storm/utils/ShellUtils.java new file mode 100755 index 000000000..1065ff92f --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/ShellUtils.java @@ -0,0 +1,498 @@ +/** + * 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 backtype.storm.utils; + +import java.io.BufferedReader; +import java.io.File; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.InputStream; +import java.util.Map; +import java.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + + +abstract public class ShellUtils { + public static Logger LOG = LoggerFactory.getLogger(ShellUtils.class); + + // OSType detection + public enum OSType { + OS_TYPE_LINUX, + OS_TYPE_WIN, + OS_TYPE_SOLARIS, + OS_TYPE_MAC, + OS_TYPE_FREEBSD, + OS_TYPE_OTHER + } + + public static final OSType osType = getOSType(); + + static private OSType getOSType() { + String osName = System.getProperty("os.name"); + if (osName.startsWith("Windows")) { + return OSType.OS_TYPE_WIN; + } else if (osName.contains("SunOS") || osName.contains("Solaris")) { + return OSType.OS_TYPE_SOLARIS; + } else if (osName.contains("Mac")) { + return OSType.OS_TYPE_MAC; + } else if (osName.contains("FreeBSD")) { + return OSType.OS_TYPE_FREEBSD; + } else if (osName.startsWith("Linux")) { + return OSType.OS_TYPE_LINUX; + } else { + // Some other form of Unix + return OSType.OS_TYPE_OTHER; + } + } + + // Helper static vars for each platform + public static final boolean WINDOWS = (osType == OSType.OS_TYPE_WIN); + public static final boolean SOLARIS = (osType == OSType.OS_TYPE_SOLARIS); + public static final boolean MAC = (osType == OSType.OS_TYPE_MAC); + public static final boolean FREEBSD = (osType == OSType.OS_TYPE_FREEBSD); + public static final boolean LINUX = (osType == OSType.OS_TYPE_LINUX); + public static final boolean OTHER = (osType == OSType.OS_TYPE_OTHER); + + + /** Token separator regex used to parse Shell tool outputs */ + public static final String TOKEN_SEPARATOR_REGEX + = WINDOWS ? "[|\n\r]" : "[ \t\n\r\f]"; + + private long interval; // refresh interval in msec + private long lastTime; // last time the command was performed + final private boolean redirectErrorStream; // merge stdout and stderr + private Map environment; // env for the command execution + private File dir; + private Process process; // sub process used to execute the command + private int exitCode; + /**Time after which the executing script would be timedout*/ + protected long timeOutInterval = 0L; + /** If or not script timed out*/ + private AtomicBoolean timedOut; + + /**If or not script finished executing*/ + private volatile AtomicBoolean completed; + + public ShellUtils() { + this(0L); + } + + public ShellUtils(long interval) { + this(interval, false); + } + + /** + * @param interval the minimum duration to wait before re-executing the + * command. + */ + public ShellUtils(long interval, boolean redirectErrorStream) { + this.interval = interval; + this.lastTime = (interval<0) ? 0 : -interval; + this.redirectErrorStream = redirectErrorStream; + } + + /** set the environment for the command + * @param env Mapping of environment variables + */ + protected void setEnvironment(Map env) { + this.environment = env; + } + + /** set the working directory + * @param dir The directory where the command would be executed + */ + protected void setWorkingDirectory(File dir) { + this.dir = dir; + } + + /** a Unix command to get the current user's groups list */ + public static String[] getGroupsCommand() { + return (WINDOWS)? new String[]{"cmd", "/c", "groups"} + : new String[]{"bash", "-c", "groups"}; + } + + /** + * a Unix command to get a given user's groups list. + * If the OS is not WINDOWS, the command will get the user's primary group + * first and finally get the groups list which includes the primary group. + * i.e. the user's primary group will be included twice. + */ + public static String[] getGroupsForUserCommand(final String user) { + //'groups username' command return is non-consistent across different unixes + return new String [] {"bash", "-c", "id -gn " + user + + "&& id -Gn " + user}; + } + + + /** check to see if a command needs to be executed and execute if needed */ + protected void run() throws IOException { + if (lastTime + interval > System.currentTimeMillis()) + return; + exitCode = 0; // reset for next run + runCommand(); + } + + /** Run a command */ + private void runCommand() throws IOException { + ProcessBuilder builder = new ProcessBuilder(getExecString()); + Timer timeOutTimer = null; + ShellTimeoutTimerTask timeoutTimerTask = null; + timedOut = new AtomicBoolean(false); + completed = new AtomicBoolean(false); + + if (environment != null) { + builder.environment().putAll(this.environment); + } + if (dir != null) { + builder.directory(this.dir); + } + + builder.redirectErrorStream(redirectErrorStream); + process = builder.start(); + + if (timeOutInterval > 0) { + timeOutTimer = new Timer("Shell command timeout"); + timeoutTimerTask = new ShellTimeoutTimerTask(this); + //One time scheduling. + timeOutTimer.schedule(timeoutTimerTask, timeOutInterval); + } + final BufferedReader errReader = + new BufferedReader(new InputStreamReader(process + .getErrorStream())); + BufferedReader inReader = + new BufferedReader(new InputStreamReader(process + .getInputStream())); + final StringBuffer errMsg = new StringBuffer(); + + // read error and input streams as this would free up the buffers + // free the error stream buffer + Thread errThread = new Thread() { + @Override + public void run() { + try { + String line = errReader.readLine(); + while((line != null) && !isInterrupted()) { + errMsg.append(line); + errMsg.append(System.getProperty("line.separator")); + line = errReader.readLine(); + } + } catch(IOException ioe) { + LOG.warn("Error reading the error stream", ioe); + } + } + }; + try { + errThread.start(); + } catch (IllegalStateException ise) { } + try { + parseExecResult(inReader); // parse the output + // clear the input stream buffer + String line = inReader.readLine(); + while(line != null) { + line = inReader.readLine(); + } + // wait for the process to finish and check the exit code + exitCode = process.waitFor(); + // make sure that the error thread exits + joinThread(errThread); + completed.set(true); + //the timeout thread handling + //taken care in finally block + if (exitCode != 0) { + throw new ExitCodeException(exitCode, errMsg.toString()); + } + } catch (InterruptedException ie) { + throw new IOException(ie.toString()); + } finally { + if (timeOutTimer != null) { + timeOutTimer.cancel(); + } + // close the input stream + try { + // JDK 7 tries to automatically drain the input streams for us + // when the process exits, but since close is not synchronized, + // it creates a race if we close the stream first and the same + // fd is recycled. the stream draining thread will attempt to + // drain that fd!! it may block, OOM, or cause bizarre behavior + // see: https://bugs.openjdk.java.net/browse/JDK-8024521 + // issue is fixed in build 7u60 + InputStream stdout = process.getInputStream(); + synchronized (stdout) { + inReader.close(); + } + } catch (IOException ioe) { + LOG.warn("Error while closing the input stream", ioe); + } + if (!completed.get()) { + errThread.interrupt(); + joinThread(errThread); + } + try { + InputStream stderr = process.getErrorStream(); + synchronized (stderr) { + errReader.close(); + } + } catch (IOException ioe) { + LOG.warn("Error while closing the error stream", ioe); + } + process.destroy(); + lastTime = System.currentTimeMillis(); + } + } + + private static void joinThread(Thread t) { + while (t.isAlive()) { + try { + t.join(); + } catch (InterruptedException ie) { + if (LOG.isWarnEnabled()) { + LOG.warn("Interrupted while joining on: " + t, ie); + } + t.interrupt(); // propagate interrupt + } + } + } + + /** return an array containing the command name & its parameters */ + protected abstract String[] getExecString(); + + /** Parse the execution result */ + protected abstract void parseExecResult(BufferedReader lines) + throws IOException; + + /** get the current sub-process executing the given command + * @return process executing the command + */ + public Process getProcess() { + return process; + } + + /** + * This is an IOException with exit code added. + */ + public static class ExitCodeException extends IOException { + int exitCode; + + public ExitCodeException(int exitCode, String message) { + super(message); + this.exitCode = exitCode; + } + + public int getExitCode() { + return exitCode; + } + } + + /** + * A simple shell command executor. + * + * ShellCommandExecutorshould be used in cases where the output + * of the command needs no explicit parsing and where the command, working + * directory and the environment remains unchanged. The output of the command + * is stored as-is and is expected to be small. + */ + public static class ShellCommandExecutor extends ShellUtils { + + private String[] command; + private StringBuffer output; + + + public ShellCommandExecutor(String[] execString) { + this(execString, null); + } + + public ShellCommandExecutor(String[] execString, File dir) { + this(execString, dir, null); + } + + public ShellCommandExecutor(String[] execString, File dir, + Map env) { + this(execString, dir, env , 0L); + } + + /** + * Create a new instance of the ShellCommandExecutor to execute a command. + * + * @param execString The command to execute with arguments + * @param dir If not-null, specifies the directory which should be set + * as the current working directory for the command. + * If null, the current working directory is not modified. + * @param env If not-null, environment of the command will include the + * key-value pairs specified in the map. If null, the current + * environment is not modified. + * @param timeout Specifies the time in milliseconds, after which the + * command will be killed and the status marked as timedout. + * If 0, the command will not be timed out. + */ + public ShellCommandExecutor(String[] execString, File dir, + Map env, long timeout) { + command = execString.clone(); + if (dir != null) { + setWorkingDirectory(dir); + } + if (env != null) { + setEnvironment(env); + } + timeOutInterval = timeout; + } + + + /** Execute the shell command. */ + public void execute() throws IOException { + this.run(); + } + + @Override + public String[] getExecString() { + return command; + } + + @Override + protected void parseExecResult(BufferedReader lines) throws IOException { + output = new StringBuffer(); + char[] buf = new char[512]; + int nRead; + while ( (nRead = lines.read(buf, 0, buf.length)) > 0 ) { + output.append(buf, 0, nRead); + } + } + + /** Get the output of the shell command.*/ + public String getOutput() { + return (output == null) ? "" : output.toString(); + } + + /** + * Returns the commands of this instance. + * Arguments with spaces in are presented with quotes round; other + * arguments are presented raw + * + * @return a string representation of the object. + */ + @Override + public String toString() { + StringBuilder builder = new StringBuilder(); + String[] args = getExecString(); + for (String s : args) { + if (s.indexOf(' ') >= 0) { + builder.append('"').append(s).append('"'); + } else { + builder.append(s); + } + builder.append(' '); + } + return builder.toString(); + } + } + + /** + * To check if the passed script to shell command executor timed out or + * not. + * + * @return if the script timed out. + */ + public boolean isTimedOut() { + return timedOut.get(); + } + + /** + * Set if the command has timed out. + * + */ + private void setTimedOut() { + this.timedOut.set(true); + } + + + /** + * Static method to execute a shell command. + * Covers most of the simple cases without requiring the user to implement + * the Shell interface. + * @param cmd shell command to execute. + * @return the output of the executed command. + */ + public static String execCommand(String ... cmd) throws IOException { + return execCommand(null, cmd, 0L); + } + + /** + * Static method to execute a shell command. + * Covers most of the simple cases without requiring the user to implement + * the Shell interface. + * @param env the map of environment key=value + * @param cmd shell command to execute. + * @param timeout time in milliseconds after which script should be marked timeout + * @return the output of the executed command.o + */ + + public static String execCommand(Map env, String[] cmd, + long timeout) throws IOException { + ShellCommandExecutor exec = new ShellCommandExecutor(cmd, null, env, + timeout); + exec.execute(); + return exec.getOutput(); + } + + /** + * Static method to execute a shell command. + * Covers most of the simple cases without requiring the user to implement + * the Shell interface. + * @param env the map of environment key=value + * @param cmd shell command to execute. + * @return the output of the executed command. + */ + public static String execCommand(Map env, String ... cmd) + throws IOException { + return execCommand(env, cmd, 0L); + } + + /** + * Timer which is used to timeout scripts spawned off by shell. + */ + private static class ShellTimeoutTimerTask extends TimerTask { + + private ShellUtils shell; + + public ShellTimeoutTimerTask(ShellUtils shell) { + this.shell = shell; + } + + @Override + public void run() { + Process p = shell.getProcess(); + try { + p.exitValue(); + } catch (Exception e) { + //Process has not terminated. + //So check if it has completed + //if not just destroy it. + if (p != null && !shell.completed.get()) { + shell.setTimedOut(); + p.destroy(); + } + } + } + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/StormBoundedExponentialBackoffRetry.java b/jstorm-core/src/main/java/backtype/storm/utils/StormBoundedExponentialBackoffRetry.java new file mode 100755 index 000000000..4aa555648 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/StormBoundedExponentialBackoffRetry.java @@ -0,0 +1,74 @@ +/** + * 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 backtype.storm.utils; + +import org.apache.curator.retry.BoundedExponentialBackoffRetry; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Random; + +public class StormBoundedExponentialBackoffRetry extends BoundedExponentialBackoffRetry { + private static final Logger LOG = LoggerFactory.getLogger(StormBoundedExponentialBackoffRetry.class); + private int stepSize; + private int expRetriesThreshold; + private final Random random = new Random(); + private final int linearBaseSleepMs; + + /** + * The class provides generic exponential-linear backoff retry strategy for + * storm. It calculates threshold for exponentially increasing sleeptime + * for retries. Beyond this threshold, the sleeptime increase is linear. + * Also adds jitter for exponential/linear retry. + * It guarantees currSleepTimeMs >= prevSleepTimeMs and + * baseSleepTimeMs <= currSleepTimeMs <= maxSleepTimeMs + */ + + public StormBoundedExponentialBackoffRetry(int baseSleepTimeMs, int maxSleepTimeMs, int maxRetries) { + super(baseSleepTimeMs, maxSleepTimeMs, maxRetries); + expRetriesThreshold = 1; + while ((1 << (expRetriesThreshold + 1)) < ((maxSleepTimeMs - baseSleepTimeMs) / 2)) + expRetriesThreshold++; + LOG.info("The baseSleepTimeMs [" + baseSleepTimeMs + "] the maxSleepTimeMs [" + maxSleepTimeMs + "] " + + "the maxRetries [" + maxRetries + "]"); + if (baseSleepTimeMs > maxSleepTimeMs) { + LOG.warn("Misconfiguration: the baseSleepTimeMs [" + baseSleepTimeMs + "] can't be greater than " + + "the maxSleepTimeMs [" + maxSleepTimeMs + "]."); + } + if( maxRetries > 0 && maxRetries > expRetriesThreshold ) { + this.stepSize = Math.max(1, (maxSleepTimeMs - (1 << expRetriesThreshold)) / (maxRetries - expRetriesThreshold)); + } else { + this.stepSize = 1; + } + this.linearBaseSleepMs = super.getBaseSleepTimeMs() + (1 << expRetriesThreshold); + } + + @Override + public int getSleepTimeMs(int retryCount, long elapsedTimeMs) { + if (retryCount < expRetriesThreshold) { + int exp = 1 << retryCount; + int jitter = random.nextInt(exp); + int sleepTimeMs = super.getBaseSleepTimeMs() + exp + jitter; + return sleepTimeMs; + } else { + int stepJitter = random.nextInt(stepSize); + return Math.min(super.getMaxSleepTimeMs(), (linearBaseSleepMs + + (stepSize * (retryCount - expRetriesThreshold)) + stepJitter)); + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/TestUtils.java b/jstorm-core/src/main/java/backtype/storm/utils/TestUtils.java new file mode 100755 index 000000000..276559c41 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/TestUtils.java @@ -0,0 +1,34 @@ +/** + * 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 backtype.storm.utils; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.retry.ExponentialBackoffRetry; +import java.util.Map; + +public class TestUtils extends Utils { + + public static void testSetupBuilder(CuratorFrameworkFactory.Builder + builder, String zkStr, Map conf, ZookeeperAuthInfo auth) + { + setupBuilder(builder, zkStr, conf, auth); + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/ThreadResourceManager.java b/jstorm-core/src/main/java/backtype/storm/utils/ThreadResourceManager.java new file mode 100755 index 000000000..e3ab03f7d --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/ThreadResourceManager.java @@ -0,0 +1,45 @@ +/** + * 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 backtype.storm.utils; + +import java.util.concurrent.ConcurrentLinkedQueue; + +public class ThreadResourceManager { + public static interface ResourceFactory { + X makeResource(); + } + + ResourceFactory _factory; + ConcurrentLinkedQueue _resources = new ConcurrentLinkedQueue(); + + public ThreadResourceManager(ResourceFactory factory) { + _factory = factory; + } + + public T acquire() { + T ret = _resources.poll(); + if (ret == null) { + ret = _factory.makeResource(); + } + return ret; + } + + public void release(T resource) { + _resources.add(resource); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/ThriftTopologyUtils.java b/jstorm-core/src/main/java/backtype/storm/utils/ThriftTopologyUtils.java new file mode 100755 index 000000000..47a48c73a --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/ThriftTopologyUtils.java @@ -0,0 +1,69 @@ +/** + * 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 backtype.storm.utils; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import backtype.storm.generated.Bolt; +import backtype.storm.generated.ComponentCommon; +import backtype.storm.generated.SpoutSpec; +import backtype.storm.generated.StateSpoutSpec; +import backtype.storm.generated.StormTopology; + +public class ThriftTopologyUtils { + public static Set getComponentIds(StormTopology topology) { + Set ret = new HashSet(); + for (StormTopology._Fields f : StormTopology.metaDataMap.keySet()) { + Map componentMap = (Map) topology.getFieldValue(f); + ret.addAll(componentMap.keySet()); + } + return ret; + } + + public static Map getComponents(StormTopology topology) { + Map ret = new HashMap(); + for (StormTopology._Fields f : StormTopology.metaDataMap.keySet()) { + Map componentMap = (Map) topology.getFieldValue(f); + ret.putAll(componentMap); + } + return ret; + } + + public static ComponentCommon getComponentCommon(StormTopology topology, String componentId) { + for (StormTopology._Fields f : StormTopology.metaDataMap.keySet()) { + Map componentMap = (Map) topology.getFieldValue(f); + if (componentMap.containsKey(componentId)) { + Object component = componentMap.get(componentId); + if (component instanceof Bolt) { + return ((Bolt) component).get_common(); + } + if (component instanceof SpoutSpec) { + return ((SpoutSpec) component).get_common(); + } + if (component instanceof StateSpoutSpec) { + return ((StateSpoutSpec) component).get_common(); + } + throw new RuntimeException("Unreachable code! No get_common conversion for component " + component); + } + } + throw new IllegalArgumentException("Could not find component common for " + componentId); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/Time.java b/jstorm-core/src/main/java/backtype/storm/utils/Time.java new file mode 100755 index 000000000..50a79fd67 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/Time.java @@ -0,0 +1,109 @@ +/** + * 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 backtype.storm.utils; + +import java.util.concurrent.atomic.AtomicLong; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class Time { + public static Logger LOG = LoggerFactory.getLogger(Time.class); + + private static AtomicBoolean simulating = new AtomicBoolean(false); + //TODO: should probably use weak references here or something + private static volatile Map threadSleepTimes; + private static final Object sleepTimesLock = new Object(); + + private static AtomicLong simulatedCurrTimeMs; //should this be a thread local that's allowed to keep advancing? + + public static void startSimulating() { + synchronized(sleepTimesLock) { + simulating.set(true); + simulatedCurrTimeMs = new AtomicLong(0); + threadSleepTimes = new ConcurrentHashMap(); + } + } + + public static void stopSimulating() { + synchronized(sleepTimesLock) { + simulating.set(false); + threadSleepTimes = null; + } + } + + public static boolean isSimulating() { + return simulating.get(); + } + + public static void sleepUntil(long targetTimeMs) throws InterruptedException { + if(simulating.get()) { + try { + synchronized(sleepTimesLock) { + threadSleepTimes.put(Thread.currentThread(), new AtomicLong(targetTimeMs)); + } + while(simulatedCurrTimeMs.get() < targetTimeMs) { + Thread.sleep(10); + } + } finally { + synchronized(sleepTimesLock) { + if (simulating.get()) { + threadSleepTimes.remove(Thread.currentThread()); + } + } + } + } else { + long sleepTime = targetTimeMs-currentTimeMillis(); + if(sleepTime>0) + Thread.sleep(sleepTime); + } + } + + public static void sleep(long ms) throws InterruptedException { + sleepUntil(currentTimeMillis()+ms); + } + + public static long currentTimeMillis() { + if(simulating.get()) { + return simulatedCurrTimeMs.get(); + } else { + return System.currentTimeMillis(); + } + } + + public static int currentTimeSecs() { + return (int) (currentTimeMillis() / 1000); + } + + public static void advanceTime(long ms) { + if(!simulating.get()) throw new IllegalStateException("Cannot simulate time unless in simulation mode"); + simulatedCurrTimeMs.set(simulatedCurrTimeMs.get() + ms); + } + + public static boolean isThreadWaiting(Thread t) { + if(!simulating.get()) throw new IllegalStateException("Must be in simulation mode"); + AtomicLong time; + synchronized(sleepTimesLock) { + time = threadSleepTimes.get(t); + } + return !t.isAlive() || time!=null && currentTimeMillis() < time.longValue(); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/TimeCacheMap.java b/jstorm-core/src/main/java/backtype/storm/utils/TimeCacheMap.java new file mode 100755 index 000000000..f0a194fab --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/TimeCacheMap.java @@ -0,0 +1,169 @@ +/** + * 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 backtype.storm.utils; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +/** + * Expires keys that have not been updated in the configured number of seconds. The algorithm used will take between expirationSecs and expirationSecs * (1 + 1 + * / (numBuckets-1)) to actually expire the message. + * + * get, put, remove, containsKey, and size take O(numBuckets) time to run. + * + * Please use com.alibaba.jstorm.utils.TimeCacheMap + */ +@Deprecated +public class TimeCacheMap { + // this default ensures things expire at most 50% past the expiration time + private static final int DEFAULT_NUM_BUCKETS = 3; + + @Deprecated + public static interface ExpiredCallback { + public void expire(K key, V val); + } + + private LinkedList> _buckets; + + private final Object _lock = new Object(); + private Thread _cleaner; + private ExpiredCallback _callback; + + public TimeCacheMap(int expirationSecs, int numBuckets, ExpiredCallback callback) { + if (numBuckets < 2) { + throw new IllegalArgumentException("numBuckets must be >= 2"); + } + _buckets = new LinkedList>(); + for (int i = 0; i < numBuckets; i++) { + _buckets.add(new HashMap()); + } + + _callback = callback; + final long expirationMillis = expirationSecs * 1000L; + final long sleepTime = expirationMillis / (numBuckets - 1); + _cleaner = new Thread(new Runnable() { + public void run() { + try { + while (true) { + Map dead = null; + Time.sleep(sleepTime); + synchronized (_lock) { + dead = _buckets.removeLast(); + _buckets.addFirst(new HashMap()); + } + if (_callback != null) { + for (Entry entry : dead.entrySet()) { + _callback.expire(entry.getKey(), entry.getValue()); + } + } + } + } catch (InterruptedException ex) { + + } + } + }); + _cleaner.setDaemon(true); + _cleaner.start(); + } + + public TimeCacheMap(int expirationSecs, ExpiredCallback callback) { + this(expirationSecs, DEFAULT_NUM_BUCKETS, callback); + } + + public TimeCacheMap(int expirationSecs) { + this(expirationSecs, DEFAULT_NUM_BUCKETS); + } + + public TimeCacheMap(int expirationSecs, int numBuckets) { + this(expirationSecs, numBuckets, null); + } + + public boolean containsKey(K key) { + synchronized (_lock) { + for (HashMap bucket : _buckets) { + if (bucket.containsKey(key)) { + return true; + } + } + return false; + } + } + + public V get(K key) { + synchronized (_lock) { + for (HashMap bucket : _buckets) { + if (bucket.containsKey(key)) { + return bucket.get(key); + } + } + return null; + } + } + + public void put(K key, V value) { + synchronized (_lock) { + Iterator> it = _buckets.iterator(); + HashMap bucket = it.next(); + bucket.put(key, value); + while (it.hasNext()) { + bucket = it.next(); + bucket.remove(key); + } + } + } + + public Object remove(K key) { + synchronized (_lock) { + for (HashMap bucket : _buckets) { + if (bucket.containsKey(key)) { + return bucket.remove(key); + } + } + return null; + } + } + + public int size() { + synchronized (_lock) { + int size = 0; + for (HashMap bucket : _buckets) { + size += bucket.size(); + } + return size; + } + } + + public void cleanup() { + _cleaner.interrupt(); + } + + public Set keySet() { + Set ret = new HashSet(); + synchronized (_lock) { + for (HashMap bucket : _buckets) { + ret.addAll(bucket.keySet()); + } + } + return ret; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/TransferDrainer.java b/jstorm-core/src/main/java/backtype/storm/utils/TransferDrainer.java new file mode 100755 index 000000000..463811799 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/TransferDrainer.java @@ -0,0 +1,113 @@ +/** + * 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 backtype.storm.utils; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; + +import backtype.storm.messaging.IConnection; +import backtype.storm.messaging.TaskMessage; + +public class TransferDrainer { + + private HashMap>> bundles = new HashMap(); + + public void add(HashMap> workerTupleSetMap) { + for (String key : workerTupleSetMap.keySet()) { + + ArrayList> bundle = bundles.get(key); + if (null == bundle) { + bundle = new ArrayList>(); + bundles.put(key, bundle); + } + + ArrayList tupleSet = workerTupleSetMap.get(key); + if (null != tupleSet && tupleSet.size() > 0) { + bundle.add(tupleSet); + } + } + } + + public void send(HashMap connections) { + for (String hostPort : bundles.keySet()) { + IConnection connection = connections.get(hostPort); + if (null != connection) { + ArrayList> bundle = bundles.get(hostPort); + for (ArrayList list : bundle) { + connection.send(list); + } + + } + } + } + + private Iterator getBundleIterator(final ArrayList> bundle) { + + if (null == bundle) { + return null; + } + + return new Iterator () { + + private int offset = 0; + private int size = 0; + { + for (ArrayList list : bundle) { + size += list.size(); + } + } + + private int bundleOffset = 0; + private Iterator iter = bundle.get(bundleOffset).iterator(); + + @Override + public boolean hasNext() { + if (offset < size) { + return true; + } + return false; + } + + @Override + public TaskMessage next() { + TaskMessage msg = null; + if (iter.hasNext()) { + msg = iter.next(); + } else { + bundleOffset++; + iter = bundle.get(bundleOffset).iterator(); + msg = iter.next(); + } + if (null != msg) { + offset++; + } + return msg; + } + + @Override + public void remove() { + throw new RuntimeException("not supported"); + } + }; + } + + public void clear() { + bundles.clear(); + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/backtype/storm/utils/TupleHelpers.java b/jstorm-core/src/main/java/backtype/storm/utils/TupleHelpers.java new file mode 100755 index 000000000..45725c990 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/TupleHelpers.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package backtype.storm.utils; + +import backtype.storm.Constants; +import backtype.storm.tuple.Tuple; + +public class TupleHelpers { + private TupleHelpers() { + + } + + public static boolean isTickTuple(Tuple tuple) { + return tuple.getSourceComponent().equals(Constants.SYSTEM_COMPONENT_ID) && tuple.getSourceStreamId().equals(Constants.SYSTEM_TICK_STREAM_ID); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/TupleUtils.java b/jstorm-core/src/main/java/backtype/storm/utils/TupleUtils.java new file mode 100755 index 000000000..f9fb2c07c --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/TupleUtils.java @@ -0,0 +1,35 @@ +/** + * 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 backtype.storm.utils; + +import backtype.storm.Constants; +import backtype.storm.tuple.Tuple; + +public final class TupleUtils { + + private TupleUtils() { + // No instantiation + } + + public static boolean isTick(Tuple tuple) { + return tuple != null + && Constants.SYSTEM_COMPONENT_ID .equals(tuple.getSourceComponent()) + && Constants.SYSTEM_TICK_STREAM_ID.equals(tuple.getSourceStreamId()); + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/Utils.java b/jstorm-core/src/main/java/backtype/storm/utils/Utils.java new file mode 100644 index 000000000..0669cfb84 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/Utils.java @@ -0,0 +1,906 @@ +/** + * 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 backtype.storm.utils; + +import java.io.BufferedReader; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.FileReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.lang.reflect.Constructor; +import java.net.URL; +import java.net.URLClassLoader; +import java.net.URLDecoder; +import java.nio.ByteBuffer; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; +import java.util.ArrayList; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; +import java.util.TreeMap; +import java.util.UUID; +import java.util.zip.GZIPInputStream; +import java.util.zip.GZIPOutputStream; + +import org.apache.commons.io.input.ClassLoaderObjectInputStream; +import org.apache.commons.lang.StringUtils; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.thrift.TException; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.data.Id; +import org.json.simple.JSONValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.yaml.snakeyaml.Yaml; +import org.yaml.snakeyaml.constructor.SafeConstructor; + +import backtype.storm.Config; +import backtype.storm.generated.ComponentCommon; +import backtype.storm.generated.ComponentObject; +import backtype.storm.generated.StormTopology; +import backtype.storm.serialization.DefaultSerializationDelegate; +import backtype.storm.serialization.SerializationDelegate; +import clojure.lang.IFn; +import clojure.lang.RT; + +import com.alibaba.jstorm.utils.LoadConf; +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; + +public class Utils { + private static final Logger LOG = LoggerFactory.getLogger(Utils.class); + public static final String DEFAULT_STREAM_ID = "default"; + + private static SerializationDelegate serializationDelegate; + + static { + Map conf = readStormConfig(); + serializationDelegate = getSerializationDelegate(conf); + } + + public static Object newInstance(String klass) { + try { + Class c = Class.forName(klass); + return c.newInstance(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public static Object newInstance(String klass, Object ...params) { + try { + Class c = Class.forName(klass); + Constructor[] constructors = c.getConstructors(); + boolean found = false; + Constructor con = null; + for (Constructor cons : constructors) { + if (cons.getParameterTypes().length == params.length) { + con = cons; + break; + } + } + + if (con == null) { + throw new RuntimeException("Cound not found the corresponding constructor, params=" + params.toString()); + } else { + if (con.getParameterTypes().length == 0) { + return c.newInstance(); + } else { + return con.newInstance(params); + } + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * Go thrift gzip serializer + * @param obj + * @return + */ + public static byte[] serialize(Object obj) { + /** + * @@@ + * JStorm disable the thrift.gz.serializer + */ + //return serializationDelegate.serialize(obj); + return javaSerialize(obj); + } + + /** + * Go thrift gzip serializer + * @param obj + * @return + */ + public static T deserialize(byte[] serialized, Class clazz) { + /** + * @@@ + * JStorm disable the thrift.gz.serializer + */ + //return serializationDelegate.deserialize(serialized, clazz); + return (T)javaDeserialize(serialized); + } + + public static byte[] javaSerialize(Object obj) { + if (obj instanceof byte[]) { + return (byte[])obj; + } + try { + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + ObjectOutputStream oos = new ObjectOutputStream(bos); + oos.writeObject(obj); + oos.close(); + return bos.toByteArray(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public static Object maybe_deserialize(byte[] data) { + if (data == null || data.length == 0) { + return null; + } + try { + return javaDeserializeWithCL(data, null); + } catch (Exception e) { + return null; + } + } + + /** + * Deserialized with ClassLoader + * @param serialized + * @param loader + * @return + */ + public static Object javaDeserializeWithCL(byte[] serialized, URLClassLoader loader) { + try { + ByteArrayInputStream bis = new ByteArrayInputStream(serialized); + Object ret = null; + if (loader != null) { + ClassLoaderObjectInputStream cis = new ClassLoaderObjectInputStream(loader, bis); + ret = cis.readObject(); + cis.close(); + } else { + ObjectInputStream ois = new ObjectInputStream(bis); + ret = ois.readObject(); + ois.close(); + } + return ret; + } catch (IOException ioe) { + throw new RuntimeException(ioe); + } catch (ClassNotFoundException e) { + throw new RuntimeException(e); + } + } + + public static Object javaDeserialize(byte[] serialized) { + return javaDeserializeWithCL(serialized, WorkerClassLoader.getInstance()); + } + + public static T javaDeserialize(byte[] serialized, Class clazz) { + return (T)javaDeserializeWithCL(serialized, WorkerClassLoader.getInstance()); + } + + public static String to_json(Object m) { + // return JSON.toJSONString(m); + return JSONValue.toJSONString(m); + } + + public static Object from_json(String json) { + if (json == null) { + return null; + } else { + // return JSON.parse(json); + return JSONValue.parse(json); + } + } + + public static String toPrettyJsonString(Object obj) { + Gson gson2 = new GsonBuilder().setPrettyPrinting().create(); + String ret = gson2.toJson(obj); + + return ret; + } + + public static byte[] gzip(byte[] data) { + try { + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + GZIPOutputStream out = new GZIPOutputStream(bos); + out.write(data); + out.close(); + return bos.toByteArray(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public static byte[] gunzip(byte[] data) { + try { + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + ByteArrayInputStream bis = new ByteArrayInputStream(data); + GZIPInputStream in = new GZIPInputStream(bis); + byte[] buffer = new byte[1024]; + int len = 0; + while ((len = in.read(buffer)) >= 0) { + bos.write(buffer, 0, len); + } + in.close(); + bos.close(); + return bos.toByteArray(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public static String join(Iterable coll, String sep) { + Iterator it = coll.iterator(); + String ret = ""; + while(it.hasNext()) { + ret = ret + it.next(); + if(it.hasNext()) { + ret = ret + sep; + } + } + return ret; + } + + public static void sleep(long millis) { + try { + Time.sleep(millis); + } catch(InterruptedException e) { + throw new RuntimeException(e); + } + } + + /** + * Please directly use LoadConf.findResources(name); + * @param name + * @return + */ + @Deprecated + public static List findResources(String name) { + return LoadConf.findResources(name); + } + + /** + * Please directly use LoadConf.findAndReadYaml(name); + * @param name + * @return + */ + @Deprecated + public static Map findAndReadConfigFile(String name, boolean mustExist) { + return LoadConf.findAndReadYaml(name, mustExist, false); + } + + + public static Map findAndReadConfigFile(String name) { + return LoadConf.findAndReadYaml(name, true, false); + } + + public static Map readDefaultConfig() { + return LoadConf.findAndReadYaml("defaults.yaml", true, false); + } + + public static Map readCommandLineOpts() { + Map ret = new HashMap(); + String commandOptions = System.getProperty("storm.options"); + if(commandOptions != null) { + String[] configs = commandOptions.split(","); + for (String config : configs) { + config = URLDecoder.decode(config); + String[] options = config.split("=", 2); + if (options.length == 2) { + Object val = JSONValue.parse(options[1]); + if (val == null) { + val = options[1]; + } + ret.put(options[0], val); + } + } + } + return ret; + } + + + public static void replaceLocalDir(Map conf) { + String stormHome = System.getProperty("jstorm.home"); + boolean isEmpty = StringUtils.isBlank(stormHome); + + Map replaceMap = new HashMap(); + + for (Entry entry : conf.entrySet()) { + Object key = entry.getKey(); + Object value = entry.getValue(); + + if (value instanceof String) { + if (StringUtils.isBlank((String) value) == true) { + continue; + } + + String str = (String) value; + if (isEmpty == true) { + // replace %JSTORM_HOME% as current directory + str = str.replace("%JSTORM_HOME%", "."); + } else { + str = str.replace("%JSTORM_HOME%", stormHome); + } + + replaceMap.put(key, str); + } + } + + conf.putAll(replaceMap); + } + + public static Map loadDefinedConf(String confFile) { + File file = new File(confFile); + if (file.exists() == false) { + return findAndReadConfigFile(confFile, true); + } + + Yaml yaml = new Yaml(); + Map ret; + try { + ret = (Map) yaml.load(new FileReader(file)); + } catch (FileNotFoundException e) { + ret = null; + } + if (ret == null) + ret = new HashMap(); + + return new HashMap(ret); + } + + public static Map readStormConfig() { + Map ret = readDefaultConfig(); + String confFile = System.getProperty("storm.conf.file"); + Map storm; + if (StringUtils.isBlank(confFile) == true) { + storm = findAndReadConfigFile("storm.yaml", false); + } else { + storm = loadDefinedConf(confFile); + } + ret.putAll(storm); + ret.putAll(readCommandLineOpts()); + + replaceLocalDir(ret); + return ret; + } + + private static Object normalizeConf(Object conf) { + if(conf==null) return new HashMap(); + if(conf instanceof Map) { + Map confMap = new HashMap((Map) conf); + for(Object key: confMap.keySet()) { + Object val = confMap.get(key); + confMap.put(key, normalizeConf(val)); + } + return confMap; + } else if(conf instanceof List) { + List confList = new ArrayList((List) conf); + for(int i=0; i stormConf) { + return normalizeConf(stormConf).equals(normalizeConf(Utils.from_json(Utils.to_json(stormConf)))); + } + + public static Object getSetComponentObject(ComponentObject obj, URLClassLoader loader) { + if (obj.getSetField() == ComponentObject._Fields.SERIALIZED_JAVA) { + return javaDeserializeWithCL(obj.get_serialized_java(), loader); + } else if (obj.getSetField() == ComponentObject._Fields.JAVA_OBJECT) { + return obj.get_java_object(); + } else { + return obj.get_shell(); + } + } + + public static T get(Map m, S key, T def) { + T ret = m.get(key); + if (ret == null) { + ret = def; + } + return ret; + } + + public static List tuple(Object... values) { + List ret = new ArrayList(); + for (Object v : values) { + ret.add(v); + } + return ret; + } + + public static void downloadFromMaster(Map conf, String file, String localFile) throws IOException, TException { + WritableByteChannel out = null; + NimbusClient client = null; + try { + client = NimbusClient.getConfiguredClient(conf, 10 * 1000); + String id = client.getClient().beginFileDownload(file); + out = Channels.newChannel(new FileOutputStream(localFile)); + while (true) { + ByteBuffer chunk = client.getClient().downloadChunk(id); + int written = out.write(chunk); + if (written == 0) { + client.getClient().finishFileDownload(id); + break; + } + } + } finally { + if (out != null) + out.close(); + if (client != null) + client.close(); + } + } + + public static IFn loadClojureFn(String namespace, String name) { + try { + clojure.lang.Compiler.eval(RT.readString("(require '" + namespace + ")")); + } catch (Exception e) { + //if playing from the repl and defining functions, file won't exist + } + return (IFn) RT.var(namespace, name).deref(); + } + + public static boolean isSystemId(String id) { + return id.startsWith("__"); + } + + public static Map reverseMap(Map map) { + Map ret = new HashMap(); + for(K key: map.keySet()) { + ret.put(map.get(key), key); + } + return ret; + } + + public static ComponentCommon getComponentCommon(StormTopology topology, String id) { + if(topology.get_spouts().containsKey(id)) { + return topology.get_spouts().get(id).get_common(); + } + if(topology.get_bolts().containsKey(id)) { + return topology.get_bolts().get(id).get_common(); + } + if(topology.get_state_spouts().containsKey(id)) { + return topology.get_state_spouts().get(id).get_common(); + } + throw new IllegalArgumentException("Could not find component with id " + id); + } + + public static Integer getInt(Object o) { + Integer result = getInt(o, null); + if (null == result) { + throw new IllegalArgumentException("Don't know how to convert null to int"); + } + return result; + } + + public static Integer getInt(Object o, Integer defaultValue) { + if (null == o) { + return defaultValue; + } + + if (o instanceof Number) { + return ((Number) o).intValue(); + } else if (o instanceof String) { + return Integer.parseInt(((String) o)); + } else { + throw new IllegalArgumentException("Don't know how to convert " + o + " to int"); + } + } + + public static long secureRandomLong() { + return UUID.randomUUID().getLeastSignificantBits(); + } + + public static class BoundedExponentialBackoffRetry extends ExponentialBackoffRetry { + + protected final int maxRetryInterval; + + public BoundedExponentialBackoffRetry(int baseSleepTimeMs, int maxRetries, int maxSleepTimeMs) { + super(baseSleepTimeMs, maxRetries); + this.maxRetryInterval = maxSleepTimeMs; + } + + public int getMaxRetryInterval() { + return this.maxRetryInterval; + } + + @Override + public int getSleepTimeMs(int count, long elapsedMs) { + return Math.min(maxRetryInterval, super.getSleepTimeMs(count, elapsedMs)); + } + + } + + public static CuratorFramework newCurator(Map conf, List servers, Object port, String root) { + return newCurator(conf, servers, port, root, null); + } + + public static CuratorFramework newCurator(Map conf, List servers, Object port, String root, ZookeeperAuthInfo auth) { + List serverPorts = new ArrayList(); + for(String zkServer: (List) servers) { + serverPorts.add(zkServer + ":" + Utils.getInt(port)); + } + String zkStr = StringUtils.join(serverPorts, ",") + root; + CuratorFrameworkFactory.Builder builder = CuratorFrameworkFactory.builder(); + + setupBuilder(builder, zkStr, conf, auth); + + return builder.build(); + } + + protected static void setupBuilder(CuratorFrameworkFactory.Builder builder, String zkStr, Map conf, ZookeeperAuthInfo auth) + { + builder.connectString(zkStr) + .connectionTimeoutMs(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_CONNECTION_TIMEOUT))) + .sessionTimeoutMs(Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_SESSION_TIMEOUT))) + .retryPolicy(new StormBoundedExponentialBackoffRetry( + Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL)), + Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_INTERVAL_CEILING)), + Utils.getInt(conf.get(Config.STORM_ZOOKEEPER_RETRY_TIMES)))); + + if(auth!=null && auth.scheme!=null && auth.payload!=null) { + builder = builder.authorization(auth.scheme, auth.payload); + } + } + + public static CuratorFramework newCurator(Map conf, List servers, Object port, ZookeeperAuthInfo auth) { + return newCurator(conf, servers, port, "", auth); + } + + public static CuratorFramework newCuratorStarted(Map conf, List servers, Object port, String root, ZookeeperAuthInfo auth) { + CuratorFramework ret = newCurator(conf, servers, port, root, auth); + ret.start(); + return ret; + } + + public static CuratorFramework newCuratorStarted(Map conf, List servers, Object port, ZookeeperAuthInfo auth) { + CuratorFramework ret = newCurator(conf, servers, port, auth); + ret.start(); + return ret; + } + + /** + * +(defn integer-divided [sum num-pieces] + (let [base (int (/ sum num-pieces)) + num-inc (mod sum num-pieces) + num-bases (- num-pieces num-inc)] + (if (= num-inc 0) + {base num-bases} + {base num-bases (inc base) num-inc} + ))) + * @param sum + * @param numPieces + * @return + */ + + public static TreeMap integerDivided(int sum, int numPieces) { + int base = sum / numPieces; + int numInc = sum % numPieces; + int numBases = numPieces - numInc; + TreeMap ret = new TreeMap(); + ret.put(base, numBases); + if(numInc!=0) { + ret.put(base+1, numInc); + } + return ret; + } + + public static byte[] toByteArray(ByteBuffer buffer) { + byte[] ret = new byte[buffer.remaining()]; + buffer.get(ret, 0, ret.length); + return ret; + } + + public static void readAndLogStream(String prefix, InputStream in) { + try { + BufferedReader r = new BufferedReader(new InputStreamReader(in)); + String line = null; + while ((line = r.readLine())!= null) { + LOG.info("{}:{}", prefix, line); + } + } catch (IOException e) { + LOG.warn("Error whiel trying to log stream", e); + } + } + + public static boolean exceptionCauseIsInstanceOf(Class klass, Throwable throwable) { + Throwable t = throwable; + while(t != null) { + if(klass.isInstance(t)) { + return true; + } + t = t.getCause(); + } + return false; + } + + /** + * Is the cluster configured to interact with ZooKeeper in a secure way? + * This only works when called from within Nimbus or a Supervisor process. + * @param conf the storm configuration, not the topology configuration + * @return true if it is configured else false. + */ + public static boolean isZkAuthenticationConfiguredStormServer(Map conf) { + return null != System.getProperty("java.security.auth.login.config") + || (conf != null + && conf.get(Config.STORM_ZOOKEEPER_AUTH_SCHEME) != null + && ! ((String)conf.get(Config.STORM_ZOOKEEPER_AUTH_SCHEME)).isEmpty()); + } + + /** + * Is the topology configured to have ZooKeeper authentication. + * @param conf the topology configuration + * @return true if ZK is configured else false + */ + public static boolean isZkAuthenticationConfiguredTopology(Map conf) { + return (conf != null + && conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME) != null + && ! ((String)conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME)).isEmpty()); + } + + public static List getWorkerACL(Map conf) { + //This is a work around to an issue with ZK where a sasl super user is not super unless there is an open SASL ACL so we are trying to give the correct perms + if (!isZkAuthenticationConfiguredTopology(conf)) { + return null; + } + String stormZKUser = (String)conf.get(Config.STORM_ZOOKEEPER_SUPERACL); + if (stormZKUser == null) { + throw new IllegalArgumentException("Authentication is enabled but "+Config.STORM_ZOOKEEPER_SUPERACL+" is not set"); + } + String[] split = stormZKUser.split(":",2); + if (split.length != 2) { + throw new IllegalArgumentException(Config.STORM_ZOOKEEPER_SUPERACL+" does not appear to be in the form scheme:acl, i.e. sasl:storm-user"); + } + ArrayList ret = new ArrayList(ZooDefs.Ids.CREATOR_ALL_ACL); + ret.add(new ACL(ZooDefs.Perms.ALL, new Id(split[0], split[1]))); + return ret; + } + + public static String threadDump() { + final StringBuilder dump = new StringBuilder(); + final java.lang.management.ThreadMXBean threadMXBean = java.lang.management.ManagementFactory.getThreadMXBean(); + final java.lang.management.ThreadInfo[] threadInfos = threadMXBean.getThreadInfo(threadMXBean.getAllThreadIds(), 100); + for (java.lang.management.ThreadInfo threadInfo : threadInfos) { + dump.append('"'); + dump.append(threadInfo.getThreadName()); + dump.append("\" "); + final Thread.State state = threadInfo.getThreadState(); + dump.append("\n java.lang.Thread.State: "); + dump.append(state); + final StackTraceElement[] stackTraceElements = threadInfo.getStackTrace(); + for (final StackTraceElement stackTraceElement : stackTraceElements) { + dump.append("\n at "); + dump.append(stackTraceElement); + } + dump.append("\n\n"); + } + return dump.toString(); + } + + // Assumes caller is synchronizing + private static SerializationDelegate getSerializationDelegate(Map stormConf) { + String delegateClassName = (String)stormConf.get(Config.STORM_META_SERIALIZATION_DELEGATE); + SerializationDelegate delegate; + try { + Class delegateClass = Class.forName(delegateClassName); + delegate = (SerializationDelegate) delegateClass.newInstance(); + } catch (ClassNotFoundException e) { + LOG.error("Failed to construct serialization delegate, falling back to default", e); + delegate = new DefaultSerializationDelegate(); + } catch (InstantiationException e) { + LOG.error("Failed to construct serialization delegate, falling back to default", e); + delegate = new DefaultSerializationDelegate(); + } catch (IllegalAccessException e) { + LOG.error("Failed to construct serialization delegate, falling back to default", e); + delegate = new DefaultSerializationDelegate(); + } + delegate.prepare(stormConf); + return delegate; + } + + public static void handleUncaughtException(Throwable t) { + if (t != null && t instanceof Error) { + if (t instanceof OutOfMemoryError) { + try { + System.err.println("Halting due to Out Of Memory Error..." + Thread.currentThread().getName()); + } catch (Throwable err) { + //Again we don't want to exit because of logging issues. + } + Runtime.getRuntime().halt(-1); + } else { + //Running in daemon mode, we would pass Error to calling thread. + throw (Error) t; + } + } + } + + + + public static List tokenize_path(String path) { + String[] toks = path.split("/"); + java.util.ArrayList rtn = new ArrayList(); + for (String str : toks) { + if (!str.isEmpty()) { + rtn.add(str); + } + } + return rtn; + } + + public static String toks_to_path(List toks) { + StringBuffer buff = new StringBuffer(); + buff.append("/"); + int size = toks.size(); + for (int i = 0; i < size; i++) { + buff.append(toks.get(i)); + if (i < (size - 1)) { + buff.append("/"); + } + + } + return buff.toString(); + } + + public static String normalize_path(String path) { + String rtn = toks_to_path(tokenize_path(path)); + return rtn; + } + + public static String printStack() { + StringBuilder sb = new StringBuilder(); + sb.append("\nCurrent call stack:\n"); + StackTraceElement[] stackElements = Thread.currentThread().getStackTrace(); + for (int i = 2; i < stackElements.length; i++) { + sb.append("\t").append(stackElements[i]).append("\n"); + } + + return sb.toString(); + } + + private static Map loadProperty(String prop) { + Map ret = new HashMap(); + Properties properties = new Properties(); + + try { + InputStream stream = new FileInputStream(prop); + properties.load(stream); + if (properties.size() == 0) { + System.out.println("WARN: Config file is empty"); + return null; + } else { + ret.putAll(properties); + } + } catch (FileNotFoundException e) { + System.out.println("No such file " + prop); + throw new RuntimeException(e.getMessage()); + } catch (Exception e1) { + e1.printStackTrace(); + throw new RuntimeException(e1.getMessage()); + } + + return ret; + } + + private static Map loadYaml(String confPath) { + Map ret = new HashMap(); + Yaml yaml = new Yaml(); + + try { + InputStream stream = new FileInputStream(confPath); + ret = (Map) yaml.load(stream); + if (ret == null || ret.isEmpty() == true) { + System.out.println("WARN: Config file is empty"); + return null; + } + } catch (FileNotFoundException e) { + System.out.println("No such file " + confPath); + throw new RuntimeException("No config file"); + } catch (Exception e1) { + e1.printStackTrace(); + throw new RuntimeException("Failed to read config file"); + } + + return ret; + } + + public static Map loadConf(String arg) { + Map ret = null; + if (arg.endsWith("yaml")) { + ret = loadYaml(arg); + } else { + ret = loadProperty(arg); + } + return ret; + } + + public static String getVersion() { + String ret = ""; + InputStream input = null; + try { + input = + Utils.class.getClassLoader().getResourceAsStream("version"); + BufferedReader in = new BufferedReader(new InputStreamReader(input)); + String s = in.readLine(); + ret = s.trim(); + + + } catch (Exception e) { + LOG.warn("Failed to get version", e); + } finally { + if (input != null) { + try { + input.close(); + } catch (Exception e) { + LOG.error("Failed to close the reader of RELEASE", e); + } + } + } + + return ret; + } + + public static void writeIntToByteArray(byte[] bytes, int offset, int value) { + bytes[offset++] = (byte) (value & 0x000000FF); + bytes[offset++] = (byte) ((value & 0x0000FF00) >> 8); + bytes[offset++] = (byte) ((value & 0x00FF0000) >> 16); + bytes[offset] = (byte) ((value & 0xFF000000) >> 24); + } + + public static int readIntFromByteArray(byte[] bytes, int offset) { + int ret = 0; + ret = ret | (bytes[offset++] & 0x000000FF); + ret = ret | ((bytes[offset++] << 8) & 0x0000FF00); + ret = ret | ((bytes[offset++] << 16) & 0x00FF0000); + ret = ret | ((bytes[offset] << 24) & 0xFF000000); + return ret; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/VersionInfo.java b/jstorm-core/src/main/java/backtype/storm/utils/VersionInfo.java new file mode 100755 index 000000000..1740e180d --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/VersionInfo.java @@ -0,0 +1,131 @@ +/* + * 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 backtype.storm.utils; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Properties; + +public class VersionInfo { + + private Properties info; + + protected VersionInfo(String component) { + info = new Properties(); + String versionInfoFile = component + "-version-info.properties"; + InputStream is = null; + try { + is = Thread.currentThread().getContextClassLoader() + .getResourceAsStream(versionInfoFile); + if (is == null) { + throw new IOException("Resource not found"); + } + info.load(is); + } catch (IOException ex) { + } finally { + if (is != null) { + try { + + is.close(); + } catch (IOException ioex) { + } + + } + } + } + + protected String _getVersion() { + return info.getProperty("version", "Unknown"); + } + + protected String _getRevision() { + return info.getProperty("revision", "Unknown"); + } + + protected String _getBranch() { + return info.getProperty("branch", "Unknown"); + } + + protected String _getDate() { + return info.getProperty("date", "Unknown"); + } + + protected String _getUser() { + return info.getProperty("user", "Unknown"); + } + + protected String _getUrl() { + return info.getProperty("url", "Unknown"); + } + + protected String _getSrcChecksum() { + return info.getProperty("srcChecksum", "Unknown"); + } + + protected String _getBuildVersion(){ + return getVersion() + + " from " + _getRevision() + + " by " + _getUser() + + " source checksum " + _getSrcChecksum(); + } + + + private static VersionInfo COMMON_VERSION_INFO = new VersionInfo("storm-core"); + + public static String getVersion() { + return COMMON_VERSION_INFO._getVersion(); + } + + public static String getRevision() { + return COMMON_VERSION_INFO._getRevision(); + } + + public static String getBranch() { + return COMMON_VERSION_INFO._getBranch(); + } + + public static String getDate() { + return COMMON_VERSION_INFO._getDate(); + } + + public static String getUser() { + return COMMON_VERSION_INFO._getUser(); + } + + public static String getUrl() { + return COMMON_VERSION_INFO._getUrl(); + } + + public static String getSrcChecksum() { + return COMMON_VERSION_INFO._getSrcChecksum(); + } + + public static String getBuildVersion(){ + return COMMON_VERSION_INFO._getBuildVersion(); + } + + + public static void main(String[] args) { + System.out.println("Storm " + getVersion()); + System.out.println("URL " + getUrl() + " -r " + getRevision()); + System.out.println("Branch " + getBranch()); + System.out.println("Compiled by " + getUser() + " on " + getDate()); + System.out.println("From source with checksum " + getSrcChecksum()); + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/VersionedStore.java b/jstorm-core/src/main/java/backtype/storm/utils/VersionedStore.java new file mode 100755 index 000000000..07ce5a8d9 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/VersionedStore.java @@ -0,0 +1,187 @@ +/** + * 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 backtype.storm.utils; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.io.File; + +import org.apache.commons.io.FileUtils; + +public class VersionedStore { + private static final String FINISHED_VERSION_SUFFIX = ".version"; + + private String _root; + + public VersionedStore(String path) throws IOException { + _root = path; + mkdirs(_root); + } + + public String getRoot() { + return _root; + } + + public String versionPath(long version) { + return new File(_root, "" + version).getAbsolutePath(); + } + + public String mostRecentVersionPath() throws IOException { + Long v = mostRecentVersion(); + if(v==null) return null; + return versionPath(v); + } + + public String mostRecentVersionPath(long maxVersion) throws IOException { + Long v = mostRecentVersion(maxVersion); + if(v==null) return null; + return versionPath(v); + } + + public Long mostRecentVersion() throws IOException { + List all = getAllVersions(); + if(all.size()==0) return null; + return all.get(0); + } + + public Long mostRecentVersion(long maxVersion) throws IOException { + List all = getAllVersions(); + for(Long v: all) { + if(v <= maxVersion) return v; + } + return null; + } + + public String createVersion() throws IOException { + Long mostRecent = mostRecentVersion(); + long version = Time.currentTimeMillis(); + if(mostRecent!=null && version <= mostRecent) { + version = mostRecent + 1; + } + return createVersion(version); + } + + public String createVersion(long version) throws IOException { + String ret = versionPath(version); + if(getAllVersions().contains(version)) + throw new RuntimeException("Version already exists or data already exists"); + else + return ret; + } + + public void failVersion(String path) throws IOException { + deleteVersion(validateAndGetVersion(path)); + } + + public void deleteVersion(long version) throws IOException { + File versionFile = new File(versionPath(version)); + File tokenFile = new File(tokenPath(version)); + + if(tokenFile.exists()) { + FileUtils.forceDelete(tokenFile); + } + + if(versionFile.exists()) { + FileUtils.forceDelete(versionFile); + } + } + + public void succeedVersion(String path) throws IOException { + long version = validateAndGetVersion(path); + // should rewrite this to do a file move + createNewFile(tokenPath(version)); + } + + public void cleanup() throws IOException { + cleanup(-1); + } + + public void cleanup(int versionsToKeep) throws IOException { + List versions = getAllVersions(); + if(versionsToKeep >= 0) { + versions = versions.subList(0, Math.min(versions.size(), versionsToKeep)); + } + HashSet keepers = new HashSet(versions); + + for(String p: listDir(_root)) { + Long v = parseVersion(p); + if(v!=null && !keepers.contains(v)) { + deleteVersion(v); + } + } + } + + /** + * Sorted from most recent to oldest + */ + public List getAllVersions() throws IOException { + List ret = new ArrayList(); + for(String s: listDir(_root)) { + if(s.endsWith(FINISHED_VERSION_SUFFIX)) { + ret.add(validateAndGetVersion(s)); + } + } + Collections.sort(ret); + Collections.reverse(ret); + return ret; + } + + private String tokenPath(long version) { + return new File(_root, "" + version + FINISHED_VERSION_SUFFIX).getAbsolutePath(); + } + + private long validateAndGetVersion(String path) { + Long v = parseVersion(path); + if(v==null) throw new RuntimeException(path + " is not a valid version"); + return v; + } + + private Long parseVersion(String path) { + String name = new File(path).getName(); + if(name.endsWith(FINISHED_VERSION_SUFFIX)) { + name = name.substring(0, name.length()-FINISHED_VERSION_SUFFIX.length()); + } + try { + return Long.parseLong(name); + } catch(NumberFormatException e) { + return null; + } + } + + private void createNewFile(String path) throws IOException { + new File(path).createNewFile(); + } + + private void mkdirs(String path) throws IOException { + new File(path).mkdirs(); + } + + private List listDir(String dir) throws IOException { + List ret = new ArrayList(); + File[] contents = new File(dir).listFiles(); + if(contents!=null) { + for(File f: contents) { + ret.add(f.getAbsolutePath()); + } + } + return ret; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/WindowedTimeThrottler.java b/jstorm-core/src/main/java/backtype/storm/utils/WindowedTimeThrottler.java new file mode 100755 index 000000000..5a288a007 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/WindowedTimeThrottler.java @@ -0,0 +1,51 @@ +/** + * 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 backtype.storm.utils; + +public class WindowedTimeThrottler { + long _windowMillis; + int _maxAmt; + long _windowStartTime; + int _windowEvents = 0; + + public WindowedTimeThrottler(Number windowMillis, Number maxAmt) { + _windowMillis = windowMillis.longValue(); + _maxAmt = maxAmt.intValue(); + _windowStartTime = System.currentTimeMillis(); + } + + public boolean isThrottled() { + resetIfNecessary(); + return _windowEvents >= _maxAmt; + } + + //returns void if the event should continue, false if the event should not be done + public void markEvent() { + resetIfNecessary(); + _windowEvents++; + + } + + private void resetIfNecessary() { + long now = System.currentTimeMillis(); + if(now - _windowStartTime > _windowMillis) { + _windowStartTime = now; + _windowEvents = 0; + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/WorkerClassLoader.java b/jstorm-core/src/main/java/backtype/storm/utils/WorkerClassLoader.java new file mode 100755 index 000000000..f3526b1a3 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/WorkerClassLoader.java @@ -0,0 +1,188 @@ +/** + * 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 backtype.storm.utils; + +import java.net.URL; +import java.net.URLClassLoader; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class WorkerClassLoader extends URLClassLoader { + + public static Logger LOG = LoggerFactory.getLogger(WorkerClassLoader.class); + + private ClassLoader defaultClassLoader; + + private ClassLoader JDKClassLoader; + + private boolean isDebug; + + protected static WorkerClassLoader instance; + + protected static boolean enable; + + protected static Map threadContextCache; + + protected WorkerClassLoader(URL[] urls, ClassLoader defaultClassLoader, ClassLoader JDKClassLoader, boolean isDebug) { + super(urls, JDKClassLoader); + this.defaultClassLoader = defaultClassLoader; + this.JDKClassLoader = JDKClassLoader; + this.isDebug = isDebug; + + // TODO Auto-generated constructor stub + } + + // for all log go through logback when enable classloader + protected boolean isLogByDefault(String name) { + if (name.startsWith("org.apache.log4j")) { + return true; + } else if (name.startsWith("org.slf4j")) { + return true; + } + + return false; + + } + + protected boolean isLoadByDefault(String name) { + if (name.startsWith("backtype.storm") == true) { + return true; + } else if (name.startsWith("com.alibaba.jstorm")) { + return true; + } else if (isLogByDefault(name)) { + return true; + } else { + return false; + } + } + + @Override + public Class loadClass(String name) throws ClassNotFoundException { + Class result = null; + try { + result = this.findLoadedClass(name); + + if (result != null) { + return result; + } + + try { + result = JDKClassLoader.loadClass(name); + if (result != null) + return result; + } catch (Exception e) { + + } + + try { + if (isLoadByDefault(name) == false) { + result = findClass(name); + + if (result != null) { + return result; + } + } + + } catch (Exception e) { + + } + + result = defaultClassLoader.loadClass(name); + return result; + + } finally { + if (result != null) { + ClassLoader resultClassLoader = result.getClassLoader(); + LOG.info("Successfully load class " + name + " by " + resultClassLoader + ",threadContextLoader:" + Thread.currentThread().getContextClassLoader()); + } else { + LOG.warn("Failed to load class " + name + ",threadContextLoader:" + Thread.currentThread().getContextClassLoader()); + } + + if (isDebug) { + LOG.info(Utils.printStack()); + } + } + + } + + public static WorkerClassLoader mkInstance(URL[] urls, ClassLoader DefaultClassLoader, ClassLoader JDKClassLoader, boolean enable, boolean isDebug) { + WorkerClassLoader.enable = enable; + if (enable == false) { + LOG.info("Don't enable UserDefine ClassLoader"); + return null; + } + + synchronized (WorkerClassLoader.class) { + if (instance == null) { + instance = new WorkerClassLoader(urls, DefaultClassLoader, JDKClassLoader, isDebug); + + threadContextCache = new ConcurrentHashMap(); + } + + } + + LOG.info("Successfully create classloader " + mk_list(urls)); + return instance; + } + + public static WorkerClassLoader getInstance() { + return instance; + } + + public static boolean isEnable() { + return enable; + } + + public static void switchThreadContext() { + if (enable == false) { + return; + } + + Thread thread = Thread.currentThread(); + ClassLoader oldClassLoader = thread.getContextClassLoader(); + threadContextCache.put(thread, oldClassLoader); + thread.setContextClassLoader(instance); + } + + public static void restoreThreadContext() { + if (enable == false) { + return; + } + + Thread thread = Thread.currentThread(); + ClassLoader oldClassLoader = threadContextCache.get(thread); + if (oldClassLoader != null) { + thread.setContextClassLoader(oldClassLoader); + } else { + LOG.info("No context classloader of " + thread.getName()); + } + } + + private static List mk_list(V... args) { + ArrayList rtn = new ArrayList(); + for (V o : args) { + rtn.add(o); + } + return rtn; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/WritableUtils.java b/jstorm-core/src/main/java/backtype/storm/utils/WritableUtils.java new file mode 100755 index 000000000..8516f9713 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/WritableUtils.java @@ -0,0 +1,375 @@ +/** + * 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. + */ +/** + * 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. + */ + +/** + * This file originally comes from the Apache Hadoop project. Changes have been made to the file. + * + */ + +package backtype.storm.utils; + +import java.io.*; + + +import java.util.zip.GZIPInputStream; +import java.util.zip.GZIPOutputStream; + +public final class WritableUtils { + + public static byte[] readCompressedByteArray(DataInput in) throws IOException { + int length = in.readInt(); + if (length == -1) return null; + byte[] buffer = new byte[length]; + in.readFully(buffer); // could/should use readFully(buffer,0,length)? + GZIPInputStream gzi = new GZIPInputStream(new ByteArrayInputStream(buffer, 0, buffer.length)); + byte[] outbuf = new byte[length]; + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + int len; + while((len=gzi.read(outbuf, 0, outbuf.length)) != -1){ + bos.write(outbuf, 0, len); + } + byte[] decompressed = bos.toByteArray(); + bos.close(); + gzi.close(); + return decompressed; + } + + public static void skipCompressedByteArray(DataInput in) throws IOException { + int length = in.readInt(); + if (length != -1) { + skipFully(in, length); + } + } + + public static int writeCompressedByteArray(DataOutput out, byte[] bytes) throws IOException { + if (bytes != null) { + ByteArrayOutputStream bos = new ByteArrayOutputStream(); + GZIPOutputStream gzout = new GZIPOutputStream(bos); + gzout.write(bytes, 0, bytes.length); + gzout.close(); + byte[] buffer = bos.toByteArray(); + int len = buffer.length; + out.writeInt(len); + out.write(buffer, 0, len); + /* debug only! Once we have confidence, can lose this. */ + return ((bytes.length != 0) ? (100*buffer.length)/bytes.length : 0); + } else { + out.writeInt(-1); + return -1; + } + } + + + /* Ugly utility, maybe someone else can do this better */ + public static String readCompressedString(DataInput in) throws IOException { + byte[] bytes = readCompressedByteArray(in); + if (bytes == null) return null; + return new String(bytes, "UTF-8"); + } + + + public static int writeCompressedString(DataOutput out, String s) throws IOException { + return writeCompressedByteArray(out, (s != null) ? s.getBytes("UTF-8") : null); + } + + /* + * + * Write a String as a Network Int n, followed by n Bytes + * Alternative to 16 bit read/writeUTF. + * Encoding standard is... ? + * + */ + public static void writeString(DataOutput out, String s) throws IOException { + if (s != null) { + byte[] buffer = s.getBytes("UTF-8"); + int len = buffer.length; + out.writeInt(len); + out.write(buffer, 0, len); + } else { + out.writeInt(-1); + } + } + + /* + * Read a String as a Network Int n, followed by n Bytes + * Alternative to 16 bit read/writeUTF. + * Encoding standard is... ? + * + */ + public static String readString(DataInput in) throws IOException{ + int length = in.readInt(); + if (length == -1) return null; + byte[] buffer = new byte[length]; + in.readFully(buffer); // could/should use readFully(buffer,0,length)? + return new String(buffer,"UTF-8"); + } + + + /* + * Write a String array as a Nework Int N, followed by Int N Byte Array Strings. + * Could be generalised using introspection. + * + */ + public static void writeStringArray(DataOutput out, String[] s) throws IOException{ + out.writeInt(s.length); + for(int i = 0; i < s.length; i++) { + writeString(out, s[i]); + } + } + + /* + * Write a String array as a Nework Int N, followed by Int N Byte Array of + * compressed Strings. Handles also null arrays and null values. + * Could be generalised using introspection. + * + */ + public static void writeCompressedStringArray(DataOutput out, String[] s) throws IOException{ + if (s == null) { + out.writeInt(-1); + return; + } + out.writeInt(s.length); + for(int i = 0; i < s.length; i++) { + writeCompressedString(out, s[i]); + } + } + + /* + * Write a String array as a Nework Int N, followed by Int N Byte Array Strings. + * Could be generalised using introspection. Actually this bit couldn't... + * + */ + public static String[] readStringArray(DataInput in) throws IOException { + int len = in.readInt(); + if (len == -1) return null; + String[] s = new String[len]; + for(int i = 0; i < len; i++) { + s[i] = readString(in); + } + return s; + } + + + /* + * Write a String array as a Nework Int N, followed by Int N Byte Array Strings. + * Could be generalised using introspection. Handles null arrays and null values. + * + */ + public static String[] readCompressedStringArray(DataInput in) throws IOException { + int len = in.readInt(); + if (len == -1) return null; + String[] s = new String[len]; + for(int i = 0; i < len; i++) { + s[i] = readCompressedString(in); + } + return s; + } + + + /* + * + * Test Utility Method Display Byte Array. + * + */ + public static void displayByteArray(byte[] record){ + int i; + for(i=0;i < record.length -1; i++){ + if (i % 16 == 0) { System.out.println(); } + System.out.print(Integer.toHexString(record[i] >> 4 & 0x0F)); + System.out.print(Integer.toHexString(record[i] & 0x0F)); + System.out.print(","); + } + System.out.print(Integer.toHexString(record[i] >> 4 & 0x0F)); + System.out.print(Integer.toHexString(record[i] & 0x0F)); + System.out.println(); + } + + + /** + * Serializes an integer to a binary stream with zero-compressed encoding. + * For -120 <= i <= 127, only one byte is used with the actual value. + * For other values of i, the first byte value indicates whether the + * integer is positive or negative, and the number of bytes that follow. + * If the first byte value v is between -121 and -124, the following integer + * is positive, with number of bytes that follow are -(v+120). + * If the first byte value v is between -125 and -128, the following integer + * is negative, with number of bytes that follow are -(v+124). Bytes are + * stored in the high-non-zero-byte-first order. + * + * @param stream Binary output stream + * @param i Integer to be serialized + * @throws java.io.IOException + */ + public static void writeVInt(DataOutput stream, int i) throws IOException { + writeVLong(stream, i); + } + + /** + * Serializes a long to a binary stream with zero-compressed encoding. + * For -112 <= i <= 127, only one byte is used with the actual value. + * For other values of i, the first byte value indicates whether the + * long is positive or negative, and the number of bytes that follow. + * If the first byte value v is between -113 and -120, the following long + * is positive, with number of bytes that follow are -(v+112). + * If the first byte value v is between -121 and -128, the following long + * is negative, with number of bytes that follow are -(v+120). Bytes are + * stored in the high-non-zero-byte-first order. + * + * @param stream Binary output stream + * @param i Long to be serialized + * @throws java.io.IOException + */ + public static void writeVLong(DataOutput stream, long i) throws IOException { + if (i >= -112 && i <= 127) { + stream.writeByte((byte)i); + return; + } + + int len = -112; + if (i < 0) { + i ^= -1L; // take one's complement' + len = -120; + } + + long tmp = i; + while (tmp != 0) { + tmp = tmp >> 8; + len--; + } + + stream.writeByte((byte)len); + + len = (len < -120) ? -(len + 120) : -(len + 112); + + for (int idx = len; idx != 0; idx--) { + int shiftbits = (idx - 1) * 8; + long mask = 0xFFL << shiftbits; + stream.writeByte((byte)((i & mask) >> shiftbits)); + } + } + + + /** + * Reads a zero-compressed encoded long from input stream and returns it. + * @param stream Binary input stream + * @throws java.io.IOException + * @return deserialized long from stream. + */ + public static long readVLong(DataInput stream) throws IOException { + byte firstByte = stream.readByte(); + int len = decodeVIntSize(firstByte); + if (len == 1) { + return firstByte; + } + long i = 0; + for (int idx = 0; idx < len-1; idx++) { + byte b = stream.readByte(); + i = i << 8; + i = i | (b & 0xFF); + } + return (isNegativeVInt(firstByte) ? (i ^ -1L) : i); + } + + /** + * Reads a zero-compressed encoded integer from input stream and returns it. + * @param stream Binary input stream + * @throws java.io.IOException + * @return deserialized integer from stream. + */ + public static int readVInt(DataInput stream) throws IOException { + return (int) readVLong(stream); + } + + /** + * Given the first byte of a vint/vlong, determine the sign + * @param value the first byte + * @return is the value negative + */ + public static boolean isNegativeVInt(byte value) { + return value < -120 || (value >= -112 && value < 0); + } + + /** + * Parse the first byte of a vint/vlong to determine the number of bytes + * @param value the first byte of the vint/vlong + * @return the total number of bytes (1 to 9) + */ + public static int decodeVIntSize(byte value) { + if (value >= -112) { + return 1; + } else if (value < -120) { + return -119 - value; + } + return -111 - value; + } + + /** + * Get the encoded length if an integer is stored in a variable-length format + * @return the encoded length + */ + public static int getVIntSize(long i) { + if (i >= -112 && i <= 127) { + return 1; + } + + if (i < 0) { + i ^= -1L; // take one's complement' + } + // find the number of bytes with non-leading zeros + int dataBits = Long.SIZE - Long.numberOfLeadingZeros(i); + // find the number of data bytes + length byte + return (dataBits + 7) / 8 + 1; + } + + /** + * Skip len number of bytes in input streamin + * @param in input stream + * @param len number of bytes to skip + * @throws IOException when skipped less number of bytes + */ + public static void skipFully(DataInput in, int len) throws IOException { + int total = 0; + int cur = 0; + + while ((total 0)) { + total += cur; + } + + if (total max) { + _port = 0; + _factory = null; + LOG.error("Failed to find a port for Zookeeper"); + throw new RuntimeException("No port is available to launch an inprocess zookeeper."); + } + } + + public int port() { + return _port; + } + + public NIOServerCnxnFactory factory() { + return _factory; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/disruptor/AbstractSequencerExt.java b/jstorm-core/src/main/java/backtype/storm/utils/disruptor/AbstractSequencerExt.java new file mode 100755 index 000000000..c7199c621 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/disruptor/AbstractSequencerExt.java @@ -0,0 +1,38 @@ +/** + * 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 backtype.storm.utils.disruptor; + +import com.lmax.disruptor.AbstractSequencer; +import com.lmax.disruptor.WaitStrategy; + +public abstract class AbstractSequencerExt extends AbstractSequencer { + private static boolean waitSleep = true; + + public static boolean isWaitSleep() { + return waitSleep; + } + + public static void setWaitSleep(boolean waitSleep) { + AbstractSequencerExt.waitSleep = waitSleep; + } + + public AbstractSequencerExt(int bufferSize, WaitStrategy waitStrategy) { + super(bufferSize, waitStrategy); + } + +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/disruptor/MultiProducerSequencer.java b/jstorm-core/src/main/java/backtype/storm/utils/disruptor/MultiProducerSequencer.java new file mode 100755 index 000000000..cb5d7f9fb --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/disruptor/MultiProducerSequencer.java @@ -0,0 +1,267 @@ +/** + * 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 backtype.storm.utils.disruptor; + +import java.util.concurrent.locks.LockSupport; + +import sun.misc.Unsafe; + +import com.lmax.disruptor.InsufficientCapacityException; +import com.lmax.disruptor.Sequence; +import com.lmax.disruptor.Sequencer; +import com.lmax.disruptor.WaitStrategy; +import com.lmax.disruptor.util.Util; + +/** + * Coordinator for claiming sequences for access to a data structure while tracking dependent {@link Sequence}s + * + * Suitable for use for sequencing across multiple publisher threads. + */ +public class MultiProducerSequencer extends AbstractSequencerExt { + + private static final Unsafe UNSAFE = Util.getUnsafe(); + private static final long BASE = UNSAFE.arrayBaseOffset(int[].class); + private static final long SCALE = UNSAFE.arrayIndexScale(int[].class); + + private final Sequence gatingSequenceCache = new Sequence(Sequencer.INITIAL_CURSOR_VALUE); + + // availableBuffer tracks the state of each ringbuffer slot + // see below for more details on the approach + private final int[] availableBuffer; + private final int indexMask; + private final int indexShift; + + /** + * Construct a Sequencer with the selected wait strategy and buffer size. + * + * @param bufferSize the size of the buffer that this will sequence over. + * @param waitStrategy for those waiting on sequences. + */ + public MultiProducerSequencer(int bufferSize, final WaitStrategy waitStrategy) { + super(bufferSize, waitStrategy); + availableBuffer = new int[bufferSize]; + indexMask = bufferSize - 1; + indexShift = Util.log2(bufferSize); + initialiseAvailableBuffer(); + } + + /** + * @see Sequencer#hasAvailableCapacity(int) + */ + @Override + public boolean hasAvailableCapacity(final int requiredCapacity) { + return hasAvailableCapacity(gatingSequences, requiredCapacity, cursor.get()); + } + + private boolean hasAvailableCapacity(Sequence[] gatingSequences, final int requiredCapacity, long cursorValue) { + long wrapPoint = (cursorValue + requiredCapacity) - bufferSize; + long cachedGatingSequence = gatingSequenceCache.get(); + + if (wrapPoint > cachedGatingSequence || cachedGatingSequence > cursorValue) { + long minSequence = Util.getMinimumSequence(gatingSequences, cursorValue); + gatingSequenceCache.set(minSequence); + + if (wrapPoint > minSequence) { + return false; + } + } + + return true; + } + + /** + * @see Sequencer#claim(long) + */ + @Override + public void claim(long sequence) { + cursor.set(sequence); + } + + /** + * @see Sequencer#next() + */ + @Override + public long next() { + return next(1); + } + + /** + * @see Sequencer#next(int) + */ + @Override + public long next(int n) { + if (n < 1) { + throw new IllegalArgumentException("n must be > 0"); + } + + long current; + long next; + + do { + current = cursor.get(); + next = current + n; + + long wrapPoint = next - bufferSize; + long cachedGatingSequence = gatingSequenceCache.get(); + + if (wrapPoint > cachedGatingSequence || cachedGatingSequence > current) { + long gatingSequence = Util.getMinimumSequence(gatingSequences, current); + + if (wrapPoint > gatingSequence) { + if (AbstractSequencerExt.isWaitSleep()) { + try { + Thread.sleep(1); + } catch (InterruptedException e) { + } + } else { + LockSupport.parkNanos(1); + } + continue; + } + + gatingSequenceCache.set(gatingSequence); + } else if (cursor.compareAndSet(current, next)) { + break; + } + } while (true); + + return next; + } + + /** + * @see Sequencer#tryNext() + */ + @Override + public long tryNext() throws InsufficientCapacityException { + return tryNext(1); + } + + /** + * @see Sequencer#tryNext(int) + */ + @Override + public long tryNext(int n) throws InsufficientCapacityException { + if (n < 1) { + throw new IllegalArgumentException("n must be > 0"); + } + + long current; + long next; + + do { + current = cursor.get(); + next = current + n; + + if (!hasAvailableCapacity(gatingSequences, n, current)) { + throw InsufficientCapacityException.INSTANCE; + } + } while (!cursor.compareAndSet(current, next)); + + return next; + } + + /** + * @see Sequencer#remainingCapacity() + */ + @Override + public long remainingCapacity() { + long consumed = Util.getMinimumSequence(gatingSequences, cursor.get()); + long produced = cursor.get(); + return getBufferSize() - (produced - consumed); + } + + private void initialiseAvailableBuffer() { + for (int i = availableBuffer.length - 1; i != 0; i--) { + setAvailableBufferValue(i, -1); + } + + setAvailableBufferValue(0, -1); + } + + /** + * @see Sequencer#publish(long) + */ + @Override + public void publish(final long sequence) { + setAvailable(sequence); + waitStrategy.signalAllWhenBlocking(); + } + + /** + * @see Sequencer#publish(long, long) + */ + @Override + public void publish(long lo, long hi) { + for (long l = lo; l <= hi; l++) { + setAvailable(l); + } + waitStrategy.signalAllWhenBlocking(); + } + + /** + * The below methods work on the availableBuffer flag. + * + * The prime reason is to avoid a shared sequence object between publisher threads. (Keeping single pointers tracking start and end would require + * coordination between the threads). + * + * -- Firstly we have the constraint that the delta between the cursor and minimum gating sequence will never be larger than the buffer size (the code in + * next/tryNext in the Sequence takes care of that). -- Given that; take the sequence value and mask off the lower portion of the sequence as the index into + * the buffer (indexMask). (aka modulo operator) -- The upper portion of the sequence becomes the value to check for availability. ie: it tells us how many + * times around the ring buffer we've been (aka division) -- Because we can't wrap without the gating sequences moving forward (i.e. the minimum gating + * sequence is effectively our last available position in the buffer), when we have new data and successfully claimed a slot we can simply write over the + * top. + */ + private void setAvailable(final long sequence) { + setAvailableBufferValue(calculateIndex(sequence), calculateAvailabilityFlag(sequence)); + } + + private void setAvailableBufferValue(int index, int flag) { + long bufferAddress = (index * SCALE) + BASE; + UNSAFE.putOrderedInt(availableBuffer, bufferAddress, flag); + } + + /** + * @see Sequencer#isAvailable(long) + */ + @Override + public boolean isAvailable(long sequence) { + int index = calculateIndex(sequence); + int flag = calculateAvailabilityFlag(sequence); + long bufferAddress = (index * SCALE) + BASE; + return UNSAFE.getIntVolatile(availableBuffer, bufferAddress) == flag; + } + + @Override + public long getHighestPublishedSequence(long lowerBound, long availableSequence) { + for (long sequence = lowerBound; sequence <= availableSequence; sequence++) { + if (!isAvailable(sequence)) { + return sequence - 1; + } + } + + return availableSequence; + } + + private int calculateAvailabilityFlag(final long sequence) { + return (int) (sequence >>> indexShift); + } + + private int calculateIndex(final long sequence) { + return ((int) sequence) & indexMask; + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/disruptor/RingBuffer.java b/jstorm-core/src/main/java/backtype/storm/utils/disruptor/RingBuffer.java new file mode 100755 index 000000000..da124f0d8 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/disruptor/RingBuffer.java @@ -0,0 +1,996 @@ +/** + * 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 backtype.storm.utils.disruptor; + +import com.lmax.disruptor.BlockingWaitStrategy; +import com.lmax.disruptor.Cursored; +import com.lmax.disruptor.DataProvider; +import com.lmax.disruptor.EventFactory; +import com.lmax.disruptor.EventProcessor; +import com.lmax.disruptor.EventTranslator; +import com.lmax.disruptor.EventTranslatorOneArg; +import com.lmax.disruptor.EventTranslatorThreeArg; +import com.lmax.disruptor.EventTranslatorTwoArg; +import com.lmax.disruptor.EventTranslatorVararg; +import com.lmax.disruptor.InsufficientCapacityException; +import com.lmax.disruptor.Sequence; +import com.lmax.disruptor.SequenceBarrier; +import com.lmax.disruptor.Sequencer; +import com.lmax.disruptor.WaitStrategy; +import com.lmax.disruptor.dsl.ProducerType; +import backtype.storm.utils.disruptor.MultiProducerSequencer; +import backtype.storm.utils.disruptor.SingleProducerSequencer; + +/** + * Ring based store of reusable entries containing the data representing an event being exchanged between event producer and {@link EventProcessor}s. + * + * @param implementation storing the data for sharing during exchange or parallel coordination of an event. + */ +public class RingBuffer implements Cursored, DataProvider { + public static final long INITIAL_CURSOR_VALUE = -1L; + + private final int indexMask; + private final Object[] entries; + private final int bufferSize; + private final Sequencer sequencer; + + /** + * Construct a RingBuffer with the full option set. + * + * @param eventFactory to newInstance entries for filling the RingBuffer + * @param sequencer sequencer to handle the ordering of events moving through the RingBuffer. + * @throws IllegalArgumentException if bufferSize is less than 1 or not a power of 2 + */ + public RingBuffer(EventFactory eventFactory, Sequencer sequencer) { + this.sequencer = sequencer; + this.bufferSize = sequencer.getBufferSize(); + + if (bufferSize < 1) { + throw new IllegalArgumentException("bufferSize must not be less than 1"); + } + if (Integer.bitCount(bufferSize) != 1) { + throw new IllegalArgumentException("bufferSize must be a power of 2"); + } + + this.indexMask = bufferSize - 1; + this.entries = new Object[sequencer.getBufferSize()]; + fill(eventFactory); + } + + /** + * Create a new multiple producer RingBuffer with the specified wait strategy. + * + * @see MultiProducerSequencer + * @param factory used to create the events within the ring buffer. + * @param bufferSize number of elements to create within the ring buffer. + * @param waitStrategy used to determine how to wait for new elements to become available. + * @throws IllegalArgumentException if bufferSize is less than 1 or not a power of 2 + */ + public static RingBuffer createMultiProducer(EventFactory factory, int bufferSize, WaitStrategy waitStrategy) { + MultiProducerSequencer sequencer = new MultiProducerSequencer(bufferSize, waitStrategy); + + return new RingBuffer(factory, sequencer); + } + + /** + * Create a new multiple producer RingBuffer using the default wait strategy {@link BlockingWaitStrategy}. + * + * @see MultiProducerSequencer + * @param factory used to create the events within the ring buffer. + * @param bufferSize number of elements to create within the ring buffer. + * @throws IllegalArgumentException if bufferSize is less than 1 or not a power of 2 + */ + public static RingBuffer createMultiProducer(EventFactory factory, int bufferSize) { + return createMultiProducer(factory, bufferSize, new BlockingWaitStrategy()); + } + + /** + * Create a new single producer RingBuffer with the specified wait strategy. + * + * @see SingleProducerSequencer + * @param factory used to create the events within the ring buffer. + * @param bufferSize number of elements to create within the ring buffer. + * @param waitStrategy used to determine how to wait for new elements to become available. + * @throws IllegalArgumentException if bufferSize is less than 1 or not a power of 2 + */ + public static RingBuffer createSingleProducer(EventFactory factory, int bufferSize, WaitStrategy waitStrategy) { + SingleProducerSequencer sequencer = new SingleProducerSequencer(bufferSize, waitStrategy); + + return new RingBuffer(factory, sequencer); + } + + /** + * Create a new single producer RingBuffer using the default wait strategy {@link BlockingWaitStrategy}. + * + * @see MultiProducerSequencer + * @param factory used to create the events within the ring buffer. + * @param bufferSize number of elements to create within the ring buffer. + * @throws IllegalArgumentException if bufferSize is less than 1 or not a power of 2 + */ + public static RingBuffer createSingleProducer(EventFactory factory, int bufferSize) { + return createSingleProducer(factory, bufferSize, new BlockingWaitStrategy()); + } + + /** + * Create a new Ring Buffer with the specified producer type (SINGLE or MULTI) + * + * @param producerType producer type to use {@link ProducerType}. + * @param factory used to create events within the ring buffer. + * @param bufferSize number of elements to create within the ring buffer. + * @param waitStrategy used to determine how to wait for new elements to become available. + * @throws IllegalArgumentException if bufferSize is less than 1 or not a power of 2 + */ + public static RingBuffer create(ProducerType producerType, EventFactory factory, int bufferSize, WaitStrategy waitStrategy) { + switch (producerType) { + case SINGLE: + return createSingleProducer(factory, bufferSize, waitStrategy); + case MULTI: + return createMultiProducer(factory, bufferSize, waitStrategy); + default: + throw new IllegalStateException(producerType.toString()); + } + } + + /** + *

+ * Get the event for a given sequence in the RingBuffer. + *

+ * + *

+ * This call has 2 uses. Firstly use this call when publishing to a ring buffer. After calling {@link RingBuffer#next()} use this call to get hold of the + * preallocated event to fill with data before calling {@link RingBuffer#publish(long)}. + *

+ * + *

+ * Secondly use this call when consuming data from the ring buffer. After calling {@link SequenceBarrier#waitFor(long)} call this method with any value + * greater than that your current consumer sequence and less than or equal to the value returned from the {@link SequenceBarrier#waitFor(long)} method. + *

+ * + * @param sequence for the event + * @return the event for the given sequence + */ + @SuppressWarnings("unchecked") + public E get(long sequence) { + return (E) entries[(int) sequence & indexMask]; + } + + /** + * @deprecated Use {@link RingBuffer#get(long)} + */ + @Deprecated + public E getPreallocated(long sequence) { + return get(sequence); + } + + /** + * @deprecated Use {@link RingBuffer#get(long)} + */ + @Deprecated + public E getPublished(long sequence) { + return get(sequence); + } + + /** + * Increment and return the next sequence for the ring buffer. Calls of this method should ensure that they always publish the sequence afterward. E.g. + * + *
+     * long sequence = ringBuffer.next();
+     * try {
+     *     Event e = ringBuffer.get(sequence);
+     *     // Do some work with the event.
+     * } finally {
+     *     ringBuffer.publish(sequence);
+     * }
+     * 
+ * + * @see RingBuffer#publish(long) + * @see RingBuffer#get(long) + * @return The next sequence to publish to. + */ + public long next() { + return sequencer.next(); + } + + /** + * The same functionality as {@link RingBuffer#next()}, but allows the caller to claim the next n sequences. + * + * @see Sequencer#next(int) + * @param n number of slots to claim + * @return sequence number of the highest slot claimed + */ + public long next(int n) { + return sequencer.next(n); + } + + /** + *

+ * Increment and return the next sequence for the ring buffer. Calls of this method should ensure that they always publish the sequence afterward. E.g. + * + *

+     * long sequence = ringBuffer.next();
+     * try {
+     *     Event e = ringBuffer.get(sequence);
+     *     // Do some work with the event.
+     * } finally {
+     *     ringBuffer.publish(sequence);
+     * }
+     * 
+ *

+ * This method will not block if there is not space available in the ring buffer, instead it will throw an {@link InsufficientCapacityException}. + * + * + * @see RingBuffer#publish(long) + * @see RingBuffer#get(long) + * @return The next sequence to publish to. + * @throws InsufficientCapacityException if the necessary space in the ring buffer is not available + */ + public long tryNext() throws InsufficientCapacityException { + return sequencer.tryNext(); + } + + /** + * The same functionality as {@link RingBuffer#tryNext()}, but allows the caller to attempt to claim the next n sequences. + * + * @param n number of slots to claim + * @return sequence number of the highest slot claimed + * @throws InsufficientCapacityException if the necessary space in the ring buffer is not available + */ + public long tryNext(int n) throws InsufficientCapacityException { + return sequencer.tryNext(n); + } + + /** + * Resets the cursor to a specific value. This can be applied at any time, but it is worth not that it is a racy thing to do and should only be used in + * controlled circumstances. E.g. during initialisation. + * + * @param sequence The sequence to reset too. + * @throws IllegalStateException If any gating sequences have already been specified. + */ + public void resetTo(long sequence) { + sequencer.claim(sequence); + sequencer.publish(sequence); + } + + /** + * Sets the cursor to a specific sequence and returns the preallocated entry that is stored there. This is another deliberately racy call, that should only + * be done in controlled circumstances, e.g. initialisation. + * + * @param sequence The sequence to claim. + * @return The preallocated event. + */ + public E claimAndGetPreallocated(long sequence) { + sequencer.claim(sequence); + return get(sequence); + } + + /** + * Determines if a particular entry has been published. + * + * @param sequence The sequence to identify the entry. + * @return If the value has been published or not. + */ + public boolean isPublished(long sequence) { + return sequencer.isAvailable(sequence); + } + + /** + * Add the specified gating sequences to this instance of the Disruptor. They will safely and atomically added to the list of gating sequences. + * + * @param gatingSequences The sequences to add. + */ + public void addGatingSequences(Sequence... gatingSequences) { + sequencer.addGatingSequences(gatingSequences); + } + + /** + * Get the minimum sequence value from all of the gating sequences added to this ringBuffer. + * + * @return The minimum gating sequence or the cursor sequence if no sequences have been added. + */ + public long getMinimumGatingSequence() { + return sequencer.getMinimumSequence(); + } + + /** + * Remove the specified sequence from this ringBuffer. + * + * @param sequence to be removed. + * @return true if this sequence was found, false otherwise. + */ + public boolean removeGatingSequence(Sequence sequence) { + return sequencer.removeGatingSequence(sequence); + } + + /** + * Create a new SequenceBarrier to be used by an EventProcessor to track which messages are available to be read from the ring buffer given a list of + * sequences to track. + * + * @see SequenceBarrier + * @param sequencesToTrack the additional sequences to track + * @return A sequence barrier that will track the specified sequences. + */ + public SequenceBarrier newBarrier(Sequence... sequencesToTrack) { + return sequencer.newBarrier(sequencesToTrack); + } + + /** + * Get the current cursor value for the ring buffer. The cursor value is the last value that was published, or the highest available sequence that can be + * consumed. + */ + public long getCursor() { + return sequencer.getCursor(); + } + + /** + * The size of the buffer. + */ + public int getBufferSize() { + return bufferSize; + } + + /** + * Given specified requiredCapacity determines if that amount of space is available. Note, you can not assume that if this method returns + * true that a call to {@link RingBuffer#next()} will not block. Especially true if this ring buffer is set up to handle multiple producers. + * + * @param requiredCapacity The capacity to check for. + * @return true If the specified requiredCapacity is available false if now. + */ + public boolean hasAvailableCapacity(int requiredCapacity) { + return sequencer.hasAvailableCapacity(requiredCapacity); + } + + /** + * Publishes an event to the ring buffer. It handles claiming the next sequence, getting the current (uninitialised) event from the ring buffer and + * publishing the claimed sequence after translation. + * + * @param translator The user specified translation for the event + */ + public void publishEvent(EventTranslator translator) { + final long sequence = sequencer.next(); + translateAndPublish(translator, sequence); + } + + /** + * Attempts to publish an event to the ring buffer. It handles claiming the next sequence, getting the current (uninitialised) event from the ring buffer + * and publishing the claimed sequence after translation. Will return false if specified capacity was not available. + * + * @param translator The user specified translation for the event + * @return true if the value was published, false if there was insufficient capacity. + */ + public boolean tryPublishEvent(EventTranslator translator) { + try { + final long sequence = sequencer.tryNext(); + translateAndPublish(translator, sequence); + return true; + } catch (InsufficientCapacityException e) { + return false; + } + } + + /** + * Allows one user supplied argument. + * + * @see #publishEvent(EventTranslator) + * @param translator The user specified translation for the event + * @param arg0 A user supplied argument. + */ + public void publishEvent(EventTranslatorOneArg translator, A arg0) { + final long sequence = sequencer.next(); + translateAndPublish(translator, sequence, arg0); + } + + /** + * Allows one user supplied argument. + * + * @see #tryPublishEvent(EventTranslator) + * @param translator The user specified translation for the event + * @param arg0 A user supplied argument. + * @return true if the value was published, false if there was insufficient capacity. + */ + public boolean tryPublishEvent(EventTranslatorOneArg translator, A arg0) { + try { + final long sequence = sequencer.tryNext(); + translateAndPublish(translator, sequence, arg0); + return true; + } catch (InsufficientCapacityException e) { + return false; + } + } + + /** + * Allows two user supplied arguments. + * + * @see #publishEvent(EventTranslator) + * @param translator The user specified translation for the event + * @param arg0 A user supplied argument. + * @param arg1 A user supplied argument. + */ + public void publishEvent(EventTranslatorTwoArg translator, A arg0, B arg1) { + final long sequence = sequencer.next(); + translateAndPublish(translator, sequence, arg0, arg1); + } + + /** + * Allows two user supplied arguments. + * + * @see #tryPublishEvent(EventTranslator) + * @param translator The user specified translation for the event + * @param arg0 A user supplied argument. + * @param arg1 A user supplied argument. + * @return true if the value was published, false if there was insufficient capacity. + */ + public boolean tryPublishEvent(EventTranslatorTwoArg translator, A arg0, B arg1) { + try { + final long sequence = sequencer.tryNext(); + translateAndPublish(translator, sequence, arg0, arg1); + return true; + } catch (InsufficientCapacityException e) { + return false; + } + } + + /** + * Allows three user supplied arguments + * + * @see #publishEvent(EventTranslator) + * @param translator The user specified translation for the event + * @param arg0 A user supplied argument. + * @param arg1 A user supplied argument. + * @param arg2 A user supplied argument. + */ + public void publishEvent(EventTranslatorThreeArg translator, A arg0, B arg1, C arg2) { + final long sequence = sequencer.next(); + translateAndPublish(translator, sequence, arg0, arg1, arg2); + } + + /** + * Allows three user supplied arguments + * + * @see #publishEvent(EventTranslator) + * @param translator The user specified translation for the event + * @param arg0 A user supplied argument. + * @param arg1 A user supplied argument. + * @param arg2 A user supplied argument. + * @return true if the value was published, false if there was insufficient capacity. + */ + public boolean tryPublishEvent(EventTranslatorThreeArg translator, A arg0, B arg1, C arg2) { + try { + final long sequence = sequencer.tryNext(); + translateAndPublish(translator, sequence, arg0, arg1, arg2); + return true; + } catch (InsufficientCapacityException e) { + return false; + } + } + + /** + * Allows a variable number of user supplied arguments + * + * @see #publishEvent(EventTranslator) + * @param translator The user specified translation for the event + * @param args User supplied arguments. + */ + public void publishEvent(EventTranslatorVararg translator, Object... args) { + final long sequence = sequencer.next(); + translateAndPublish(translator, sequence, args); + } + + /** + * Allows a variable number of user supplied arguments + * + * @see #publishEvent(EventTranslator) + * @param translator The user specified translation for the event + * @param args User supplied arguments. + * @return true if the value was published, false if there was insufficient capacity. + */ + public boolean tryPublishEvent(EventTranslatorVararg translator, Object... args) { + try { + final long sequence = sequencer.tryNext(); + translateAndPublish(translator, sequence, args); + return true; + } catch (InsufficientCapacityException e) { + return false; + } + } + + /** + * Publishes multiple events to the ring buffer. It handles claiming the next sequence, getting the current (uninitialised) event from the ring buffer and + * publishing the claimed sequence after translation. + * + * @param translators The user specified translation for each event + */ + public void publishEvents(EventTranslator[] translators) { + publishEvents(translators, 0, translators.length); + } + + /** + * Publishes multiple events to the ring buffer. It handles claiming the next sequence, getting the current (uninitialised) event from the ring buffer and + * publishing the claimed sequence after translation. + * + * @param translators The user specified translation for each event + * @param batchStartsAt The first element of the array which is within the batch. + * @param batchSize The actual size of the batch + */ + public void publishEvents(EventTranslator[] translators, int batchStartsAt, int batchSize) { + checkBounds(translators, batchStartsAt, batchSize); + final long finalSequence = sequencer.next(batchSize); + translateAndPublishBatch(translators, batchStartsAt, batchSize, finalSequence); + } + + /** + * Attempts to publish multiple events to the ring buffer. It handles claiming the next sequence, getting the current (uninitialised) event from the ring + * buffer and publishing the claimed sequence after translation. Will return false if specified capacity was not available. + * + * @param translators The user specified translation for the event + * @return true if the value was published, false if there was insufficient capacity. + */ + public boolean tryPublishEvents(EventTranslator[] translators) { + return tryPublishEvents(translators, 0, translators.length); + } + + /** + * Attempts to publish multiple events to the ring buffer. It handles claiming the next sequence, getting the current (uninitialised) event from the ring + * buffer and publishing the claimed sequence after translation. Will return false if specified capacity was not available. + * + * @param translators The user specified translation for the event + * @param batchStartsAt The first element of the array which is within the batch. + * @param batchSize The actual size of the batch + * @return true if all the values were published, false if there was insufficient capacity. + */ + public boolean tryPublishEvents(EventTranslator[] translators, int batchStartsAt, int batchSize) { + checkBounds(translators, batchStartsAt, batchSize); + try { + final long finalSequence = sequencer.tryNext(batchSize); + translateAndPublishBatch(translators, batchStartsAt, batchSize, finalSequence); + return true; + } catch (InsufficientCapacityException e) { + return false; + } + } + + /** + * Allows one user supplied argument per event. + * + * @param translator The user specified translation for the event + * @param arg0 A user supplied argument. + * @see #publishEvents(com.lmax.disruptor.EventTranslator[]) + */ + public void publishEvents(EventTranslatorOneArg translator, A[] arg0) { + publishEvents(translator, 0, arg0.length, arg0); + } + + /** + * Allows one user supplied argument per event. + * + * @param translator The user specified translation for each event + * @param batchStartsAt The first element of the array which is within the batch. + * @param batchSize The actual size of the batch + * @param arg0 An array of user supplied arguments, one element per event. + * @see #publishEvents(EventTranslator[]) + */ + public void publishEvents(EventTranslatorOneArg translator, int batchStartsAt, int batchSize, A[] arg0) { + checkBounds(arg0, batchStartsAt, batchSize); + final long finalSequence = sequencer.next(batchSize); + translateAndPublishBatch(translator, arg0, batchStartsAt, batchSize, finalSequence); + } + + /** + * Allows one user supplied argument. + * + * @param translator The user specified translation for each event + * @param arg0 An array of user supplied arguments, one element per event. + * @return true if the value was published, false if there was insufficient capacity. + * @see #tryPublishEvents(com.lmax.disruptor.EventTranslator[]) + */ + public boolean tryPublishEvents(EventTranslatorOneArg translator, A[] arg0) { + return tryPublishEvents(translator, 0, arg0.length, arg0); + } + + /** + * Allows one user supplied argument. + * + * @param translator The user specified translation for each event + * @param batchStartsAt The first element of the array which is within the batch. + * @param batchSize The actual size of the batch + * @param arg0 An array of user supplied arguments, one element per event. + * @return true if the value was published, false if there was insufficient capacity. + * @see #tryPublishEvents(EventTranslator[]) + */ + public boolean tryPublishEvents(EventTranslatorOneArg translator, int batchStartsAt, int batchSize, A[] arg0) { + checkBounds(arg0, batchStartsAt, batchSize); + try { + final long finalSequence = sequencer.tryNext(batchSize); + translateAndPublishBatch(translator, arg0, batchStartsAt, batchSize, finalSequence); + return true; + } catch (InsufficientCapacityException e) { + return false; + } + } + + /** + * Allows two user supplied arguments per event. + * + * @param translator The user specified translation for the event + * @param arg0 An array of user supplied arguments, one element per event. + * @param arg1 An array of user supplied arguments, one element per event. + * @see #publishEvents(com.lmax.disruptor.EventTranslator[]) + */ + public void publishEvents(EventTranslatorTwoArg translator, A[] arg0, B[] arg1) { + publishEvents(translator, 0, arg0.length, arg0, arg1); + } + + /** + * Allows two user supplied arguments per event. + * + * @param translator The user specified translation for the event + * @param batchStartsAt The first element of the array which is within the batch. + * @param batchSize The actual size of the batch. + * @param arg0 An array of user supplied arguments, one element per event. + * @param arg1 An array of user supplied arguments, one element per event. + * @see #publishEvents(EventTranslator[]) + */ + public void publishEvents(EventTranslatorTwoArg translator, int batchStartsAt, int batchSize, A[] arg0, B[] arg1) { + checkBounds(arg0, arg1, batchStartsAt, batchSize); + final long finalSequence = sequencer.next(batchSize); + translateAndPublishBatch(translator, arg0, arg1, batchStartsAt, batchSize, finalSequence); + } + + /** + * Allows two user supplied arguments per event. + * + * @param translator The user specified translation for the event + * @param arg0 An array of user supplied arguments, one element per event. + * @param arg1 An array of user supplied arguments, one element per event. + * @return true if the value was published, false if there was insufficient capacity. + * @see #tryPublishEvents(com.lmax.disruptor.EventTranslator[]) + */ + public boolean tryPublishEvents(EventTranslatorTwoArg translator, A[] arg0, B[] arg1) { + return tryPublishEvents(translator, 0, arg0.length, arg0, arg1); + } + + /** + * Allows two user supplied arguments per event. + * + * @param translator The user specified translation for the event + * @param batchStartsAt The first element of the array which is within the batch. + * @param batchSize The actual size of the batch. + * @param arg0 An array of user supplied arguments, one element per event. + * @param arg1 An array of user supplied arguments, one element per event. + * @return true if the value was published, false if there was insufficient capacity. + * @see #tryPublishEvents(EventTranslator[]) + */ + public boolean tryPublishEvents(EventTranslatorTwoArg translator, int batchStartsAt, int batchSize, A[] arg0, B[] arg1) { + checkBounds(arg0, arg1, batchStartsAt, batchSize); + try { + final long finalSequence = sequencer.tryNext(batchSize); + translateAndPublishBatch(translator, arg0, arg1, batchStartsAt, batchSize, finalSequence); + return true; + } catch (InsufficientCapacityException e) { + return false; + } + } + + /** + * Allows three user supplied arguments per event. + * + * @param translator The user specified translation for the event + * @param arg0 An array of user supplied arguments, one element per event. + * @param arg1 An array of user supplied arguments, one element per event. + * @param arg2 An array of user supplied arguments, one element per event. + * @see #publishEvents(com.lmax.disruptor.EventTranslator[]) + */ + public void publishEvents(EventTranslatorThreeArg translator, A[] arg0, B[] arg1, C[] arg2) { + publishEvents(translator, 0, arg0.length, arg0, arg1, arg2); + } + + /** + * Allows three user supplied arguments per event. + * + * @param translator The user specified translation for the event + * @param batchStartsAt The first element of the array which is within the batch. + * @param batchSize The number of elements in the batch. + * @param arg0 An array of user supplied arguments, one element per event. + * @param arg1 An array of user supplied arguments, one element per event. + * @param arg2 An array of user supplied arguments, one element per event. + * @see #publishEvents(EventTranslator[]) + */ + public void publishEvents(EventTranslatorThreeArg translator, int batchStartsAt, int batchSize, A[] arg0, B[] arg1, C[] arg2) { + checkBounds(arg0, arg1, arg2, batchStartsAt, batchSize); + final long finalSequence = sequencer.next(batchSize); + translateAndPublishBatch(translator, arg0, arg1, arg2, batchStartsAt, batchSize, finalSequence); + } + + /** + * Allows three user supplied arguments per event. + * + * @param translator The user specified translation for the event + * @param arg0 An array of user supplied arguments, one element per event. + * @param arg1 An array of user supplied arguments, one element per event. + * @param arg2 An array of user supplied arguments, one element per event. + * @return true if the value was published, false if there was insufficient capacity. + * @see #publishEvents(com.lmax.disruptor.EventTranslator[]) + */ + public boolean tryPublishEvents(EventTranslatorThreeArg translator, A[] arg0, B[] arg1, C[] arg2) { + return tryPublishEvents(translator, 0, arg0.length, arg0, arg1, arg2); + } + + /** + * Allows three user supplied arguments per event. + * + * @param translator The user specified translation for the event + * @param batchStartsAt The first element of the array which is within the batch. + * @param batchSize The actual size of the batch. + * @param arg0 An array of user supplied arguments, one element per event. + * @param arg1 An array of user supplied arguments, one element per event. + * @param arg2 An array of user supplied arguments, one element per event. + * @return true if the value was published, false if there was insufficient capacity. + * @see #publishEvents(EventTranslator[]) + */ + public boolean tryPublishEvents(EventTranslatorThreeArg translator, int batchStartsAt, int batchSize, A[] arg0, B[] arg1, C[] arg2) { + checkBounds(arg0, arg1, arg2, batchStartsAt, batchSize); + try { + final long finalSequence = sequencer.tryNext(batchSize); + translateAndPublishBatch(translator, arg0, arg1, arg2, batchStartsAt, batchSize, finalSequence); + return true; + } catch (InsufficientCapacityException e) { + return false; + } + } + + /** + * Allows a variable number of user supplied arguments per event. + * + * @param translator The user specified translation for the event + * @param args User supplied arguments, one Object[] per event. + * @see #publishEvents(com.lmax.disruptor.EventTranslator[]) + */ + public void publishEvents(EventTranslatorVararg translator, Object[]... args) { + publishEvents(translator, 0, args.length, args); + } + + /** + * Allows a variable number of user supplied arguments per event. + * + * @param translator The user specified translation for the event + * @param batchStartsAt The first element of the array which is within the batch. + * @param batchSize The actual size of the batch + * @param args User supplied arguments, one Object[] per event. + * @see #publishEvents(EventTranslator[]) + */ + public void publishEvents(EventTranslatorVararg translator, int batchStartsAt, int batchSize, Object[]... args) { + checkBounds(batchStartsAt, batchSize, args); + final long finalSequence = sequencer.next(batchSize); + translateAndPublishBatch(translator, batchStartsAt, batchSize, finalSequence, args); + } + + /** + * Allows a variable number of user supplied arguments per event. + * + * @param translator The user specified translation for the event + * @param args User supplied arguments, one Object[] per event. + * @return true if the value was published, false if there was insufficient capacity. + * @see #publishEvents(com.lmax.disruptor.EventTranslator[]) + */ + public boolean tryPublishEvents(EventTranslatorVararg translator, Object[]... args) { + return tryPublishEvents(translator, 0, args.length, args); + } + + /** + * Allows a variable number of user supplied arguments per event. + * + * @param translator The user specified translation for the event + * @param batchStartsAt The first element of the array which is within the batch. + * @param batchSize The actual size of the batch. + * @param args User supplied arguments, one Object[] per event. + * @return true if the value was published, false if there was insufficient capacity. + * @see #publishEvents(EventTranslator[]) + */ + public boolean tryPublishEvents(EventTranslatorVararg translator, int batchStartsAt, int batchSize, Object[]... args) { + checkBounds(args, batchStartsAt, batchSize); + try { + final long finalSequence = sequencer.tryNext(batchSize); + translateAndPublishBatch(translator, batchStartsAt, batchSize, finalSequence, args); + return true; + } catch (InsufficientCapacityException e) { + return false; + } + } + + /** + * Publish the specified sequence. This action marks this particular message as being available to be read. + * + * @param sequence the sequence to publish. + */ + public void publish(long sequence) { + sequencer.publish(sequence); + } + + /** + * Publish the specified sequences. This action marks these particular messages as being available to be read. + * + * @see Sequencer#next(int) + * @param lo the lowest sequence number to be published + * @param hi the highest sequence number to be published + */ + public void publish(long lo, long hi) { + sequencer.publish(lo, hi); + } + + /** + * Get the remaining capacity for this ringBuffer. + * + * @return The number of slots remaining. + */ + public long remainingCapacity() { + return sequencer.remainingCapacity(); + } + + private void checkBounds(final EventTranslator[] translators, final int batchStartsAt, final int batchSize) { + checkBatchSizing(batchStartsAt, batchSize); + batchOverRuns(translators, batchStartsAt, batchSize); + } + + private void checkBatchSizing(int batchStartsAt, int batchSize) { + if (batchStartsAt < 0 || batchSize < 0) { + throw new IllegalArgumentException("Both batchStartsAt and batchSize must be positive but got: batchStartsAt " + batchStartsAt + " and batchSize " + batchSize); + } else if (batchSize > bufferSize) { + throw new IllegalArgumentException("The ring buffer cannot accommodate " + batchSize + " it only has space for " + bufferSize + " entities."); + } + } + + private void checkBounds(final A[] arg0, final int batchStartsAt, final int batchSize) { + checkBatchSizing(batchStartsAt, batchSize); + batchOverRuns(arg0, batchStartsAt, batchSize); + } + + private void checkBounds(final A[] arg0, final B[] arg1, final int batchStartsAt, final int batchSize) { + checkBatchSizing(batchStartsAt, batchSize); + batchOverRuns(arg0, batchStartsAt, batchSize); + batchOverRuns(arg1, batchStartsAt, batchSize); + } + + private void checkBounds(final A[] arg0, final B[] arg1, final C[] arg2, final int batchStartsAt, final int batchSize) { + checkBatchSizing(batchStartsAt, batchSize); + batchOverRuns(arg0, batchStartsAt, batchSize); + batchOverRuns(arg1, batchStartsAt, batchSize); + batchOverRuns(arg2, batchStartsAt, batchSize); + } + + private void checkBounds(final int batchStartsAt, final int batchSize, final Object[][] args) { + checkBatchSizing(batchStartsAt, batchSize); + batchOverRuns(args, batchStartsAt, batchSize); + } + + private void batchOverRuns(final A[] arg0, final int batchStartsAt, final int batchSize) { + if (batchStartsAt + batchSize > arg0.length) { + throw new IllegalArgumentException("A batchSize of: " + batchSize + " with batchStatsAt of: " + batchStartsAt + " will overrun the available number of arguments: " + (arg0.length - batchStartsAt)); + } + } + + private void translateAndPublish(EventTranslator translator, long sequence) { + try { + translator.translateTo(get(sequence), sequence); + } finally { + sequencer.publish(sequence); + } + } + + private void translateAndPublish(EventTranslatorOneArg translator, long sequence, A arg0) { + try { + translator.translateTo(get(sequence), sequence, arg0); + } finally { + sequencer.publish(sequence); + } + } + + private void translateAndPublish(EventTranslatorTwoArg translator, long sequence, A arg0, B arg1) { + try { + translator.translateTo(get(sequence), sequence, arg0, arg1); + } finally { + sequencer.publish(sequence); + } + } + + private void translateAndPublish(EventTranslatorThreeArg translator, long sequence, A arg0, B arg1, C arg2) { + try { + translator.translateTo(get(sequence), sequence, arg0, arg1, arg2); + } finally { + sequencer.publish(sequence); + } + } + + private void translateAndPublish(EventTranslatorVararg translator, long sequence, Object... args) { + try { + translator.translateTo(get(sequence), sequence, args); + } finally { + sequencer.publish(sequence); + } + } + + private void translateAndPublishBatch(final EventTranslator[] translators, int batchStartsAt, final int batchSize, final long finalSequence) { + final long initialSequence = finalSequence - (batchSize - 1); + try { + long sequence = initialSequence; + final int batchEndsAt = batchStartsAt + batchSize; + for (int i = batchStartsAt; i < batchEndsAt; i++) { + final EventTranslator translator = translators[i]; + translator.translateTo(get(sequence), sequence++); + } + } finally { + sequencer.publish(initialSequence, finalSequence); + } + } + + private void translateAndPublishBatch(final EventTranslatorOneArg translator, final A[] arg0, int batchStartsAt, final int batchSize, final long finalSequence) { + final long initialSequence = finalSequence - (batchSize - 1); + try { + long sequence = initialSequence; + final int batchEndsAt = batchStartsAt + batchSize; + for (int i = batchStartsAt; i < batchEndsAt; i++) { + translator.translateTo(get(sequence), sequence++, arg0[i]); + } + } finally { + sequencer.publish(initialSequence, finalSequence); + } + } + + private void translateAndPublishBatch(final EventTranslatorTwoArg translator, final A[] arg0, final B[] arg1, int batchStartsAt, int batchSize, final long finalSequence) { + final long initialSequence = finalSequence - (batchSize - 1); + try { + long sequence = initialSequence; + final int batchEndsAt = batchStartsAt + batchSize; + for (int i = batchStartsAt; i < batchEndsAt; i++) { + translator.translateTo(get(sequence), sequence++, arg0[i], arg1[i]); + } + } finally { + sequencer.publish(initialSequence, finalSequence); + } + } + + private void translateAndPublishBatch(final EventTranslatorThreeArg translator, final A[] arg0, final B[] arg1, final C[] arg2, int batchStartsAt, final int batchSize, final long finalSequence) { + final long initialSequence = finalSequence - (batchSize - 1); + try { + long sequence = initialSequence; + final int batchEndsAt = batchStartsAt + batchSize; + for (int i = batchStartsAt; i < batchEndsAt; i++) { + translator.translateTo(get(sequence), sequence++, arg0[i], arg1[i], arg2[i]); + } + } finally { + sequencer.publish(initialSequence, finalSequence); + } + } + + private void translateAndPublishBatch(final EventTranslatorVararg translator, int batchStartsAt, final int batchSize, final long finalSequence, final Object[][] args) { + final long initialSequence = finalSequence - (batchSize - 1); + try { + long sequence = initialSequence; + final int batchEndsAt = batchStartsAt + batchSize; + for (int i = batchStartsAt; i < batchEndsAt; i++) { + translator.translateTo(get(sequence), sequence++, args[i]); + } + } finally { + sequencer.publish(initialSequence, finalSequence); + } + } + + private void fill(EventFactory eventFactory) { + for (int i = 0; i < entries.length; i++) { + entries[i] = eventFactory.newInstance(); + } + } +} diff --git a/jstorm-core/src/main/java/backtype/storm/utils/disruptor/SingleProducerSequencer.java b/jstorm-core/src/main/java/backtype/storm/utils/disruptor/SingleProducerSequencer.java new file mode 100755 index 000000000..5ca272410 --- /dev/null +++ b/jstorm-core/src/main/java/backtype/storm/utils/disruptor/SingleProducerSequencer.java @@ -0,0 +1,197 @@ +/** + * 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 backtype.storm.utils.disruptor; + +import java.util.concurrent.locks.LockSupport; + +import com.lmax.disruptor.InsufficientCapacityException; +import com.lmax.disruptor.Sequence; +import com.lmax.disruptor.Sequencer; +import com.lmax.disruptor.WaitStrategy; +import com.lmax.disruptor.util.Util; + +/** + *

+ * Coordinator for claiming sequences for access to a data structure while tracking dependent {@link Sequence}s. + *

+ * + *

+ * Generally not safe for use from multiple threads as it does not implement any barriers. + *

+ */ +public class SingleProducerSequencer extends AbstractSequencerExt { + @SuppressWarnings("unused") + private static class Padding { + /** Set to -1 as sequence starting point */ + public long nextValue = -1L, cachedValue = -1L, p2, p3, p4, p5, p6, p7; + } + + private final Padding pad = new Padding(); + + /** + * Construct a Sequencer with the selected wait strategy and buffer size. + * + * @param bufferSize the size of the buffer that this will sequence over. + * @param waitStrategy for those waiting on sequences. + */ + public SingleProducerSequencer(int bufferSize, final WaitStrategy waitStrategy) { + super(bufferSize, waitStrategy); + } + + /** + * @see Sequencer#hasAvailableCapacity(int) + */ + @Override + public boolean hasAvailableCapacity(final int requiredCapacity) { + long nextValue = pad.nextValue; + + long wrapPoint = (nextValue + requiredCapacity) - bufferSize; + long cachedGatingSequence = pad.cachedValue; + + if (wrapPoint > cachedGatingSequence || cachedGatingSequence > nextValue) { + long minSequence = Util.getMinimumSequence(gatingSequences, nextValue); + pad.cachedValue = minSequence; + + if (wrapPoint > minSequence) { + return false; + } + } + + return true; + } + + /** + * @see Sequencer#next() + */ + @Override + public long next() { + return next(1); + } + + /** + * @see Sequencer#next(int) + */ + @Override + public long next(int n) { + if (n < 1) { + throw new IllegalArgumentException("n must be > 0"); + } + + long nextValue = pad.nextValue; + + long nextSequence = nextValue + n; + long wrapPoint = nextSequence - bufferSize; + long cachedGatingSequence = pad.cachedValue; + + if (wrapPoint > cachedGatingSequence || cachedGatingSequence > nextValue) { + long minSequence; + while (wrapPoint > (minSequence = Util.getMinimumSequence(gatingSequences, nextValue))) { + if (AbstractSequencerExt.isWaitSleep()) { + try { + Thread.sleep(1); + } catch (InterruptedException e) { + } + } else { + LockSupport.parkNanos(1); + } + } + + pad.cachedValue = minSequence; + } + + pad.nextValue = nextSequence; + + return nextSequence; + } + + /** + * @see Sequencer#tryNext() + */ + @Override + public long tryNext() throws InsufficientCapacityException { + return tryNext(1); + } + + /** + * @see Sequencer#tryNext(int) + */ + @Override + public long tryNext(int n) throws InsufficientCapacityException { + if (n < 1) { + throw new IllegalArgumentException("n must be > 0"); + } + + if (!hasAvailableCapacity(n)) { + throw InsufficientCapacityException.INSTANCE; + } + + long nextSequence = pad.nextValue += n; + + return nextSequence; + } + + /** + * @see Sequencer#remainingCapacity() + */ + @Override + public long remainingCapacity() { + long nextValue = pad.nextValue; + + long consumed = Util.getMinimumSequence(gatingSequences, nextValue); + long produced = nextValue; + return getBufferSize() - (produced - consumed); + } + + /** + * @see Sequencer#claim(long) + */ + @Override + public void claim(long sequence) { + pad.nextValue = sequence; + } + + /** + * @see Sequencer#publish(long) + */ + @Override + public void publish(long sequence) { + cursor.set(sequence); + waitStrategy.signalAllWhenBlocking(); + } + + /** + * @see Sequencer#publish(long, long) + */ + @Override + public void publish(long lo, long hi) { + publish(hi); + } + + /** + * @see Sequencer#isAvailable(long) + */ + @Override + public boolean isAvailable(long sequence) { + return sequence <= cursor.get(); + } + + @Override + public long getHighestPublishedSequence(long lowerBound, long availableSequence) { + return availableSequence; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/batch/BatchId.java b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/BatchId.java new file mode 100755 index 000000000..807c5ec78 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/BatchId.java @@ -0,0 +1,83 @@ +/** + * 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.batch; + +import java.io.Serializable; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; + +public class BatchId implements Serializable { + private static final long serialVersionUID = 5720810158625748049L; + protected final long id; + + protected BatchId(long id) { + this.id = id; + } + + public long getId() { + return id; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + (int) (id ^ (id >>> 32)); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + BatchId other = (BatchId) obj; + if (id != other.id) + return false; + return true; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } + + private static AtomicLong staticId = new AtomicLong(0); + + public static void updateId(long id) { + staticId.set(id); + } + + public static BatchId mkInstance() { + long id = staticId.incrementAndGet(); + + return new BatchId(id); + } + + public static BatchId incBatchId(BatchId old) { + long other = old.getId(); + return new BatchId(other + 1); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/batch/BatchTopologyBuilder.java b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/BatchTopologyBuilder.java new file mode 100755 index 000000000..85dec6cd6 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/BatchTopologyBuilder.java @@ -0,0 +1,88 @@ +/** + * 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.batch; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.topology.BoltDeclarer; +import backtype.storm.topology.IBasicBolt; +import backtype.storm.topology.SpoutDeclarer; +import backtype.storm.topology.TopologyBuilder; + +import com.alibaba.jstorm.batch.impl.BatchSpoutTrigger; +import com.alibaba.jstorm.batch.impl.CoordinatedBolt; +import com.alibaba.jstorm.batch.util.BatchDef; + +public class BatchTopologyBuilder { + private static final Logger LOG = LoggerFactory + .getLogger(BatchTopologyBuilder.class); + + private TopologyBuilder topologyBuilder; + + private SpoutDeclarer spoutDeclarer; + + public BatchTopologyBuilder(String topologyName) { + topologyBuilder = new TopologyBuilder(); + + spoutDeclarer = + topologyBuilder.setSpout(BatchDef.SPOUT_TRIGGER, + new BatchSpoutTrigger(), 1); + } + + public BoltDeclarer setSpout(String id, IBatchSpout spout, int paralel) { + + BoltDeclarer boltDeclarer = + this.setBolt(id, (IBatchSpout) spout, paralel); + boltDeclarer.allGrouping(BatchDef.SPOUT_TRIGGER, + BatchDef.COMPUTING_STREAM_ID); + + return boltDeclarer; + } + + public BoltDeclarer setBolt(String id, IBasicBolt bolt, int paralel) { + CoordinatedBolt coordinatedBolt = new CoordinatedBolt(bolt); + + BoltDeclarer boltDeclarer = + topologyBuilder.setBolt(id, coordinatedBolt, paralel); + + if (bolt instanceof IPrepareCommit) { + boltDeclarer.allGrouping(BatchDef.SPOUT_TRIGGER, + BatchDef.PREPARE_STREAM_ID); + } + + if (bolt instanceof ICommitter) { + boltDeclarer.allGrouping(BatchDef.SPOUT_TRIGGER, + BatchDef.COMMIT_STREAM_ID); + boltDeclarer.allGrouping(BatchDef.SPOUT_TRIGGER, + BatchDef.REVERT_STREAM_ID); + } + + if (bolt instanceof IPostCommit) { + boltDeclarer.allGrouping(BatchDef.SPOUT_TRIGGER, + BatchDef.POST_STREAM_ID); + } + + return boltDeclarer; + } + + public TopologyBuilder getTopologyBuilder() { + return topologyBuilder; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/batch/IBatchSpout.java b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/IBatchSpout.java new file mode 100755 index 000000000..591f0f09e --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/IBatchSpout.java @@ -0,0 +1,55 @@ +/** + * 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.batch; + +import java.io.Serializable; + +import backtype.storm.topology.IBasicBolt; + +public interface IBatchSpout extends IBasicBolt, ICommitter, Serializable { + + /** + * input's filed 0 is BatchId + * + * execute only receive trigger message + * + * do emitBatch operation in execute whose streamID is + * "batch/compute-stream" + */ + // void execute(Tuple input, IBasicOutputCollector collector); + /** + * begin to ack batchId's data + * + * return value will be stored in ZK, so sometimes don't need special action + * + * @param id + */ + // byte[] commit(BatchId id) throws FailedException; + + /** + * begin to revert batchId's data + * + * If current task fails to commit batchId, it won't call revert(batchId) If + * current task fails to revert batchId, JStorm won't call revert again. + * + * if not transaction, it can don't care revert + * + * @param id + */ + // void revert(BatchId id, byte[] commitResult); +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/batch/ICommitter.java b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/ICommitter.java new file mode 100755 index 000000000..16f10daa6 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/ICommitter.java @@ -0,0 +1,52 @@ +/** + * 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.batch; + +import java.io.Serializable; + +import backtype.storm.topology.FailedException; + +/** + * The less committer, the state is more stable. Don't need to do + * + * @author zhongyan.feng + * @version + */ +public interface ICommitter extends Serializable { + /** + * begin to commit batchId's data, then return the commit result The + * commitResult will store into outside storage + * + * if failed to commit, please throw FailedException + * + * + * + * @param id + */ + byte[] commit(BatchId id) throws FailedException; + + /** + * begin to revert batchId's data + * + * If current task fails to commit batchId, it won't call revert(batchId) If + * current task fails to revert batchId, JStorm won't call revert again. + * + * @param id + */ + void revert(BatchId id, byte[] commitResult); +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/batch/IPostCommit.java b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/IPostCommit.java new file mode 100755 index 000000000..375bf1921 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/IPostCommit.java @@ -0,0 +1,29 @@ +/** + * 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.batch; + +import backtype.storm.topology.BasicOutputCollector; + +public interface IPostCommit { + /** + * Do after commit Don't care failure of postCommit + * + * @param id + */ + void postCommit(BatchId id, BasicOutputCollector collector); +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/batch/IPrepareCommit.java b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/IPrepareCommit.java new file mode 100755 index 000000000..aa75f9e12 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/IPrepareCommit.java @@ -0,0 +1,38 @@ +/** + * 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.batch; + +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.FailedException; + +/** + * Called before commit, after finish batch + * + * @author zhongyan.feng + */ +public interface IPrepareCommit { + + /** + * Do prepare before commit + * + * @param id + * @param collector + */ + void prepareCommit(BatchId id, BasicOutputCollector collector) + throws FailedException; +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/batch/impl/BatchSpoutMsgId.java b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/impl/BatchSpoutMsgId.java new file mode 100755 index 000000000..99b191591 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/impl/BatchSpoutMsgId.java @@ -0,0 +1,65 @@ +/** + * 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.batch.impl; + +import java.io.Serializable; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; + +import com.alibaba.jstorm.batch.BatchId; +import com.alibaba.jstorm.batch.util.BatchStatus; + +public class BatchSpoutMsgId implements Serializable { + + /** */ + private static final long serialVersionUID = 2899009971479957517L; + + private final BatchId batchId; + private BatchStatus batchStatus; + + protected BatchSpoutMsgId(BatchId batchId, BatchStatus batchStatus) { + this.batchId = batchId; + this.batchStatus = batchStatus; + } + + public static BatchSpoutMsgId mkInstance() { + BatchId batchId = BatchId.mkInstance(); + BatchStatus batchStatus = BatchStatus.COMPUTING; + + return new BatchSpoutMsgId(batchId, batchStatus); + } + + public BatchStatus getBatchStatus() { + return batchStatus; + } + + public void setBatchStatus(BatchStatus batchStatus) { + this.batchStatus = batchStatus; + } + + public BatchId getBatchId() { + return batchId; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/batch/impl/BatchSpoutTrigger.java b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/impl/BatchSpoutTrigger.java new file mode 100755 index 000000000..c1cdae40a --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/impl/BatchSpoutTrigger.java @@ -0,0 +1,333 @@ +/** + * 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.batch.impl; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.IRichSpout; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; + +import com.alibaba.jstorm.batch.BatchId; +import com.alibaba.jstorm.batch.util.BatchCommon; +import com.alibaba.jstorm.batch.util.BatchDef; +import com.alibaba.jstorm.batch.util.BatchStatus; +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.ClusterState; +import com.alibaba.jstorm.utils.IntervalCheck; +import com.alibaba.jstorm.utils.JStormUtils; + +/** + * Strong Sequence + * + * @author zhongyan.feng + * @version + */ +public class BatchSpoutTrigger implements IRichSpout { + /** */ + private static final long serialVersionUID = 7215109169247425954L; + + private static final Logger LOG = LoggerFactory + .getLogger(BatchSpoutTrigger.class); + + private LinkedBlockingQueue batchQueue; + + private transient ClusterState zkClient; + + private transient SpoutOutputCollector collector; + + private static final String ZK_NODE_PATH = "/trigger"; + + private static BatchId currentBatchId = null; + + private Map conf; + + private String taskName; + + private IntervalCheck intervalCheck; + + /** + * @throws Exception + * + */ + public void initMsgId() throws Exception { + Long zkMsgId = null; + byte[] data = zkClient.get_data(ZK_NODE_PATH, false); + if (data != null) { + String value = new String(data); + try { + zkMsgId = Long.valueOf(value); + LOG.info("ZK msgId:" + zkMsgId); + } catch (Exception e) { + LOG.warn("Failed to get msgId ", e); + + } + + } + + if (zkMsgId != null) { + BatchId.updateId(zkMsgId); + } + + int max_spout_pending = + JStormUtils.parseInt( + conf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING), 1); + + for (int i = 0; i < max_spout_pending; i++) { + BatchSpoutMsgId msgId = BatchSpoutMsgId.mkInstance(); + if (currentBatchId == null) { + currentBatchId = msgId.getBatchId(); + } + batchQueue.offer(msgId); + LOG.info("Push into queue," + msgId); + } + + } + + @Override + public void open(Map conf, TopologyContext context, + SpoutOutputCollector collector) { + batchQueue = new LinkedBlockingQueue(); + this.collector = collector; + this.conf = conf; + taskName = context.getThisComponentId() + "_" + context.getThisTaskId(); + + intervalCheck = new IntervalCheck(); + + try { + zkClient = BatchCommon.getZkClient(conf); + + initMsgId(); + + } catch (Exception e) { + LOG.error("", e); + throw new RuntimeException("Failed to init"); + } + LOG.info("Successfully open " + taskName); + } + + @Override + public void close() { + zkClient.close(); + } + + @Override + public void activate() { + LOG.info("Activate " + taskName); + } + + @Override + public void deactivate() { + LOG.info("Deactivate " + taskName); + } + + protected String getStreamId(BatchStatus batchStatus) { + if (batchStatus == BatchStatus.COMPUTING) { + return BatchDef.COMPUTING_STREAM_ID; + } else if (batchStatus == BatchStatus.PREPARE_COMMIT) { + return BatchDef.PREPARE_STREAM_ID; + } else if (batchStatus == BatchStatus.COMMIT) { + return BatchDef.COMMIT_STREAM_ID; + } else if (batchStatus == BatchStatus.POST_COMMIT) { + return BatchDef.POST_STREAM_ID; + } else if (batchStatus == BatchStatus.REVERT_COMMIT) { + return BatchDef.REVERT_STREAM_ID; + } else { + LOG.error("Occur unkonw type BatchStatus " + batchStatus); + throw new RuntimeException(); + } + } + + protected boolean isCommitStatus(BatchStatus batchStatus) { + if (batchStatus == BatchStatus.COMMIT) { + return true; + } else if (batchStatus == BatchStatus.REVERT_COMMIT) { + return true; + } else { + return false; + } + } + + protected boolean isCommitWait(BatchSpoutMsgId msgId) { + + if (isCommitStatus(msgId.getBatchStatus()) == false) { + return false; + } + + // left status is commit status + if (currentBatchId.getId() >= msgId.getBatchId().getId()) { + return false; + } + + return true; + } + + @Override + public void nextTuple() { + BatchSpoutMsgId msgId = null; + try { + msgId = batchQueue.poll(10, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + LOG.error("", e); + } + if (msgId == null) { + return; + } + + if (isCommitWait(msgId)) { + + batchQueue.offer(msgId); + if (intervalCheck.check()) { + LOG.info("Current msgId " + msgId + + ", but current commit BatchId is " + currentBatchId); + } else { + LOG.debug("Current msgId " + msgId + + ", but current commit BatchId is " + currentBatchId); + } + + return; + } + + String streamId = getStreamId(msgId.getBatchStatus()); + List outTasks = + collector.emit(streamId, new Values(msgId.getBatchId()), msgId); + if (outTasks.isEmpty()) { + forward(msgId); + } + return; + + } + + protected void mkMsgId(BatchSpoutMsgId oldMsgId) { + synchronized (BatchSpoutMsgId.class) { + if (currentBatchId.getId() <= oldMsgId.getBatchId().getId()) { + // this is normal case + + byte[] data = String.valueOf(currentBatchId.getId()).getBytes(); + try { + zkClient.set_data(ZK_NODE_PATH, data); + } catch (Exception e) { + LOG.error("Failed to update to ZK " + oldMsgId, e); + } + + currentBatchId = BatchId.incBatchId(oldMsgId.getBatchId()); + + } else { + // bigger batchId has been failed, when old msgId finish + // it will go here + + } + + } + + BatchSpoutMsgId newMsgId = BatchSpoutMsgId.mkInstance(); + batchQueue.offer(newMsgId); + StringBuilder sb = new StringBuilder(); + sb.append("Create new BatchId,"); + sb.append("old:").append(oldMsgId); + sb.append("new:").append(newMsgId); + sb.append("currentBatchId:").append(currentBatchId); + LOG.info(sb.toString()); + } + + protected void forward(BatchSpoutMsgId msgId) { + BatchStatus status = msgId.getBatchStatus(); + + BatchStatus newStatus = status.forward(); + if (newStatus == null) { + // create new status + mkMsgId(msgId); + LOG.info("Finish old batch " + msgId); + + } else { + msgId.setBatchStatus(newStatus); + batchQueue.offer(msgId); + LOG.info("Forward batch " + msgId); + } + } + + @Override + public void ack(Object msgId) { + if (msgId instanceof BatchSpoutMsgId) { + forward((BatchSpoutMsgId) msgId); + return; + } else { + LOG.warn("Unknown type msgId " + msgId.getClass().getName() + ":" + + msgId); + return; + } + } + + protected void handleFail(BatchSpoutMsgId msgId) { + LOG.info("Failed batch " + msgId); + BatchStatus status = msgId.getBatchStatus(); + + BatchStatus newStatus = status.error(); + if (newStatus == BatchStatus.ERROR) { + // create new status + mkMsgId(msgId); + + } else { + + msgId.setBatchStatus(newStatus); + batchQueue.offer(msgId); + + } + } + + @Override + public void fail(Object msgId) { + if (msgId instanceof BatchSpoutMsgId) { + handleFail((BatchSpoutMsgId) msgId); + } else { + LOG.warn("Unknown type msgId " + msgId.getClass().getName() + ":" + + msgId); + return; + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declareStream(BatchDef.COMPUTING_STREAM_ID, new Fields( + "BatchId")); + declarer.declareStream(BatchDef.PREPARE_STREAM_ID, + new Fields("BatchId")); + declarer.declareStream(BatchDef.COMMIT_STREAM_ID, new Fields("BatchId")); + declarer.declareStream(BatchDef.REVERT_STREAM_ID, new Fields("BatchId")); + declarer.declareStream(BatchDef.POST_STREAM_ID, new Fields("BatchId")); + } + + @Override + public Map getComponentConfiguration() { + Map map = new HashMap(); + ConfigExtension.setSpoutSingleThread(map, true); + return map; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/batch/impl/CoordinatedBolt.java b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/impl/CoordinatedBolt.java new file mode 100755 index 000000000..c9bf0b595 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/impl/CoordinatedBolt.java @@ -0,0 +1,300 @@ +/** + * 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.batch.impl; + +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.FailedException; +import backtype.storm.topology.IBasicBolt; +import backtype.storm.topology.IRichBolt; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.ReportedFailedException; +import backtype.storm.tuple.Tuple; +import backtype.storm.utils.TimeCacheMap; + +import com.alibaba.jstorm.batch.BatchId; +import com.alibaba.jstorm.batch.ICommitter; +import com.alibaba.jstorm.batch.IPostCommit; +import com.alibaba.jstorm.batch.IPrepareCommit; +import com.alibaba.jstorm.batch.util.BatchCommon; +import com.alibaba.jstorm.batch.util.BatchDef; +import com.alibaba.jstorm.batch.util.BatchStatus; +import com.alibaba.jstorm.cluster.ClusterState; + +public class CoordinatedBolt implements IRichBolt { + private static final long serialVersionUID = 5720810158625748046L; + + public static Logger LOG = LoggerFactory.getLogger(CoordinatedBolt.class); + + private IBasicBolt delegate; + private BasicOutputCollector basicCollector; + private OutputCollector collector; + + private String taskId; + private String taskName; + + private boolean isCommiter = false; + private String zkCommitPath; + private TimeCacheMap commited; + + public CoordinatedBolt(IBasicBolt delegate) { + + this.delegate = delegate; + + } + + // use static variable to reduce zk connection + private static ClusterState zkClient = null; + + public void mkCommitDir(Map conf) { + + try { + zkClient = BatchCommon.getZkClient(conf); + + zkCommitPath = + BatchDef.ZK_COMMIT_DIR + BatchDef.ZK_SEPERATOR + taskId; + if (zkClient.node_existed(zkCommitPath, false)) { + zkClient.delete_node(zkCommitPath); + } + zkClient.mkdirs(zkCommitPath); + + LOG.info(taskName + " successfully create commit path" + + zkCommitPath); + } catch (Exception e) { + LOG.error("Failed to create zk node", e); + throw new RuntimeException(); + } + } + + public void prepare(Map conf, TopologyContext context, + OutputCollector collector) { + + taskId = String.valueOf(context.getThisTaskId()); + taskName = context.getThisComponentId() + "_" + context.getThisTaskId(); + + this.basicCollector = new BasicOutputCollector(collector); + this.collector = collector; + + if (delegate instanceof ICommitter) { + isCommiter = true; + commited = + new TimeCacheMap( + context.maxTopologyMessageTimeout()); + mkCommitDir(conf); + } + + delegate.prepare(conf, context); + + } + + public void removeUseless(String path, int reserveSize) throws Exception { + List childs = zkClient.get_children(path, false); + Collections.sort(childs, new Comparator() { + + @Override + public int compare(String o1, String o2) { + try { + Long v1 = Long.valueOf(o1); + Long v2 = Long.valueOf(o2); + return v1.compareTo(v2); + } catch (Exception e) { + return o1.compareTo(o2); + } + + } + + }); + + for (int index = 0; index < childs.size() - reserveSize; index++) { + zkClient.delete_node(path + BatchDef.ZK_SEPERATOR + + childs.get(index)); + } + } + + public String getCommitPath(BatchId id) { + return zkCommitPath + BatchDef.ZK_SEPERATOR + id.getId(); + } + + public void updateToZk(Object id, byte[] commitResult) { + try { + + removeUseless(zkCommitPath, BatchDef.ZK_COMMIT_RESERVER_NUM); + + String path = getCommitPath((BatchId) id); + byte[] data = commitResult; + if (data == null) { + data = new byte[0]; + } + zkClient.set_data(path, data); + LOG.info("Update " + path + " to zk"); + } catch (Exception e) { + LOG.warn("Failed to update to zk,", e); + + } + + } + + public byte[] getCommittedData(Object id) { + try { + String path = getCommitPath((BatchId) id); + byte[] data = zkClient.get_data(path, false); + + return data; + } catch (Exception e) { + LOG.error("Failed to visit ZK,", e); + return null; + } + } + + public void handleRegular(Tuple tuple) { + basicCollector.setContext(tuple); + try { + delegate.execute(tuple, basicCollector); + collector.ack(tuple); + } catch (FailedException e) { + if (e instanceof ReportedFailedException) { + collector.reportError(e); + } + collector.fail(tuple); + } + + } + + public void handlePrepareCommit(Tuple tuple) { + basicCollector.setContext(tuple); + try { + BatchId id = (BatchId) tuple.getValue(0); + ((IPrepareCommit) delegate).prepareCommit(id, basicCollector); + collector.ack(tuple); + } catch (FailedException e) { + if (e instanceof ReportedFailedException) { + collector.reportError(e); + } + collector.fail(tuple); + } + + } + + public void handleCommit(Tuple tuple) { + Object id = tuple.getValue(0); + try { + byte[] commitResult = ((ICommitter) delegate).commit((BatchId) id); + + collector.ack(tuple); + + updateToZk(id, commitResult); + commited.put(id, commitResult); + } catch (Exception e) { + LOG.error("Failed to commit ", e); + collector.fail(tuple); + } + } + + public void handleRevert(Tuple tuple) { + try { + Object id = tuple.getValue(0); + byte[] commitResult = null; + + if (commited.containsKey(id)) { + commitResult = (byte[]) commited.get(id); + } else { + commitResult = getCommittedData(id); + } + + if (commitResult != null) { + ((ICommitter) delegate).revert((BatchId) id, commitResult); + } + } catch (Exception e) { + LOG.error("Failed to revert,", e); + } + + collector.ack(tuple); + } + + public void handlePostCommit(Tuple tuple) { + + basicCollector.setContext(tuple); + try { + BatchId id = (BatchId) tuple.getValue(0); + ((IPostCommit) delegate).postCommit(id, basicCollector); + + } catch (Exception e) { + LOG.info("Failed to do postCommit,", e); + } + collector.ack(tuple); + } + + public void execute(Tuple tuple) { + + BatchStatus batchStatus = getBatchStatus(tuple); + + if (batchStatus == BatchStatus.COMPUTING) { + handleRegular(tuple); + } else if (batchStatus == BatchStatus.PREPARE_COMMIT) { + handlePrepareCommit(tuple); + } else if (batchStatus == BatchStatus.COMMIT) { + handleCommit(tuple); + } else if (batchStatus == BatchStatus.REVERT_COMMIT) { + handleRevert(tuple); + } else if (batchStatus == BatchStatus.POST_COMMIT) { + handlePostCommit(tuple); + } else { + throw new RuntimeException( + "Receive commit tuple, but not committer"); + } + } + + public void cleanup() { + delegate.cleanup(); + } + + public void declareOutputFields(OutputFieldsDeclarer declarer) { + delegate.declareOutputFields(declarer); + } + + @Override + public Map getComponentConfiguration() { + return delegate.getComponentConfiguration(); + } + + private BatchStatus getBatchStatus(Tuple tuple) { + String streamId = tuple.getSourceStreamId(); + + if (streamId.equals(BatchDef.PREPARE_STREAM_ID)) { + return BatchStatus.PREPARE_COMMIT; + } else if (streamId.equals(BatchDef.COMMIT_STREAM_ID)) { + return BatchStatus.COMMIT; + } else if (streamId.equals(BatchDef.REVERT_STREAM_ID)) { + return BatchStatus.REVERT_COMMIT; + } else if (streamId.equals(BatchDef.POST_STREAM_ID)) { + return BatchStatus.POST_COMMIT; + } else { + return BatchStatus.COMPUTING; + } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/batch/util/BatchCommon.java b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/util/BatchCommon.java new file mode 100755 index 000000000..fcc54faed --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/util/BatchCommon.java @@ -0,0 +1,89 @@ +/** + * 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.batch.util; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; + +import com.alibaba.jstorm.cluster.ClusterState; +import com.alibaba.jstorm.cluster.DistributedClusterState; +import com.alibaba.jstorm.utils.JStormUtils; + +public class BatchCommon { + private static final Logger LOG = LoggerFactory + .getLogger(BatchCommon.class); + + private static ClusterState zkClient = null; + + public static ClusterState getZkClient(Map conf) throws Exception { + synchronized (BatchCommon.class) { + if (zkClient != null) { + return zkClient; + } + + List zkServers = null; + if (conf.get(Config.TRANSACTIONAL_ZOOKEEPER_SERVERS) != null) { + zkServers = + (List) conf + .get(Config.TRANSACTIONAL_ZOOKEEPER_SERVERS); + } else if (conf.get(Config.STORM_ZOOKEEPER_SERVERS) != null) { + zkServers = + (List) conf.get(Config.STORM_ZOOKEEPER_SERVERS); + } else { + throw new RuntimeException("No setting zk"); + } + + int port = 2181; + if (conf.get(Config.TRANSACTIONAL_ZOOKEEPER_PORT) != null) { + port = + JStormUtils.parseInt( + conf.get(Config.TRANSACTIONAL_ZOOKEEPER_PORT), + 2181); + } else if (conf.get(Config.STORM_ZOOKEEPER_PORT) != null) { + port = + JStormUtils.parseInt( + conf.get(Config.STORM_ZOOKEEPER_PORT), 2181); + } + + String root = BatchDef.BATCH_ZK_ROOT; + if (conf.get(Config.TRANSACTIONAL_ZOOKEEPER_ROOT) != null) { + root = (String) conf.get(Config.TRANSACTIONAL_ZOOKEEPER_ROOT); + } + + root = + root + BatchDef.ZK_SEPERATOR + + conf.get(Config.TOPOLOGY_NAME); + + Map tmpConf = new HashMap(); + tmpConf.putAll(conf); + tmpConf.put(Config.STORM_ZOOKEEPER_SERVERS, zkServers); + tmpConf.put(Config.STORM_ZOOKEEPER_ROOT, root); + zkClient = new DistributedClusterState(tmpConf); + + LOG.info("Successfully connect ZK"); + return zkClient; + } + + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/batch/util/BatchDef.java b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/util/BatchDef.java new file mode 100755 index 000000000..89a6c36c3 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/util/BatchDef.java @@ -0,0 +1,41 @@ +/** + * 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.batch.util; + +public class BatchDef { + public static final String COMPUTING_STREAM_ID = "batch/compute-stream"; + + public static final String PREPARE_STREAM_ID = "batch/parepare-stream"; + + public static final String COMMIT_STREAM_ID = "batch/commit-stream"; + + public static final String REVERT_STREAM_ID = "batch/revert-stream"; + + public static final String POST_STREAM_ID = "batch/post-stream"; + + public static final String SPOUT_TRIGGER = "spout_trigger"; + + public static final String BATCH_ZK_ROOT = "batch"; + + public static final String ZK_COMMIT_DIR = "/commit"; + + public static final int ZK_COMMIT_RESERVER_NUM = 3; + + public static final String ZK_SEPERATOR = "/"; + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/batch/util/BatchStatus.java b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/util/BatchStatus.java new file mode 100755 index 000000000..77ea2dae7 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/batch/util/BatchStatus.java @@ -0,0 +1,43 @@ +/** + * 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.batch.util; + +public enum BatchStatus { + COMPUTING, PREPARE_COMMIT, COMMIT, REVERT_COMMIT, POST_COMMIT, ERROR; + + public BatchStatus forward() { + if (this == COMPUTING) { + return PREPARE_COMMIT; + } else if (this == PREPARE_COMMIT) { + return COMMIT; + } else if (this == COMMIT) { + return POST_COMMIT; + } else { + return null; + } + } + + public BatchStatus error() { + if (this == COMMIT) { + return REVERT_COMMIT; + } else { + return ERROR; + } + } + +}; diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/cache/JStormCache.java b/jstorm-core/src/main/java/com/alibaba/jstorm/cache/JStormCache.java new file mode 100755 index 000000000..a5a6835a8 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/cache/JStormCache.java @@ -0,0 +1,49 @@ +/** + * 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.cache; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Map; + +import com.alibaba.jstorm.client.ConfigExtension; + + + +public interface JStormCache extends Serializable { + public static final String TAG_TIMEOUT_LIST = ConfigExtension.CACHE_TIMEOUT_LIST; + + void init(Map conf)throws Exception; + void cleanup(); + + Object get(String key) ; + + void getBatch(Map map); + + void remove(String key); + + void removeBatch(Collection keys); + + void put(String key, Object value, int timeoutSecond); + + void put(String key, Object value); + + void putBatch(Map map) ; + + void putBatch(Map map, int timeoutSeconds); +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/cache/RocksDBCache.java b/jstorm-core/src/main/java/com/alibaba/jstorm/cache/RocksDBCache.java new file mode 100755 index 000000000..e72e3d0a3 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/cache/RocksDBCache.java @@ -0,0 +1,311 @@ +/** + * 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.cache; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.apache.commons.lang.StringUtils; +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.Options; +import org.rocksdb.RocksDB; +import org.rocksdb.WriteBatch; +import org.rocksdb.WriteOptions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.PathUtils; + +public class RocksDBCache implements JStormCache { + private static final long serialVersionUID = 705938812240167583L; + private static Logger LOG = LoggerFactory.getLogger(RocksDBCache.class); + + static { + RocksDB.loadLibrary(); + } + + public static final String ROCKSDB_ROOT_DIR = "rocksdb.root.dir"; + public static final String ROCKSDB_RESET = "rocksdb.reset"; + protected RocksDB db; + protected String rootDir; + + public void initDir(Map conf) { + String confDir = (String) conf.get(ROCKSDB_ROOT_DIR); + if (StringUtils.isBlank(confDir) == true) { + throw new RuntimeException("Doesn't set rootDir of rocksDB"); + } + + boolean clean = ConfigExtension.getNimbusCacheReset(conf); + LOG.info("RocksDB reset is " + clean); + if (clean == true) { + try { + PathUtils.rmr(confDir); + } catch (IOException e) { + // TODO Auto-generated catch block + throw new RuntimeException("Failed to cleanup rooDir of rocksDB " + confDir); + } + } + + File file = new File(confDir); + if (file.exists() == false) { + try { + PathUtils.local_mkdirs(confDir); + file = new File(confDir); + } catch (IOException e) { + // TODO Auto-generated catch block + throw new RuntimeException("Failed to mkdir rooDir of rocksDB " + confDir); + } + } + + rootDir = file.getAbsolutePath(); + } + + public void initDb(List list) throws Exception { + LOG.info("Begin to init rocksDB of {}", rootDir); + + Options dbOptions = null; + + try { + dbOptions = new Options().setCreateMissingColumnFamilies(true).setCreateIfMissing(true); + + List columnFamilyHandleList = new ArrayList(); + + db = RocksDB.open(dbOptions, rootDir); + + LOG.info("Successfully init rocksDB of {}", rootDir); + } finally { + + if (dbOptions != null) { + dbOptions.dispose(); + } + } + } + + @Override + public void init(Map conf) throws Exception { + // TODO Auto-generated method stub + initDir(conf); + + List list = new ArrayList(); + if (conf.get(TAG_TIMEOUT_LIST) != null) { + for (Object obj : (List) ConfigExtension.getCacheTimeoutList(conf)) { + Integer timeoutSecond = JStormUtils.parseInt(obj); + if (timeoutSecond == null || timeoutSecond <= 0) { + continue; + } + + list.add(timeoutSecond); + } + } + + // Add retry logic + boolean isSuccess = false; + for (int i = 0; i < 3; i++) { + try { + initDb(list); + isSuccess = true; + break; + } catch (Exception e) { + LOG.warn("Failed to init rocksDB " + rootDir, e); + try { + PathUtils.rmr(rootDir); + } catch (IOException e1) { + // TODO Auto-generated catch block + + } + } + } + + if (isSuccess == false) { + throw new RuntimeException("Failed to init rocksDB " + rootDir); + } + } + + @Override + public void cleanup() { + LOG.info("Begin to close rocketDb of {}", rootDir); + + if (db != null) { + db.close(); + } + + LOG.info("Successfully closed rocketDb of {}", rootDir); + } + + @Override + public Object get(String key) { + // TODO Auto-generated method stub + try { + byte[] data = db.get(key.getBytes()); + if (data != null) { + try { + return Utils.javaDeserialize(data); + } catch (Exception e) { + LOG.error("Failed to deserialize obj of " + key); + db.remove(key.getBytes()); + return null; + } + } + + } catch (Exception e) { + + } + + return null; + } + + @Override + public void getBatch(Map map) { + List lookupKeys = new ArrayList(); + for (String key : map.keySet()) { + lookupKeys.add(key.getBytes()); + } + + try { + Map results = db.multiGet(lookupKeys); + if (results == null || results.size() == 0) { + return; + } + + for (Entry resultEntry : results.entrySet()) { + byte[] keyByte = resultEntry.getKey(); + byte[] valueByte = resultEntry.getValue(); + + if (keyByte == null || valueByte == null) { + continue; + } + + Object value = null; + try { + value = Utils.javaDeserialize(valueByte); + } catch (Exception e) { + LOG.error("Failed to deserialize obj of " + new String(keyByte)); + db.remove(keyByte); + continue; + } + + map.put(new String(keyByte), value); + } + + return; + } catch (Exception e) { + LOG.error("Failed to query " + map.keySet() + ", in window: "); + } + + return; + } + + @Override + public void remove(String key) { + try { + db.remove(key.getBytes()); + + } catch (Exception e) { + LOG.error("Failed to remove " + key); + } + + } + + @Override + public void removeBatch(Collection keys) { + // TODO Auto-generated method stub + for (String key : keys) { + remove(key); + } + } + + @Override + public void put(String key, Object value, int timeoutSecond) { + // TODO Auto-generated method stub + + put(key, value); + + } + + @Override + public void put(String key, Object value) { + byte[] data = Utils.javaSerialize(value); + try { + db.put(key.getBytes(), data); + } catch (Exception e) { + LOG.error("Failed put into cache, " + key, e); + return; + } + } + + @Override + public void putBatch(Map map) { + // TODO Auto-generated method stub + WriteOptions writeOpts = null; + WriteBatch writeBatch = null; + + Set putKeys = new HashSet(); + + try { + writeOpts = new WriteOptions(); + writeBatch = new WriteBatch(); + + for (Entry entry : map.entrySet()) { + String key = entry.getKey(); + Object value = entry.getValue(); + + byte[] data = Utils.javaSerialize(value); + + if (StringUtils.isBlank(key) || data == null || data.length == 0) { + continue; + } + + byte[] keyByte = key.getBytes(); + writeBatch.put(keyByte, data); + + putKeys.add(keyByte); + } + + db.write(writeOpts, writeBatch); + } catch (Exception e) { + LOG.error("Failed to putBatch into DB, " + map.keySet(), e); + } finally { + if (writeOpts != null) { + writeOpts.dispose(); + } + + if (writeBatch != null) { + writeBatch.dispose(); + } + } + + } + + @Override + public void putBatch(Map map, int timeoutSeconds) { + // TODO Auto-generated method stub + putBatch(map); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/cache/RocksTTLDBCache.java b/jstorm-core/src/main/java/com/alibaba/jstorm/cache/RocksTTLDBCache.java new file mode 100755 index 000000000..20b5f468e --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/cache/RocksTTLDBCache.java @@ -0,0 +1,469 @@ +/** + * 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.cache; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.TreeMap; + +import org.apache.commons.lang.StringUtils; +import org.rocksdb.ColumnFamilyDescriptor; +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.DBOptions; +import org.rocksdb.RocksDB; +import org.rocksdb.TtlDB; +import org.rocksdb.WriteBatch; +import org.rocksdb.WriteOptions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.PathUtils; + +public class RocksTTLDBCache implements JStormCache { + private static final long serialVersionUID = 705938812240167583L; + private static Logger LOG = LoggerFactory.getLogger(RocksTTLDBCache.class); + + static { + RocksDB.loadLibrary(); + } + + public static final String ROCKSDB_ROOT_DIR = "rocksdb.root.dir"; + public static final String ROCKSDB_RESET = "rocksdb.reset"; + protected TtlDB ttlDB; + protected String rootDir; + protected TreeMap windowHandlers = new TreeMap(); + + public void initDir(Map conf) { + String confDir = (String) conf.get(ROCKSDB_ROOT_DIR); + if (StringUtils.isBlank(confDir) == true) { + throw new RuntimeException("Doesn't set rootDir of rocksDB"); + } + + boolean clean = ConfigExtension.getNimbusCacheReset(conf); + LOG.info("RocksDB reset is " + clean); + if (clean == true) { + try { + PathUtils.rmr(confDir); + } catch (IOException e) { + // TODO Auto-generated catch block + throw new RuntimeException("Failed to cleanup rooDir of rocksDB " + confDir); + } + } + + File file = new File(confDir); + if (file.exists() == false) { + try { + PathUtils.local_mkdirs(confDir); + file = new File(confDir); + } catch (IOException e) { + // TODO Auto-generated catch block + throw new RuntimeException("Failed to mkdir rooDir of rocksDB " + confDir); + } + } + + rootDir = file.getAbsolutePath(); + } + + public void initDb(List list) throws Exception{ + LOG.info("Begin to init rocksDB of {}", rootDir); + + DBOptions dbOptions = null; + + List columnFamilyNames = new ArrayList(); + columnFamilyNames.add(new ColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY)); + for (Integer timeout : list) { + columnFamilyNames.add(new ColumnFamilyDescriptor(String.valueOf(timeout).getBytes())); + } + + List ttlValues = new ArrayList(); + // Default column family with infinite lifetime + // ATTENSION, the first must be 0, RocksDB.java API has this limitation + ttlValues.add(0); + // new column family with list second ttl + ttlValues.addAll(list); + + try { + dbOptions = new DBOptions().setCreateMissingColumnFamilies(true).setCreateIfMissing(true); + + List columnFamilyHandleList = new ArrayList(); + + ttlDB = TtlDB.open(dbOptions, rootDir, columnFamilyNames, columnFamilyHandleList, ttlValues, false); + + for (int i = 0; i < ttlValues.size(); i++) { + windowHandlers.put(ttlValues.get(i), columnFamilyHandleList.get(i)); + } + + LOG.info("Successfully init rocksDB of {}", rootDir); + } finally { + + if (dbOptions != null) { + dbOptions.dispose(); + } + } + } + + @Override + public void init(Map conf) throws Exception{ + // TODO Auto-generated method stub + initDir(conf); + + List list = new ArrayList(); + if (conf.get(TAG_TIMEOUT_LIST) != null) { + for (Object obj : (List) ConfigExtension.getCacheTimeoutList(conf)) { + Integer timeoutSecond = JStormUtils.parseInt(obj); + if (timeoutSecond == null || timeoutSecond <= 0) { + continue; + } + + list.add(timeoutSecond); + } + } + + // Add retry logic + boolean isSuccess = false; + for (int i = 0; i < 3; i++) { + try { + initDb(list); + isSuccess = true; + break; + }catch(Exception e) { + LOG.warn("Failed to init rocksDB " + rootDir, e); + try { + PathUtils.rmr(rootDir); + } catch (IOException e1) { + // TODO Auto-generated catch block + + } + } + } + + if (isSuccess == false) { + throw new RuntimeException("Failed to init rocksDB " + rootDir); + } + } + + @Override + public void cleanup() { + LOG.info("Begin to close rocketDb of {}", rootDir); + + for (ColumnFamilyHandle columnFamilyHandle : windowHandlers.values()) { + columnFamilyHandle.dispose(); + } + + if (ttlDB != null) { + ttlDB.close(); + } + + LOG.info("Successfully closed rocketDb of {}", rootDir); + } + + @Override + public Object get(String key) { + // TODO Auto-generated method stub + for (Entry entry : windowHandlers.entrySet()) { + try { + byte[] data = ttlDB.get(entry.getValue(), + key.getBytes()); + if (data != null) { + try { + return Utils.javaDeserialize(data); + }catch(Exception e) { + LOG.error("Failed to deserialize obj of " + key); + ttlDB.remove(entry.getValue(), + key.getBytes()); + return null; + } + } + + + }catch(Exception e) { + + } + } + + return null; + } + + @Override + public void getBatch(Map map) { + List lookupKeys = new ArrayList(); + for (String key : map.keySet()) { + lookupKeys.add(key.getBytes()); + } + for (Entry entry : windowHandlers.entrySet()) { + + List cfHandlers = new ArrayList(); + for (String key : map.keySet()) { + cfHandlers.add(entry.getValue()); + } + + try { + Map results = ttlDB.multiGet(cfHandlers, lookupKeys); + if (results == null || results.size() == 0) { + continue; + } + + for (Entry resultEntry : results.entrySet()) { + byte[] keyByte = resultEntry.getKey(); + byte[] valueByte = resultEntry.getValue(); + + if (keyByte == null || valueByte == null) { + continue; + } + + Object value = null; + try { + value = Utils.javaDeserialize(valueByte); + } catch (Exception e) { + LOG.error("Failed to deserialize obj of " + new String(keyByte)); + ttlDB.remove(entry.getValue(), keyByte); + continue; + } + + map.put(new String(keyByte), value); + } + + return ; + } catch (Exception e) { + LOG.error("Failed to query " + map.keySet() + ", in window: " + entry.getKey()); + } + } + + return; + } + + + @Override + public void remove(String key) { + for (Entry entry : windowHandlers.entrySet()) { + try { + ttlDB.remove(entry.getValue(), + key.getBytes()); + + + + }catch(Exception e) { + LOG.error("Failed to remove " + key); + } + } + } + + @Override + public void removeBatch(Collection keys) { + // TODO Auto-generated method stub + for (String key : keys) { + remove(key); + } + } + + protected void put(String key, Object value, Entry entry) { + + byte[] data = Utils.javaSerialize(value); + try { + ttlDB.put(entry.getValue(), key.getBytes(), data); + }catch(Exception e) { + LOG.error("Failed put into cache, " + key, e); + return ; + } + + for (Entry removeEntry : windowHandlers.entrySet()) { + if (removeEntry.getKey().equals(entry.getKey())) { + continue; + } + + try { + ttlDB.remove(removeEntry.getValue(), key.getBytes()); + } catch (Exception e) { + // TODO Auto-generated catch block + LOG.warn("Failed to remove other " + key); + } + } + } + + protected Entry getHandler(int timeoutSecond) { + ColumnFamilyHandle cfHandler = null; + Entry ceilingEntry = windowHandlers.ceilingEntry(timeoutSecond); + if (ceilingEntry != null) { + return ceilingEntry; + }else { + return windowHandlers.firstEntry(); + } + } + + @Override + public void put(String key, Object value, int timeoutSecond) { + // TODO Auto-generated method stub + + + put(key, value, getHandler(timeoutSecond)); + + } + + @Override + public void put(String key, Object value) { + put(key, value, windowHandlers.firstEntry()); + } + + protected void putBatch(Map map, Entry putEntry ) { + // TODO Auto-generated method stub + WriteOptions writeOpts = null; + WriteBatch writeBatch = null; + + Set putKeys = new HashSet(); + + try { + writeOpts = new WriteOptions(); + writeBatch = new WriteBatch(); + + for (Entry entry : map.entrySet()) { + String key = entry.getKey(); + Object value = entry.getValue(); + + + byte[] data = Utils.javaSerialize(value); + + if (StringUtils.isBlank(key) || data == null || data.length == 0) { + continue; + } + + byte[] keyByte = key.getBytes(); + writeBatch.put(putEntry.getValue(), keyByte, data); + + putKeys.add(keyByte); + } + + ttlDB.write(writeOpts, writeBatch); + }catch(Exception e) { + LOG.error("Failed to putBatch into DB, " + map.keySet(), e); + }finally { + if (writeOpts != null) { + writeOpts.dispose(); + } + + if (writeBatch != null) { + writeBatch.dispose(); + } + } + + for (Entry entry : windowHandlers.entrySet()) { + if (entry.getKey().equals(putEntry.getKey())) { + continue; + } + for (byte[] keyByte : putKeys) { + try { + ttlDB.remove(entry.getValue(), keyByte); + + } catch (Exception e) { + LOG.error("Failed to remove other's " + new String(keyByte)); + } + } + } + } + + @Override + public void putBatch(Map map) { + // TODO Auto-generated method stub + putBatch(map, windowHandlers.firstEntry()); + } + + @Override + public void putBatch(Map map, int timeoutSeconds) { + // TODO Auto-generated method stub + putBatch(map, getHandler(timeoutSeconds)); + } + +// public void put() throws Exception { + +// } +// +// public void write() throws Exception { +// Options options = null; +// WriteBatch wb1 = null; +// WriteBatch wb2 = null; +// WriteOptions opts = null; +// try { +// options = new Options().setMergeOperator(new StringAppendOperator()).setCreateIfMissing(true); +// db = RocksDB.open(options, dbFolder.getRoot().getAbsolutePath()); +// opts = new WriteOptions(); +// wb1 = new WriteBatch(); +// wb1.put("key1".getBytes(), "aa".getBytes()); +// wb1.merge("key1".getBytes(), "bb".getBytes()); +// wb2 = new WriteBatch(); +// wb2.put("key2".getBytes(), "xx".getBytes()); +// wb2.merge("key2".getBytes(), "yy".getBytes()); +// db.write(opts, wb1); +// db.write(opts, wb2); +// assertThat(db.get("key1".getBytes())).isEqualTo("aa,bb".getBytes()); +// assertThat(db.get("key2".getBytes())).isEqualTo("xx,yy".getBytes()); +// } finally { +// if (db != null) { +// db.close(); +// } +// if (wb1 != null) { +// wb1.dispose(); +// } +// if (wb2 != null) { +// wb2.dispose(); +// } +// if (options != null) { +// options.dispose(); +// } +// if (opts != null) { +// opts.dispose(); +// } +// } +// } +// + +// +// public void remove() throws Exception { +// RocksDB db = null; +// WriteOptions wOpt; +// try { +// wOpt = new WriteOptions(); +// db = RocksDB.open(dbFolder.getRoot().getAbsolutePath()); +// db.put("key1".getBytes(), "value".getBytes()); +// db.put("key2".getBytes(), "12345678".getBytes()); +// assertThat(db.get("key1".getBytes())).isEqualTo("value".getBytes()); +// assertThat(db.get("key2".getBytes())).isEqualTo("12345678".getBytes()); +// db.remove("key1".getBytes()); +// db.remove(wOpt, "key2".getBytes()); +// assertThat(db.get("key1".getBytes())).isNull(); +// assertThat(db.get("key2".getBytes())).isNull(); +// } finally { +// if (db != null) { +// db.close(); +// } +// } +// } +// +// public void ttlDbOpenWithColumnFamilies() throws Exception, InterruptedException { +// +// } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/cache/TimeoutMemCache.java b/jstorm-core/src/main/java/com/alibaba/jstorm/cache/TimeoutMemCache.java new file mode 100755 index 000000000..d4d99053d --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/cache/TimeoutMemCache.java @@ -0,0 +1,178 @@ +/** + * 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.cache; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.TreeMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.TimeCacheMap; + +public class TimeoutMemCache implements JStormCache { + private static final long serialVersionUID = 705938812240167583L; + private static Logger LOG = LoggerFactory.getLogger(TimeoutMemCache.class); + + + protected int defaultTimeout; + protected final TreeMap> cacheWindows = new TreeMap>(); + + public TimeoutMemCache() { + + } + + protected void registerCacheWindow(int timeoutSecond) { + synchronized (this) { + if (cacheWindows.get(timeoutSecond) == null) { + TimeCacheMap cacheWindow = new TimeCacheMap(timeoutSecond); + cacheWindows.put(timeoutSecond, cacheWindow); + + LOG.info("Successfully register CacheWindow: " + timeoutSecond); + } else { + LOG.info("CacheWindow: " + timeoutSecond + " has been registered"); + } + } + } + + @Override + public void init(Map conf) { + // TODO Auto-generated method stub + this.defaultTimeout = ConfigExtension.getDefaultCacheTimeout(conf); + registerCacheWindow(defaultTimeout); + + List list = (List) ConfigExtension.getCacheTimeoutList(conf); + if (list != null) { + for (Object obj : list) { + Integer timeoutSecond = JStormUtils.parseInt(obj); + if (timeoutSecond == null) { + continue; + } + + registerCacheWindow(timeoutSecond); + } + } + } + + @Override + public void cleanup() { + // TODO Auto-generated method stub + + } + + @Override + public Object get(String key) { + // TODO Auto-generated method stub + // @@@ TODO + // in order to improve performance, it can be query from defaultWindow firstly, then others + for (TimeCacheMap cacheWindow : cacheWindows.values()) { + Object ret = cacheWindow.get(key); + if (ret != null) { + return ret; + } + } + return null; + } + + @Override + public void getBatch(Map map) { + // TODO Auto-generated method stub + for (String key : map.keySet()) { + Object obj = get(key); + map.put(key, obj); + } + + return; + } + + @Override + public void remove(String key) { + // TODO Auto-generated method stub + for (TimeCacheMap cacheWindow : cacheWindows.values()) { + Object ret = cacheWindow.remove(key); + if (ret != null) { + return; + } + } + } + + @Override + public void removeBatch(Collection keys) { + // TODO Auto-generated method stub + for (String key : keys) { + remove(key); + } + + return; + } + + @Override + public void put(String key, Object value, int timeoutSecond) { + + // TODO Auto-generated method stub + Entry> ceilingEntry = cacheWindows.ceilingEntry(timeoutSecond); + if (ceilingEntry == null) { + put(key, value); + return ; + }else { + remove(key); + ceilingEntry.getValue().put(key, value); + } + + } + + @Override + public void put(String key, Object value) { + remove(key); + TimeCacheMap bestWindow = cacheWindows.get(defaultTimeout); + bestWindow.put(key, value); + } + + @Override + public void putBatch(Map map) { + // TODO Auto-generated method stub + for (Entry entry : map.entrySet()) { + put(entry.getKey(), entry.getValue()); + } + + } + + @Override + public void putBatch(Map map, int timeoutSeconds) { + // TODO Auto-generated method stub + for (Entry entry : map.entrySet()) { + put(entry.getKey(), entry.getValue(), timeoutSeconds); + } + + } + + public int getDefaultTimeout() { + return defaultTimeout; + } + + public void setDefaultTimeout(int defaultTimeout) { + this.defaultTimeout = defaultTimeout; + } + + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/callback/AsyncLoopDefaultKill.java b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/AsyncLoopDefaultKill.java new file mode 100755 index 000000000..805cf8140 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/AsyncLoopDefaultKill.java @@ -0,0 +1,42 @@ +/** + * 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.callback; + +import com.alibaba.jstorm.utils.JStormUtils; + +/** + * Killer callback + * + * @author yannian + * + */ + +public class AsyncLoopDefaultKill extends RunnableCallback { + + @Override + public Object execute(T... args) { + Exception e = (Exception) args[0]; + JStormUtils.halt_process(1, "Async loop died!"); + return e; + } + + @Override + public void run() { + JStormUtils.halt_process(1, "Async loop died!"); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/callback/AsyncLoopRunnable.java b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/AsyncLoopRunnable.java new file mode 100755 index 000000000..d21cc4a57 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/AsyncLoopRunnable.java @@ -0,0 +1,127 @@ +/** + * 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.callback; + +import java.util.concurrent.atomic.AtomicBoolean; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.utils.JStormUtils; + +/** + * AsyncLoopThread 's runnable + * + * The class wrapper RunnableCallback fn, if occur exception, run killfn + * + * @author yannian + * + */ +public class AsyncLoopRunnable implements Runnable { + private static Logger LOG = LoggerFactory + .getLogger(AsyncLoopRunnable.class); + + // set shutdown as false is to + private static AtomicBoolean shutdown = new AtomicBoolean(false); + + public static AtomicBoolean getShutdown() { + return shutdown; + } + + private RunnableCallback fn; + private RunnableCallback killfn; + private long lastTime = System.currentTimeMillis(); + + public AsyncLoopRunnable(RunnableCallback fn, RunnableCallback killfn) { + this.fn = fn; + this.killfn = killfn; + } + + private boolean needQuit(Object rtn) { + if (rtn != null) { + long sleepTime = Long.parseLong(String.valueOf(rtn)); + if (sleepTime < 0) { + return true; + } else if (sleepTime > 0) { + long now = System.currentTimeMillis(); + long cost = now - lastTime; + long sleepMs = sleepTime * 1000 - cost; + if (sleepMs > 0) { + JStormUtils.sleepMs(sleepMs); + lastTime = System.currentTimeMillis(); + } else { + lastTime = now; + } + + } + } + return false; + } + + private void shutdown() { + fn.postRun(); + fn.shutdown(); + LOG.info("Succefully shutdown"); + } + + @Override + public void run() { + + if (fn == null) { + LOG.error("fn==null"); + throw new RuntimeException("AsyncLoopRunnable no core function "); + } + + fn.preRun(); + + try { + while (shutdown.get() == false) { + Exception e = null; + + fn.run(); + + if (shutdown.get() == true) { + shutdown(); + return; + } + + e = fn.error(); + if (e != null) { + throw e; + } + Object rtn = fn.getResult(); + if (this.needQuit(rtn)) { + shutdown(); + return; + } + + } + } catch (Throwable e) { + if (shutdown.get() == true) { + shutdown(); + return; + } else { + LOG.error("Async loop died!!!" + e.getMessage(), e); + killfn.execute(e); + } + + } + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/callback/AsyncLoopThread.java b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/AsyncLoopThread.java new file mode 100755 index 000000000..ce49c5159 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/AsyncLoopThread.java @@ -0,0 +1,140 @@ +/** + * 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.callback; + +import java.lang.Thread.UncaughtExceptionHandler; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.utils.Time; + +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.SmartThread; + +/** + * Wrapper Timer thread Every several seconds execute afn, if something is run, + * run kill_fn + * + * + * @author yannian + * + */ +public class AsyncLoopThread implements SmartThread { + private static final Logger LOG = LoggerFactory + .getLogger(AsyncLoopThread.class); + + private Thread thread; + + private RunnableCallback afn; + + public AsyncLoopThread(RunnableCallback afn) { + this.init(afn, false, Thread.NORM_PRIORITY, true); + } + + public AsyncLoopThread(RunnableCallback afn, boolean daemon, int priority, + boolean start) { + this.init(afn, daemon, priority, start); + } + + public AsyncLoopThread(RunnableCallback afn, boolean daemon, + RunnableCallback kill_fn, int priority, boolean start) { + this.init(afn, daemon, kill_fn, priority, start); + } + + public void init(RunnableCallback afn, boolean daemon, int priority, + boolean start) { + RunnableCallback kill_fn = new AsyncLoopDefaultKill(); + this.init(afn, daemon, kill_fn, priority, start); + } + + /** + * + * @param afn + * @param daemon + * @param kill_fn (Exception e) + * @param priority + * @param args_fn + * @param start + */ + private void init(RunnableCallback afn, boolean daemon, + RunnableCallback kill_fn, int priority, boolean start) { + if (kill_fn == null) { + kill_fn = new AsyncLoopDefaultKill(); + } + + Runnable runable = new AsyncLoopRunnable(afn, kill_fn); + thread = new Thread(runable); + String threadName = afn.getThreadName(); + if (threadName == null) { + threadName = afn.getClass().getSimpleName(); + } + thread.setName(threadName); + thread.setDaemon(daemon); + thread.setPriority(priority); + thread.setUncaughtExceptionHandler(new UncaughtExceptionHandler() { + @Override + public void uncaughtException(Thread t, Throwable e) { + LOG.error("UncaughtException", e); + JStormUtils.halt_process(1, "UncaughtException"); + } + }); + + this.afn = afn; + + if (start) { + thread.start(); + } + + } + + @Override + public void start() { + thread.start(); + } + + @Override + public void join() throws InterruptedException { + thread.join(); + } + + // for test + public void join(int times) throws InterruptedException { + thread.join(times); + } + + @Override + public void interrupt() { + thread.interrupt(); + } + + @Override + public Boolean isSleeping() { + return Time.isThreadWaiting(thread); + } + + public Thread getThread() { + return thread; + } + + @Override + public void cleanup() { + // TODO Auto-generated method stub + afn.shutdown(); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/callback/BaseCallback.java b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/BaseCallback.java new file mode 100755 index 000000000..b1a8df90e --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/BaseCallback.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.callback; + +public class BaseCallback implements Callback { + + @Override + public Object execute(T... args) { + // TODO Auto-generated method stub + return null; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/callback/Callback.java b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/Callback.java new file mode 100755 index 000000000..6675ca8bb --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/Callback.java @@ -0,0 +1,30 @@ +/** + * 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.callback; + +/** + * Callback interface + * + * @author lixin 2012-3-12 + * + */ +public interface Callback { + + public Object execute(T... args); + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/callback/ClusterStateCallback.java b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/ClusterStateCallback.java new file mode 100755 index 000000000..2275e61fb --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/ClusterStateCallback.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.callback; + +public class ClusterStateCallback extends BaseCallback { + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/callback/DefaultWatcherCallBack.java b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/DefaultWatcherCallBack.java new file mode 100755 index 000000000..4f1764b76 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/DefaultWatcherCallBack.java @@ -0,0 +1,45 @@ +/** + * 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.callback; + +import org.apache.zookeeper.Watcher.Event.EventType; +import org.apache.zookeeper.Watcher.Event.KeeperState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.zk.ZkEventTypes; +import com.alibaba.jstorm.zk.ZkKeeperStates; + +/** + * Default ZK watch callback + * + * @author yannian + * + */ +public class DefaultWatcherCallBack implements WatcherCallBack { + + private static Logger LOG = LoggerFactory + .getLogger(DefaultWatcherCallBack.class); + + @Override + public void execute(KeeperState state, EventType type, String path) { + LOG.info("Zookeeper state update:" + ZkKeeperStates.getStateName(state) + + "," + ZkEventTypes.getStateName(type) + "," + path); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/callback/RunnableCallback.java b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/RunnableCallback.java new file mode 100755 index 000000000..d759af6c7 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/RunnableCallback.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.callback; + +import backtype.storm.daemon.Shutdownable; + +/** + * Base Runnable/Callback function + * + * @author yannian + * + */ +public class RunnableCallback implements Runnable, Callback, Shutdownable { + + @Override + public Object execute(T... args) { + return null; + } + + public void preRun() { + + } + + @Override + public void run() { + + } + + public void postRun() { + + } + + public Exception error() { + return null; + } + + public Object getResult() { + return null; + } + + public void shutdown() { + + } + + public String getThreadName() { + return null; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/callback/WatcherCallBack.java b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/WatcherCallBack.java new file mode 100755 index 000000000..b30749dca --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/WatcherCallBack.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.callback; + +import org.apache.zookeeper.Watcher.Event.EventType; +import org.apache.zookeeper.Watcher.Event.KeeperState; + +public interface WatcherCallBack { + public void execute(KeeperState state, EventType type, String path); +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/ActiveTransitionCallback.java b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/ActiveTransitionCallback.java new file mode 100755 index 000000000..67a9913fc --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/ActiveTransitionCallback.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.callback.impl; + +import com.alibaba.jstorm.callback.BaseCallback; +import com.alibaba.jstorm.cluster.StormStatus; +import com.alibaba.jstorm.daemon.nimbus.StatusType; + +/** + * Set the topology status as Active + * + */ +public class ActiveTransitionCallback extends BaseCallback { + + @Override + public Object execute(T... args) { + + return new StormStatus(StatusType.active); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/DelayStatusTransitionCallback.java b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/DelayStatusTransitionCallback.java new file mode 100755 index 000000000..79ef6334b --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/DelayStatusTransitionCallback.java @@ -0,0 +1,111 @@ +/** + * 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.callback.impl; + +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; + +import com.alibaba.jstorm.callback.BaseCallback; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.cluster.StormStatus; +import com.alibaba.jstorm.daemon.nimbus.NimbusData; +import com.alibaba.jstorm.daemon.nimbus.StatusType; +import com.alibaba.jstorm.schedule.DelayEventRunnable; +import com.alibaba.jstorm.utils.JStormUtils; + +/** + * + * + * The action when nimbus receive kill command 1. set the topology status as + * target 2. wait 2 * Timeout seconds later, do removing topology from ZK + * + * @author Longda + */ +public class DelayStatusTransitionCallback extends BaseCallback { + + private static Logger LOG = LoggerFactory + .getLogger(DelayStatusTransitionCallback.class); + + public static final int DEFAULT_DELAY_SECONDS = 30; + + protected NimbusData data; + protected String topologyid; + protected StormStatus oldStatus; + protected StatusType newType; + protected StatusType nextAction; + + public DelayStatusTransitionCallback(NimbusData data, String topologyid, + StormStatus oldStatus, StatusType newType, StatusType nextAction) { + this.data = data; + this.topologyid = topologyid; + this.oldStatus = oldStatus; + this.newType = newType; + this.nextAction = nextAction; + } + + public int getDelaySeconds(Object[] args) { + if (oldStatus != null && oldStatus.getDelaySecs() > 0) { + return oldStatus.getDelaySecs(); + } + + Integer delaySecs = DelayStatusTransitionCallback.DEFAULT_DELAY_SECONDS; + if (args == null || args.length == 0 || args[0] == null) { + Map map = null; + try { + + map = + StormConfig.read_nimbus_topology_conf(data.getConf(), + topologyid); + delaySecs = + JStormUtils.parseInt( + map.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS), + DEFAULT_DELAY_SECONDS); + } catch (Exception e) { + LOG.info("Failed to get topology configuration " + topologyid); + } + + } else { + delaySecs = JStormUtils.parseInt(args[0]); + } + + if (delaySecs == null || delaySecs < 0) { + delaySecs = DelayStatusTransitionCallback.DEFAULT_DELAY_SECONDS; + } + + return delaySecs; + } + + @Override + public Object execute(T... args) { + int delaySecs = getDelaySeconds(args); + LOG.info("Delaying event " + newType + " for " + delaySecs + + " secs for " + topologyid); + + data.getScheduExec().schedule( + new DelayEventRunnable(data, topologyid, nextAction, args), + delaySecs, TimeUnit.SECONDS); + + return new StormStatus(delaySecs, newType); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/DoRebalanceTransitionCallback.java b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/DoRebalanceTransitionCallback.java new file mode 100644 index 000000000..41706b322 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/DoRebalanceTransitionCallback.java @@ -0,0 +1,307 @@ +/** + * 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.callback.impl; + +import java.util.*; +import java.util.Map.Entry; + +import org.apache.log4j.Logger; + +import backtype.storm.Config; +import backtype.storm.generated.Bolt; +import backtype.storm.generated.SpoutSpec; +import backtype.storm.generated.StormTopology; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.callback.BaseCallback; +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.Common; +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.cluster.StormStatus; +import com.alibaba.jstorm.daemon.nimbus.NimbusData; +import com.alibaba.jstorm.daemon.nimbus.NimbusUtils; +import com.alibaba.jstorm.daemon.nimbus.StatusType; +import com.alibaba.jstorm.daemon.nimbus.TopologyAssign; +import com.alibaba.jstorm.daemon.nimbus.TopologyAssignEvent; +import com.alibaba.jstorm.task.TaskInfo; +import com.alibaba.jstorm.task.TkHbCacheTime; +import com.alibaba.jstorm.utils.JStormUtils; + +/** + * Do real rebalance action. + * + * After nimbus receive one rebalance command, it will do as following: 1. set + * topology status as rebalancing 2. delay 2 * timeout seconds 3. do this + * callback + * + * @author Xin.Li/Longda + * + */ +public class DoRebalanceTransitionCallback extends BaseCallback { + + private static Logger LOG = Logger + .getLogger(DoRebalanceTransitionCallback.class); + + private NimbusData data; + private String topologyid; + private StormStatus oldStatus; + private Set newTasks; + + public DoRebalanceTransitionCallback(NimbusData data, String topologyid, + StormStatus status) { + this.data = data; + this.topologyid = topologyid; + this.oldStatus = status; + this.newTasks = new HashSet(); + } + + @Override + public Object execute(T... args) { + boolean isSetTaskInfo = false; + try { + Boolean reassign = (Boolean) args[1]; + Map conf = (Map) args[2]; // args[0]: + // delay, + // args[1]: + // reassign_flag, + // args[2]: + // conf + if (conf != null) { + boolean isConfUpdate = false; + Map stormConf = data.getConf(); + + // Update topology code + Map topoConf = + StormConfig.read_nimbus_topology_conf(stormConf, + topologyid); + StormTopology rawOldTopology = + StormConfig.read_nimbus_topology_code(stormConf, + topologyid); + StormTopology rawNewTopology = + NimbusUtils.normalizeTopology(conf, rawOldTopology, + true); + StormTopology sysOldTopology = rawOldTopology.deepCopy(); + StormTopology sysNewTopology = rawNewTopology.deepCopy(); + if (conf.get(Config.TOPOLOGY_ACKER_EXECUTORS) != null) { + Common.add_acker(topoConf, sysOldTopology); + Common.add_acker(conf, sysNewTopology); + int ackerNum = + JStormUtils.parseInt(conf + .get(Config.TOPOLOGY_ACKER_EXECUTORS)); + int oldAckerNum = + JStormUtils.parseInt(topoConf + .get(Config.TOPOLOGY_ACKER_EXECUTORS)); + LOG.info("Update acker from oldAckerNum=" + oldAckerNum + + " to ackerNum=" + ackerNum); + topoConf.put(Config.TOPOLOGY_ACKER_EXECUTORS, ackerNum); + isConfUpdate = true; + } + + // If scale-out, setup task info for new added tasks + setTaskInfo(sysOldTopology, sysNewTopology); + isSetTaskInfo = true; + + // If everything is OK, write topology code into disk + StormConfig.write_nimbus_topology_code(stormConf, topologyid, + Utils.serialize(rawNewTopology)); + + // Update topology conf if worker num has been updated + Set keys = conf.keySet(); + Integer workerNum = + JStormUtils.parseInt(conf.get(Config.TOPOLOGY_WORKERS)); + if (workerNum != null) { + Integer oldWorkerNum = + JStormUtils.parseInt(topoConf + .get(Config.TOPOLOGY_WORKERS)); + topoConf.put(Config.TOPOLOGY_WORKERS, workerNum); + isConfUpdate = true; + + LOG.info("Update worker num from " + oldWorkerNum + " to " + + workerNum); + } + + if (keys.contains(Config.ISOLATION_SCHEDULER_MACHINES)) { + topoConf.put(Config.ISOLATION_SCHEDULER_MACHINES, + conf.get(Config.ISOLATION_SCHEDULER_MACHINES)); + } + + if (isConfUpdate) { + StormConfig.write_nimbus_topology_conf(stormConf, + topologyid, topoConf); + } + } + + TopologyAssignEvent event = new TopologyAssignEvent(); + + event.setTopologyId(topologyid); + event.setScratch(true); + event.setOldStatus(oldStatus); + event.setReassign(reassign); + + TopologyAssign.push(event); + } catch (Exception e) { + LOG.error("do-rebalance error!", e); + // Rollback the changes on ZK + if (isSetTaskInfo) { + try { + StormClusterState clusterState = + data.getStormClusterState(); + clusterState.remove_task(topologyid, newTasks); + } catch (Exception e1) { + LOG.error( + "Failed to rollback the changes on ZK for task-" + + newTasks, e); + } + } + } + + DelayStatusTransitionCallback delayCallback = + new DelayStatusTransitionCallback(data, topologyid, oldStatus, + StatusType.rebalancing, StatusType.done_rebalance); + return delayCallback.execute(); + } + + private void setTaskInfo(StormTopology oldTopology, + StormTopology newTopology) throws Exception { + StormClusterState clusterState = data.getStormClusterState(); + // Retrieve the max task ID + TreeSet taskIds = + new TreeSet(clusterState.task_ids(topologyid)); + int cnt = taskIds.descendingIterator().next(); + + cnt = setBoltInfo(oldTopology, newTopology, cnt, clusterState); + cnt = setSpoutInfo(oldTopology, newTopology, cnt, clusterState); + } + + private int setBoltInfo(StormTopology oldTopology, + StormTopology newTopology, int cnt, StormClusterState clusterState) + throws Exception { + Map oldBolts = oldTopology.get_bolts(); + Map bolts = newTopology.get_bolts(); + for (Entry entry : oldBolts.entrySet()) { + String boltName = entry.getKey(); + Bolt oldBolt = entry.getValue(); + Bolt bolt = bolts.get(boltName); + if (oldBolt.get_common().get_parallelism_hint() > bolt.get_common() + .get_parallelism_hint()) { + int removedTaskNum = + oldBolt.get_common().get_parallelism_hint() + - bolt.get_common().get_parallelism_hint(); + TreeSet taskIds = + new TreeSet( + clusterState.task_ids_by_componentId( + topologyid, boltName)); + Iterator descendIterator = + taskIds.descendingIterator(); + while (--removedTaskNum >= 0) { + int taskId = descendIterator.next(); + removeTask(topologyid, taskId, clusterState); + LOG.info("Remove bolt task, taskId=" + taskId + " for " + + boltName); + } + } else if (oldBolt.get_common().get_parallelism_hint() == bolt + .get_common().get_parallelism_hint()) { + continue; + } else { + int delta = + bolt.get_common().get_parallelism_hint() + - oldBolt.get_common().get_parallelism_hint(); + Map taskInfoMap = new HashMap(); + + for (int i = 1; i <= delta; i++) { + cnt++; + TaskInfo taskInfo = + new TaskInfo((String) entry.getKey(), "bolt"); + taskInfoMap.put(cnt, taskInfo); + newTasks.add(cnt); + LOG.info("Setup new bolt task, taskId=" + cnt + " for " + + boltName); + } + clusterState.add_task(topologyid, taskInfoMap); + } + } + + return cnt; + } + + private int setSpoutInfo(StormTopology oldTopology, + StormTopology newTopology, int cnt, StormClusterState clusterState) + throws Exception { + Map oldSpouts = oldTopology.get_spouts(); + Map spouts = newTopology.get_spouts(); + for (Entry entry : oldSpouts.entrySet()) { + String spoutName = entry.getKey(); + SpoutSpec oldSpout = entry.getValue(); + SpoutSpec spout = spouts.get(spoutName); + if (oldSpout.get_common().get_parallelism_hint() > spout + .get_common().get_parallelism_hint()) { + int removedTaskNum = + oldSpout.get_common().get_parallelism_hint() + - spout.get_common().get_parallelism_hint(); + TreeSet taskIds = + new TreeSet( + clusterState.task_ids_by_componentId( + topologyid, spoutName)); + Iterator descendIterator = + taskIds.descendingIterator(); + while (--removedTaskNum >= 0) { + int taskId = descendIterator.next(); + removeTask(topologyid, taskId, clusterState); + LOG.info("Remove spout task, taskId=" + taskId + " for " + + spoutName); + } + + + + } else if (oldSpout.get_common().get_parallelism_hint() == spout + .get_common().get_parallelism_hint()) { + continue; + } else { + int delta = + spout.get_common().get_parallelism_hint() + - oldSpout.get_common().get_parallelism_hint(); + Map taskInfoMap = new HashMap(); + + for (int i = 1; i <= delta; i++) { + cnt++; + TaskInfo taskInfo = + new TaskInfo((String) entry.getKey(), "spout"); + taskInfoMap.put(cnt, taskInfo); + newTasks.add(cnt); + LOG.info("Setup new spout task, taskId=" + cnt + " for " + + spoutName); + } + clusterState.add_task(topologyid, taskInfoMap); + } + } + + return cnt; + } + + private void removeTask(String topologyId, int taskId, + StormClusterState clusterState) throws Exception { + Set taskIds = new HashSet(taskId); + clusterState.remove_task(topologyid, taskIds); + Map TkHbs = + data.getTaskHeartbeatsCache(topologyid, false); + if (TkHbs != null) { + TkHbs.remove(taskId); + } + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/DoneRebalanceTransitionCallback.java b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/DoneRebalanceTransitionCallback.java new file mode 100755 index 000000000..964bbffc5 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/DoneRebalanceTransitionCallback.java @@ -0,0 +1,22 @@ +package com.alibaba.jstorm.callback.impl; + +import com.alibaba.jstorm.callback.BaseCallback; +import com.alibaba.jstorm.cluster.StormStatus; +import com.alibaba.jstorm.daemon.nimbus.NimbusData; +import com.alibaba.jstorm.daemon.nimbus.StatusType; + +public class DoneRebalanceTransitionCallback extends BaseCallback { + + private NimbusData data; + private String topologyId; + + public DoneRebalanceTransitionCallback(NimbusData data, String topologyId) { + this.data = data; + this.topologyId = topologyId; + } + + @Override + public Object execute(T... args) { + return new StormStatus(StatusType.active); + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/InactiveTransitionCallback.java b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/InactiveTransitionCallback.java new file mode 100755 index 000000000..f902c68cc --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/InactiveTransitionCallback.java @@ -0,0 +1,38 @@ +/** + * 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.callback.impl; + +import com.alibaba.jstorm.callback.BaseCallback; +import com.alibaba.jstorm.cluster.StormStatus; +import com.alibaba.jstorm.daemon.nimbus.StatusType; + +/** + * + * set Topology status as inactive + * + * Here just return inactive status Later, it will set inactive status to ZK + */ +public class InactiveTransitionCallback extends BaseCallback { + + @Override + public Object execute(T... args) { + + return new StormStatus(StatusType.inactive); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/KillTransitionCallback.java b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/KillTransitionCallback.java new file mode 100755 index 000000000..4dad8909f --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/KillTransitionCallback.java @@ -0,0 +1,38 @@ +/** + * 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.callback.impl; + +import com.alibaba.jstorm.daemon.nimbus.NimbusData; +import com.alibaba.jstorm.daemon.nimbus.StatusType; + +/** + * The action when nimbus receive killed command. + * + * 1. change current topology status as killed 2. one TIMEOUT seconds later, do + * remove action, which remove topology from ZK + * + * @author Longda + * + */ +public class KillTransitionCallback extends DelayStatusTransitionCallback { + + public KillTransitionCallback(NimbusData data, String topologyid) { + super(data, topologyid, null, StatusType.killed, StatusType.remove); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/ReassignTransitionCallback.java b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/ReassignTransitionCallback.java new file mode 100755 index 000000000..1b4841ced --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/ReassignTransitionCallback.java @@ -0,0 +1,66 @@ +/** + * 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.callback.impl; + +import com.alibaba.jstorm.callback.BaseCallback; +import com.alibaba.jstorm.cluster.StormStatus; +import com.alibaba.jstorm.daemon.nimbus.NimbusData; +import com.alibaba.jstorm.daemon.nimbus.TopologyAssign; +import com.alibaba.jstorm.daemon.nimbus.TopologyAssignEvent; + +/** + * 1. every Config.NIMBUS_MONITOR_FREQ_SECS will call MonitorRunnable 2. + * MonitorRunnable will call NimbusData.transition 3. NimbusData.transition will + * this callback + * + * + */ +public class ReassignTransitionCallback extends BaseCallback { + + private NimbusData data; + private String topologyid; + private StormStatus oldStatus; + + public ReassignTransitionCallback(NimbusData data, String topologyid) { + this.data = data; + this.topologyid = topologyid; + this.oldStatus = null; + } + + public ReassignTransitionCallback(NimbusData data, String topologyid, + StormStatus oldStatus) { + this.data = data; + this.topologyid = topologyid; + this.oldStatus = oldStatus; + } + + @Override + public Object execute(T... args) { + + // default is true + TopologyAssignEvent assignEvent = new TopologyAssignEvent(); + assignEvent.setTopologyId(topologyid); + assignEvent.setScratch(false); + assignEvent.setOldStatus(oldStatus); + + TopologyAssign.push(assignEvent); + + return null; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/RebalanceTransitionCallback.java b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/RebalanceTransitionCallback.java new file mode 100755 index 000000000..f65f542ae --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/RebalanceTransitionCallback.java @@ -0,0 +1,42 @@ +/** + * 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.callback.impl; + +import com.alibaba.jstorm.cluster.StormStatus; +import com.alibaba.jstorm.daemon.nimbus.NimbusData; +import com.alibaba.jstorm.daemon.nimbus.StatusType; + +/** + * The action when nimbus receive rebalance command. Rebalance command is only + * valid when current status is active + * + * 1. change current topology status as rebalancing 2. do_rebalance action after + * 2 * TIMEOUT seconds + * + * @author Lixin/Longda + * + */ +public class RebalanceTransitionCallback extends DelayStatusTransitionCallback { + + public RebalanceTransitionCallback(NimbusData data, String topologyid, + StormStatus status) { + super(data, topologyid, status, StatusType.rebalancing, + StatusType.do_rebalance); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/RemoveTransitionCallback.java b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/RemoveTransitionCallback.java new file mode 100644 index 000000000..231d8e912 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/RemoveTransitionCallback.java @@ -0,0 +1,71 @@ +/** + * 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.callback.impl; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.callback.BaseCallback; +import com.alibaba.jstorm.cluster.StormBase; +import com.alibaba.jstorm.daemon.nimbus.NimbusData; +import com.alibaba.jstorm.daemon.nimbus.NimbusUtils; + +/** + * Remove topology /ZK-DIR/topology data + * + * remove this ZK node will trigger watch on this topology + * + * And Monitor thread every 10 seconds will clean these disappear topology + * + */ +public class RemoveTransitionCallback extends BaseCallback { + + private static Logger LOG = LoggerFactory + .getLogger(RemoveTransitionCallback.class); + + private NimbusData data; + private String topologyid; + + public RemoveTransitionCallback(NimbusData data, String topologyid) { + this.data = data; + this.topologyid = topologyid; + } + + @Override + public Object execute(T... args) { + LOG.info("Begin to remove topology: " + topologyid); + try { + + StormBase stormBase = + data.getStormClusterState().storm_base(topologyid, null); + if (stormBase == null) { + LOG.info("Topology " + topologyid + " has been removed "); + return null; + } + data.getStormClusterState().remove_storm(topologyid); + NimbusUtils.removeTopologyTaskTimeout(data, topologyid); + LOG.info("Successfully removed ZK items topology: " + topologyid); + + } catch (Exception e) { + // TODO Auto-generated catch block + LOG.warn("Failed to remove StormBase " + topologyid + " from ZK", e); + } + return null; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/UpdateConfTransitionCallback.java b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/UpdateConfTransitionCallback.java new file mode 100644 index 000000000..ca4e0eed6 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/callback/impl/UpdateConfTransitionCallback.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.callback.impl; + +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.callback.BaseCallback; +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.cluster.StormStatus; +import com.alibaba.jstorm.daemon.nimbus.NimbusData; +import com.alibaba.jstorm.schedule.Assignment; +import com.alibaba.jstorm.schedule.Assignment.AssignmentType; + +/** + * Update user configuration + * + * @author Basti.lj + */ +public class UpdateConfTransitionCallback extends BaseCallback { + + private static Logger LOG = LoggerFactory + .getLogger(DelayStatusTransitionCallback.class); + + public static final int DEFAULT_DELAY_SECONDS = 30; + + private NimbusData data; + private String topologyId; + private StormStatus currentStatus; + + public UpdateConfTransitionCallback(NimbusData data, String topologyId, + StormStatus currentStatus) { + this.data = data; + this.topologyId = topologyId; + this.currentStatus = currentStatus; + } + + @Override + public Object execute(T... args) { + StormClusterState clusterState = data.getStormClusterState(); + try { + Map userConf = (Map) args[0]; + Map topoConf = + StormConfig.read_nimbus_topology_conf(data.getConf(), + topologyId); + topoConf.putAll(userConf); + StormConfig.write_nimbus_topology_conf(data.getConf(), topologyId, topoConf); + + Assignment assignment = + clusterState.assignment_info(topologyId, null); + assignment.setAssignmentType(AssignmentType.Config); + assignment.updateTimeStamp(); + clusterState.set_assignment(topologyId, assignment); + LOG.info("Successfully update new config to ZK for " + topologyId); + } catch (Exception e) { + LOG.error("Failed to update user configuartion.", e); + } + return currentStatus; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/client/ConfigExtension.java b/jstorm-core/src/main/java/com/alibaba/jstorm/client/ConfigExtension.java new file mode 100644 index 000000000..26a068c89 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/client/ConfigExtension.java @@ -0,0 +1,867 @@ +/** + * 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.client; + +import java.security.InvalidParameterException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.commons.lang.StringUtils; + +import backtype.storm.Config; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.utils.JStormUtils; + +public class ConfigExtension { + /** + * if this configure has been set, the spout or bolt will log all receive + * tuples + * + * topology.debug just for logging all sent tuples + */ + protected static final String TOPOLOGY_DEBUG_RECV_TUPLE = + "topology.debug.recv.tuple"; + + public static void setTopologyDebugRecvTuple(Map conf, boolean debug) { + conf.put(TOPOLOGY_DEBUG_RECV_TUPLE, Boolean.valueOf(debug)); + } + + public static Boolean isTopologyDebugRecvTuple(Map conf) { + return JStormUtils.parseBoolean(conf.get(TOPOLOGY_DEBUG_RECV_TUPLE), + false); + } + + /** + * port number of deamon httpserver server + */ + private static final Integer DEFAULT_DEAMON_HTTPSERVER_PORT = 7621; + + protected static final String SUPERVISOR_DEAMON_HTTPSERVER_PORT = + "supervisor.deamon.logview.port"; + + public static Integer getSupervisorDeamonHttpserverPort(Map conf) { + return JStormUtils.parseInt( + conf.get(SUPERVISOR_DEAMON_HTTPSERVER_PORT), + DEFAULT_DEAMON_HTTPSERVER_PORT + 1); + } + + protected static final String NIMBUS_DEAMON_HTTPSERVER_PORT = + "nimbus.deamon.logview.port"; + + public static Integer getNimbusDeamonHttpserverPort(Map conf) { + return JStormUtils.parseInt(conf.get(NIMBUS_DEAMON_HTTPSERVER_PORT), + DEFAULT_DEAMON_HTTPSERVER_PORT); + } + + /** + * Worker gc parameter + * + * + */ + protected static final String WORKER_GC_CHILDOPTS = "worker.gc.childopts"; + + public static void setWorkerGc(Map conf, String gc) { + conf.put(WORKER_GC_CHILDOPTS, gc); + } + + public static String getWorkerGc(Map conf) { + return (String) conf.get(WORKER_GC_CHILDOPTS); + } + + protected static final String WOREKER_REDIRECT_OUTPUT = + "worker.redirect.output"; + + public static boolean getWorkerRedirectOutput(Map conf) { + Object result = conf.get(WOREKER_REDIRECT_OUTPUT); + if (result == null) + return true; + return (Boolean) result; + } + + protected static final String WOREKER_REDIRECT_OUTPUT_FILE = + "worker.redirect.output.file"; + + public static void setWorkerRedirectOutputFile(Map conf, String outputPath) { + conf.put(WOREKER_REDIRECT_OUTPUT_FILE, outputPath); + } + + public static String getWorkerRedirectOutputFile(Map conf) { + return (String) conf.get(WOREKER_REDIRECT_OUTPUT_FILE); + } + + /** + * Usually, spout finish prepare before bolt, so spout need wait several + * seconds so that bolt finish preparation + * + * By default, the setting is 30 seconds + */ + protected static final String SPOUT_DELAY_RUN = "spout.delay.run"; + + public static void setSpoutDelayRunSeconds(Map conf, int delay) { + conf.put(SPOUT_DELAY_RUN, Integer.valueOf(delay)); + } + + public static int getSpoutDelayRunSeconds(Map conf) { + return JStormUtils.parseInt(conf.get(SPOUT_DELAY_RUN), 30); + } + + /** + * Default ZMQ Pending queue size + */ + public static final int DEFAULT_ZMQ_MAX_QUEUE_MSG = 1000; + + /** + * One task will alloc how many memory slot, the default setting is 1 + */ + protected static final String MEM_SLOTS_PER_TASK = "memory.slots.per.task"; + + @Deprecated + public static void setMemSlotPerTask(Map conf, int slotNum) { + if (slotNum < 1) { + throw new InvalidParameterException(); + } + conf.put(MEM_SLOTS_PER_TASK, Integer.valueOf(slotNum)); + } + + /** + * One task will use cpu slot number, the default setting is 1 + */ + protected static final String CPU_SLOTS_PER_TASK = "cpu.slots.per.task"; + + @Deprecated + public static void setCpuSlotsPerTask(Map conf, int slotNum) { + if (slotNum < 1) { + throw new InvalidParameterException(); + } + conf.put(CPU_SLOTS_PER_TASK, Integer.valueOf(slotNum)); + } + + /** + * if the setting has been set, the component's task must run different node + * This is conflict with USE_SINGLE_NODE + */ + protected static final String TASK_ON_DIFFERENT_NODE = + "task.on.differ.node"; + + public static void setTaskOnDifferentNode(Map conf, boolean isIsolate) { + conf.put(TASK_ON_DIFFERENT_NODE, Boolean.valueOf(isIsolate)); + } + + public static boolean isTaskOnDifferentNode(Map conf) { + return JStormUtils + .parseBoolean(conf.get(TASK_ON_DIFFERENT_NODE), false); + } + + protected static final String SUPERVISOR_ENABLE_CGROUP = + "supervisor.enable.cgroup"; + + public static boolean isEnableCgroup(Map conf) { + return JStormUtils.parseBoolean(conf.get(SUPERVISOR_ENABLE_CGROUP), + false); + } + + /** + * If component or topology configuration set "use.old.assignment", will try + * use old assignment firstly + */ + protected static final String USE_OLD_ASSIGNMENT = "use.old.assignment"; + + 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 + */ + protected static final String SUPERVISOR_HOSTNAME = "supervisor.hostname"; + public static final Object SUPERVISOR_HOSTNAME_SCHEMA = String.class; + + public static String getSupervisorHost(Map conf) { + return (String) conf.get(SUPERVISOR_HOSTNAME); + } + + protected static final String SUPERVISOR_USE_IP = "supervisor.use.ip"; + + public static boolean isSupervisorUseIp(Map conf) { + return JStormUtils.parseBoolean(conf.get(SUPERVISOR_USE_IP), false); + } + + protected static final String NIMBUS_USE_IP = "nimbus.use.ip"; + + public static boolean isNimbusUseIp(Map conf) { + return JStormUtils.parseBoolean(conf.get(NIMBUS_USE_IP), false); + } + + protected static final String TOPOLOGY_ENABLE_CLASSLOADER = + "topology.enable.classloader"; + + public static boolean isEnableTopologyClassLoader(Map conf) { + return JStormUtils.parseBoolean(conf.get(TOPOLOGY_ENABLE_CLASSLOADER), + false); + } + + 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"; + + /** + * Get to know whether nimbus is run under Apsara/Yarn container + * + * @param conf + * @return + */ + public static boolean isEnableContainerNimbus() { + String path = System.getenv(CONTAINER_NIMBUS_HEARTBEAT); + + if (StringUtils.isBlank(path)) { + return false; + } else { + return true; + } + } + + /** + * Get Apsara/Yarn nimbus container's hearbeat dir + * + * @param conf + * @return + */ + public static String getContainerNimbusHearbeat() { + return System.getenv(CONTAINER_NIMBUS_HEARTBEAT); + } + + protected static final String CONTAINER_SUPERVISOR_HEARTBEAT = + "container.supervisor.heartbeat"; + + /** + * Get to know whether supervisor is run under Apsara/Yarn supervisor + * container + * + * @param conf + * @return + */ + public static boolean isEnableContainerSupervisor() { + String path = System.getenv(CONTAINER_SUPERVISOR_HEARTBEAT); + + if (StringUtils.isBlank(path)) { + return false; + } else { + return true; + } + } + + /** + * Get Apsara/Yarn supervisor container's hearbeat dir + * + * @param conf + * @return + */ + public static String getContainerSupervisorHearbeat() { + return (String) System.getenv(CONTAINER_SUPERVISOR_HEARTBEAT); + } + + protected static final String CONTAINER_HEARTBEAT_TIMEOUT_SECONDS = + "container.heartbeat.timeout.seconds"; + + public static int getContainerHeartbeatTimeoutSeconds(Map conf) { + return JStormUtils.parseInt( + conf.get(CONTAINER_HEARTBEAT_TIMEOUT_SECONDS), 240); + } + + protected static final String CONTAINER_HEARTBEAT_FREQUENCE = + "container.heartbeat.frequence"; + + public static int getContainerHeartbeatFrequence(Map conf) { + return JStormUtils + .parseInt(conf.get(CONTAINER_HEARTBEAT_FREQUENCE), 10); + } + + protected static final String JAVA_SANDBOX_ENABLE = "java.sandbox.enable"; + + public static boolean isJavaSandBoxEnable(Map conf) { + return JStormUtils.parseBoolean(conf.get(JAVA_SANDBOX_ENABLE), false); + } + + protected static String SPOUT_SINGLE_THREAD = "spout.single.thread"; + + public static boolean isSpoutSingleThread(Map conf) { + return JStormUtils.parseBoolean(conf.get(SPOUT_SINGLE_THREAD), false); + } + + public static void setSpoutSingleThread(Map conf, boolean enable) { + conf.put(SPOUT_SINGLE_THREAD, enable); + } + + protected static String WORKER_STOP_WITHOUT_SUPERVISOR = + "worker.stop.without.supervisor"; + + public static boolean isWorkerStopWithoutSupervisor(Map conf) { + return JStormUtils.parseBoolean( + conf.get(WORKER_STOP_WITHOUT_SUPERVISOR), false); + } + + protected static String CGROUP_ROOT_DIR = "supervisor.cgroup.rootdir"; + + public static String getCgroupRootDir(Map conf) { + return (String) conf.get(CGROUP_ROOT_DIR); + } + + protected static String NETTY_TRANSFER_ASYNC_AND_BATCH = + "storm.messaging.netty.transfer.async.batch"; + + public static boolean isNettyTransferAsyncBatch(Map conf) { + return JStormUtils.parseBoolean( + conf.get(NETTY_TRANSFER_ASYNC_AND_BATCH), true); + } + + protected static String NETTY_PENDING_BUFFER_TIMEOUT = + "storm.messaging.netty.pending.buffer.timeout"; + + public static void setNettyPendingBufferTimeout(Map conf, Long timeout) { + conf.put(NETTY_PENDING_BUFFER_TIMEOUT, timeout); + } + + public static long getNettyPendingBufferTimeout(Map conf) { + int messageTimeout = JStormUtils.parseInt( + conf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS), 120); + return JStormUtils.parseLong( + conf.get(NETTY_PENDING_BUFFER_TIMEOUT), messageTimeout*1000); + } + + protected static final String USE_USERDEFINE_ASSIGNMENT = + "use.userdefine.assignment"; + + public static void setUserDefineAssignment(Map conf, + List userDefines) { + List ret = new ArrayList(); + for (WorkerAssignment worker : userDefines) { + ret.add(Utils.to_json(worker)); + } + 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"; + + public static void setMemSizePerWorker(Map conf, long memSize) { + conf.put(MEMSIZE_PER_WORKER, memSize); + } + + public static void setMemSizePerWorkerByKB(Map conf, long memSize) { + long size = memSize * 1024l; + setMemSizePerWorker(conf, size); + } + + public static void setMemSizePerWorkerByMB(Map conf, long memSize) { + long size = memSize * 1024l; + setMemSizePerWorkerByKB(conf, size); + } + + 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"; + + public static boolean isEnablePerformanceMetrics(Map conf) { + return JStormUtils.parseBoolean(conf.get(TOPOLOGY_PERFORMANCE_METRICS), + true); + } + + public static void setPerformanceMetrics(Map conf, boolean isEnable) { + conf.put(TOPOLOGY_PERFORMANCE_METRICS, isEnable); + } + + protected static String NETTY_BUFFER_THRESHOLD_SIZE = + "storm.messaging.netty.buffer.threshold"; + + public static long getNettyBufferThresholdSize(Map conf) { + return JStormUtils.parseLong(conf.get(NETTY_BUFFER_THRESHOLD_SIZE), + 8 * JStormUtils.SIZE_1_M); + } + + public static void setNettyBufferThresholdSize(Map conf, long size) { + conf.put(NETTY_BUFFER_THRESHOLD_SIZE, size); + } + + protected static String NETTY_MAX_SEND_PENDING = + "storm.messaging.netty.max.pending"; + + public static void setNettyMaxSendPending(Map conf, long pending) { + conf.put(NETTY_MAX_SEND_PENDING, pending); + } + + public static long getNettyMaxSendPending(Map conf) { + return JStormUtils.parseLong(conf.get(NETTY_MAX_SEND_PENDING), 4); + } + + protected static String DISRUPTOR_USE_SLEEP = "disruptor.use.sleep"; + + public static boolean isDisruptorUseSleep(Map conf) { + return JStormUtils.parseBoolean(conf.get(DISRUPTOR_USE_SLEEP), true); + } + + public static void setDisruptorUseSleep(Map conf, boolean useSleep) { + conf.put(DISRUPTOR_USE_SLEEP, useSleep); + } + + public static boolean isTopologyContainAcker(Map conf) { + int num = + JStormUtils.parseInt(conf.get(Config.TOPOLOGY_ACKER_EXECUTORS), + 1); + if (num > 0) { + return true; + } else { + return false; + } + } + + protected static String NETTY_SYNC_MODE = "storm.messaging.netty.sync.mode"; + + public static boolean isNettySyncMode(Map conf) { + return JStormUtils.parseBoolean(conf.get(NETTY_SYNC_MODE), false); + } + + public static void setNettySyncMode(Map conf, boolean sync) { + conf.put(NETTY_SYNC_MODE, sync); + } + + protected static String NETTY_ASYNC_BLOCK = + "storm.messaging.netty.async.block"; + + public static boolean isNettyASyncBlock(Map conf) { + return JStormUtils.parseBoolean(conf.get(NETTY_ASYNC_BLOCK), true); + } + + public static void setNettyASyncBlock(Map conf, boolean block) { + conf.put(NETTY_ASYNC_BLOCK, block); + } + + protected static String ALIMONITOR_METRICS_POST = + "topology.alimonitor.metrics.post"; + + public static boolean isAlimonitorMetricsPost(Map conf) { + return JStormUtils + .parseBoolean(conf.get(ALIMONITOR_METRICS_POST), true); + } + + public static void setAlimonitorMetricsPost(Map conf, boolean post) { + conf.put(ALIMONITOR_METRICS_POST, post); + } + + public static String TASK_CLEANUP_TIMEOUT_SEC = + "task.cleanup.timeout.sec"; + + public static int getTaskCleanupTimeoutSec(Map conf) { + return JStormUtils.parseInt(conf.get(TASK_CLEANUP_TIMEOUT_SEC), 10); + } + + public static void setTaskCleanupTimeoutSec(Map conf, int timeout) { + conf.put(TASK_CLEANUP_TIMEOUT_SEC, timeout); + } + + protected static String UI_CLUSTERS = "ui.clusters"; + protected static String UI_CLUSTER_NAME = "name"; + protected static String UI_CLUSTER_ZK_ROOT = "zkRoot"; + protected static String UI_CLUSTER_ZK_SERVERS = "zkServers"; + protected static String UI_CLUSTER_ZK_PORT = "zkPort"; + + public static List getUiClusters(Map conf) { + return (List) conf.get(UI_CLUSTERS); + } + + public static void setUiClusters(Map conf, List uiClusters) { + conf.put(UI_CLUSTERS, uiClusters); + } + + public static Map getUiClusterInfo(List uiClusters, String name) { + Map ret = null; + for (Map cluster : uiClusters) { + String clusterName = getUiClusterName(cluster); + if (clusterName.equals(name)) { + ret = cluster; + break; + } + } + + return ret; + } + + public static String getUiClusterName(Map uiCluster) { + return (String) uiCluster.get(UI_CLUSTER_NAME); + } + + public static String getUiClusterZkRoot(Map uiCluster) { + return (String) uiCluster.get(UI_CLUSTER_ZK_ROOT); + } + + public static List getUiClusterZkServers(Map uiCluster) { + return (List) uiCluster.get(UI_CLUSTER_ZK_SERVERS); + } + + public static Integer getUiClusterZkPort(Map uiCluster) { + return JStormUtils.parseInt(uiCluster.get(UI_CLUSTER_ZK_PORT)); + } + + protected static String SPOUT_PEND_FULL_SLEEP = "spout.pending.full.sleep"; + + public static boolean isSpoutPendFullSleep(Map conf) { + return JStormUtils.parseBoolean(conf.get(SPOUT_PEND_FULL_SLEEP), false); + } + + public static void setSpoutPendFullSleep(Map conf, boolean sleep) { + conf.put(SPOUT_PEND_FULL_SLEEP, sleep); + + } + + protected static String LOGVIEW_ENCODING = + "supervisor.deamon.logview.encoding"; + protected static String UTF8 = "utf-8"; + + public static String getLogViewEncoding(Map conf) { + String ret = (String) conf.get(LOGVIEW_ENCODING); + if (ret == null) + ret = UTF8; + return ret; + } + + public static void setLogViewEncoding(Map conf, String enc) { + conf.put(LOGVIEW_ENCODING, enc); + } + + protected static String LOG_PAGE_SIZE = "log.page.size"; + + public static int getLogPageSize(Map conf) { + return JStormUtils.parseInt(conf.get(LOG_PAGE_SIZE), 32 * 1024); + } + + public static void setLogPageSize(Map conf, int pageSize) { + conf.put(LOG_PAGE_SIZE, pageSize); + } + + public static String TASK_STATUS_ACTIVE = "Active"; + public static String TASK_STATUS_STARTING = "Starting"; + + protected static String ALIMONITOR_TOPO_METIRC_NAME = + "topology.alimonitor.topo.metrics.name"; + protected static String ALIMONITOR_TASK_METIRC_NAME = + "topology.alimonitor.task.metrics.name"; + protected static String ALIMONITOR_WORKER_METIRC_NAME = + "topology.alimonitor.worker.metrics.name"; + protected static String ALIMONITOR_USER_METIRC_NAME = + "topology.alimonitor.user.metrics.name"; + + public static String getAlmonTopoMetricName(Map conf) { + return (String) conf.get(ALIMONITOR_TOPO_METIRC_NAME); + } + + public static String getAlmonTaskMetricName(Map conf) { + return (String) conf.get(ALIMONITOR_TASK_METIRC_NAME); + } + + public static String getAlmonWorkerMetricName(Map conf) { + return (String) conf.get(ALIMONITOR_WORKER_METIRC_NAME); + } + + public static String getAlmonUserMetricName(Map conf) { + return (String) conf.get(ALIMONITOR_USER_METIRC_NAME); + } + + protected static String SPOUT_PARALLELISM = "topology.spout.parallelism"; + protected static String BOLT_PARALLELISM = "topology.bolt.parallelism"; + + public static Integer getSpoutParallelism(Map conf, String componentName) { + Integer ret = null; + Map map = + (Map) (conf.get(SPOUT_PARALLELISM)); + if (map != null) + ret = JStormUtils.parseInt(map.get(componentName)); + return ret; + } + + public static Integer getBoltParallelism(Map conf, String componentName) { + Integer ret = null; + Map map = + (Map) (conf.get(BOLT_PARALLELISM)); + if (map != null) + ret = JStormUtils.parseInt(map.get(componentName)); + return ret; + } + + protected static String TOPOLOGY_BUFFER_SIZE_LIMITED = + "topology.buffer.size.limited"; + + public static void setTopologyBufferSizeLimited(Map conf, boolean limited) { + conf.put(TOPOLOGY_BUFFER_SIZE_LIMITED, limited); + } + + public static boolean getTopologyBufferSizeLimited(Map conf) { + boolean isSynchronized = isNettySyncMode(conf); + if (isSynchronized == true) { + return true; + } + + 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); + } + + protected static String DEFAULT_CACHE_TIMEOUT = "default.cache.timeout"; + + public static int getDefaultCacheTimeout(Map conf) { + return JStormUtils.parseInt(conf.get(DEFAULT_CACHE_TIMEOUT), 60); + } + + public static void setDefaultCacheTimeout(Map conf, int timeout) { + conf.put(DEFAULT_CACHE_TIMEOUT, timeout); + } + + protected static String WORKER_MERTRIC_REPORT_FREQUENCY = + "worker.metric.report.frequency.secs"; + + public static int getWorkerMetricReportFrequency(Map conf) { + return JStormUtils.parseInt(conf.get(WORKER_MERTRIC_REPORT_FREQUENCY), + 60); + } + + public static void setWorkerMetricReportFrequency(Map conf, int frequence) { + conf.put(WORKER_MERTRIC_REPORT_FREQUENCY, frequence); + } + + /** + * Store local worker port/workerId/supervisorId to configuration + */ + protected static String LOCAL_WORKER_PORT = "local.worker.port"; + protected static String LOCLA_WORKER_ID = "local.worker.id"; + protected static String LOCAL_SUPERVISOR_ID = "local.supervisor.id"; + + public static int getLocalWorkerPort(Map conf) { + return JStormUtils.parseInt(conf.get(LOCAL_WORKER_PORT)); + } + + public static void setLocalWorkerPort(Map conf, int port) { + conf.put(LOCAL_WORKER_PORT, port); + } + + public static String getLocalWorkerId(Map conf) { + return (String)conf.get(LOCLA_WORKER_ID); + } + + public static void setLocalWorkerId(Map conf, String workerId) { + conf.put(LOCLA_WORKER_ID, workerId); + } + + public static String getLocalSupervisorId(Map conf) { + return (String)conf.get(LOCAL_SUPERVISOR_ID); + } + + public static void setLocalSupervisorId(Map conf, String supervisorId) { + conf.put(LOCAL_SUPERVISOR_ID, supervisorId); + } + + protected static String WORKER_CPU_CORE_UPPER_LIMIT = + "worker.cpu.core.upper.limit"; + + public static Integer getWorkerCpuCoreUpperLimit(Map conf) { + return JStormUtils.parseInt(conf.get(WORKER_CPU_CORE_UPPER_LIMIT), 1); + } + + public static void setWorkerCpuCoreUpperLimit(Map conf, + Integer cpuUpperLimit) { + conf.put(WORKER_CPU_CORE_UPPER_LIMIT, cpuUpperLimit); + } + + protected static String CLUSTER_NAME = "cluster.name"; + + public static String getClusterName(Map conf) { + return (String) conf.get(CLUSTER_NAME); + } + + public static void setClusterName(Map conf, String clusterName) { + conf.put(CLUSTER_NAME, clusterName); + } + + protected static final String NIMBUS_CACHE_CLASS = "nimbus.cache.class"; + + public static String getNimbusCacheClass(Map conf) { + return (String)conf.get(NIMBUS_CACHE_CLASS); + } + + /** + * if this is set, nimbus cache db will be clean when start nimbus + */ + protected static final String NIMBUS_CACHE_RESET = "nimbus.cache.reset"; + + public static boolean getNimbusCacheReset(Map conf) { + return JStormUtils.parseBoolean(conf.get(NIMBUS_CACHE_RESET), true); + } + + public static final String CACHE_TIMEOUT_LIST = "cache.timeout.list"; + + public static List getCacheTimeoutList(Map conf) { + return (List)conf.get(CACHE_TIMEOUT_LIST); + } + + protected static final String NIMBUS_METRICS_THREAD_NUM = "nimbus.metrics.thread.num"; + public static int getNimbusMetricThreadNum(Map conf) { + return JStormUtils.parseInt(conf.get(NIMBUS_METRICS_THREAD_NUM), 2); + } + + protected static String TASK_MSG_BATCH_SIZE = "task.msg.batch.size"; + + public static Integer getTaskMsgBatchSize(Map conf) { + return JStormUtils.parseInt(conf.get(TASK_MSG_BATCH_SIZE), 1); + } + + public static void setTaskMsgBatchSize(Map conf, Integer batchSize) { + conf.put(TASK_MSG_BATCH_SIZE, batchSize); + } + + protected static String TASK_BATCH_TUPLE = "task.batch.tuple"; + + public static Boolean isTaskBatchTuple(Map conf) { + return JStormUtils.parseBoolean(conf.get(TASK_BATCH_TUPLE), false); + } + + public static void setTaskBatchTuple(Map conf, boolean isBatchTuple) { + conf.put(TASK_BATCH_TUPLE, isBatchTuple); + } + + protected static String TOPOLOGY_ENABLE_NETTY_METRICS = "topology.enable.netty.metrics"; + public static void setTopologyNettyMetrics(Map conf, boolean enable) { + conf.put(TOPOLOGY_ENABLE_NETTY_METRICS, enable); + } + + public static Boolean isEnableTopologyNettyMetrics(Map conf) { + return (Boolean)conf.get(TOPOLOGY_ENABLE_NETTY_METRICS); + } + + protected static String UI_ONE_TABLE_PAGE_SIZE = "ui.one.table.page.size"; + public static long getUiOneTablePageSize(Map conf) { + return JStormUtils.parseLong(conf.get(UI_ONE_TABLE_PAGE_SIZE), 200); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/client/WorkerAssignment.java b/jstorm-core/src/main/java/com/alibaba/jstorm/client/WorkerAssignment.java new file mode 100755 index 000000000..c994858cf --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/client/WorkerAssignment.java @@ -0,0 +1,283 @@ +/** + * 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.client; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; +import org.json.simple.JSONAware; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.scheduler.WorkerSlot; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.utils.JStormUtils; + +public class WorkerAssignment extends WorkerSlot implements Serializable, + JSONAware { + private static final Logger LOG = LoggerFactory + .getLogger(WorkerAssignment.class); + + private static final long serialVersionUID = -3483047434535537861L; + + private Map componentToNum = + new HashMap(); + + private long mem; + + private int cpu; + + private String hostName; + + private String jvm; + + private static final String COMPONENTTONUM_TAG = "componentToNum"; + private static final String MEM_TAG = "mem"; + private static final String CPU_TAG = "cpu"; + private static final String HOSTNAME_TAG = "hostName"; + private static final String JVM_TAG = "jvm"; + private static final String NODEID_TAG = "nodeId"; + private static final String PORT_TAG = "port"; + + public WorkerAssignment(String nodeId, Number port) { + super(nodeId, port); + // TODO Auto-generated constructor stub + } + + public WorkerAssignment() { + + } + + public void addComponent(String compenentName, Integer num) { + componentToNum.put(compenentName, num); + } + + public Map getComponentToNum() { + return componentToNum; + } + + public String getHostName() { + return hostName; + } + + public void setHostName(String hostName) { + this.hostName = hostName; + } + + public void setJvm(String jvm) { + this.jvm = jvm; + } + + public String getJvm() { + return jvm; + } + + public long getMem() { + return mem; + } + + public void setMem(long mem) { + this.mem = mem; + } + + public int getCpu() { + return cpu; + } + + public void setCpu(int cpu) { + this.cpu = cpu; + } + + @Override + public String toJSONString() { + // StringBuilder sb = new StringBuilder(); + + // sb.append("["); + // sb.append("\"" + this.getNodeId() + "\""); + // sb.append(","); + // sb.append("\"" + this.hostName + "\""); + // sb.append(","); + // sb.append("\"" + String.valueOf(this.getPort()) + "\""); + // sb.append(","); + // sb.append("\"" + this.jvm + "\""); + // sb.append(","); + // sb.append("\"" + String.valueOf(this.mem) + "\""); + // sb.append(","); + // sb.append("\"" + String.valueOf(this.cpu) + "\""); + // sb.append(","); + // sb.append("{"); + // for (Entry entry : componentToNum.entrySet()) { + // sb.append("\"" + entry.getKey() + "\":"); + // sb.append("\"" + String.valueOf(entry.getValue()) + "\""); + // sb.append(","); + // } + // sb.append("}"); + // sb.append("]"); + + Map map = new HashMap(); + + map.put(COMPONENTTONUM_TAG, Utils.to_json(componentToNum)); + map.put(MEM_TAG, String.valueOf(mem)); + map.put(CPU_TAG, String.valueOf(cpu)); + map.put(HOSTNAME_TAG, hostName); + map.put(JVM_TAG, jvm); + map.put(NODEID_TAG, getNodeId()); + map.put(PORT_TAG, String.valueOf(getPort())); + + return Utils.to_json(map); + } + + public static WorkerAssignment parseFromObj(Object obj) { + if (obj == null) { + return null; + } + + if (obj instanceof Map == false) { + return null; + } + + try { + Map map = (Map) obj; + + String supervisorId = map.get(NODEID_TAG); + String hostname = map.get(HOSTNAME_TAG); + Integer port = JStormUtils.parseInt(map.get(PORT_TAG)); + String jvm = map.get(JVM_TAG); + Long mem = JStormUtils.parseLong(map.get(MEM_TAG)); + Integer cpu = JStormUtils.parseInt(map.get(CPU_TAG)); + Map componentToNum = + (Map) Utils.from_json(map + .get(COMPONENTTONUM_TAG)); + + WorkerAssignment ret = new WorkerAssignment(supervisorId, port); + + ret.hostName = hostname; + ret.setNodeId(supervisorId); + ret.setJvm(jvm); + if (port != null) { + ret.setPort(port); + } + if (mem != null) { + ret.setMem(mem); + } + if (cpu != null) { + ret.setCpu(cpu); + } + + for (Entry entry : componentToNum.entrySet()) { + ret.addComponent(entry.getKey(), + JStormUtils.parseInt(entry.getValue())); + } + return ret; + } catch (Exception e) { + LOG.error("Failed to convert to WorkerAssignment, raw:" + obj, e); + return null; + } + + } + + public static String getStringFromJson(String text) { + return text.equals("null") ? null : text; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = super.hashCode(); + result = + prime + * result + + ((componentToNum == null) ? 0 : componentToNum + .hashCode()); + result = prime * result + cpu; + result = + prime * result + ((hostName == null) ? 0 : hostName.hashCode()); + result = prime * result + ((jvm == null) ? 0 : jvm.hashCode()); + result = prime * result + (int) (mem ^ (mem >>> 32)); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (!super.equals(obj)) + return false; + if (getClass() != obj.getClass()) + return false; + WorkerAssignment other = (WorkerAssignment) obj; + if (componentToNum == null) { + if (other.componentToNum != null) + return false; + } else if (!componentToNum.equals(other.componentToNum)) + return false; + if (cpu != other.cpu) + return false; + if (hostName == null) { + if (other.hostName != null) + return false; + } else if (!hostName.equals(other.hostName)) + return false; + if (jvm == null) { + if (other.jvm != null) + return false; + } else if (!jvm.equals(other.jvm)) + return false; + if (mem != other.mem) + return false; + return true; + } + + public static void main(String[] args) { + WorkerAssignment input = new WorkerAssignment(); + + input.setJvm("sb"); + + input.setCpu(1); + + input.setMem(2); + + input.addComponent("2b", 2); + + String outString = Utils.to_json(input); + + System.out.println(input); + + // String outString = + // "[componentToNum={},mem=1610612736,cpu=1,hostName=mobilejstorm-60-1,jvm=,nodeId=,port=0]"; + + Object object = Utils.from_json(outString); + System.out.println(object); + + System.out.println(parseFromObj(object)); + + System.out.print(input.equals(parseFromObj(object))); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/client/spout/IAckValueSpout.java b/jstorm-core/src/main/java/com/alibaba/jstorm/client/spout/IAckValueSpout.java new file mode 100755 index 000000000..df88ad8c4 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/client/spout/IAckValueSpout.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.client.spout; + +import java.util.List; + +/** + * This interface will list emit values when tuple success + * + * if spout implement this interface, spout won't call ISpout.ack() when tuple + * success + * + * @author longda + */ +public interface IAckValueSpout { + void ack(Object msgId, List values); +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/client/spout/IFailValueSpout.java b/jstorm-core/src/main/java/com/alibaba/jstorm/client/spout/IFailValueSpout.java new file mode 100755 index 000000000..9bebfa4bf --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/client/spout/IFailValueSpout.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.client.spout; + +import java.util.List; + +/** + * This interface will list emit values when tuple fails + * + * if spout implement this interface, spout won't call ISpout.fail() when tuple + * fail + * + * @author longda + */ +public interface IFailValueSpout { + void fail(Object msgId, List values); +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/Cluster.java b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/Cluster.java new file mode 100644 index 000000000..f7390873e --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/Cluster.java @@ -0,0 +1,398 @@ +/** + * 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.cluster; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.TreeMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; +import com.alibaba.jstorm.schedule.Assignment; +import com.alibaba.jstorm.task.TaskInfo; +import com.alibaba.jstorm.task.error.TaskError; +import com.alibaba.jstorm.task.heartbeat.TaskHeartbeat; +import com.alibaba.jstorm.utils.TimeUtils; + +/** + * storm operation ZK + * + * @author yannian/longda/zhiyuan.ls + * + */ +public class Cluster { + + // TODO Need Migrate constants to ZkConstant + + private static Logger LOG = LoggerFactory.getLogger(Cluster.class); + + public static final String ZK_SEPERATOR = "/"; + + public static final String ASSIGNMENTS_ROOT = "assignments"; + public static final String ASSIGNMENTS_BAK = "assignments_bak"; + public static final String TASKS_ROOT = "tasks"; + public static final String STORMS_ROOT = "topology"; + public static final String SUPERVISORS_ROOT = "supervisors"; + public static final String TASKBEATS_ROOT = "taskbeats"; + public static final String TASKERRORS_ROOT = "taskerrors"; + public static final String MASTER_ROOT = "nimbus_master"; + public static final String NIMBUS_SLAVE_ROOT = "nimbus_slave"; + public static final String METRIC_ROOT = "metrics"; + + public static final String LAST_ERROR = "last_error"; + + public static final String ASSIGNMENTS_SUBTREE; + public static final String ASSIGNMENTS_BAK_SUBTREE; + public static final String TASKS_SUBTREE; + public static final String STORMS_SUBTREE; + public static final String SUPERVISORS_SUBTREE; + public static final String TASKBEATS_SUBTREE; + public static final String TASKERRORS_SUBTREE; + public static final String MASTER_SUBTREE; + public static final String NIMBUS_SLAVE_SUBTREE; + public static final String METRIC_SUBTREE; + + static { + ASSIGNMENTS_SUBTREE = ZK_SEPERATOR + ASSIGNMENTS_ROOT; + ASSIGNMENTS_BAK_SUBTREE = ZK_SEPERATOR + ASSIGNMENTS_BAK; + TASKS_SUBTREE = ZK_SEPERATOR + TASKS_ROOT; + STORMS_SUBTREE = ZK_SEPERATOR + STORMS_ROOT; + SUPERVISORS_SUBTREE = ZK_SEPERATOR + SUPERVISORS_ROOT; + TASKBEATS_SUBTREE = ZK_SEPERATOR + TASKBEATS_ROOT; + TASKERRORS_SUBTREE = ZK_SEPERATOR + TASKERRORS_ROOT; + MASTER_SUBTREE = ZK_SEPERATOR + MASTER_ROOT; + NIMBUS_SLAVE_SUBTREE = ZK_SEPERATOR + NIMBUS_SLAVE_ROOT; + METRIC_SUBTREE = ZK_SEPERATOR + METRIC_ROOT; + } + + public static String supervisor_path(String id) { + return SUPERVISORS_SUBTREE + ZK_SEPERATOR + id; + } + + public static String assignment_path(String id) { + return ASSIGNMENTS_SUBTREE + ZK_SEPERATOR + id; + } + + public static String storm_path(String id) { + return STORMS_SUBTREE + ZK_SEPERATOR + id; + } + + public static String storm_task_root(String topology_id) { + return TASKS_SUBTREE + ZK_SEPERATOR + topology_id; + } + + public static String taskbeat_storm_root(String topology_id) { + return TASKBEATS_SUBTREE + ZK_SEPERATOR + topology_id; + } + + public static String taskbeat_path(String topology_id, int task_id) { + return taskbeat_storm_root(topology_id) + ZK_SEPERATOR + task_id; + } + + public static String taskerror_storm_root(String topology_id) { + return TASKERRORS_SUBTREE + ZK_SEPERATOR + topology_id; + } + + public static String lasterror_path(String topology_id) { + return taskerror_storm_root(topology_id) + ZK_SEPERATOR + LAST_ERROR; + } + + public static String taskerror_path(String topology_id, int task_id) { + return taskerror_storm_root(topology_id) + ZK_SEPERATOR + task_id; + } + + public static String metric_path(String topology_id) { + return METRIC_SUBTREE + ZK_SEPERATOR + topology_id; + } + + public static String assignment_bak_path(String id) { + return ASSIGNMENTS_BAK_SUBTREE + ZK_SEPERATOR + id; + } + + @SuppressWarnings("rawtypes") + public static StormClusterState mk_storm_cluster_state( + Map cluster_state_spec) throws Exception { + return new StormZkClusterState(cluster_state_spec); + } + + public static StormClusterState mk_storm_cluster_state( + ClusterState cluster_state_spec) throws Exception { + return new StormZkClusterState(cluster_state_spec); + } + + public static Map get_all_taskInfo( + StormClusterState zkCluster, String topologyId) throws Exception { + return zkCluster.task_all_info(topologyId); + } + + + public static Map get_all_task_component( + StormClusterState zkCluster, String topologyId, + Map taskInfoMap) throws Exception { + if (taskInfoMap == null) { + taskInfoMap = get_all_taskInfo(zkCluster, topologyId); + } + + if (taskInfoMap == null) { + return null; + } + + return Common.getTaskToComponent(taskInfoMap); + } + + public static Map get_all_task_type( + StormClusterState zkCluster, String topologyId, + Map taskInfoMap) throws Exception { + if (taskInfoMap == null) { + taskInfoMap = get_all_taskInfo(zkCluster, topologyId); + } + + if (taskInfoMap == null) { + return null; + } + + return Common.getTaskToType(taskInfoMap); + } + + public static Map get_all_task_heartbeat( + StormClusterState zkCluster, String topologyId) throws Exception { + Map ret = new HashMap(); + + List taskList = zkCluster.heartbeat_tasks(topologyId); + for (String taskId : taskList) { + TaskHeartbeat hb = + zkCluster.task_heartbeat(topologyId, + Integer.valueOf(taskId)); + if (hb == null) { + LOG.error("Failed to get hearbeat of " + topologyId + ":" + + taskId); + continue; + } + + ret.put(taskId, hb); + } + + return ret; + } + + /** + * if one topology's name equal the input storm_name, then return the + * topology id, otherwise return null + * + * @param zkCluster + * @param storm_name + * @return + * @throws Exception + */ + public static String get_topology_id(StormClusterState zkCluster, + String storm_name) throws Exception { + List active_storms = zkCluster.active_storms(); + String rtn = null; + if (active_storms != null) { + for (String topology_id : active_storms) { + + if (topology_id.indexOf(storm_name) < 0) { + continue; + } + + String zkTopologyName = Common.topologyIdToName(topology_id); + if (storm_name.endsWith(zkTopologyName)) { + return topology_id; + } + + } + } + return rtn; + } + + /** + * get all topology's StormBase + * + * @param zkCluster + * @return + * @throws Exception + */ + public static HashMap get_all_StormBase( + StormClusterState zkCluster) throws Exception { + HashMap rtn = new HashMap(); + List active_storms = zkCluster.active_storms(); + if (active_storms != null) { + for (String topology_id : active_storms) { + StormBase base = zkCluster.storm_base(topology_id, null); + if (base != null) { + rtn.put(topology_id, base); + } + } + } + return rtn; + } + + /** + * get all SupervisorInfo of storm cluster + * + * @param stormClusterState + * @param callback + * @return Map String: supervisorId SupervisorInfo: + * [time-secs hostname worker-ports uptime-secs] + * @throws Exception + */ + public static Map get_all_SupervisorInfo( + StormClusterState stormClusterState, RunnableCallback callback) + throws Exception { + + Map rtn = new TreeMap(); + // get /ZK/supervisors + List supervisorIds = stormClusterState.supervisors(callback); + if (supervisorIds != null) { + for (Iterator iter = supervisorIds.iterator(); iter + .hasNext();) { + + String supervisorId = iter.next(); + // get /supervisors/supervisorid + SupervisorInfo supervisorInfo = + stormClusterState.supervisor_info(supervisorId); + if (supervisorInfo == null) { + LOG.warn("Failed to get SupervisorInfo of " + supervisorId); + } else { + + rtn.put(supervisorId, supervisorInfo); + } + } + } else { + LOG.info("No alive supervisor"); + } + + return rtn; + } + + public static Map get_all_assignment( + StormClusterState stormClusterState, RunnableCallback callback) + throws Exception { + Map ret = new HashMap(); + + // get /assignments {topology_id} + List assignments = stormClusterState.assignments(callback); + if (assignments == null) { + LOG.debug("No assignment of ZK"); + return ret; + } + + for (String topology_id : assignments) { + + Assignment assignment = + stormClusterState.assignment_info(topology_id, callback); + + if (assignment == null) { + LOG.error("Failed to get Assignment of " + topology_id + + " from ZK"); + continue; + } + + ret.put(topology_id, assignment); + } + + return ret; + } + + public static Map get_all_nimbus_slave( + StormClusterState stormClusterState) throws Exception { + List hosts = stormClusterState.get_nimbus_slaves(); + if (hosts == null || hosts.size() == 0) { + return null; + } + + Map ret = new HashMap(); + for (String host : hosts) { + String time = stormClusterState.get_nimbus_slave_time(host); + ret.put(host, time); + } + + return ret; + } + + public static String get_supervisor_hostname( + StormClusterState stormClusterState, String supervisorId) + throws Exception { + SupervisorInfo supervisorInfo = + stormClusterState.supervisor_info(supervisorId); + if (supervisorInfo == null) { + return null; + } else { + return supervisorInfo.getHostName(); + } + } + + public static boolean is_topology_exist_error( + StormClusterState stormClusterState, String topologyId) + throws Exception { + + Map lastErrMap = + stormClusterState.topo_lastErr_time(topologyId); + if (lastErrMap == null || lastErrMap.size() == 0) { + return false; + } + + int now = TimeUtils.current_time_secs(); + for (Entry entry : lastErrMap.entrySet()) { + Integer timeout = entry.getKey(); + Integer timestamp = Integer.valueOf(entry.getValue()); + + if (now - timestamp <= timeout) { + return true; + } + } + + return false; + } + + public static Map> get_all_task_errors( + StormClusterState stormClusterState, String topologyId) { + Map> ret = new HashMap>(); + try { + List errorTasks = stormClusterState.task_error_ids(topologyId); + if (errorTasks == null || errorTasks.size() == 0) { + return ret; + } + + for (String taskIdStr : errorTasks) { + Integer taskId = -1; + try { + taskId = Integer.valueOf(taskIdStr); + }catch(Exception e) { + // skip last_error + continue; + } + + List taskErrorList = stormClusterState.task_errors(topologyId, taskId); + ret.put(taskId, taskErrorList); + } + return ret; + } catch (Exception e) { + // TODO Auto-generated catch block + return ret; + } + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/ClusterState.java b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/ClusterState.java new file mode 100755 index 000000000..8cba0732b --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/ClusterState.java @@ -0,0 +1,56 @@ +/** + * 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.cluster; + +import java.util.List; +import java.util.UUID; + +import com.alibaba.jstorm.callback.ClusterStateCallback; + +/** + * All ZK interface + * + * @author yannian + * + */ +public interface ClusterState { + public void set_ephemeral_node(String path, byte[] data) throws Exception; + + public void delete_node(String path) throws Exception; + + public void set_data(String path, byte[] data) throws Exception; + + public byte[] get_data(String path, boolean watch) throws Exception; + + public byte[] get_data_sync(String path, boolean watch) throws Exception; + + public List get_children(String path, boolean watch) + throws Exception; + + public void mkdirs(String path) throws Exception; + + public void tryToBeLeader(String path, byte[] host) throws Exception; + + public void close(); + + public UUID register(ClusterStateCallback callback); + + public ClusterStateCallback unregister(UUID id); + + public boolean node_existed(String path, boolean watch) throws Exception; +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/Common.java b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/Common.java new file mode 100644 index 000000000..a9e3e0b73 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/Common.java @@ -0,0 +1,845 @@ +/** + * 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.cluster; + +import java.io.IOException; +import java.net.URLClassLoader; +import java.security.InvalidParameterException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.Map.Entry; +import java.util.Set; + +import org.apache.log4j.Logger; + +import backtype.storm.Config; +import backtype.storm.Constants; +import backtype.storm.generated.Bolt; +import backtype.storm.generated.ComponentCommon; +import backtype.storm.generated.ComponentObject; +import backtype.storm.generated.GlobalStreamId; +import backtype.storm.generated.Grouping; +import backtype.storm.generated.InvalidTopologyException; +import backtype.storm.generated.JavaObject; +import backtype.storm.generated.ShellComponent; +import backtype.storm.generated.SpoutSpec; +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; +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Fields; +import backtype.storm.utils.ThriftTopologyUtils; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.daemon.worker.WorkerData; +import com.alibaba.jstorm.schedule.default_assign.DefaultTopologyAssignContext; +import com.alibaba.jstorm.task.Task; +import com.alibaba.jstorm.task.TaskInfo; +import com.alibaba.jstorm.task.acker.Acker; +import com.alibaba.jstorm.task.group.MkGrouper; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.Thrift; +import com.alibaba.jstorm.utils.TimeUtils; +import com.google.common.collect.Maps; + +/** + * Base utility function + * + * 1. base topology validation 2. add streams/inputs + * + * @author yannian/Longda + * + */ +public class Common { + private final static Logger LOG = Logger.getLogger(Common.class); + + public static final String ACKER_COMPONENT_ID = Acker.ACKER_COMPONENT_ID; + public static final String ACKER_INIT_STREAM_ID = + Acker.ACKER_INIT_STREAM_ID; + public static final String ACKER_ACK_STREAM_ID = Acker.ACKER_ACK_STREAM_ID; + public static final String ACKER_FAIL_STREAM_ID = + Acker.ACKER_FAIL_STREAM_ID; + + public static final String SYSTEM_STREAM_ID = "__system"; + + public static final String LS_WORKER_HEARTBEAT = "worker-heartbeat"; + public static final String LS_ID = "supervisor-id"; + public static final String LS_LOCAL_ASSIGNMENTS = "local-assignments"; + public static final String LS_APPROVED_WORKERS = "approved-workers"; + public static final String LS_TASK_CLEANUP_TIMEOUT = "task-cleanup-timeout"; + + public static final String compErrorInfo = + "ID can only contains a-z, A-Z, 0-9, '-', '_', '.', '$', and should not start with \"__\"."; + public static final String nameErrorInfo = + "Name can only contains a-z, A-Z, 0-9, '-', '_', '.'"; + + public static boolean system_id(String id) { + return Utils.isSystemId(id); + } + + private static void validate_component(Object obj) + throws InvalidTopologyException { + + if (obj instanceof StateSpoutSpec) { + StateSpoutSpec spec = (StateSpoutSpec) obj; + for (String id : spec.get_common().get_streams().keySet()) { + if (system_id(id) || !charComponentValidate(id)) { + throw new InvalidTopologyException(id + + " is not a valid component id. " + compErrorInfo); + } + } + + } else if (obj instanceof SpoutSpec) { + SpoutSpec spec = (SpoutSpec) obj; + for (String id : spec.get_common().get_streams().keySet()) { + if (system_id(id) || !charComponentValidate(id)) { + throw new InvalidTopologyException(id + + " is not a valid component id. " + compErrorInfo); + } + } + } else if (obj instanceof Bolt) { + Bolt spec = (Bolt) obj; + for (String id : spec.get_common().get_streams().keySet()) { + if (system_id(id) || !charComponentValidate(id)) { + throw new InvalidTopologyException(id + + " is not a valid component id. " + compErrorInfo); + } + } + } else { + throw new InvalidTopologyException("Unknow type component"); + } + + } + + public static String topologyNameToId(String topologyName, int counter) { + return topologyName + "-" + counter + "-" + + TimeUtils.current_time_secs(); + } + + public static String getTopologyNameById(String topologyId) { + String topologyName = null; + try { + topologyName = topologyIdToName(topologyId); + } catch (InvalidTopologyException e) { + LOG.error("Invalid topologyId=" + topologyId); + } + return topologyName; + } + + /** + * Convert topologyId to topologyName. TopologyId = + * topoloygName-counter-timeStamp + * + * @param topologyId + * @return + */ + public static String topologyIdToName(String topologyId) + throws InvalidTopologyException { + String ret = null; + int index = topologyId.lastIndexOf('-'); + if (index != -1 && index > 2) { + index = topologyId.lastIndexOf('-', index - 1); + if (index != -1 && index > 0) + ret = topologyId.substring(0, index); + else + throw new InvalidTopologyException(topologyId + + " is not a valid topologyId"); + } else + throw new InvalidTopologyException(topologyId + + " is not a valid topologyId"); + return ret; + } + + /** + * Validation of topology name chars. Only alpha char, number, '-', '_', '.' + * are valid. + * + * @return + */ + public static boolean charValidate(String name) { + return name.matches("[a-zA-Z0-9-_.]+"); + } + + /** + * Validation of topology component chars. Only alpha char, number, '-', + * '_', '.', '$' are valid. + * + * @return + */ + public static boolean charComponentValidate(String name) { + return name.matches("[a-zA-Z0-9-_/.$]+"); + } + + /** + * Check Whether ID of Bolt or spout is system_id + * + * @param topology + * @throws InvalidTopologyException + */ + @SuppressWarnings("unchecked") + public static void validate_ids(StormTopology topology, String topologyId) + throws InvalidTopologyException { + String topologyName = topologyIdToName(topologyId); + if (!charValidate(topologyName)) { + throw new InvalidTopologyException(topologyName + + " is not a valid topology name. " + nameErrorInfo); + } + + List list = new ArrayList(); + + for (StormTopology._Fields field : Thrift.STORM_TOPOLOGY_FIELDS) { + Object value = topology.getFieldValue(field); + if (value != null) { + Map obj_map = (Map) value; + + Set commids = obj_map.keySet(); + + for (String id : commids) { + if (system_id(id) || !charComponentValidate(id)) { + throw new InvalidTopologyException(id + + " is not a valid component id. " + + compErrorInfo); + } + } + + for (Object obj : obj_map.values()) { + validate_component(obj); + } + + list.addAll(commids); + } + } + + List offending = JStormUtils.getRepeat(list); + if (offending.isEmpty() == false) { + throw new InvalidTopologyException("Duplicate component ids: " + + offending); + } + + } + + private static void validate_component_inputs(Object obj) + throws InvalidTopologyException { + if (obj instanceof StateSpoutSpec) { + StateSpoutSpec spec = (StateSpoutSpec) obj; + if (!spec.get_common().get_inputs().isEmpty()) { + throw new InvalidTopologyException( + "May not declare inputs for a spout"); + } + + } + + if (obj instanceof SpoutSpec) { + SpoutSpec spec = (SpoutSpec) obj; + if (!spec.get_common().get_inputs().isEmpty()) { + throw new InvalidTopologyException( + "May not declare inputs for a spout"); + } + } + } + + /** + * Validate the topology 1. component id name is valid or not 2. check some + * spout's input is empty or not + * + * @param topology + * @throws InvalidTopologyException + */ + public static void validate_basic(StormTopology topology, + Map totalStormConf, String topologyid) + throws InvalidTopologyException { + validate_ids(topology, topologyid); + + for (StormTopology._Fields field : Thrift.SPOUT_FIELDS) { + Object value = topology.getFieldValue(field); + if (value != null) { + Map obj_map = (Map) value; + for (Object obj : obj_map.values()) { + validate_component_inputs(obj); + } + } + + } + + Integer workerNum = + JStormUtils.parseInt(totalStormConf + .get(Config.TOPOLOGY_WORKERS)); + if (workerNum == null || workerNum <= 0) { + String errMsg = + "There are no Config.TOPOLOGY_WORKERS in configuration of " + + topologyid; + throw new InvalidParameterException(errMsg); + } + + Integer ackerNum = + JStormUtils.parseInt(totalStormConf + .get(Config.TOPOLOGY_ACKER_EXECUTORS)); + if (ackerNum != null && ackerNum < 0) { + String errMsg = + "Invalide Config.TOPOLOGY_ACKERS in configuration of " + + topologyid; + throw new InvalidParameterException(errMsg); + } + + } + + /** + * Generate acker's input Map + * + * for spout for bolt + * + * + * + * @param topology + * @return + */ + public static Map acker_inputs( + StormTopology topology) { + Map spout_inputs = + new HashMap(); + Map spout_ids = topology.get_spouts(); + for (Entry spout : spout_ids.entrySet()) { + String id = spout.getKey(); + + GlobalStreamId stream = + new GlobalStreamId(id, ACKER_INIT_STREAM_ID); + + Grouping group = Thrift.mkFieldsGrouping(JStormUtils.mk_list("id")); + + spout_inputs.put(stream, group); + } + + Map bolt_ids = topology.get_bolts(); + Map bolt_inputs = + new HashMap(); + for (Entry bolt : bolt_ids.entrySet()) { + String id = bolt.getKey(); + + GlobalStreamId streamAck = + new GlobalStreamId(id, ACKER_ACK_STREAM_ID); + Grouping groupAck = + Thrift.mkFieldsGrouping(JStormUtils.mk_list("id")); + + GlobalStreamId streamFail = + new GlobalStreamId(id, ACKER_FAIL_STREAM_ID); + Grouping groupFail = + Thrift.mkFieldsGrouping(JStormUtils.mk_list("id")); + + bolt_inputs.put(streamAck, groupAck); + bolt_inputs.put(streamFail, groupFail); + } + + Map allInputs = + new HashMap(); + allInputs.putAll(bolt_inputs); + allInputs.putAll(spout_inputs); + return allInputs; + } + + /** + * Add acker bolt to topology + * + * @param num_tasks + * @param 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(); + fields.add("id"); + + outputs.put(ACKER_ACK_STREAM_ID, Thrift.directOutputFields(fields)); + outputs.put(ACKER_FAIL_STREAM_ID, Thrift.directOutputFields(fields)); + + IBolt ackerbolt = new Acker(); + + // generate inputs + Map inputs = acker_inputs(ret); + + // generate acker which will be stored in topology + Bolt acker_bolt = Thrift.mkBolt(inputs, ackerbolt, outputs, ackerNum); + + // add every bolt two output stream + // ACKER_ACK_STREAM_ID/ACKER_FAIL_STREAM_ID + for (Entry e : ret.get_bolts().entrySet()) { + + Bolt bolt = e.getValue(); + + ComponentCommon common = bolt.get_common(); + + List ackList = JStormUtils.mk_list("id", "ack-val"); + + common.put_to_streams(ACKER_ACK_STREAM_ID, + Thrift.outputFields(ackList)); + + List failList = JStormUtils.mk_list("id"); + common.put_to_streams(ACKER_FAIL_STREAM_ID, + Thrift.outputFields(failList)); + + bolt.set_common(common); + } + + // add every spout output stream ACKER_INIT_STREAM_ID + // add every spout two intput source + // ((ACKER_COMPONENT_ID, ACKER_ACK_STREAM_ID), directGrouping) + // ((ACKER_COMPONENT_ID, ACKER_FAIL_STREAM_ID), directGrouping) + for (Entry kv : ret.get_spouts().entrySet()) { + SpoutSpec bolt = kv.getValue(); + ComponentCommon common = bolt.get_common(); + List initList = + JStormUtils.mk_list("id", "init-val", "spout-task"); + common.put_to_streams(ACKER_INIT_STREAM_ID, + Thrift.outputFields(initList)); + + GlobalStreamId ack_ack = + new GlobalStreamId(ACKER_COMPONENT_ID, ACKER_ACK_STREAM_ID); + common.put_to_inputs(ack_ack, Thrift.mkDirectGrouping()); + + GlobalStreamId ack_fail = + new GlobalStreamId(ACKER_COMPONENT_ID, ACKER_FAIL_STREAM_ID); + common.put_to_inputs(ack_fail, Thrift.mkDirectGrouping()); + } + + ret.put_to_bolts(ACKER_COMPONENT_ID, acker_bolt); + } + + public static List all_components(StormTopology topology) { + List rtn = new ArrayList(); + for (StormTopology._Fields field : Thrift.STORM_TOPOLOGY_FIELDS) { + Object fields = topology.getFieldValue(field); + if (fields != null) { + rtn.addAll(((Map) fields).values()); + } + } + return rtn; + } + + private static List sysEventFields = JStormUtils.mk_list("event"); + + private static void add_component_system_streams(Object obj) { + ComponentCommon common = null; + if (obj instanceof StateSpoutSpec) { + StateSpoutSpec spec = (StateSpoutSpec) obj; + common = spec.get_common(); + } + + if (obj instanceof SpoutSpec) { + SpoutSpec spec = (SpoutSpec) obj; + common = spec.get_common(); + } + + if (obj instanceof Bolt) { + Bolt spec = (Bolt) obj; + common = spec.get_common(); + } + + if (common != null) { + StreamInfo sinfo = Thrift.outputFields(sysEventFields); + common.put_to_streams(SYSTEM_STREAM_ID, sinfo); + } + } + + /** + * Add every bolt or spout one output stream + * + * @param topology + */ + public static void add_system_streams(StormTopology topology) { + for (Object obj : all_components(topology)) { + add_component_system_streams(obj); + } + } + + 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(); + + add_acker(storm_conf, ret); + + add_metrics_component(ret); + + add_system_components(ret); + + return ret; + } + + /** + * get component configuration + * + * @param storm_conf + * @param topology_context + * @param component_id + * @return + */ + @SuppressWarnings("unchecked") + public static Map component_conf(Map storm_conf, + TopologyContext topology_context, String component_id) { + List to_remove = StormConfig.All_CONFIGS(); + to_remove.remove(Config.TOPOLOGY_DEBUG); + to_remove.remove(Config.TOPOLOGY_MAX_SPOUT_PENDING); + to_remove.remove(Config.TOPOLOGY_MAX_TASK_PARALLELISM); + to_remove.remove(Config.TOPOLOGY_TRANSACTIONAL_ID); + + Map componentConf = new HashMap(); + + String jconf = + topology_context.getComponentCommon(component_id) + .get_json_conf(); + if (jconf != null) { + componentConf = (Map) JStormUtils.from_json(jconf); + } + + /** + * @@@ Don't know why need remove system configuration from component + * conf? // + */ + // for (Object p : to_remove) { + // componentConf.remove(p); + // } + + Map ret = new HashMap(); + ret.putAll(storm_conf); + ret.putAll(componentConf); + + return ret; + } + + /** + * get object of component_id + * + * @param topology + * @param component_id + * @return + */ + public static Object get_task_object(StormTopology topology, + String component_id, URLClassLoader loader) { + Map spouts = topology.get_spouts(); + Map bolts = topology.get_bolts(); + Map state_spouts = topology.get_state_spouts(); + + ComponentObject obj = null; + if (spouts.containsKey(component_id)) { + obj = spouts.get(component_id).get_spout_object(); + } else if (bolts.containsKey(component_id)) { + obj = bolts.get(component_id).get_bolt_object(); + } else if (state_spouts.containsKey(component_id)) { + obj = state_spouts.get(component_id).get_state_spout_object(); + } + + if (obj == null) { + throw new RuntimeException("Could not find " + component_id + + " in " + topology.toString()); + } + + Object componentObject = Utils.getSetComponentObject(obj, loader); + + Object rtn = null; + + if (componentObject instanceof JavaObject) { + rtn = Thrift.instantiateJavaObject((JavaObject) componentObject); + } else if (componentObject instanceof ShellComponent) { + if (spouts.containsKey(component_id)) { + rtn = new ShellSpout((ShellComponent) componentObject); + } else { + rtn = new ShellBolt((ShellComponent) componentObject); + } + } else { + rtn = componentObject; + } + return rtn; + + } + + /** + * get current task's output > + * + * @param topology_context + * @return + */ + public static Map> outbound_components( + TopologyContext topology_context, WorkerData workerData) { + Map> rr = + new HashMap>(); + + // > + Map> output_groupings = + topology_context.getThisTargets(); + + for (Entry> entry : output_groupings + .entrySet()) { + + String stream_id = entry.getKey(); + Map component_grouping = entry.getValue(); + + Fields out_fields = topology_context.getThisOutputFields(stream_id); + + Map componentGrouper = + new HashMap(); + + for (Entry cg : component_grouping.entrySet()) { + + String component = cg.getKey(); + Grouping tgrouping = cg.getValue(); + + List outTasks = + topology_context.getComponentTasks(component); + // ATTENTION: If topology set one component parallelism as 0 + // so we don't need send tuple to it + if (outTasks.size() > 0) { + MkGrouper grouper = + new MkGrouper(topology_context, out_fields, + tgrouping, outTasks, stream_id, workerData); + componentGrouper.put(component, grouper); + } + LOG.info("outbound_components, outTasks=" + outTasks + + " for task-" + topology_context.getThisTaskId()); + } + if (componentGrouper.size() > 0) { + rr.put(stream_id, componentGrouper); + } + } + return rr; + } + + /** + * get the component's configuration + * + * @param topology_context + * @param task_id + * @return component's configurations + */ + public static Map getComponentMap(DefaultTopologyAssignContext context, + Integer task) { + String componentName = context.getTaskToComponent().get(task); + ComponentCommon componentCommon = + ThriftTopologyUtils.getComponentCommon( + context.getSysTopology(), componentName); + + Map componentMap = + (Map) JStormUtils.from_json(componentCommon.get_json_conf()); + if (componentMap == null) { + componentMap = Maps.newHashMap(); + } + return componentMap; + } + + /** + * get all bolts' inputs and spouts' outputs > + * > + * + * @param topology_context + * @return all bolts' inputs and spouts' outputs + */ + public static Map> buildSpoutOutoputAndBoltInputMap( + DefaultTopologyAssignContext context) { + Set bolts = context.getRawTopology().get_bolts().keySet(); + Set spouts = context.getRawTopology().get_spouts().keySet(); + Map> relationship = + new HashMap>(); + for (Entry entry : context.getRawTopology().get_bolts() + .entrySet()) { + Map inputs = + entry.getValue().get_common().get_inputs(); + Set input = new HashSet(); + relationship.put(entry.getKey(), input); + for (Entry inEntry : inputs.entrySet()) { + String component = inEntry.getKey().get_componentId(); + input.add(component); + if (!bolts.contains(component)) { + // spout + Set spoutOutput = relationship.get(component); + if (spoutOutput == null) { + spoutOutput = new HashSet(); + relationship.put(component, spoutOutput); + } + spoutOutput.add(entry.getKey()); + } + } + } + for (String spout : spouts) { + if (relationship.get(spout) == null) + relationship.put(spout, new HashSet()); + } + for (String bolt : bolts) { + if (relationship.get(bolt) == null) + relationship.put(bolt, new HashSet()); + } + return relationship; + } + + public static Map getTaskToComponent(Map taskInfoMap) { + Map ret = new TreeMap(); + for (Entry entry :taskInfoMap.entrySet()) { + ret.put(entry.getKey(), entry.getValue().getComponentId()); + } + + return ret; + } + + public static Map getTaskToType(Map taskInfoMap) { + Map ret = new TreeMap(); + for (Entry entry :taskInfoMap.entrySet()) { + ret.put(entry.getKey(), entry.getValue().getComponentType()); + } + + return ret; + } + + @SuppressWarnings({"rawtypes", "unchecked"}) + public static Integer mkTaskMaker(Map stormConf, + Map cidSpec, + Map rtn, + Integer cnt) { + if (cidSpec == null) { + LOG.warn("Component map is empty"); + return cnt; + } + + Set entrySet = cidSpec.entrySet(); + for (Iterator it = entrySet.iterator(); it.hasNext();) { + Entry entry = (Entry) it.next(); + Object obj = entry.getValue(); + + ComponentCommon common = null; + String componentType = "bolt"; + if (obj instanceof Bolt) { + common = ((Bolt) obj).get_common(); + componentType = "bolt"; + } else if (obj instanceof SpoutSpec) { + common = ((SpoutSpec) obj).get_common(); + componentType = "spout"; + } else if (obj instanceof StateSpoutSpec) { + common = ((StateSpoutSpec) obj).get_common(); + componentType = "spout"; + } + + if (common == null) { + throw new RuntimeException("No ComponentCommon of " + entry.getKey()); + } + + int declared = Thrift.parallelismHint(common); + Integer parallelism = declared; + // Map tmp = (Map) Utils_clj.from_json(common.get_json_conf()); + + Map newStormConf = new HashMap(stormConf); + // newStormConf.putAll(tmp); + Integer maxParallelism = JStormUtils.parseInt(newStormConf.get(Config.TOPOLOGY_MAX_TASK_PARALLELISM)); + if (maxParallelism != null) { + parallelism = Math.min(maxParallelism, declared); + } + + for (int i = 0; i < parallelism; i++) { + cnt++; + TaskInfo taskInfo = new TaskInfo((String) entry.getKey(), componentType); + rtn.put(cnt, taskInfo); + } + } + return cnt; + } + + public static Map mkTaskInfo( + Map stormConf, + StormTopology sysTopology, + String topologyid) { + + // use TreeMap to make task as sequence + Map rtn = new TreeMap(); + + Integer count = 0; + count = mkTaskMaker(stormConf, sysTopology.get_bolts(), rtn, count); + count = mkTaskMaker(stormConf, sysTopology.get_spouts(), rtn, count); + count = mkTaskMaker(stormConf, sysTopology.get_state_spouts(), rtn, count); + + return rtn; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/DaemonCommon.java b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/DaemonCommon.java new file mode 100755 index 000000000..86e343248 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/DaemonCommon.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.cluster; + +public interface DaemonCommon { + public boolean waiting(); +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/DistributedClusterState.java b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/DistributedClusterState.java new file mode 100755 index 000000000..2ebce8384 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/DistributedClusterState.java @@ -0,0 +1,242 @@ +/** + * 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.cluster; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.Watcher.Event.EventType; +import org.apache.zookeeper.Watcher.Event.KeeperState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; + +import com.alibaba.jstorm.cache.JStormCache; +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; + +/** + * All ZK interface implementation + * + * @author yannian.mu + * + */ +public class DistributedClusterState implements ClusterState { + + private static Logger LOG = LoggerFactory + .getLogger(DistributedClusterState.class); + + private Zookeeper zkobj = new Zookeeper(); + private CuratorFramework zk; + private WatcherCallBack watcher; + + /** + * why run all callbacks, when receive one event + */ + private ConcurrentHashMap callbacks = + new ConcurrentHashMap(); + + private Map conf; + private AtomicBoolean active; + + private JStormCache zkCache; + + public DistributedClusterState(Map _conf) throws Exception { + conf = _conf; + + // just mkdir STORM_ZOOKEEPER_ROOT dir + CuratorFramework _zk = mkZk(); + String path = String.valueOf(conf.get(Config.STORM_ZOOKEEPER_ROOT)); + zkobj.mkdirs(_zk, path); + _zk.close(); + + active = new AtomicBoolean(true); + + watcher = new WatcherCallBack() { + @Override + public void execute(KeeperState state, EventType type, String path) { + if (active.get()) { + if (!(state.equals(KeeperState.SyncConnected))) { + LOG.warn("Received event " + state + ":" + type + ":" + + path + " with disconnected Zookeeper."); + } else { + LOG.info("Received event " + state + ":" + type + ":" + + path); + } + + if (!type.equals(EventType.None)) { + for (Entry e : callbacks + .entrySet()) { + ClusterStateCallback fn = e.getValue(); + fn.execute(type, path); + } + } + } + } + }; + zk = null; + zk = mkZk(watcher); + + } + + @SuppressWarnings("unchecked") + private CuratorFramework mkZk() throws IOException { + return zkobj.mkClient(conf, + (List) conf.get(Config.STORM_ZOOKEEPER_SERVERS), + conf.get(Config.STORM_ZOOKEEPER_PORT), ""); + } + + @SuppressWarnings("unchecked") + private CuratorFramework mkZk(WatcherCallBack watcher) + throws NumberFormatException, IOException { + return zkobj.mkClient(conf, + (List) conf.get(Config.STORM_ZOOKEEPER_SERVERS), + conf.get(Config.STORM_ZOOKEEPER_PORT), + String.valueOf(conf.get(Config.STORM_ZOOKEEPER_ROOT)), watcher); + } + + @Override + public void close() { + this.active.set(false); + zk.close(); + } + + @Override + public void delete_node(String path) throws Exception { + if (zkCache != null) { + zkCache.remove(path); + } + zkobj.deletereRcursive(zk, path); + } + + @Override + public List get_children(String path, boolean watch) + throws Exception { + return zkobj.getChildren(zk, path, watch); + } + + @Override + public byte[] get_data(String path, boolean watch) throws Exception { + byte[] ret = null; + if (watch == false && zkCache != null) { + ret = (byte[]) zkCache.get(path); + } + if (ret != null) { + return ret; + } + + ret = zkobj.getData(zk, path, watch); + if (zkCache != null) { + zkCache.put(path, ret); + } + + return ret; + } + + @Override + public byte[] get_data_sync(String path, boolean watch) throws Exception { + byte[] ret = null; + ret = zkobj.getData(zk, path, watch); + if (zkCache != null && ret != null) { + zkCache.put(path, ret); + } + return ret; + } + + @Override + public void mkdirs(String path) throws Exception { + zkobj.mkdirs(zk, path); + + } + + @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!, data size is " + data.length); + } + if (zkobj.exists(zk, path, false)) { + zkobj.setData(zk, path, data); + } else { + zkobj.mkdirs(zk, PathUtils.parent_path(path)); + zkobj.createNode(zk, path, data, CreateMode.PERSISTENT); + } + + if (zkCache != null) { + zkCache.put(path, data); + } + + } + + @Override + public void set_ephemeral_node(String path, byte[] data) throws Exception { + zkobj.mkdirs(zk, PathUtils.parent_path(path)); + if (zkobj.exists(zk, path, false)) { + zkobj.setData(zk, path, data); + } else { + zkobj.createNode(zk, path, data, CreateMode.EPHEMERAL); + } + + if (zkCache != null) { + zkCache.put(path, data); + } + } + + @Override + public UUID register(ClusterStateCallback callback) { + UUID id = UUID.randomUUID(); + this.callbacks.put(id, callback); + return id; + } + + @Override + public ClusterStateCallback unregister(UUID id) { + return this.callbacks.remove(id); + } + + @Override + public boolean node_existed(String path, boolean watch) throws Exception { + // TODO Auto-generated method stub + return zkobj.existsNode(zk, path, watch); + } + + @Override + public void tryToBeLeader(String path, byte[] host) throws Exception { + // TODO Auto-generated method stub + zkobj.createNode(zk, path, host, CreateMode.EPHEMERAL); + } + + public JStormCache getZkCache() { + return zkCache; + } + + public void setZkCache(JStormCache zkCache) { + this.zkCache = zkCache; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/StormBase.java b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/StormBase.java new file mode 100755 index 000000000..e6438ddba --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/StormBase.java @@ -0,0 +1,144 @@ +/** + * 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.cluster; + +import java.io.Serializable; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; + +import com.alibaba.jstorm.daemon.nimbus.StatusType; + +/** + * Topology stored in ZK + */ + +public class StormBase implements Serializable { + + private static final long serialVersionUID = -3013095336395395213L; + private String stormName; + private int lanchTimeSecs; + private StormStatus status; + private boolean enableMonitor = true; + private String group; + + public StormBase(String stormName, int lanchTimeSecs, StormStatus status, + String group) { + this.stormName = stormName; + this.lanchTimeSecs = lanchTimeSecs; + this.status = status; + this.setGroup(group); + } + + public String getStormName() { + return stormName; + } + + public void setStormName(String stormName) { + this.stormName = stormName; + } + + public int getLanchTimeSecs() { + return lanchTimeSecs; + } + + public void setLanchTimeSecs(int lanchTimeSecs) { + this.lanchTimeSecs = lanchTimeSecs; + } + + public StormStatus getStatus() { + return status; + } + + public void setStatus(StormStatus status) { + this.status = status; + } + + public String getStatusString() { + StatusType t = status.getStatusType(); + return t.getStatus().toUpperCase(); + } + + public String getGroup() { + return group; + } + + public void setGroup(String group) { + this.group = group; + } + + public boolean isEnableMonitor() { + return enableMonitor; + } + + public void setEnableMonitor(boolean enableMonitor) { + this.enableMonitor = enableMonitor; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + (enableMonitor ? 1231 : 1237); + result = prime * result + ((group == null) ? 0 : group.hashCode()); + result = prime * result + lanchTimeSecs; + result = prime * result + ((status == null) ? 0 : status.hashCode()); + result = + prime * result + + ((stormName == null) ? 0 : stormName.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + StormBase other = (StormBase) obj; + if (enableMonitor != other.enableMonitor) + return false; + if (group == null) { + if (other.group != null) + return false; + } else if (!group.equals(other.group)) + return false; + if (lanchTimeSecs != other.lanchTimeSecs) + return false; + if (status == null) { + if (other.status != null) + return false; + } else if (!status.equals(other.status)) + return false; + if (stormName == null) { + if (other.stormName != null) + return false; + } else if (!stormName.equals(other.stormName)) + return false; + return true; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/StormClusterState.java b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/StormClusterState.java new file mode 100644 index 000000000..6486d5e51 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/StormClusterState.java @@ -0,0 +1,162 @@ +/** + * 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.cluster; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; +import com.alibaba.jstorm.schedule.Assignment; +import com.alibaba.jstorm.schedule.AssignmentBak; +import com.alibaba.jstorm.task.TaskInfo; +import com.alibaba.jstorm.task.error.TaskError; +import com.alibaba.jstorm.task.heartbeat.TaskHeartbeat; + +/** + * all storm in zk operation interface + */ +public interface StormClusterState { + public void disconnect() throws Exception; + + public void remove_storm(String topology_id) throws Exception; + + public void try_remove_storm(String topology_id); + + public List assignments(RunnableCallback callback) throws Exception; + + public Assignment assignment_info(String topology_id, + RunnableCallback callback) throws Exception; + + public void set_assignment(String topology_id, Assignment info) + throws Exception; + + public AssignmentBak assignment_bak(String topologyName) throws Exception; + + public void backup_assignment(String topology_id, AssignmentBak info) + throws Exception; + + public List active_storms() throws Exception; + + public StormBase storm_base(String topology_id, RunnableCallback callback) + throws Exception; + + public void activate_storm(String topology_id, StormBase storm_base) + throws Exception; + + public void update_storm(String topology_id, StormStatus new_elems) + throws Exception; + + public void set_storm_monitor(String topologyId, boolean isEnable) + throws Exception; + + public void remove_storm_base(String topology_id) throws Exception; + + public List task_storms() throws Exception; + + public Set task_ids(String topology_id) throws Exception; + + public Set task_ids_by_componentId(String topologyId, + String componentId) throws Exception; + + public void set_task(String topologyId, Map taskInfoMap) throws Exception; + public void add_task(String topology_id, Map taskInfoMap) + throws Exception; + + public void remove_task(String topologyId, Set taskIds) throws Exception; + + public Map task_all_info(String topology_id) throws Exception; + + public void setup_heartbeats(String topology_id) throws Exception; + + public List heartbeat_storms() throws Exception; + + public List heartbeat_tasks(String topology_id) throws Exception; + + public TaskHeartbeat task_heartbeat(String topology_id, int task_id) + throws Exception; + + public void task_heartbeat(String topology_id, int task_id, + TaskHeartbeat info) throws Exception; + + public void teardown_heartbeats(String topology_id) throws Exception; + + public void remove_task_heartbeat(String topology_id, int task_id) + throws Exception; + + public List task_error_storms() throws Exception; + + public List task_error_ids(String topologyId) throws Exception; + + public void report_task_error(String topology_id, int task_id, + Throwable error) throws Exception; + + public void report_task_error(String topology_id, int task_id, String error) + throws Exception; + + public Map topo_lastErr_time(String topologyId) + throws Exception; + + public void remove_lastErr_time(String topologyId) throws Exception; + + public List task_errors(String topology_id, int task_id) + throws Exception; + + public void remove_task_error(String topologyId, int taskId) + 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; + + public void teardown_task_errors(String topology_id) throws Exception; + + public List supervisors(RunnableCallback callback) throws Exception; + + public SupervisorInfo supervisor_info(String supervisor_id) + throws Exception; + + public void supervisor_heartbeat(String supervisor_id, SupervisorInfo info) + throws Exception; + + public boolean try_to_be_leader(String path, String host, + RunnableCallback callback) throws Exception; + + public String get_leader_host() throws Exception; + + public boolean leader_existed() throws Exception; + + public List get_nimbus_slaves() throws Exception; + + public void update_nimbus_slave(String host, int time) throws Exception; + + public String get_nimbus_slave_time(String host) throws Exception; + + public void unregister_nimbus_host(String host) throws Exception; + + public void set_topology_metric(String topologyId, Object metric) + throws Exception; + + public Object get_topology_metric(String topologyId) throws Exception; + + public List get_metrics() throws Exception; + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/StormConfig.java b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/StormConfig.java new file mode 100755 index 000000000..3d1cd2925 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/StormConfig.java @@ -0,0 +1,549 @@ +/** + * 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.cluster; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.commons.io.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.generated.StormTopology; +import backtype.storm.utils.LocalState; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.PathUtils; + +public class StormConfig { + private final static Logger LOG = LoggerFactory + .getLogger(StormConfig.class); + public final static String RESOURCES_SUBDIR = "resources"; + public final static String WORKER_DATA_SUBDIR = "worker_shared_data"; + + public static final String FILE_SEPERATEOR = File.separator; + + public static String clojureConfigName(String name) { + return name.toUpperCase().replace("_", "-"); + } + + public static Map read_storm_config() { + return Utils.readStormConfig(); + } + + public static Map read_yaml_config(String name) { + return Utils.findAndReadConfigFile(name, true); + } + + public static Map read_default_config() { + return Utils.readDefaultConfig(); + } + + public static List All_CONFIGS() { + List rtn = new ArrayList(); + Config config = new Config(); + Class ConfigClass = config.getClass(); + Field[] fields = ConfigClass.getFields(); + for (int i = 0; i < fields.length; i++) { + try { + Object obj = fields[i].get(null); + rtn.add(obj); + } catch (IllegalArgumentException e) { + LOG.error(e.getMessage(), e); + } catch (IllegalAccessException e) { + LOG.error(e.getMessage(), e); + } + } + return rtn; + } + + public static HashMap getClassFields(Class cls) + throws IllegalArgumentException, IllegalAccessException { + java.lang.reflect.Field[] list = cls.getDeclaredFields(); + HashMap rtn = new HashMap(); + for (java.lang.reflect.Field f : list) { + String name = f.getName(); + rtn.put(name, f.get(null).toString()); + + } + return rtn; + } + + public static String cluster_mode(Map conf) { + String mode = (String) conf.get(Config.STORM_CLUSTER_MODE); + return mode; + + } + + public static boolean local_mode(Map conf) { + String mode = (String) conf.get(Config.STORM_CLUSTER_MODE); + if (mode != null) { + if (mode.equals("local")) { + return true; + } + + if (mode.equals("distributed")) { + return false; + } + } + throw new IllegalArgumentException("Illegal cluster mode in conf:" + + mode); + + } + + /** + * validate whether the mode is distributed + * + * @param conf + */ + public static void validate_distributed_mode(Map conf) { + if (StormConfig.local_mode(conf)) { + throw new IllegalArgumentException( + "Cannot start server in local mode!"); + } + + } + + public static void validate_local_mode(Map conf) { + if (!StormConfig.local_mode(conf)) { + throw new IllegalArgumentException( + "Cannot start server in distributed mode!"); + } + + } + + public static String worker_root(Map conf) throws IOException { + String ret = + String.valueOf(conf.get(Config.STORM_LOCAL_DIR)) + + FILE_SEPERATEOR + "workers"; + FileUtils.forceMkdir(new File(ret)); + return ret; + } + + public static String worker_root(Map conf, String id) throws IOException { + String ret = worker_root(conf) + FILE_SEPERATEOR + id; + FileUtils.forceMkdir(new File(ret)); + return ret; + } + + public static String worker_pids_root(Map conf, String id) + throws IOException { + String ret = worker_root(conf, id) + FILE_SEPERATEOR + "pids"; + FileUtils.forceMkdir(new File(ret)); + return ret; + } + + public static String worker_pid_path(Map conf, String id, String pid) + throws IOException { + String ret = worker_pids_root(conf, id) + FILE_SEPERATEOR + pid; + return ret; + } + + public static String worker_heartbeats_root(Map conf, String id) + throws IOException { + String ret = worker_root(conf, id) + FILE_SEPERATEOR + "heartbeats"; + FileUtils.forceMkdir(new File(ret)); + return ret; + } + + public static String default_worker_shared_dir(Map conf) throws IOException { + String ret = + String.valueOf(conf.get(Config.STORM_LOCAL_DIR)) + + FILE_SEPERATEOR + WORKER_DATA_SUBDIR; + + FileUtils.forceMkdir(new File(ret)); + return ret; + } + + private static String supervisor_local_dir(Map conf) throws IOException { + String ret = + String.valueOf(conf.get(Config.STORM_LOCAL_DIR)) + + FILE_SEPERATEOR + "supervisor"; + FileUtils.forceMkdir(new File(ret)); + return ret; + } + + public static String supervisor_stormdist_root(Map conf) throws IOException { + String ret = stormdist_path(supervisor_local_dir(conf)); + FileUtils.forceMkdir(new File(ret)); + return ret; + } + + public static String supervisor_stormdist_root(Map conf, String topologyId) + throws IOException { + return supervisor_stormdist_root(conf) + FILE_SEPERATEOR + topologyId; + } + + /** + * Return supervisor's pid dir + * + * @param conf + * @return + * @throws IOException + */ + public static String supervisorPids(Map conf) throws IOException { + String ret = supervisor_local_dir(conf) + FILE_SEPERATEOR + "pids"; + try { + FileUtils.forceMkdir(new File(ret)); + } catch (IOException e) { + LOG.error("Failed to create dir " + ret, e); + throw e; + } + return ret; + } + + /** + * Return nimbus's heartbeat dir for apsara + * + * @param conf + * @return + * @throws IOException + */ + public static String supervisorHearbeatForContainer(Map conf) + throws IOException { + String ret = + supervisor_local_dir(conf) + FILE_SEPERATEOR + + "supervisor.heartbeat"; + try { + FileUtils.forceMkdir(new File(ret)); + } catch (IOException e) { + LOG.error("Failed to create dir " + ret, e); + throw e; + } + return ret; + } + + public static String stormjar_path(String stormroot) { + return stormroot + FILE_SEPERATEOR + "stormjar.jar"; + } + + public static String stormcode_path(String stormroot) { + return stormroot + FILE_SEPERATEOR + "stormcode.ser"; + } + + public static String stormconf_path(String stormroot) { + return stormroot + FILE_SEPERATEOR + "stormconf.ser"; + } + + public static String stormlib_path(String stormroot, String libname) { + return stormroot + FILE_SEPERATEOR + "lib" + FILE_SEPERATEOR + libname; + } + + public static String stormlib_path(String stormroot) { + return stormroot + FILE_SEPERATEOR + "lib"; + } + + public static String stormdist_path(String stormroot) { + return stormroot + FILE_SEPERATEOR + "stormdist"; + } + + public static String supervisor_storm_resources_path(String stormroot) { + return stormroot + FILE_SEPERATEOR + RESOURCES_SUBDIR; + } + + public static String stormtmp_path(String stormroot) { + return stormroot + FILE_SEPERATEOR + "tmp"; + } + + public static String stormts_path(String stormroot) { + return stormroot + FILE_SEPERATEOR + "timestamp"; + } + + public static LocalState worker_state(Map conf, String id) + throws IOException { + String path = worker_heartbeats_root(conf, id); + + LocalState rtn = new LocalState(path); + return rtn; + + } + + public static String masterLocalDir(Map conf) throws IOException { + String ret = + String.valueOf(conf.get(Config.STORM_LOCAL_DIR)) + + FILE_SEPERATEOR + "nimbus"; + try { + FileUtils.forceMkdir(new File(ret)); + } catch (IOException e) { + LOG.error("Failed to create dir " + ret, e); + throw e; + } + return ret; + } + + public static String masterStormdistRoot(Map conf) throws IOException { + String ret = stormdist_path(masterLocalDir(conf)); + FileUtils.forceMkdir(new File(ret)); + return ret; + } + + public static String masterStormdistRoot(Map conf, String topologyId) + throws IOException { + return masterStormdistRoot(conf) + FILE_SEPERATEOR + topologyId; + } + + public static String masterStormTmpRoot(Map conf) throws IOException { + String ret = stormtmp_path(masterLocalDir(conf)); + FileUtils.forceMkdir(new File(ret)); + return ret; + } + + public static String masterStormTmpRoot(Map conf, String topologyId) + throws IOException { + return masterStormTmpRoot(conf) + FILE_SEPERATEOR + topologyId; + } + + public static String masterInbox(Map conf) throws IOException { + String ret = masterLocalDir(conf) + FILE_SEPERATEOR + "inbox"; + try { + FileUtils.forceMkdir(new File(ret)); + } catch (IOException e) { + LOG.error("Failed to create dir " + ret, e); + throw e; + } + return ret; + } + + public static String masterInimbus(Map conf) throws IOException { + String ret = masterLocalDir(conf) + FILE_SEPERATEOR + "ininumbus"; + try { + FileUtils.forceMkdir(new File(ret)); + } catch (IOException e) { + LOG.error("Failed to create dir " + ret, e); + throw e; + } + return ret; + } + + /** + * Return nimbus's pid dir + * + * @param conf + * @return + * @throws IOException + */ + public static String masterPids(Map conf) throws IOException { + String ret = masterLocalDir(conf) + FILE_SEPERATEOR + "pids"; + try { + FileUtils.forceMkdir(new File(ret)); + } catch (IOException e) { + LOG.error("Failed to create dir " + ret, e); + throw e; + } + return ret; + } + + /** + * Return nimbus's heartbeat dir for apsara + * + * @param conf + * @return + * @throws IOException + */ + public static String masterHearbeatForContainer(Map conf) + throws IOException { + String ret = + masterLocalDir(conf) + FILE_SEPERATEOR + "nimbus.heartbeat"; + try { + FileUtils.forceMkdir(new File(ret)); + } catch (IOException e) { + LOG.error("Failed to create dir " + ret, e); + throw e; + } + return ret; + } + + public static String masterDbDir(Map conf) throws IOException { + return masterLocalDir(conf) + FILE_SEPERATEOR + "rocksdb"; + } + + public static String supervisorTmpDir(Map conf) throws IOException { + String ret = null; + try { + ret = supervisor_local_dir(conf) + FILE_SEPERATEOR + "tmp"; + FileUtils.forceMkdir(new File(ret)); + } catch (IOException e) { + LOG.error("Failed to create dir " + ret, e); + throw e; + + } + + return ret; + } + + public static LocalState supervisorState(Map conf) throws IOException { + LocalState localState = null; + try { + String localstateDir = + supervisor_local_dir(conf) + FILE_SEPERATEOR + "localstate"; + FileUtils.forceMkdir(new File(localstateDir)); + localState = new LocalState(localstateDir); + } catch (IOException e) { + LOG.error("Failed to create supervisor LocalState", e); + throw e; + } + return localState; + } + + /** + * stormconf is mergered into clusterconf + * + * @param conf + * @param topologyId + * @return + * @throws IOException + */ + public static Map read_supervisor_topology_conf(Map conf, String topologyId) + throws IOException { + String topologyRoot = + StormConfig.supervisor_stormdist_root(conf, topologyId); + String confPath = StormConfig.stormconf_path(topologyRoot); + return (Map) readLocalObject(topologyId, confPath); + } + + public static StormTopology read_supervisor_topology_code(Map conf, + String topologyId) throws IOException { + String topologyRoot = + StormConfig.supervisor_stormdist_root(conf, topologyId); + String codePath = StormConfig.stormcode_path(topologyRoot); + return (StormTopology) readLocalObject(topologyId, codePath); + } + + @SuppressWarnings("rawtypes") + public static List get_supervisor_toplogy_list(Map conf) + throws IOException { + + // get the path: STORM-LOCAL-DIR/supervisor/stormdist/ + String path = StormConfig.supervisor_stormdist_root(conf); + + List topologyids = PathUtils.read_dir_contents(path); + + return topologyids; + } + + public static Map read_nimbus_topology_conf(Map conf, String topologyId) + throws IOException { + String topologyRoot = StormConfig.masterStormdistRoot(conf, topologyId); + return read_topology_conf(topologyRoot, topologyId); + } + + public static void write_nimbus_topology_conf(Map conf, String topologyId, + Map topoConf) throws IOException { + String topologyRoot = StormConfig.masterStormdistRoot(conf, topologyId); + String confPath = StormConfig.stormconf_path(topologyRoot); + FileUtils.writeByteArrayToFile(new File(confPath), + Utils.serialize(topoConf)); + } + + public static Map read_nimbusTmp_topology_conf(Map conf, String topologyId) + throws IOException { + String topologyRoot = StormConfig.masterStormTmpRoot(conf, topologyId); + return read_topology_conf(topologyRoot, topologyId); + } + + public static Map read_topology_conf(String topologyRoot, String topologyId) + throws IOException { + String readFile = StormConfig.stormconf_path(topologyRoot); + return (Map) readLocalObject(topologyId, readFile); + } + + public static StormTopology read_nimbus_topology_code(Map conf, + String topologyId) throws IOException { + String topologyRoot = StormConfig.masterStormdistRoot(conf, topologyId); + String codePath = StormConfig.stormcode_path(topologyRoot); + return (StormTopology) readLocalObject(topologyId, codePath); + } + + public static void write_nimbus_topology_code(Map conf, String topologyId, + byte[] data) throws IOException { + String topologyRoot = StormConfig.masterStormdistRoot(conf, topologyId); + String codePath = StormConfig.stormcode_path(topologyRoot); + FileUtils.writeByteArrayToFile(new File(codePath), data); + } + + public static long read_supervisor_topology_timestamp(Map conf, + String topologyId) throws IOException { + String stormRoot = supervisor_stormdist_root(conf, topologyId); + String timeStampPath = stormts_path(stormRoot); + + byte[] data = FileUtils.readFileToByteArray(new File(timeStampPath)); + return JStormUtils.bytesToLong(data); + } + + public static void write_supervisor_topology_timestamp(Map conf, + String topologyId, long timeStamp) throws IOException { + String stormRoot = supervisor_stormdist_root(conf, topologyId); + String timeStampPath = stormts_path(stormRoot); + + byte[] data = JStormUtils.longToBytes(timeStamp); + FileUtils.writeByteArrayToFile(new File(timeStampPath), data); + } + + /** + * stormconf has mergered into clusterconf + * + * @param topologyId + * @param readFile + * @return + * @throws IOException + */ + @SuppressWarnings("unchecked") + public static Object readLocalObject(String topologyId, String readFile) + throws IOException { + + String errMsg = + "Failed to get topology configuration of " + topologyId + + " file:" + readFile; + + byte[] bconf = FileUtils.readFileToByteArray(new File(readFile)); + if (bconf == null) { + errMsg += ", due to failed to read"; + LOG.error(errMsg); + throw new IOException(errMsg); + } + + Object ret = null; + try { + ret = Utils.javaDeserialize(bconf); + } catch (Exception e) { + errMsg += ", due to failed to serialized the data"; + LOG.error(errMsg); + throw new IOException(errMsg); + } + + return ret; + } + + public static long get_supervisor_topology_Bianrymodify_time(Map conf, + String topologyId) throws IOException { + String topologyRoot = + StormConfig.supervisor_stormdist_root(conf, topologyId); + File f = new File(topologyRoot); + long modifyTime = f.lastModified(); + return modifyTime; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/StormMonitor.java b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/StormMonitor.java new file mode 100755 index 000000000..935a63864 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/StormMonitor.java @@ -0,0 +1,50 @@ +/** + * 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.cluster; + +import java.io.Serializable; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; + +/** + * Topology metrics monitor in ZK + */ + +public class StormMonitor implements Serializable { + private static final long serialVersionUID = -6023196346496305314L; + private boolean metricsMonitor; + + public StormMonitor(boolean metricsMonitor) { + this.metricsMonitor = metricsMonitor; + } + + public void setMetrics(boolean metrics) { + this.metricsMonitor = metrics; + } + + public boolean getMetrics() { + return this.metricsMonitor; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/StormStatus.java b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/StormStatus.java new file mode 100755 index 000000000..5ad70cbf4 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/StormStatus.java @@ -0,0 +1,123 @@ +/** + * 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.cluster; + +import java.io.Serializable; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; + +import com.alibaba.jstorm.daemon.nimbus.StatusType; + +/** + * author: lixin/longda + * + * Dedicate Topology status + * + * Topology status: active/inactive/killed/rebalancing killTimeSecs: when status + * isn't killed, it is -1 and useless. when status is killed, do kill operation + * after killTimeSecs seconds when status is rebalancing, do rebalancing opation + * after delaySecs seconds restore oldStatus as current status + */ +public class StormStatus implements Serializable { + + private static final long serialVersionUID = -2276901070967900100L; + private StatusType type; + @Deprecated + private int killTimeSecs; + private int delaySecs; + private StormStatus oldStatus = null; + + public StormStatus(StatusType type) { + this(0, type); + } + + public StormStatus(int delaySecs, StatusType type) { + this(type, delaySecs, null); + } + + public StormStatus(StatusType type, int delaySecs, StormStatus oldStatus) { + this.type = type; + this.delaySecs = delaySecs; + this.killTimeSecs = delaySecs; + this.oldStatus = oldStatus; + } + + public StatusType getStatusType() { + return type; + } + + public void setStatusType(StatusType type) { + this.type = type; + } + + @Deprecated + public Integer getKillTimeSecs() { + return killTimeSecs; + } + + @Deprecated + public void setKillTimeSecs(int killTimeSecs) { + this.killTimeSecs = killTimeSecs; + } + + public Integer getDelaySecs() { + return delaySecs; + } + + public void setDelaySecs(int delaySecs) { + this.delaySecs = delaySecs; + } + + public StormStatus getOldStatus() { + return oldStatus; + } + + public void setOldStatus(StormStatus oldStatus) { + this.oldStatus = oldStatus; + } + + @Override + public boolean equals(Object base) { + if ((base instanceof StormStatus) == false) { + return false; + } + + StormStatus check = (StormStatus) base; + if (check.getStatusType().equals(getStatusType()) + && check.getKillTimeSecs() == getKillTimeSecs() + && check.getDelaySecs().equals(getDelaySecs())) { + return true; + } + return false; + } + + @Override + public int hashCode() { + return this.getStatusType().hashCode() + + this.getKillTimeSecs().hashCode() + + this.getDelaySecs().hashCode(); + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/StormZkClusterState.java b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/StormZkClusterState.java new file mode 100644 index 000000000..bd60d45ca --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/cluster/StormZkClusterState.java @@ -0,0 +1,759 @@ +/** + * 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.cluster; + +import java.util.*; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.commons.lang.StringUtils; +import org.apache.zookeeper.KeeperException.NodeExistsException; +import org.apache.zookeeper.Watcher.Event.EventType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.cache.JStormCache; +import com.alibaba.jstorm.callback.ClusterStateCallback; +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.common.metric.QueueGauge; +import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; +import com.alibaba.jstorm.schedule.Assignment; +import com.alibaba.jstorm.schedule.AssignmentBak; +import com.alibaba.jstorm.task.TaskInfo; +import com.alibaba.jstorm.task.error.TaskError; +import com.alibaba.jstorm.task.heartbeat.TaskHeartbeat; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.PathUtils; +import com.alibaba.jstorm.utils.TimeUtils; + +public class StormZkClusterState implements StormClusterState { + private static Logger LOG = LoggerFactory + .getLogger(StormZkClusterState.class); + + private ClusterState cluster_state; + + private ConcurrentHashMap assignment_info_callback; + private AtomicReference supervisors_callback; + private AtomicReference assignments_callback; + private ConcurrentHashMap storm_base_callback; + private AtomicReference master_callback; + + private UUID state_id; + + private boolean solo; + + public StormZkClusterState(Object cluster_state_spec) throws Exception { + + if (cluster_state_spec instanceof ClusterState) { + solo = false; + cluster_state = (ClusterState) cluster_state_spec; + } else { + + solo = true; + cluster_state = + new DistributedClusterState((Map) cluster_state_spec); + } + + assignment_info_callback = + new ConcurrentHashMap(); + supervisors_callback = new AtomicReference(null); + assignments_callback = new AtomicReference(null); + storm_base_callback = new ConcurrentHashMap(); + master_callback = new AtomicReference(null); + + state_id = cluster_state.register(new ClusterStateCallback() { + + public Object execute(T... args) { + if (args == null) { + LOG.warn("Input args is null"); + return null; + } else if (args.length < 2) { + LOG.warn("Input args is invalid, args length:" + + args.length); + return null; + } + + EventType zkEventTypes = (EventType) args[0]; + String path = (String) args[1]; + + List toks = PathUtils.tokenize_path(path); + int size = toks.size(); + if (size >= 1) { + String params = null; + String root = toks.get(0); + RunnableCallback fn = null; + if (root.equals(Cluster.ASSIGNMENTS_ROOT)) { + if (size == 1) { + // set null and get the old value + fn = assignments_callback.getAndSet(null); + } else { + params = toks.get(1); + fn = assignment_info_callback.remove(params); + } + + } else if (root.equals(Cluster.SUPERVISORS_ROOT)) { + fn = supervisors_callback.getAndSet(null); + } else if (root.equals(Cluster.STORMS_ROOT) && size > 1) { + params = toks.get(1); + fn = storm_base_callback.remove(params); + } else if (root.equals(Cluster.MASTER_ROOT)) { + fn = master_callback.getAndSet(null); + } else { + LOG.error("Unknown callback for subtree " + path); + } + + if (fn != null) { + // FIXME How to set the args + // fn.setArgs(params, zkEventTypes, path); + fn.run(); + } + } + + return null; + } + + }); + + String[] pathlist = + JStormUtils.mk_arr(Cluster.SUPERVISORS_SUBTREE, + Cluster.STORMS_SUBTREE, Cluster.ASSIGNMENTS_SUBTREE, + Cluster.ASSIGNMENTS_BAK_SUBTREE, Cluster.TASKS_SUBTREE, + Cluster.TASKBEATS_SUBTREE, Cluster.TASKERRORS_SUBTREE, + Cluster.METRIC_SUBTREE); + for (String path : pathlist) { + cluster_state.mkdirs(path); + } + + } + + /** + * @@@ TODO + * + * Just add cache in lower ZK level In fact, for some Object + * Assignment/TaskInfo/StormBase These object can be cache for long time + * + * @param simpleCache + */ + public void setCache(JStormCache simpleCache) { + if (cluster_state instanceof DistributedClusterState) { + ((DistributedClusterState) cluster_state).setZkCache(simpleCache); + } + } + + public Object getObject(String path, boolean callback) throws Exception { + byte[] data = cluster_state.get_data(path, callback); + + return Utils.maybe_deserialize(data); + } + + public Object getObjectSync(String path, boolean callback) throws Exception { + byte[] data = cluster_state.get_data_sync(path, callback); + + return Utils.maybe_deserialize(data); + } + + public String getString(String path, boolean callback) throws Exception { + byte[] data = cluster_state.get_data(path, callback); + + return new String(data); + } + + public void deleteObject(String path) { + try { + cluster_state.delete_node(path); + } catch (Exception e) { + LOG.warn("Failed to delete node " + path); + } + } + + public void setObject(String path, Object obj) throws Exception { + if (obj instanceof byte[]) { + cluster_state.set_data(path, (byte[]) obj); + } else if (obj instanceof String) { + cluster_state.set_data(path, ((String) obj).getBytes()); + } else { + cluster_state.set_data(path, Utils.serialize(obj)); + } + } + + public void setTempObject(String path, Object obj) throws Exception { + if (obj instanceof byte[]) { + cluster_state.set_ephemeral_node(path, (byte[]) obj); + } else if (obj instanceof String) { + cluster_state.set_ephemeral_node(path, ((String) obj).getBytes()); + } else { + cluster_state.set_ephemeral_node(path, Utils.serialize(obj)); + } + } + + @Override + public void disconnect() { + cluster_state.unregister(state_id); + if (solo == true) { + cluster_state.close(); + } + } + + public void remove_storm(String topologyId, boolean needSleep) { + deleteObject(Cluster.assignment_path(topologyId)); + // wait 10 seconds, so supervisor will kill worker smoothly + if (needSleep) { + JStormUtils.sleepMs(10000); + } + try { + deleteObject(Cluster.storm_task_root(topologyId)); + teardown_heartbeats(topologyId); + teardown_task_errors(topologyId); + deleteObject(Cluster.metric_path(topologyId)); + } catch (Exception e) { + LOG.warn("Failed to delete task root and monitor root for" + + topologyId); + } + remove_storm_base(topologyId); + } + + @Override + public void remove_storm(String topologyId) throws Exception { + remove_storm(topologyId, true); + } + + @Override + public void try_remove_storm(String topologyId) { + remove_storm(topologyId, false); + } + + @Override + public Assignment assignment_info(String topologyId, + RunnableCallback callback) throws Exception { + if (callback != null) { + assignment_info_callback.put(topologyId, callback); + } + + String assgnmentPath = Cluster.assignment_path(topologyId); + + return (Assignment) getObject(assgnmentPath, callback != null); + + } + + @Override + public List assignments(RunnableCallback callback) throws Exception { + if (callback != null) { + assignments_callback.set(callback); + } + return cluster_state.get_children(Cluster.ASSIGNMENTS_SUBTREE, + callback != null); + } + + @Override + public void set_assignment(String topologyId, Assignment info) + throws Exception { + setObject(Cluster.assignment_path(topologyId), info); + } + + @Override + public AssignmentBak assignment_bak(String topologyName) throws Exception { + String assgnmentBakPath = Cluster.assignment_bak_path(topologyName); + + return (AssignmentBak) getObject(assgnmentBakPath, false); + + } + + @Override + public void backup_assignment(String topologyName, AssignmentBak info) + throws Exception { + setObject(Cluster.assignment_bak_path(topologyName), info); + } + + @Override + public StormBase storm_base(String topologyId, RunnableCallback callback) + throws Exception { + if (callback != null) { + storm_base_callback.put(topologyId, callback); + } + + return (StormBase) getObject(Cluster.storm_path(topologyId), + callback != null); + + } + + @Override + public void activate_storm(String topologyId, StormBase stormBase) + throws Exception { + String stormPath = Cluster.storm_path(topologyId); + + setObject(stormPath, stormBase); + } + + @Override + public void remove_storm_base(String topologyId) { + deleteObject(Cluster.storm_path(topologyId)); + } + + @Override + public void update_storm(String topologyId, StormStatus newElems) + throws Exception { + /** + * FIXME, maybe overwrite old callback + */ + + StormBase base = this.storm_base(topologyId, null); + + if (base != null) { + base.setStatus(newElems); + setObject(Cluster.storm_path(topologyId), base); + } + + } + + @Override + public void set_storm_monitor(String topologyId, boolean isEnable) + throws Exception { + // TODO Auto-generated method stub + StormBase base = this.storm_base(topologyId, null); + + if (base != null) { + base.setEnableMonitor(isEnable); + setObject(Cluster.storm_path(topologyId), base); + } + } + + @Override + public List active_storms() throws Exception { + return cluster_state.get_children(Cluster.STORMS_SUBTREE, false); + } + + @Override + public void setup_heartbeats(String topologyId) throws Exception { + String taskbeatPath = Cluster.taskbeat_storm_root(topologyId); + + cluster_state.mkdirs(taskbeatPath); + } + + @Override + public List heartbeat_storms() throws Exception { + return cluster_state.get_children(Cluster.TASKBEATS_SUBTREE, false); + } + + @Override + public List heartbeat_tasks(String topologyId) throws Exception { + String taskbeatPath = Cluster.taskbeat_storm_root(topologyId); + + return cluster_state.get_children(taskbeatPath, false); + } + + @Override + public void remove_task_heartbeat(String topologyId, int taskId) + throws Exception { + String taskbeatPath = Cluster.taskbeat_path(topologyId, taskId); + + deleteObject(taskbeatPath); + } + + @Override + public void teardown_heartbeats(String topologyId) { + try { + String taskbeatPath = Cluster.taskbeat_storm_root(topologyId); + + deleteObject(taskbeatPath); + } catch (Exception e) { + LOG.warn("Could not teardown heartbeats for " + topologyId, e); + } + + } + + @Override + public void report_task_error(String topologyId, int taskId, Throwable error) + throws Exception { + report_task_error(topologyId, taskId, + new String(JStormUtils.getErrorInfo(error))); + } + + public void report_task_error(String topologyId, int taskId, String error) + throws Exception { + boolean found = false; + String path = Cluster.taskerror_path(topologyId, taskId); + cluster_state.mkdirs(path); + + List children = new ArrayList(); + + String timeStamp = String.valueOf(TimeUtils.current_time_secs()); + String timestampPath = path + Cluster.ZK_SEPERATOR + timeStamp; + + for (String str : cluster_state.get_children(path, false)) { + String errorPath = path + "/" + str; + String errorInfo = getString(errorPath, false); + if (StringUtils.isBlank(errorInfo)) { + deleteObject(errorPath); + continue; + } + if (errorInfo.equals(error)) { + deleteObject(errorPath); + setObject(timestampPath, error); + found = true; + break; + } + + children.add(Integer.parseInt(str)); + } + + if (found == false) { + Collections.sort(children); + + while (children.size() >= 3) { + deleteObject(path + Cluster.ZK_SEPERATOR + children.remove(0)); + } + + setObject(timestampPath, error); + } + + setLastErrInfo(topologyId, error, timeStamp); + } + + private static final String TASK_IS_DEAD = "is dead on"; // Full string is + // "task-id is dead on hostname:port" + + private void setLastErrInfo(String topologyId, String error, + String timeStamp) throws Exception { + // Set error information in task error topology patch + // Last Error information format in ZK: map + // report_duration means only the errors will presented in web ui if the + // error happens within this duration. + // Currently, the duration for "queue full" error is 180sec(3min) while + // the duration for other errors is 1800sec(30min). + String lastErrTopoPath = Cluster.lasterror_path(topologyId); + Map lastErrInfo = null; + try { + lastErrInfo = + (Map) getObject(lastErrTopoPath, false); + + } catch (Exception e) { + LOG.error( + "Failed to get last error time. Remove the corrupt node for " + + topologyId, e); + remove_lastErr_time(topologyId); + lastErrInfo = null; + } + if (lastErrInfo == null) + lastErrInfo = new HashMap(); + + // The error time is used to indicate how long the error info is present + // in UI + if (error.indexOf(QueueGauge.QUEUE_IS_FULL) != -1) + lastErrInfo.put(JStormUtils.MIN_1 * 3, timeStamp); + else if (error.indexOf(TASK_IS_DEAD) != -1) + lastErrInfo.put(JStormUtils.DAY_1 * 3, timeStamp); + else + lastErrInfo.put(JStormUtils.MIN_30, timeStamp); + + setObject(lastErrTopoPath, lastErrInfo); + } + + @Override + public void remove_task_error(String topologyId, int taskId) + throws Exception { + String path = Cluster.taskerror_path(topologyId, taskId); + cluster_state.delete_node(path); + } + + @Override + public Map topo_lastErr_time(String topologyId) + throws Exception { + String path = Cluster.lasterror_path(topologyId); + + return (Map) getObject(path, false); + } + + @Override + public void remove_lastErr_time(String topologyId) throws Exception { + String path = Cluster.lasterror_path(topologyId); + deleteObject(path); + } + + @Override + public List task_error_storms() throws Exception { + return cluster_state.get_children(Cluster.TASKERRORS_SUBTREE, false); + } + + @Override + public List task_error_ids(String topologyId) throws Exception { + return cluster_state.get_children(Cluster.taskerror_storm_root(topologyId), 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 void remove_task(String topologyId, Set taskIds) throws Exception { + String tasksPath = Cluster.storm_task_root(topologyId); + Object data = getObject(tasksPath, false); + if (data != null) { + Map taskInfoMap = ((Map)data); + for (Integer taskId : taskIds){ + taskInfoMap.remove(taskId); + } + //update zk node of tasks + setObject(tasksPath, taskInfoMap); + } + } + + @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 getString(path, false); + } + + @Override + public List task_errors(String topologyId, int taskId) + throws Exception { + String path = Cluster.taskerror_path(topologyId, taskId); + cluster_state.mkdirs(path); + + List children = cluster_state.get_children(path, false); + List errors = new ArrayList(); + + for (String str : children) { + byte[] v = cluster_state.get_data(path + "/" + str, false); + if (v != null) { + TaskError error = + new TaskError(new String(v), Integer.parseInt(str)); + errors.add(error); + } + } + + Collections.sort(errors, new Comparator() { + + @Override + public int compare(TaskError o1, TaskError o2) { + if (o1.getTimSecs() > o2.getTimSecs()) { + return 1; + } + if (o1.getTimSecs() < o2.getTimSecs()) { + return -1; + } + return 0; + } + }); + + return errors; + + } + + @Override + public void teardown_task_errors(String topologyId) { + try { + String taskerrPath = Cluster.taskerror_storm_root(topologyId); + deleteObject(taskerrPath); + } catch (Exception e) { + LOG.error("Could not teardown errors for " + topologyId, e); + } + } + @Override + public void set_task(String topologyId, Map taskInfoMap) + throws Exception { + String stormTaskPath = Cluster.storm_task_root(topologyId); + if (taskInfoMap != null){ + //reupdate zk node of tasks + setObject(stormTaskPath, taskInfoMap); + } + } + @Override + public void add_task(String topologyId, Map taskInfoMap) + throws Exception { + String stormTaskPath = Cluster.storm_task_root(topologyId); + Object data = getObject(stormTaskPath, false); + if (data != null){ + ((Map)data).putAll(taskInfoMap); + //reupdate zk node of tasks + setObject(stormTaskPath, data); + } + } + + @Override + public TaskHeartbeat task_heartbeat(String topologyId, int taskId) + throws Exception { + String taskbeatPath = Cluster.taskbeat_path(topologyId, taskId); + + return (TaskHeartbeat) getObjectSync(taskbeatPath, false); + + } + + @Override + public void task_heartbeat(String topologyId, int taskId, TaskHeartbeat info) + throws Exception { + String taskPath = Cluster.taskbeat_path(topologyId, taskId); + + setObject(taskPath, info); + } + + @Override + public List task_storms() throws Exception { + return cluster_state.get_children(Cluster.TASKS_SUBTREE, false); + } + + @Override + public Set task_ids(String stromId) throws Exception { + + String stormTaskPath = Cluster.storm_task_root(stromId); + Object data = getObject(stormTaskPath, false); + if (data == null) { + return null; + } + return ((Map)data).keySet(); + } + + @Override + public Set task_ids_by_componentId(String topologyId, + String componentId) throws Exception { + String stormTaskPath = Cluster.storm_task_root(topologyId); + Object data = getObject(stormTaskPath, false); + if (data == null) { + return null; + } + Map taskInfoMap = (Map)data; + Set rtn = new HashSet(); + Set taskIds = taskInfoMap.keySet(); + for(Integer taskId : taskIds){ + TaskInfo taskInfo = taskInfoMap.get(taskId); + if (taskInfo != null){ + if (taskInfo.getComponentId().equalsIgnoreCase(componentId)) + rtn.add(taskId); + } + } + return rtn; + } + + @Override + public Map task_all_info(String topologyId) throws Exception { + + String taskPath = Cluster.storm_task_root(topologyId); + + Object data = getObject(taskPath, false); + if (data == null) { + return null; + } + return (Map) data; + } + + @Override + public SupervisorInfo supervisor_info(String supervisorId) throws Exception { + String supervisorPath = Cluster.supervisor_path(supervisorId); + + return (SupervisorInfo) getObject(supervisorPath, false); + + } + + @Override + public List supervisors(RunnableCallback callback) throws Exception { + if (callback != null) { + supervisors_callback.set(callback); + } + return cluster_state.get_children(Cluster.SUPERVISORS_SUBTREE, + callback != null); + } + + @Override + public void supervisor_heartbeat(String supervisorId, SupervisorInfo info) + throws Exception { + + String supervisorPath = Cluster.supervisor_path(supervisorId); + + setTempObject(supervisorPath, info); + } + + @Override + public String get_leader_host() throws Exception { + // TODO Auto-generated method stub + return new String(cluster_state.get_data(Cluster.MASTER_SUBTREE, false)); + } + + @Override + public boolean leader_existed() throws Exception { + // TODO Auto-generated method stub + return cluster_state.node_existed(Cluster.MASTER_SUBTREE, false); + } + + @Override + public List get_nimbus_slaves() throws Exception { + return cluster_state.get_children(Cluster.NIMBUS_SLAVE_SUBTREE, false); + } + + public String get_nimbus_slave_time(String host) throws Exception { + String path = + Cluster.NIMBUS_SLAVE_SUBTREE + Cluster.ZK_SEPERATOR + host; + return (String) getObject(path, false); + } + + @Override + public void update_nimbus_slave(String host, int time) throws Exception { + setTempObject(Cluster.NIMBUS_SLAVE_SUBTREE + Cluster.ZK_SEPERATOR + + host, String.valueOf(time)); + } + + @Override + public void unregister_nimbus_host(String host) throws Exception { + deleteObject(Cluster.NIMBUS_SLAVE_SUBTREE + Cluster.ZK_SEPERATOR + host); + } + + @Override + public boolean try_to_be_leader(String path, String host, + RunnableCallback callback) throws Exception { + // TODO Auto-generated method stub + if (callback != null) + this.master_callback.set(callback); + try { + cluster_state.tryToBeLeader(path, host.getBytes()); + } catch (NodeExistsException e) { + cluster_state.node_existed(path, true); + LOG.info("leader is alive"); + return false; + } + return true; + } + + @Override + public void set_topology_metric(String topologyId, Object metric) + throws Exception { + // TODO Auto-generated method stub + String path = Cluster.metric_path(topologyId); + + setObject(path, metric); + } + + @Override + public Object get_topology_metric(String topologyId) throws Exception { + // TODO Auto-generated method stub + return getObject(Cluster.metric_path(topologyId), false); + } + + @Override + public List get_metrics() throws Exception { + // TODO Auto-generated method stub + return cluster_state.get_children(Cluster.METRIC_SUBTREE, false); + } + +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Counter.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Counter.java new file mode 100755 index 000000000..f9e97ddd5 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Counter.java @@ -0,0 +1,55 @@ +/** + * 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.common.metric; + +import com.alibaba.jstorm.common.metric.operator.convert.DefaultConvertor; +import com.alibaba.jstorm.common.metric.operator.merger.SumMerger; +import com.alibaba.jstorm.common.metric.operator.updater.AddUpdater; +import com.alibaba.jstorm.common.metric.window.Metric; + +/** + * The class is similar to com.codahale.metrics.Counter + * + * Sum all window's value + * + * how to use Counter , please refer to Sampling Interface + * + * @author zhongyan.feng + * + * @param + */ +public class Counter extends Metric { + private static final long serialVersionUID = -1362345159511508074L; + + /** + * + * @param defaultValue + */ + public Counter(T zero) { + updater = new AddUpdater(); + merger = new SumMerger(); + convertor = new DefaultConvertor(); + defaultValue = zero; + + init(); + } + + public static void main(String[] args) { + + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Gauge.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Gauge.java new file mode 100755 index 000000000..30fa11022 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Gauge.java @@ -0,0 +1,61 @@ +/** + * 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.common.metric; + +import java.util.Map; +import java.util.TreeMap; + +import com.alibaba.jstorm.common.metric.window.Metric; +import com.alibaba.jstorm.common.metric.window.StatBuckets; + +public class Gauge extends Metric { + private static final long serialVersionUID = 1985614006717750790L; + + protected com.codahale.metrics.Gauge gauge; + + public Gauge(com.codahale.metrics.Gauge gauge) { + this.gauge = gauge; + + init(); + } + + @Override + public void init() { + + } + + @Override + public void update(Number obj) { + // TODO Auto-generated method stub + } + + @Override + public Map getSnapshot() { + // TODO Auto-generated method stub + Number value = gauge.getValue(); + + Map ret = new TreeMap(); + for (Integer timeKey : windowSeconds) { + ret.put(timeKey, value); + } + ret.put(StatBuckets.ALL_TIME_WINDOW, value); + + return ret; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Histogram.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Histogram.java new file mode 100755 index 000000000..7276fdf72 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Histogram.java @@ -0,0 +1,104 @@ +/** + * 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.common.metric; + +import com.alibaba.jstorm.common.metric.operator.convert.Convertor; +import com.alibaba.jstorm.common.metric.operator.merger.AvgMerger; +import com.alibaba.jstorm.common.metric.operator.updater.AvgUpdater; +import com.alibaba.jstorm.common.metric.window.Metric; + +/** + * Meter is used to compute tps + * + * Attention: 1. + * + * @author zhongyan.feng + * + */ +public class Histogram extends Metric { + private static final long serialVersionUID = -1362345159511508074L; + + public Histogram() { + defaultValue = + new HistorgramPair(); + updater = new AvgUpdater(); + merger = new AvgMerger(); + convertor = new HistogramConvertor(); + + init(); + } + + public static class HistogramConvertor implements + Convertor { + private static final long serialVersionUID = -1569170826785657226L; + + @Override + public Double convert(HistorgramPair from) { + // TODO Auto-generated method stub + if (from == null) { + return 0.0d; + } + + if (from.getTimes() == 0) { + return 0.0d; + } else { + return from.getSum()/ from.getTimes(); + } + } + + } + + public static class HistorgramPair { + private double sum; + private long times; + + public HistorgramPair() { + + } + + public HistorgramPair(double sum, long times){ + this.sum = sum; + this.times = times; + } + + public double getSum() { + return sum; + } + + public void setSum(double sum) { + this.sum = sum; + } + + public void addValue(double value) { + sum += value; + } + + public long getTimes() { + return times; + } + + public void setTimes(long times) { + this.times = times; + } + + public void addTimes(long time) { + times += time; + } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Histogram.java.bak b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Histogram.java.bak new file mode 100755 index 000000000..b8307891b --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Histogram.java.bak @@ -0,0 +1,71 @@ +/** + * 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.common.metric; + +import java.util.concurrent.atomic.AtomicLong; + +import com.alibaba.jstorm.common.metric.operator.convert.Convertor; +import com.alibaba.jstorm.common.metric.operator.merger.AvgMerger2; +import com.alibaba.jstorm.common.metric.operator.updater.AvgUpdater2; +import com.alibaba.jstorm.common.metric.window.Metric; +import com.alibaba.jstorm.utils.Pair; +import com.google.common.util.concurrent.AtomicDouble; + +/** + * Meter is used to compute tps + * + * Attention: 1. + * + * @author zhongyan.feng + * + */ +public class Histogram extends Metric> { + private static final long serialVersionUID = -1362345159511508074L; + + public Histogram() { + defaultValue = + new Pair(new AtomicDouble(0.0), + new AtomicLong(0)); + updater = new AvgUpdater2(); + merger = new AvgMerger2(); + convertor = new HistogramConvertor(); + + init(); + } + + public static class HistogramConvertor implements + Convertor, Double> { + private static final long serialVersionUID = -1569170826785657226L; + + @Override + public Double convert(Pair from) { + // TODO Auto-generated method stub + if (from == null) { + return 0.0d; + } + + if (from.getSecond().get() == 0) { + return 0.0d; + } else { + return from.getFirst().get() / from.getSecond().get(); + } + } + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/LongCounter.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/LongCounter.java new file mode 100755 index 000000000..ac589128e --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/LongCounter.java @@ -0,0 +1,39 @@ +/** + * 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.common.metric; + +import java.util.concurrent.atomic.AtomicLong; + +import com.alibaba.jstorm.common.metric.operator.convert.AtomicLongToLong; +import com.alibaba.jstorm.common.metric.operator.merger.LongSumMerger; +import com.alibaba.jstorm.common.metric.operator.updater.LongAddUpdater; +import com.alibaba.jstorm.common.metric.window.Metric; + +public class LongCounter extends Metric { + private static final long serialVersionUID = -1362345159511508074L; + + public LongCounter() { + super.defaultValue = new AtomicLong(0); + super.updater = new LongAddUpdater(); + super.merger = new LongSumMerger(); + super.convertor = new AtomicLongToLong(); + + init(); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Meter.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Meter.java new file mode 100755 index 000000000..e56d0250a --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Meter.java @@ -0,0 +1,50 @@ +/** + * 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.common.metric; + +import com.alibaba.jstorm.common.metric.operator.convert.DefaultConvertor; +import com.alibaba.jstorm.common.metric.operator.merger.TpsMerger; +import com.alibaba.jstorm.common.metric.operator.updater.AddUpdater; +import com.alibaba.jstorm.common.metric.window.Metric; +import com.alibaba.jstorm.common.metric.window.RollingWindow; + +/** + * Meter is used to compute tps + * + * Attention: 1. + * + * @author zhongyan.feng + * + */ +public class Meter extends Metric { + private static final long serialVersionUID = -1362345159511508074L; + + public Meter() { + defaultValue = 0.0d; + updater = new AddUpdater(); + merger = new TpsMerger(); + convertor = new DefaultConvertor(); + + init(); + } + + public void update() { + update(Double.valueOf(1)); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/MetricFilter.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/MetricFilter.java new file mode 100755 index 000000000..92b1f6be3 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/MetricFilter.java @@ -0,0 +1,46 @@ +/** + * 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.common.metric; + +import java.io.Serializable; + +import com.alibaba.jstorm.common.metric.window.Metric; + +public interface MetricFilter extends Serializable { + /** + * Matches all metrics, regardless of type or name. + */ + MetricFilter ALL = new MetricFilter() { + private static final long serialVersionUID = 7089987006352295530L; + + @Override + public boolean matches(String name, Metric metric) { + return true; + } + }; + + /** + * Returns {@code true} if the metric matches the filter; {@code false} + * otherwise. + * + * @param name the metric's name + * @param metric the metric + * @return {@code true} if the metric matches the filter + */ + boolean matches(String name, Metric metric); +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/MetricRegistry.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/MetricRegistry.java new file mode 100755 index 000000000..982c5f660 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/MetricRegistry.java @@ -0,0 +1,316 @@ +/** + * 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.common.metric; + +import java.util.Collections; +import java.util.Map; +import java.util.SortedMap; +import java.util.SortedSet; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.common.metric.window.Metric; + +public class MetricRegistry implements MetricSet { + private static final Logger LOG = LoggerFactory.getLogger(MetricRegistry.class); + + private static final long serialVersionUID = 8184106900230111064L; + public static final String NAME_SEPERATOR = "."; + + /** + * Concatenates elements to form a dotted name, eliding any null values or + * empty strings. + * + * @param name the first element of the name + * @param names the remaining elements of the name + * @return {@code name} and {@code names} concatenated by periods + */ + public static String name(String name, String... names) { + final StringBuilder builder = new StringBuilder(); + append(builder, name); + if (names != null) { + for (String s : names) { + append(builder, s); + } + } + return builder.toString(); + } + + /** + * Concatenates a class name and elements to form a dotted name, eliding any + * null values or empty strings. + * + * @param klass the first element of the name + * @param names the remaining elements of the name + * @return {@code klass} and {@code names} concatenated by periods + */ + public static String name(Class klass, String... names) { + return name(klass.getName(), names); + } + + private static void append(StringBuilder builder, String part) { + if (part != null && !part.isEmpty()) { + if (builder.length() > 0) { + builder.append(NAME_SEPERATOR); + } + builder.append(part); + } + } + + protected final ConcurrentMap metrics; + + /** + * Creates a new {@link MetricRegistry}. + */ + public MetricRegistry() { + this.metrics = buildMap(); + } + + /** + * Creates a new {@link ConcurrentMap} implementation for use inside the + * registry. Override this to create a {@link MetricRegistry} with space- or + * time-bounded metric lifecycles, for example. + * + * @return a new {@link ConcurrentMap} + */ + protected ConcurrentMap buildMap() { + return new ConcurrentHashMap(); + } + + /** + * Given a {@link Metric}, registers it under the given name. + * + * @param name the name of the metric + * @param metric the metric + * @param the type of the metric + * @return {@code metric} + * @throws IllegalArgumentException if the name is already registered + */ + @SuppressWarnings("unchecked") + public T register(String name, T metric) + throws IllegalArgumentException { + if (metric instanceof MetricSet) { + registerAll(name, (MetricSet) metric); + } else { + final Metric existing = metrics.putIfAbsent(name, metric); + if (existing == null) { + // add one listener to notify + LOG.info("Successfully register metric of {}", name); + } else { + throw new IllegalArgumentException("A metric named " + name + + " already exists"); + } + } + return metric; + } + + /** + * Given a metric set, registers them. + * + * @param metrics a set of metrics + * @throws IllegalArgumentException if any of the names are already + * registered + */ + public void registerAll(MetricSet metrics) throws IllegalArgumentException { + registerAll(null, metrics); + } + + /** + * Removes the metric with the given name. + * + * @param name the name of the metric + * @return whether or not the metric was removed + */ + public boolean remove(String name) { + final Metric metric = metrics.remove(name); + if (metric != null) { + // call listener to notify remove + LOG.info("Successfully unregister metric of {}", name); + return true; + } + return false; + } + + /** + * Removes all metrics which match the given filter. + * + * @param filter a filter + */ + public void removeMatching(MetricFilter filter) { + for (Map.Entry entry : metrics.entrySet()) { + if (filter.matches(entry.getKey(), entry.getValue())) { + remove(entry.getKey()); + } + } + } + + /** + * Returns a set of the names of all the metrics in the registry. + * + * @return the names of all the metrics + */ + public SortedSet getNames() { + return Collections.unmodifiableSortedSet(new TreeSet(metrics + .keySet())); + } + + /** + * Returns a map of all the gauges in the registry and their names. + * + * @return all the gauges in the registry + */ + public SortedMap getGauges() { + return getGauges(MetricFilter.ALL); + } + + /** + * Returns a map of all the gauges in the registry and their names which + * match the given filter. + * + * @param filter the metric filter to match + * @return all the gauges in the registry + */ + public SortedMap getGauges(MetricFilter filter) { + return getMetrics(Gauge.class, filter); + } + + /** + * Returns a map of all the counters in the registry and their names. + * + * @return all the counters in the registry + */ + public SortedMap getCounters() { + return getCounters(MetricFilter.ALL); + } + + /** + * Returns a map of all the counters in the registry and their names which + * match the given filter. + * + * @param filter the metric filter to match + * @return all the counters in the registry + */ + public SortedMap getCounters(MetricFilter filter) { + return getMetrics(Counter.class, filter); + } + + /** + * Returns a map of all the histograms in the registry and their names. + * + * @return all the histograms in the registry + */ + public SortedMap getHistograms() { + return getHistograms(MetricFilter.ALL); + } + + /** + * Returns a map of all the histograms in the registry and their names which + * match the given filter. + * + * @param filter the metric filter to match + * @return all the histograms in the registry + */ + public SortedMap getHistograms(MetricFilter filter) { + return getMetrics(Histogram.class, filter); + } + + /** + * Returns a map of all the meters in the registry and their names. + * + * @return all the meters in the registry + */ + public SortedMap getMeters() { + return getMeters(MetricFilter.ALL); + } + + /** + * Returns a map of all the meters in the registry and their names which + * match the given filter. + * + * @param filter the metric filter to match + * @return all the meters in the registry + */ + public SortedMap getMeters(MetricFilter filter) { + return getMetrics(Meter.class, filter); + } + + /** + * Returns a map of all the timers in the registry and their names. + * + * @return all the timers in the registry + */ + public SortedMap getTimers() { + return getTimers(MetricFilter.ALL); + } + + /** + * Returns a map of all the timers in the registry and their names which + * match the given filter. + * + * @param filter the metric filter to match + * @return all the timers in the registry + */ + public SortedMap getTimers(MetricFilter filter) { + return getMetrics(Timer.class, filter); + } + + @SuppressWarnings("unchecked") + private SortedMap getMetrics(Class klass, + MetricFilter filter) { + final TreeMap timers = new TreeMap(); + for (Map.Entry entry : metrics.entrySet()) { + if (klass.isInstance(entry.getValue()) + && filter.matches(entry.getKey(), entry.getValue())) { + timers.put(entry.getKey(), (T) entry.getValue()); + } + } + return Collections.unmodifiableSortedMap(timers); + } + + private void registerAll(String prefix, MetricSet metrics) + throws IllegalArgumentException { + for (Map.Entry entry : metrics.getMetrics().entrySet()) { + if (entry.getValue() instanceof MetricSet) { + registerAll(name(prefix, entry.getKey()), + (MetricSet) entry.getValue()); + } else { + register(name(prefix, entry.getKey()), entry.getValue()); + } + } + } + + @Override + public Map getMetrics() { + return Collections.unmodifiableMap(metrics); + } + + /** + * Expose metrics is to improve performance + * + * @return + */ + public Metric getMetric(String name) { + return metrics.get(name); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/MetricSet.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/MetricSet.java new file mode 100755 index 000000000..243f9b83b --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/MetricSet.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.common.metric; + +import java.io.Serializable; +import java.util.Map; + +import com.alibaba.jstorm.common.metric.window.Metric; + +public interface MetricSet extends Serializable { + Map getMetrics(); + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/QueueGauge.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/QueueGauge.java new file mode 100644 index 000000000..0ff964edf --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/QueueGauge.java @@ -0,0 +1,61 @@ +/** + * 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.common.metric; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.utils.DisruptorQueue; + +import com.codahale.metrics.Gauge; +import com.codahale.metrics.health.HealthCheck; + +public class QueueGauge extends HealthCheck implements Gauge { + private static final Logger LOG = LoggerFactory.getLogger(QueueGauge.class); + + public static final String QUEUE_IS_FULL = " is full"; + + DisruptorQueue queue; + String name; + Result healthy; + + public QueueGauge(String name, DisruptorQueue queue) { + this.queue = queue; + this.name = name; + this.healthy = HealthCheck.Result.healthy(); + } + + @Override + public Double getValue() { + Double ret = (double) queue.pctFull(); + + return ret; + } + + @Override + protected Result check() throws Exception { + // TODO Auto-generated method stub + Double ret = (double) queue.pctFull(); + if (ret > 0.9) { + return HealthCheck.Result.unhealthy(name + QUEUE_IS_FULL); + } else { + return healthy; + } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Timer.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Timer.java new file mode 100755 index 000000000..daf56335d --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Timer.java @@ -0,0 +1,108 @@ +/** + * 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.common.metric; + +import java.io.Closeable; +import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; + +/** + * Use com.codahale.metrics's interface + * + * @author zhongyan.feng + * + */ +public class Timer extends Histogram { + private static final long serialVersionUID = 5915881891513771108L; + + /** + * A timing context. + * + * @see Timer#time() + */ + public static class Context implements Closeable { + private final Timer timer; + private final long startTime; + + private Context(Timer timer) { + this.timer = timer; + this.startTime = System.currentTimeMillis(); + } + + /** + * Stops recording the elapsed time, updates the timer and returns the + * elapsed time in nanoseconds. + */ + public long stop() { + final long elapsed = System.currentTimeMillis() - startTime; + timer.update(elapsed, TimeUnit.MILLISECONDS); + return elapsed; + } + + @Override + public void close() { + stop(); + } + } + + public Timer() { + init(); + } + + /** + * Adds a recorded duration. + * + * @param duration the length of the duration + * @param unit the scale unit of {@code duration} + */ + public void update(long duration, TimeUnit unit) { + update(unit.toMillis(duration)); + } + + /** + * Times and records the duration of event. + * + * @param event a {@link Callable} whose {@link Callable#call()} method + * implements a process whose duration should be timed + * @param the type of the value returned by {@code event} + * @return the value returned by {@code event} + * @throws Exception if {@code event} throws an {@link Exception} + */ + public T time(Callable event) throws Exception { + final long startTime = System.currentTimeMillis(); + try { + return event.call(); + } finally { + update(System.currentTimeMillis() - startTime); + } + } + + /** + * Returns a new {@link Context}. + * + * @return a new {@link Context} + * @see Context + */ + public Context time() { + return new Context(this); + } + + public long getCount() { + return allWindow.getSnapshot().getTimes(); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/TimerRatio.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/TimerRatio.java new file mode 100755 index 000000000..0a0e7e252 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/TimerRatio.java @@ -0,0 +1,73 @@ +/** + * 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.common.metric; + +import com.codahale.metrics.Gauge; + +/** + * + * @author zhongyan.feng + * @version $Id: + */ +public class TimerRatio implements Gauge { + + private long lastUpdateTime = 0; + private long sum = 0; + private long lastGaugeTime; + + public void init() { + lastGaugeTime = System.nanoTime(); + } + + public synchronized void start() { + if (lastUpdateTime == 0) { + lastUpdateTime = System.nanoTime(); + } + } + + public synchronized void stop() { + if (lastUpdateTime != 0) { + long now = System.nanoTime(); + long cost = now - lastUpdateTime; + lastUpdateTime = 0; + sum += cost; + } + + } + + @Override + public Double getValue() { + synchronized (this) { + stop(); + + long now = System.nanoTime(); + long cost = now - lastGaugeTime; + if (cost == 0) { + return 1.0; + } + + lastGaugeTime = now; + double ratio = ((double) sum) / cost; + sum = 0; + return ratio; + + } + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Top.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Top.java new file mode 100755 index 000000000..00ccc980c --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/Top.java @@ -0,0 +1,157 @@ +/** + * 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.common.metric; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Comparator; +import java.util.List; +import java.util.TreeSet; + +import com.alibaba.jstorm.common.metric.operator.convert.Convertor; +import com.alibaba.jstorm.common.metric.operator.merger.Merger; +import com.alibaba.jstorm.common.metric.operator.updater.Updater; +import com.alibaba.jstorm.common.metric.window.Metric; + +public class Top extends Metric, TreeSet> { + private static final long serialVersionUID = 4990212679365713831L; + + final protected Comparator comparator; + final protected int n; + + public Top(Comparator comparator, int n) { + this.comparator = comparator; + this.n = n; + + this.defaultValue = new TreeSet(comparator); + this.updater = new Top.TopUpdator(comparator, n); + this.merger = new Top.TopMerger(comparator, n); + this.convertor = new Top.SetToList(); + + init(); + } + + public static class TopUpdator implements Updater> { + private static final long serialVersionUID = -3940041101182079146L; + + final protected Comparator comparator; + final protected int n; + + public TopUpdator(Comparator comparator, int n) { + this.comparator = comparator; + this.n = n; + } + + @SuppressWarnings("unchecked") + @Override + public TreeSet update(Number object, TreeSet cache, + Object... others) { + // TODO Auto-generated method stub + if (cache == null) { + cache = new TreeSet(comparator); + } + + cache.add((T) object); + + if (cache.size() > n) { + cache.remove(cache.last()); + } + + return cache; + } + + @Override + public TreeSet updateBatch(TreeSet object, TreeSet cache, + Object... objects) { + // TODO Auto-generated method stub + if (cache == null) { + cache = new TreeSet(comparator); + } + + cache.addAll(object); + + while (cache.size() > n) { + cache.remove(cache.last()); + } + + return cache; + } + + } + + public static class TopMerger implements Merger> { + + private static final long serialVersionUID = 4478867986986581638L; + final protected Comparator comparator; + final protected int n; + + public TopMerger(Comparator comparator, int n) { + this.comparator = comparator; + this.n = n; + } + + @Override + public TreeSet merge(Collection> objs, + TreeSet unflushed, Object... others) { + // TODO Auto-generated method stub + TreeSet temp = new TreeSet(comparator); + if (unflushed != null) { + temp.addAll(unflushed); + } + + for (TreeSet set : objs) { + temp.addAll(set); + } + + if (temp.size() <= n) { + return temp; + } + + TreeSet ret = new TreeSet(comparator); + int i = 0; + for (T item : temp) { + if (i < n) { + ret.add(item); + i++; + } else { + break; + } + } + return ret; + } + + } + + public static class SetToList implements Convertor, List> { + private static final long serialVersionUID = 4968816655779625255L; + + @Override + public List convert(TreeSet set) { + // TODO Auto-generated method stub + List ret = new ArrayList(); + if (set != null) { + for (T item : set) { + ret.add(item); + } + } + return ret; + } + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/Sampling.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/Sampling.java new file mode 100755 index 000000000..3d32cc988 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/Sampling.java @@ -0,0 +1,38 @@ +/** + * 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.common.metric.operator; + +import java.io.Serializable; + +public interface Sampling extends Serializable { + + /** + * Update object into Metric + * + * @param obj + */ + void update(Number obj); + + /** + * + * Get snapshot of Metric + * + * @return + */ + V getSnapshot(); +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/StartTime.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/StartTime.java new file mode 100755 index 000000000..0b6173f5d --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/StartTime.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.common.metric.operator; + +public interface StartTime { + long getStartTime(); +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/convert/AtomicLongToLong.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/convert/AtomicLongToLong.java new file mode 100755 index 000000000..8f142f10e --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/convert/AtomicLongToLong.java @@ -0,0 +1,35 @@ +/** + * 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.common.metric.operator.convert; + +import java.util.concurrent.atomic.AtomicLong; + +public class AtomicLongToLong implements Convertor { + private static final long serialVersionUID = -2755066621494409063L; + + @Override + public Long convert(AtomicLong obj) { + // TODO Auto-generated method stub + if (obj == null) { + return null; + } else { + return obj.get(); + } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/convert/Convertor.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/convert/Convertor.java new file mode 100755 index 000000000..73cdceb5d --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/convert/Convertor.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.common.metric.operator.convert; + +import java.io.Serializable; + +public interface Convertor extends Serializable { + + To convert(From obj); +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/convert/DefaultConvertor.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/convert/DefaultConvertor.java new file mode 100755 index 000000000..47065d0fd --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/convert/DefaultConvertor.java @@ -0,0 +1,29 @@ +/** + * 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.common.metric.operator.convert; + +public class DefaultConvertor implements Convertor { + private static final long serialVersionUID = -647209923903679727L; + + @Override + public T convert(T obj) { + // TODO Auto-generated method stub + return obj; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/convert/SetToList.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/convert/SetToList.java new file mode 100755 index 000000000..489122255 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/convert/SetToList.java @@ -0,0 +1,39 @@ +/** + * 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.common.metric.operator.convert; + +import java.util.ArrayList; +import java.util.List; +import java.util.Set; + +public class SetToList implements Convertor, List> { + private static final long serialVersionUID = 4968816655779625255L; + + @Override + public List convert(Set set) { + // TODO Auto-generated method stub + List ret = new ArrayList(); + if (set != null) { + for (T item : set) { + ret.add(item); + } + } + return ret; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/merger/AvgMerger.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/merger/AvgMerger.java new file mode 100755 index 000000000..3ad94f21f --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/merger/AvgMerger.java @@ -0,0 +1,52 @@ +/** + * 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.common.metric.operator.merger; + +import java.util.Collection; + +import com.alibaba.jstorm.common.metric.Histogram; +import com.alibaba.jstorm.utils.Pair; + +public class AvgMerger implements Merger { + private static final long serialVersionUID = -3892281208959055221L; + + @Override + public Histogram.HistorgramPair merge( + Collection objs, + Histogram.HistorgramPair unflushed, Object... others) { + // TODO Auto-generated method stub + double sum = 0.0d; + long times = 0l; + + if (unflushed != null) { + sum = sum + unflushed.getSum(); + times = times + unflushed.getTimes(); + } + + for (Histogram.HistorgramPair item : objs) { + if (item == null) { + continue; + } + sum = sum + item.getSum(); + times = times + item.getTimes(); + } + + return new Histogram.HistorgramPair(sum, times); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/merger/AvgMerger.java.bak b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/merger/AvgMerger.java.bak new file mode 100755 index 000000000..6f8288824 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/merger/AvgMerger.java.bak @@ -0,0 +1,53 @@ +/** + * 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.common.metric.operator.merger; + +import java.util.Collection; +import java.util.concurrent.atomic.AtomicLong; + +import com.alibaba.jstorm.utils.Pair; +import com.google.common.util.concurrent.AtomicDouble; + +public class AvgMerger2 implements Merger> { + private static final long serialVersionUID = -3892281208959055221L; + + @Override + public Pair merge( + Collection> objs, + Pair unflushed, Object... others) { + // TODO Auto-generated method stub + AtomicDouble sum = new AtomicDouble(0.0); + AtomicLong times = new AtomicLong(0); + + if (unflushed != null) { + sum.addAndGet(unflushed.getFirst().get()); + times.addAndGet(unflushed.getSecond().get()); + } + + for (Pair item : objs) { + if (item == null) { + continue; + } + sum.addAndGet(item.getFirst().get()); + times.addAndGet(item.getSecond().get()); + } + + return new Pair(sum, times); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/merger/LongSumMerger.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/merger/LongSumMerger.java new file mode 100755 index 000000000..30ded3491 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/merger/LongSumMerger.java @@ -0,0 +1,43 @@ +/** + * 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.common.metric.operator.merger; + +import java.util.Collection; +import java.util.concurrent.atomic.AtomicLong; + +public class LongSumMerger implements Merger { + private static final long serialVersionUID = -3500779273677666691L; + + @Override + public AtomicLong merge(Collection objs, AtomicLong unflushed, + Object... others) { + AtomicLong ret = new AtomicLong(0); + if (unflushed != null) { + ret.addAndGet(unflushed.get()); + } + + for (AtomicLong item : objs) { + if (item == null) { + continue; + } + ret.addAndGet(item.get()); + } + return ret; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/merger/Merger.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/merger/Merger.java new file mode 100755 index 000000000..048345844 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/merger/Merger.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.common.metric.operator.merger; + +import java.io.Serializable; +import java.util.Collection; + +public interface Merger extends Serializable { + V merge(Collection objs, V unflushed, Object... others); +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/merger/SumMerger.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/merger/SumMerger.java new file mode 100755 index 000000000..ead3c53a9 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/merger/SumMerger.java @@ -0,0 +1,39 @@ +/** + * 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.common.metric.operator.merger; + +import java.util.Collection; + +import com.alibaba.jstorm.utils.JStormUtils; + +public class SumMerger implements Merger { + private static final long serialVersionUID = -7026523452570138433L; + + @SuppressWarnings("unchecked") + @Override + public T merge(Collection objs, T unflushed, Object... others) { + // TODO Auto-generated method stub + T ret = unflushed; + for (T obj : objs) { + ret = (T) JStormUtils.add(ret, obj); + } + + return ret; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/merger/TpsMerger.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/merger/TpsMerger.java new file mode 100755 index 000000000..859f6429b --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/merger/TpsMerger.java @@ -0,0 +1,65 @@ +/** + * 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.common.metric.operator.merger; + +import java.util.Collection; + +import com.alibaba.jstorm.common.metric.operator.StartTime; + +public class TpsMerger implements Merger { + private static final long serialVersionUID = -4534840881635955942L; + protected final long createTime; + + public TpsMerger() { + createTime = System.currentTimeMillis(); + } + + public long getRunMillis(Object... args) { + long startTime = createTime; + + if (args != null) { + if (args[0] != null && args[0] instanceof StartTime) { + StartTime rollingWindow = (StartTime) args[0]; + + startTime = rollingWindow.getStartTime(); + } + } + + return (System.currentTimeMillis() - startTime); + } + + @Override + public Double merge(Collection objs, Double unflushed, + Object... others) { + // TODO Auto-generated method stub + double sum = 0.0d; + if (unflushed != null) { + sum += unflushed; + } + + for (Double item : objs) { + if (item != null) { + sum += item; + } + } + + Double ret = (sum * 1000) / getRunMillis(others); + return ret; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/updater/AddUpdater.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/updater/AddUpdater.java new file mode 100755 index 000000000..4fdf81367 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/updater/AddUpdater.java @@ -0,0 +1,38 @@ +/** + * 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.common.metric.operator.updater; + +import com.alibaba.jstorm.utils.JStormUtils; + +public class AddUpdater implements Updater { + private static final long serialVersionUID = -7955740095421752763L; + + @SuppressWarnings("unchecked") + @Override + public T update(Number object, T cache, Object... others) { + // TODO Auto-generated method stub + return (T) JStormUtils.add(cache, object); + } + + @Override + public T updateBatch(T object, T cache, Object... objects) { + // TODO Auto-generated method stub + return (T) JStormUtils.add(cache, object); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/updater/AvgUpdater.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/updater/AvgUpdater.java new file mode 100755 index 000000000..30ae46c4a --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/updater/AvgUpdater.java @@ -0,0 +1,62 @@ +/** + * 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.common.metric.operator.updater; + +import com.alibaba.jstorm.common.metric.Histogram; + +public class AvgUpdater implements Updater { + private static final long serialVersionUID = 2562836921724586449L; + + @Override + public Histogram.HistorgramPair update(Number object, + Histogram.HistorgramPair cache, Object... others) { + // TODO Auto-generated method stub + if (object == null) { + return cache; + } + if (cache == null) { + cache = + new Histogram.HistorgramPair(); + } + + cache.addValue(object.doubleValue()); + cache.addTimes(1l); + + return cache; + } + + @Override + public Histogram.HistorgramPair updateBatch( + Histogram.HistorgramPair object, + Histogram.HistorgramPair cache, Object... objects) { + // TODO Auto-generated method stub + if (object == null) { + return cache; + } + if (cache == null) { + cache = + new Histogram.HistorgramPair(); + } + + cache.addValue(object.getSum()); + cache.addTimes(object.getTimes()); + + return cache; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/updater/AvgUpdater.java.bak b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/updater/AvgUpdater.java.bak new file mode 100755 index 000000000..44cc70d31 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/updater/AvgUpdater.java.bak @@ -0,0 +1,73 @@ +/** + * 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.common.metric.operator.updater; + +import java.util.concurrent.atomic.AtomicLong; + +import com.alibaba.jstorm.utils.Pair; +import com.google.common.util.concurrent.AtomicDouble; + +public class AvgUpdater2 implements Updater> { + private static final long serialVersionUID = 2562836921724586449L; + + @Override + public Pair update(Number object, + Pair cache, Object... others) { + // TODO Auto-generated method stub + if (object == null) { + return cache; + } + if (cache == null) { + cache = + new Pair(new AtomicDouble(0.0), + new AtomicLong(0)); + } + + AtomicDouble sum = cache.getFirst(); + AtomicLong times = cache.getSecond(); + + sum.addAndGet(object.doubleValue()); + times.incrementAndGet(); + + return cache; + } + + @Override + public Pair updateBatch( + Pair object, + Pair cache, Object... objects) { + // TODO Auto-generated method stub + if (object == null) { + return cache; + } + if (cache == null) { + cache = + new Pair(new AtomicDouble(0.0), + new AtomicLong(0)); + } + + AtomicDouble sum = cache.getFirst(); + AtomicLong times = cache.getSecond(); + + sum.addAndGet(object.getFirst().get()); + times.addAndGet(object.getSecond().get()); + + return cache; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/updater/DoubleAddUpdater.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/updater/DoubleAddUpdater.java new file mode 100755 index 000000000..e3b640aca --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/updater/DoubleAddUpdater.java @@ -0,0 +1,45 @@ +/** + * 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.common.metric.operator.updater; + +import com.google.common.util.concurrent.AtomicDouble; + +public class DoubleAddUpdater implements Updater { + private static final long serialVersionUID = -1293565961076552462L; + + @Override + public AtomicDouble update(Number object, AtomicDouble cache, + Object... others) { + // TODO Auto-generated method stub + if (cache == null) { + cache = new AtomicDouble(0.0); + } + if (object != null) { + cache.addAndGet(object.doubleValue()); + } + return cache; + } + + @Override + public AtomicDouble updateBatch(AtomicDouble object, AtomicDouble cache, + Object... objects) { + // TODO Auto-generated method stub + return update(object, cache, objects); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/updater/LongAddUpdater.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/updater/LongAddUpdater.java new file mode 100755 index 000000000..4986146f0 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/updater/LongAddUpdater.java @@ -0,0 +1,45 @@ +/** + * 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.common.metric.operator.updater; + +import java.util.concurrent.atomic.AtomicLong; + +public class LongAddUpdater implements Updater { + private static final long serialVersionUID = -2185639264737912405L; + + @Override + public AtomicLong update(Number object, AtomicLong cache, Object... others) { + // TODO Auto-generated method stub + if (cache == null) { + cache = new AtomicLong(0); + } + + if (object != null) { + cache.addAndGet(object.longValue()); + } + return cache; + } + + @Override + public AtomicLong updateBatch(AtomicLong object, AtomicLong cache, + Object... objects) { + // TODO Auto-generated method stub + return update(object, cache, objects); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/updater/Updater.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/updater/Updater.java new file mode 100755 index 000000000..cb22c4cf3 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/operator/updater/Updater.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.common.metric.operator.updater; + +import java.io.Serializable; + +public interface Updater extends Serializable { + V update(Number object, V cache, Object... others); + V updateBatch(V object, V cache, Object... objects ); +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/window/AllWindow.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/window/AllWindow.java new file mode 100755 index 000000000..8475e4c82 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/window/AllWindow.java @@ -0,0 +1,78 @@ +/** + * 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.common.metric.window; + +import java.util.ArrayList; + +import com.alibaba.jstorm.common.metric.operator.Sampling; +import com.alibaba.jstorm.common.metric.operator.StartTime; +import com.alibaba.jstorm.common.metric.operator.merger.Merger; +import com.alibaba.jstorm.common.metric.operator.updater.Updater; + +public class AllWindow implements Sampling, StartTime { + + private static final long serialVersionUID = -8523514907315740812L; + + protected V unflushed; + protected V defaultValue; + + protected Updater updater; + protected Merger merger; + protected long startTime; + + AllWindow(V defaultValue, Updater updater, Merger merger) { + + this.updater = updater; + this.merger = merger; + + this.defaultValue = defaultValue; + this.startTime = System.currentTimeMillis(); + } + + @Override + public void update(Number obj) { + // TODO Auto-generated method stub + synchronized (this) { + unflushed = updater.update(obj, unflushed); + } + } + + public void updateBatch(V batch) { + synchronized (this) { + unflushed = updater.updateBatch(batch, unflushed); + } + } + + @Override + public V getSnapshot() { + // TODO Auto-generated method stub + V ret = merger.merge(new ArrayList(), unflushed, this); + if (ret == null) { + return defaultValue; + } else { + return ret; + } + } + + @Override + public long getStartTime() { + // TODO Auto-generated method stub + return startTime; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/window/Metric.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/window/Metric.java new file mode 100755 index 000000000..63a725aa9 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/window/Metric.java @@ -0,0 +1,231 @@ +/** + * 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.common.metric.window; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.callback.Callback; +import com.alibaba.jstorm.common.metric.operator.Sampling; +import com.alibaba.jstorm.common.metric.operator.convert.Convertor; +import com.alibaba.jstorm.common.metric.operator.merger.Merger; +import com.alibaba.jstorm.common.metric.operator.updater.Updater; +import com.alibaba.jstorm.utils.IntervalCheck; + +public class Metric implements Sampling> { + private static final long serialVersionUID = -1362345159511508074L; + private static final Logger LOG = LoggerFactory.getLogger(Metric.class); + + protected static boolean enable; + + public static void setEnable(boolean e) { + enable = e; + } + + protected List> rollingWindows; + protected AllWindow allWindow; + + protected int[] windowSeconds = { StatBuckets.MINUTE_WINDOW, + StatBuckets.HOUR_WINDOW, StatBuckets.DAY_WINDOW }; + protected int bucketSize = StatBuckets.NUM_STAT_BUCKETS; + protected V defaultValue; + protected Updater updater; + protected Merger merger; + protected Convertor convertor; + protected Callback callback; + + protected int interval; // unit is second + protected IntervalCheck intervalCheck; + protected V unflushed; + + public Metric() { + } + + public int getInterval() { + if (windowSeconds == null || windowSeconds.length == 0) { + return StatBuckets.NUM_STAT_BUCKETS; + } + + int intervals[] = new int[windowSeconds.length]; + int smallest = Integer.MAX_VALUE; + for (int i = 0; i < windowSeconds.length; i++) { + int interval = windowSeconds[i] / bucketSize; + intervals[i] = interval; + if (interval < smallest) { + smallest = interval; + } + } + + for (int goodInterval = smallest; goodInterval > 1; goodInterval--) { + boolean good = true; + for (int interval : intervals) { + if (interval % goodInterval != 0) { + good = false; + break; + } + } + + if (good == true) { + return goodInterval; + } + } + + return 1; + } + + public void init() { + if (defaultValue == null || updater == null || merger == null + || convertor == null) { + throw new IllegalArgumentException("Invalid argements"); + } + + rollingWindows = new ArrayList>(); + if (windowSeconds != null) { + rollingWindows.clear(); + for (int windowSize : windowSeconds) { + RollingWindow rollingWindow = + new RollingWindow(defaultValue, windowSize + / bucketSize, windowSize, updater, merger); + + rollingWindows.add(rollingWindow); + } + + } + allWindow = new AllWindow(defaultValue, updater, merger); + + this.interval = getInterval(); + this.intervalCheck = new IntervalCheck(); + this.intervalCheck.setInterval(interval); + } + + /** + * In order to improve performance + * Do + */ + @Override + public void update(Number obj) { + if (enable == false) { + return; + } + + if (intervalCheck.check()) { + flush(); + } + synchronized (this) { + unflushed = updater.update(obj, unflushed); + } + } + + public synchronized void flush() { + if (unflushed == null) { + return; + } + for (RollingWindow rollingWindow : rollingWindows) { + rollingWindow.updateBatch(unflushed); + } + allWindow.updateBatch(unflushed); + unflushed = null; + } + + @Override + public Map getSnapshot() { + // TODO Auto-generated method stub + flush(); + + Map ret = new TreeMap(); + for (RollingWindow rollingWindow : rollingWindows) { + V value = rollingWindow.getSnapshot(); + + ret.put(rollingWindow.getWindowSecond(), convertor.convert(value)); + } + + ret.put(StatBuckets.ALL_TIME_WINDOW, + convertor.convert(allWindow.getSnapshot())); + + if (callback != null) { + callback.execute(this); + } + return ret; + } + + public T getAllTimeValue() { + return convertor.convert(allWindow.getSnapshot()); + } + + public int[] getWindowSeconds() { + return windowSeconds; + } + + public void setWindowSeconds(int[] windowSeconds) { + this.windowSeconds = windowSeconds; + } + + public int getBucketSize() { + return bucketSize; + } + + public void setBucketSize(int bucketSize) { + this.bucketSize = bucketSize; + } + + public V getDefaultValue() { + return defaultValue; + } + + public void setDefaultValue(V defaultValue) { + this.defaultValue = defaultValue; + } + + public Updater getUpdater() { + return updater; + } + + public void setUpdater(Updater updater) { + this.updater = updater; + } + + public Merger getMerger() { + return merger; + } + + public void setMerger(Merger merger) { + this.merger = merger; + } + + public Convertor getConvertor() { + return convertor; + } + + public void setConvertor(Convertor convertor) { + this.convertor = convertor; + } + + public Callback getCallback() { + return callback; + } + + public void setCallback(Callback callback) { + this.callback = callback; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/window/RollingWindow.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/window/RollingWindow.java new file mode 100755 index 000000000..54047a640 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/window/RollingWindow.java @@ -0,0 +1,194 @@ +/** + * 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.common.metric.window; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.TreeMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.common.metric.operator.Sampling; +import com.alibaba.jstorm.common.metric.operator.StartTime; +import com.alibaba.jstorm.common.metric.operator.merger.Merger; +import com.alibaba.jstorm.common.metric.operator.updater.Updater; +import com.alibaba.jstorm.utils.IntervalCheck; +import com.alibaba.jstorm.utils.TimeUtils; + +public class RollingWindow implements Sampling, StartTime { + private static final long serialVersionUID = 3794478417380003279L; + private static final Logger LOG = LoggerFactory + .getLogger(RollingWindow.class); + + protected long startTime; + protected Integer currBucketTime; + protected int interval; // unit is second + protected int windowSecond; + protected IntervalCheck intervalCheck; + + protected TreeMap buckets; + protected Integer bucketNum; + protected V unflushed; + protected V defaultValue; + + protected Updater updater; + protected Merger merger; + + RollingWindow(V defaultValue, int interval, int windowSecond, + Updater updater, Merger merger) { + this.startTime = System.currentTimeMillis(); + this.interval = interval; + this.intervalCheck = new IntervalCheck(); + this.intervalCheck.setInterval(interval); + this.currBucketTime = getCurrBucketTime(); + + this.bucketNum = windowSecond / interval; + this.windowSecond = (bucketNum) * interval; + + this.buckets = new TreeMap(); + + this.updater = updater; + this.merger = merger; + + this.defaultValue = defaultValue; + + } + + + @Override + public void update(Number obj) { + // TODO Auto-generated method stub + + if (intervalCheck.check()) { + rolling(); + } + synchronized (this) { + unflushed = updater.update(obj, unflushed); + + } + + } + + /** + * In order to improve performance + * Flush one batch to rollingWindow + * + */ + public void updateBatch(V batch) { + + if (intervalCheck.check()) { + rolling(); + } + synchronized (this) { + unflushed = updater.updateBatch(batch, unflushed); + } + + } + + @Override + public V getSnapshot() { + // TODO Auto-generated method stub + if (intervalCheck.check()) { + rolling(); + } + + cleanExpiredBuckets(); + // @@@ Testing + //LOG.info("Raw Data:" + buckets + ",unflushed:" + unflushed); + + Collection values = buckets.values(); + + V ret = merger.merge(values, unflushed, this); + if (ret == null) { + + // @@@ testing + //LOG.warn("!!!!Exist null data !!!!!"); + return defaultValue; + } + return ret; + } + + /* + * Move the "current bucket time" index and clean the expired buckets + */ + protected void rolling() { + synchronized (this) { + if (unflushed != null) { + buckets.put(currBucketTime, unflushed); + unflushed = null; + } + + currBucketTime = getCurrBucketTime(); + + return ; + } + } + + protected void cleanExpiredBuckets() { + int nowSec = TimeUtils.current_time_secs(); + int startRemove = nowSec - (interval - 1) - windowSecond; + + List removeList = new ArrayList(); + + for (Integer keyTime : buckets.keySet()) { + if (keyTime < startRemove) { + removeList.add(keyTime); + } else if (keyTime >= startRemove) { + break; + } + } + + for (Integer removeKey : removeList) { + buckets.remove(removeKey); + // @@@ Testing + //LOG.info("Remove key:" + removeKey + ", diff:" + (nowSec - removeKey)); + + } + + if (buckets.isEmpty() == false) { + Integer first = buckets.firstKey(); + startTime = first.longValue() * 1000; + } + } + + public int getWindowSecond() { + return windowSecond; + } + + public long getStartTime() { + return startTime; + } + + public int getInterval() { + return interval; + } + + public Integer getBucketNum() { + return bucketNum; + } + + public V getDefaultValue() { + return defaultValue; + } + + private Integer getCurrBucketTime() { + return (TimeUtils.current_time_secs() / interval) * interval; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/window/StatBuckets.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/window/StatBuckets.java new file mode 100755 index 000000000..3e9b021d8 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/metric/window/StatBuckets.java @@ -0,0 +1,153 @@ +/** + * 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.common.metric.window; + +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; + +public class StatBuckets { + + public static final Integer NUM_STAT_BUCKETS = 20; + + public static final Integer MINUTE_WINDOW = 600; + public static final Integer HOUR_WINDOW = 10800; + public static final Integer DAY_WINDOW = 86400; + public static final Integer ALL_TIME_WINDOW = 0; + public static Set TIME_WINDOWS = new TreeSet(); + static { + TIME_WINDOWS.add(ALL_TIME_WINDOW); + TIME_WINDOWS.add(MINUTE_WINDOW); + TIME_WINDOWS.add(HOUR_WINDOW); + TIME_WINDOWS.add(DAY_WINDOW); + } + + public static final String MINUTE_WINDOW_STR = "0d0h10m0s"; + public static final String HOUR_WINDOW_STR = "0d3h0m0s"; + public static final String DAY_WINDOW_STR = "1d0h0m0s"; + public static final String ALL_WINDOW_STR = "All-time"; + + public static Integer[] STAT_BUCKETS = { MINUTE_WINDOW / NUM_STAT_BUCKETS, + HOUR_WINDOW / NUM_STAT_BUCKETS, DAY_WINDOW / NUM_STAT_BUCKETS }; + + private static final String[][] PRETTYSECDIVIDERS = { + new String[] { "s", "60" }, new String[] { "m", "60" }, + new String[] { "h", "24" }, new String[] { "d", null } }; + + /** + * Service b + * + * @param key + * @return + */ + public static String parseTimeKey(Integer key) { + if (key == 0) { + return ALL_WINDOW_STR; + } else { + return String.valueOf(key); + } + } + + /** + * + * Default is the latest result + * + * @param showKey + * @return + */ + public static Integer getTimeKey(String showKey) { + Integer window = null; + if (showKey == null) { + window = (MINUTE_WINDOW); + } else if (showKey.equals(MINUTE_WINDOW_STR)) { + window = (MINUTE_WINDOW); + } else if (showKey.equals(HOUR_WINDOW_STR)) { + window = (HOUR_WINDOW); + } else if (showKey.equals(DAY_WINDOW_STR)) { + window = (DAY_WINDOW); + } else if (showKey.equals(ALL_WINDOW_STR)) { + window = ALL_TIME_WINDOW; + } else { + window = MINUTE_WINDOW; + } + + return window; + } + + /** + * Default is the latest result + * + * @param showStr + * @return + */ + public static String getShowTimeStr(Integer time) { + if (time == null) { + return MINUTE_WINDOW_STR; + } else if (time.equals(MINUTE_WINDOW)) { + return MINUTE_WINDOW_STR; + } else if (time.equals(HOUR_WINDOW)) { + return HOUR_WINDOW_STR; + } else if (time.equals(DAY_WINDOW)) { + return DAY_WINDOW_STR; + } else if (time.equals(ALL_TIME_WINDOW)) { + return ALL_WINDOW_STR; + } else { + return MINUTE_WINDOW_STR; + } + + } + + /** + * seconds to string like 1d20h30m40s + * + * @param secs + * @return + */ + public static String prettyUptimeStr(int secs) { + int diversize = PRETTYSECDIVIDERS.length; + + List tmp = new ArrayList(); + int div = secs; + for (int i = 0; i < diversize; i++) { + if (PRETTYSECDIVIDERS[i][1] != null) { + Integer d = Integer.parseInt(PRETTYSECDIVIDERS[i][1]); + tmp.add(div % d + PRETTYSECDIVIDERS[i][0]); + div = div / d; + } else { + tmp.add(div + PRETTYSECDIVIDERS[i][0]); + } + } + + String rtn = ""; + int tmpSzie = tmp.size(); + for (int j = tmpSzie - 1; j > -1; j--) { + rtn += tmp.get(j); + } + return rtn; + } + + /** + * @param args + */ + public static void main(String[] args) { + // TODO Auto-generated method stub + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/common/stats/StaticsType.java b/jstorm-core/src/main/java/com/alibaba/jstorm/common/stats/StaticsType.java new file mode 100755 index 000000000..2dbab6f14 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/common/stats/StaticsType.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.common.stats; + +public enum StaticsType { + emitted, send_tps, recv_tps, acked, failed, transferred, process_latencies; +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/CgroupCenter.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/CgroupCenter.java new file mode 100644 index 000000000..d9148db35 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/CgroupCenter.java @@ -0,0 +1,224 @@ +/** + * 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.container; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +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 org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.container.cgroup.CgroupCommon; +import com.alibaba.jstorm.utils.SystemOperation; + +public class CgroupCenter implements CgroupOperation { + + public static Logger LOG = LoggerFactory.getLogger(CgroupCenter.class); + + private static CgroupCenter instance; + + private CgroupCenter() { + + } + + /** + * Thread unsafe + * + * @return + */ + public synchronized static CgroupCenter getInstance() { + if (instance == null) + instance = new CgroupCenter(); + return CgroupUtils.enabled() ? instance : null; + } + + @Override + public List getHierarchies() { + // TODO Auto-generated method stub + Map hierarchies = new HashMap(); + FileReader reader = null; + BufferedReader br = null; + try { + reader = new FileReader(Constants.MOUNT_STATUS_FILE); + br = new BufferedReader(reader); + String str = null; + while ((str = br.readLine()) != null) { + String[] strSplit = str.split(" "); + if (!strSplit[2].equals("cgroup")) + continue; + String name = strSplit[0]; + String type = strSplit[3]; + String dir = strSplit[1]; + Hierarchy h = hierarchies.get(type); + h = new Hierarchy(name, CgroupUtils.analyse(type), dir); + hierarchies.put(type, h); + } + return new ArrayList(hierarchies.values()); + } catch (Exception e) { + LOG.error("Get hierarchies error", e); + } finally { + CgroupUtils.close(reader, br); + } + return null; + } + + @Override + public Set getSubSystems() { + // TODO Auto-generated method stub + Set subSystems = new HashSet(); + FileReader reader = null; + BufferedReader br = null; + try { + reader = new FileReader(Constants.CGROUP_STATUS_FILE); + br = new BufferedReader(reader); + String str = null; + while ((str = br.readLine()) != null) { + String[] split = str.split("\t"); + SubSystemType type = SubSystemType.getSubSystem(split[0]); + if (type == null) + continue; + subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), + Integer.valueOf(split[2]), Integer.valueOf(split[3]) + .intValue() == 1 ? true : false)); + } + return subSystems; + } catch (Exception e) { + LOG.error("Get subSystems error ", e); + } finally { + CgroupUtils.close(reader, br); + } + return null; + } + + @Override + public boolean enabled(SubSystemType subsystem) { + // TODO Auto-generated method stub + Set subSystems = this.getSubSystems(); + for (SubSystem subSystem : subSystems) { + if (subSystem.getType() == subsystem) + return true; + } + return false; + } + + @Override + public Hierarchy busy(SubSystemType subsystem) { + List hierarchies = this.getHierarchies(); + for (Hierarchy hierarchy : hierarchies) { + for (SubSystemType type : hierarchy.getSubSystems()) { + if (type == subsystem) + return hierarchy; + } + } + return null; + } + + @Override + public Hierarchy mounted(Hierarchy hierarchy) { + // TODO Auto-generated method stub + List hierarchies = this.getHierarchies(); + if (CgroupUtils.dirExists(hierarchy.getDir())) { + for (Hierarchy h : hierarchies) { + if (h.equals(hierarchy)) + return h; + } + } + return null; + } + + @Override + public void mount(Hierarchy hierarchy) throws IOException { + // TODO Auto-generated method stub + if (this.mounted(hierarchy) != null) { + LOG.error(hierarchy.getDir() + " is mounted"); + return; + } + Set subsystems = hierarchy.getSubSystems(); + for (SubSystemType type : subsystems) { + if (this.busy(type) != null) { + LOG.error("subsystem: " + type.name() + " is busy"); + subsystems.remove(type); + } + } + if (subsystems.size() == 0) + return; + if (!CgroupUtils.dirExists(hierarchy.getDir())) + new File(hierarchy.getDir()).mkdirs(); + String subSystems = CgroupUtils.reAnalyse(subsystems); + SystemOperation.mount(subSystems, hierarchy.getDir(), "cgroup", + subSystems); + + } + + @Override + public void umount(Hierarchy hierarchy) throws IOException { + // TODO Auto-generated method stub + if (this.mounted(hierarchy) != null) { + hierarchy.getRootCgroups().delete(); + SystemOperation.umount(hierarchy.getDir()); + CgroupUtils.deleteDir(hierarchy.getDir()); + } + } + + @Override + public void create(CgroupCommon cgroup) throws SecurityException { + // TODO Auto-generated method stub + if (cgroup.isRoot()) { + LOG.error("You can't create rootCgroup in this function"); + return; + } + CgroupCommon parent = cgroup.getParent(); + while (parent != null) { + if (!CgroupUtils.dirExists(parent.getDir())) { + LOG.error(parent.getDir() + "is not existed"); + return; + } + parent = parent.getParent(); + } + Hierarchy h = cgroup.getHierarchy(); + if (mounted(h) == null) { + LOG.error(h.getDir() + " is not mounted"); + return; + } + if (CgroupUtils.dirExists(cgroup.getDir())) { + LOG.error(cgroup.getDir() + " is existed"); + return; + } + (new File(cgroup.getDir())).mkdir(); + } + + @Override + public void delete(CgroupCommon cgroup) throws IOException { + // TODO Auto-generated method stub + cgroup.delete(); + } + + public static void main(String args[]) { + System.out.println(CgroupCenter.getInstance().getHierarchies().get(0) + .getRootCgroups().getChildren().size()); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/CgroupOperation.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/CgroupOperation.java new file mode 100755 index 000000000..70306ac35 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/CgroupOperation.java @@ -0,0 +1,46 @@ +/** + * 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.container; + +import java.io.IOException; +import java.util.List; +import java.util.Set; + +import com.alibaba.jstorm.container.cgroup.CgroupCommon; + +public interface CgroupOperation { + + public List getHierarchies(); + + public Set getSubSystems(); + + public boolean enabled(SubSystemType subsystem); + + public Hierarchy busy(SubSystemType subsystem); + + public Hierarchy mounted(Hierarchy hierarchy); + + public void mount(Hierarchy hierarchy) throws IOException; + + public void umount(Hierarchy hierarchy) throws IOException; + + public void create(CgroupCommon cgroup) throws SecurityException; + + public void delete(CgroupCommon cgroup) throws IOException; + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/CgroupUtils.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/CgroupUtils.java new file mode 100644 index 000000000..4de2d5ab2 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/CgroupUtils.java @@ -0,0 +1,178 @@ +/** + * 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.container; + +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.File; +import java.io.FileReader; +import java.io.FileWriter; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.utils.SystemOperation; + +public class CgroupUtils { + + public static final Logger LOG = LoggerFactory.getLogger(CgroupUtils.class); + + public static void deleteDir(String dir) { + try { + String cmd = "rmdir " + dir; + SystemOperation.exec(cmd); + } catch (IOException e) { + // TODO Auto-generated catch block + LOG.error("rm " + dir + " fail!", e); + } + } + + public static boolean fileExists(String dir) { + File file = new File(dir); + return file.exists(); + } + + public static boolean dirExists(String dir) { + File file = new File(dir); + return file.isDirectory(); + } + + public static Set analyse(String str) { + Set result = new HashSet(); + String[] subSystems = str.split(","); + for (String subSystem : subSystems) { + SubSystemType type = SubSystemType.getSubSystem(subSystem); + if (type != null) + result.add(type); + } + return result; + } + + public static String reAnalyse(Set subSystems) { + StringBuilder sb = new StringBuilder(); + if (subSystems.size() == 0) + return sb.toString(); + for (SubSystemType type : subSystems) { + sb.append(type.name()).append(","); + } + return sb.toString().substring(0, sb.length() - 1); + } + + public static boolean enabled() { + return CgroupUtils.fileExists(Constants.CGROUP_STATUS_FILE); + } + + public static List readFileByLine(String fileDir) + throws IOException { + List result = new ArrayList(); + FileReader fileReader = null; + BufferedReader reader = null; + try { + File file = new File(fileDir); + fileReader = new FileReader(file); + reader = new BufferedReader(fileReader); + String tempString = null; + while ((tempString = reader.readLine()) != null) { + result.add(tempString); + } + } finally { + CgroupUtils.close(fileReader, reader); + } + return result; + } + + public static void writeFileByLine(String fileDir, List strings) + throws IOException { + FileWriter writer = null; + BufferedWriter bw = null; + try { + File file = new File(fileDir); + if (!file.exists()) { + LOG.error(fileDir + " is no existed"); + return; + } + writer = new FileWriter(file, true); + bw = new BufferedWriter(writer); + for (String string : strings) { + bw.write(string); + bw.newLine(); + bw.flush(); + } + } finally { + CgroupUtils.close(writer, bw); + } + } + + public static void writeFileByLine(String fileDir, String string) + throws IOException { + FileWriter writer = null; + BufferedWriter bw = null; + try { + File file = new File(fileDir); + if (!file.exists()) { + LOG.error(fileDir + " is no existed"); + return; + } + writer = new FileWriter(file, true); + bw = new BufferedWriter(writer); + bw.write(string); + bw.newLine(); + bw.flush(); + + } finally { + CgroupUtils.close(writer, bw); + } + } + + public static void close(FileReader reader, BufferedReader br) { + try { + if (reader != null) + reader.close(); + if (br != null) + br.close(); + } catch (IOException e) { + // TODO Auto-generated catch block + + } + } + + public static void close(FileWriter writer, BufferedWriter bw) { + try { + if (writer != null) + writer.close(); + if (bw != null) + bw.close(); + } catch (IOException e) { + // TODO Auto-generated catch block + + } + } + + public static void sleep(long s) { + try { + Thread.sleep(s); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + } + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/Constants.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/Constants.java new file mode 100755 index 000000000..774c36778 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/Constants.java @@ -0,0 +1,30 @@ +/** + * 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.container; + +public class Constants { + + public static final String CGROUP_STATUS_FILE = "/proc/cgroups"; + + public static final String MOUNT_STATUS_FILE = "/proc/mounts"; + + public static String getDir(String dir, String constant) { + return dir + constant; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/Hierarchy.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/Hierarchy.java new file mode 100755 index 000000000..b8716d1bc --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/Hierarchy.java @@ -0,0 +1,109 @@ +/** + * 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.container; + +import java.util.Set; + +import com.alibaba.jstorm.container.cgroup.CgroupCommon; + +public class Hierarchy { + + private final String name; + + private final Set subSystems; + + private final String type; + + private final String dir; + + private final CgroupCommon rootCgroups; + + public Hierarchy(String name, Set subSystems, String dir) { + this.name = name; + this.subSystems = subSystems; + this.dir = dir; + this.rootCgroups = new CgroupCommon(this, dir); + this.type = CgroupUtils.reAnalyse(subSystems); + } + + public Set getSubSystems() { + return subSystems; + } + + public String getType() { + return type; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((dir == null) ? 0 : dir.hashCode()); + result = prime * result + ((name == null) ? 0 : name.hashCode()); + result = prime * result + ((type == null) ? 0 : type.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + Hierarchy other = (Hierarchy) obj; + if (dir == null) { + if (other.dir != null) + return false; + } else if (!dir.equals(other.dir)) + return false; + if (name == null) { + if (other.name != null) + return false; + } else if (!name.equals(other.name)) + return false; + if (type == null) { + if (other.type != null) + return false; + } else if (!type.equals(other.type)) + return false; + return true; + } + + public String getDir() { + return dir; + } + + public CgroupCommon getRootCgroups() { + return rootCgroups; + } + + public String getName() { + return name; + } + + public boolean subSystemMounted(SubSystemType subsystem) { + for (SubSystemType type : this.subSystems) { + if (type == subsystem) + return true; + } + return false; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/SubSystem.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/SubSystem.java new file mode 100755 index 000000000..1655e49f3 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/SubSystem.java @@ -0,0 +1,70 @@ +/** + * 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.container; + +public class SubSystem { + + private SubSystemType type; + + private int hierarchyID; + + private int cgroupsNum; + + private boolean enable; + + public SubSystem(SubSystemType type, int hierarchyID, int cgroupNum, + boolean enable) { + this.type = type; + this.hierarchyID = hierarchyID; + this.cgroupsNum = cgroupNum; + this.enable = enable; + } + + public SubSystemType getType() { + return type; + } + + public void setType(SubSystemType type) { + this.type = type; + } + + public int getHierarchyID() { + return hierarchyID; + } + + public void setHierarchyID(int hierarchyID) { + this.hierarchyID = hierarchyID; + } + + public int getCgroupsNum() { + return cgroupsNum; + } + + public void setCgroupsNum(int cgroupsNum) { + this.cgroupsNum = cgroupsNum; + } + + public boolean isEnable() { + return enable; + } + + public void setEnable(boolean enable) { + this.enable = enable; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/SubSystemType.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/SubSystemType.java new file mode 100755 index 000000000..bd710bd2a --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/SubSystemType.java @@ -0,0 +1,48 @@ +/** + * 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.container; + +public enum SubSystemType { + + // net_cls,ns is not supposted in ubuntu + blkio, cpu, cpuacct, cpuset, devices, freezer, memory, perf_event, net_cls, net_prio; + + public static SubSystemType getSubSystem(String str) { + if (str.equals("blkio")) + return blkio; + else if (str.equals("cpu")) + return cpu; + else if (str.equals("cpuacct")) + return cpuacct; + else if (str.equals("cpuset")) + return cpuset; + else if (str.equals("devices")) + return devices; + else if (str.equals("freezer")) + return freezer; + else if (str.equals("memory")) + return memory; + else if (str.equals("perf_event")) + return perf_event; + else if (str.equals("net_cls")) + return net_cls; + else if (str.equals("net_prio")) + return net_prio; + return null; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/CgroupCommon.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/CgroupCommon.java new file mode 100755 index 000000000..0a772f6a7 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/CgroupCommon.java @@ -0,0 +1,249 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.container.cgroup; + +import java.io.File; +import java.io.IOException; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import com.alibaba.jstorm.container.CgroupUtils; +import com.alibaba.jstorm.container.Constants; +import com.alibaba.jstorm.container.Hierarchy; +import com.alibaba.jstorm.container.SubSystemType; +import com.alibaba.jstorm.container.cgroup.core.CgroupCore; + +public class CgroupCommon implements CgroupCommonOperation { + + public static final String TASKS = "/tasks"; + public static final String NOTIFY_ON_RELEASE = "/notify_on_release"; + public static final String RELEASE_AGENT = "/release_agent"; + public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children"; + public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control"; + public static final String CGROUP_PROCS = "/cgroup.procs"; + + private final Hierarchy hierarchy; + + private final String name; + + private final String dir; + + private final CgroupCommon parent; + + private final Map cores; + + private final boolean isRoot; + + private final Set children = new HashSet(); + + public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) { + this.name = parent.getName() + "/" + name; + this.hierarchy = hierarchy; + this.parent = parent; + this.dir = parent.getDir() + "/" + name; + this.init(); + cores = + CgroupCoreFactory.getInstance(this.hierarchy.getSubSystems(), + this.dir); + this.isRoot = false; + } + + /** + * rootCgroup + */ + public CgroupCommon(Hierarchy hierarchy, String dir) { + this.name = ""; + this.hierarchy = hierarchy; + this.parent = null; + this.dir = dir; + this.init(); + cores = + CgroupCoreFactory.getInstance(this.hierarchy.getSubSystems(), + this.dir); + this.isRoot = true; + } + + @Override + public void addTask(int taskId) throws IOException { + // TODO Auto-generated method stub + CgroupUtils.writeFileByLine(Constants.getDir(this.dir, TASKS), + String.valueOf(taskId)); + } + + @Override + public Set getTasks() throws IOException { + List stringTasks = + CgroupUtils.readFileByLine(Constants.getDir(this.dir, TASKS)); + Set tasks = new HashSet(); + for (String task : stringTasks) { + tasks.add(Integer.valueOf(task)); + } + return tasks; + } + + @Override + public void addProcs(int pid) throws IOException { + // TODO Auto-generated method stub + CgroupUtils.writeFileByLine(Constants.getDir(this.dir, CGROUP_PROCS), + String.valueOf(pid)); + } + + @Override + public Set getPids() throws IOException { + // TODO Auto-generated method stub + List stringPids = + CgroupUtils.readFileByLine(Constants.getDir(this.dir, + CGROUP_PROCS)); + Set pids = new HashSet(); + for (String task : stringPids) { + pids.add(Integer.valueOf(task)); + } + return pids; + } + + @Override + public void setNotifyOnRelease(boolean flag) throws IOException { + // TODO Auto-generated method stub + CgroupUtils + .writeFileByLine(Constants.getDir(this.dir, NOTIFY_ON_RELEASE), + flag ? "1" : "0"); + } + + @Override + public boolean getNotifyOnRelease() throws IOException { + return CgroupUtils + .readFileByLine(Constants.getDir(this.dir, NOTIFY_ON_RELEASE)) + .get(0).equals("1") ? true : false; + } + + @Override + public void setReleaseAgent(String command) throws IOException { + // TODO Auto-generated method stub + if (!this.isRoot) + return; + CgroupUtils.writeFileByLine(Constants.getDir(this.dir, RELEASE_AGENT), + command); + } + + @Override + public String getReleaseAgent() throws IOException { + if (!this.isRoot) + return null; + return CgroupUtils.readFileByLine( + Constants.getDir(this.dir, RELEASE_AGENT)).get(0); + } + + @Override + public void setCgroupCloneChildren(boolean flag) throws IOException { + // TODO Auto-generated method stub + if (!this.cores.keySet().contains(SubSystemType.cpuset)) + return; + CgroupUtils.writeFileByLine(Constants.getDir(this.dir, + CGROUP_CLONE_CHILDREN), flag ? "1" : "0"); + } + + @Override + public boolean getCgroupCloneChildren() throws IOException { + return CgroupUtils + .readFileByLine( + Constants.getDir(this.dir, CGROUP_CLONE_CHILDREN)) + .get(0).equals("1") ? true : false; + } + + @Override + public void setEventControl(String eventFd, String controlFd, + String... args) throws IOException { + // TODO Auto-generated method stub + StringBuilder sb = new StringBuilder(); + sb.append(eventFd); + sb.append(' '); + sb.append(controlFd); + for (String arg : args) { + sb.append(' '); + sb.append(arg); + } + CgroupUtils + .writeFileByLine( + Constants.getDir(this.dir, CGROUP_EVENT_CONTROL), + sb.toString()); + } + + public Hierarchy getHierarchy() { + return hierarchy; + } + + public String getName() { + return name; + } + + public String getDir() { + return dir; + } + + public CgroupCommon getParent() { + return parent; + } + + public Set getChildren() { + return children; + } + + public boolean isRoot() { + return isRoot; + } + + public Map getCores() { + return cores; + } + + public void delete() throws IOException { + this.free(); + if (!this.isRoot) + this.parent.getChildren().remove(this); + } + + private void free() throws IOException { + for (CgroupCommon child : this.children) + child.free(); + if (this.isRoot) + return; + Set tasks = this.getTasks(); + if (tasks != null) { + for (Integer task : tasks) { + this.parent.addTask(task); + } + } + CgroupUtils.deleteDir(this.dir); + } + + private void init() { + File file = new File(this.dir); + File[] files = file.listFiles(); + if (files == null) + return; + for (File child : files) { + if (child.isDirectory()) { + this.children.add(new CgroupCommon(child.getName(), + this.hierarchy, this)); + } + } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/CgroupCommonOperation.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/CgroupCommonOperation.java new file mode 100755 index 000000000..3f9090fdd --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/CgroupCommonOperation.java @@ -0,0 +1,48 @@ +/** + * 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.container.cgroup; + +import java.io.IOException; +import java.util.Set; + +public interface CgroupCommonOperation { + + public void addTask(int taskid) throws IOException; + + public Set getTasks() throws IOException; + + public void addProcs(int pid) throws IOException; + + public Set getPids() throws IOException; + + public void setNotifyOnRelease(boolean flag) throws IOException; + + public boolean getNotifyOnRelease() throws IOException; + + public void setReleaseAgent(String command) throws IOException; + + public String getReleaseAgent() throws IOException; + + public void setCgroupCloneChildren(boolean flag) throws IOException; + + public boolean getCgroupCloneChildren() throws IOException; + + public void setEventControl(String eventFd, String controlFd, + String... args) throws IOException; + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/CgroupCoreFactory.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/CgroupCoreFactory.java new file mode 100755 index 000000000..279366aef --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/CgroupCoreFactory.java @@ -0,0 +1,78 @@ +/** + * 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.container.cgroup; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import com.alibaba.jstorm.container.SubSystemType; +import com.alibaba.jstorm.container.cgroup.core.BlkioCore; +import com.alibaba.jstorm.container.cgroup.core.CgroupCore; +import com.alibaba.jstorm.container.cgroup.core.CpuCore; +import com.alibaba.jstorm.container.cgroup.core.CpuacctCore; +import com.alibaba.jstorm.container.cgroup.core.CpusetCore; +import com.alibaba.jstorm.container.cgroup.core.DevicesCore; +import com.alibaba.jstorm.container.cgroup.core.FreezerCore; +import com.alibaba.jstorm.container.cgroup.core.MemoryCore; +import com.alibaba.jstorm.container.cgroup.core.NetClsCore; +import com.alibaba.jstorm.container.cgroup.core.NetPrioCore; + +public class CgroupCoreFactory { + + public static Map getInstance( + Set types, String dir) { + Map result = + new HashMap(); + for (SubSystemType type : types) { + switch (type) { + case blkio: + result.put(SubSystemType.blkio, new BlkioCore(dir)); + break; + case cpuacct: + result.put(SubSystemType.cpuacct, new CpuacctCore(dir)); + break; + case cpuset: + result.put(SubSystemType.cpuset, new CpusetCore(dir)); + break; + case cpu: + result.put(SubSystemType.cpu, new CpuCore(dir)); + break; + case devices: + result.put(SubSystemType.devices, new DevicesCore(dir)); + break; + case freezer: + result.put(SubSystemType.freezer, new FreezerCore(dir)); + break; + case memory: + result.put(SubSystemType.memory, new MemoryCore(dir)); + break; + case net_cls: + result.put(SubSystemType.net_cls, new NetClsCore(dir)); + break; + case net_prio: + result.put(SubSystemType.net_prio, new NetPrioCore(dir)); + break; + default: + break; + } + } + return result; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/Device.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/Device.java new file mode 100755 index 000000000..fff2c673a --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/Device.java @@ -0,0 +1,68 @@ +/** + * 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.container.cgroup; + +public class Device { + + public final int major; + public final int minor; + + public Device(int major, int minor) { + this.major = major; + this.minor = minor; + } + + public Device(String str) { + String[] strArgs = str.split(":"); + this.major = Integer.valueOf(strArgs[0]); + this.minor = Integer.valueOf(strArgs[1]); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(major).append(":").append(minor); + return sb.toString(); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + major; + result = prime * result + minor; + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + Device other = (Device) obj; + if (major != other.major) + return false; + if (minor != other.minor) + return false; + return true; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/BlkioCore.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/BlkioCore.java new file mode 100755 index 000000000..5d487ec25 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/BlkioCore.java @@ -0,0 +1,306 @@ +/** + * 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.container.cgroup.core; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import com.alibaba.jstorm.container.CgroupUtils; +import com.alibaba.jstorm.container.Constants; +import com.alibaba.jstorm.container.SubSystemType; +import com.alibaba.jstorm.container.cgroup.Device; + +public class BlkioCore implements CgroupCore { + + public static final String BLKIO_WEIGHT = "/blkio.weight"; + public static final String BLKIO_WEIGHT_DEVICE = "/blkio.weight_device"; + public static final String BLKIO_RESET_STATS = "/blkio.reset_stats"; + + public static final String BLKIO_THROTTLE_READ_BPS_DEVICE = + "/blkio.throttle.read_bps_device"; + public static final String BLKIO_THROTTLE_WRITE_BPS_DEVICE = + "/blkio.throttle.write_bps_device"; + public static final String BLKIO_THROTTLE_READ_IOPS_DEVICE = + "/blkio.throttle.read_iops_device"; + public static final String BLKIO_THROTTLE_WRITE_IOPS_DEVICE = + "/blkio.throttle.write_iops_device"; + + public static final String BLKIO_THROTTLE_IO_SERVICED = + "/blkio.throttle.io_serviced"; + public static final String BLKIO_THROTTLE_IO_SERVICE_BYTES = + "/blkio.throttle.io_service_bytes"; + + public static final String BLKIO_TIME = "/blkio.time"; + public static final String BLKIO_SECTORS = "/blkio.sectors"; + public static final String BLKIO_IO_SERVICED = "/blkio.io_serviced"; + public static final String BLKIO_IO_SERVICE_BYTES = + "/blkio.io_service_bytes"; + public static final String BLKIO_IO_SERVICE_TIME = "/blkio.io_service_time"; + public static final String BLKIO_IO_WAIT_TIME = "/blkio.io_wait_time"; + public static final String BLKIO_IO_MERGED = "/blkio.io_merged"; + public static final String BLKIO_IO_QUEUED = "/blkio.io_queued"; + + private final String dir; + + public BlkioCore(String dir) { + this.dir = dir; + } + + @Override + public SubSystemType getType() { + // TODO Auto-generated method stub + return SubSystemType.blkio; + } + + /* weight: 100-1000 */ + public void setBlkioWeight(int weight) throws IOException { + CgroupUtils.writeFileByLine(Constants.getDir(this.dir, BLKIO_WEIGHT), + String.valueOf(weight)); + } + + public int getBlkioWeight() throws IOException { + return Integer.valueOf( + CgroupUtils.readFileByLine( + Constants.getDir(this.dir, BLKIO_WEIGHT)).get(0)) + .intValue(); + } + + public void setBlkioWeightDevice(Device device, int weight) + throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, BLKIO_WEIGHT_DEVICE), + makeContext(device, weight)); + } + + public Map getBlkioWeightDevice() throws IOException { + List strings = + CgroupUtils.readFileByLine(Constants.getDir(this.dir, + BLKIO_WEIGHT_DEVICE)); + Map result = new HashMap(); + for (String string : strings) { + String[] strArgs = string.split(" "); + Device device = new Device(strArgs[0]); + Integer weight = Integer.valueOf(strArgs[1]); + result.put(device, weight); + } + return result; + } + + public void setReadBps(Device device, long bps) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, BLKIO_THROTTLE_READ_BPS_DEVICE), + makeContext(device, bps)); + } + + public Map getReadBps() throws IOException { + List strings = + CgroupUtils.readFileByLine(Constants.getDir(this.dir, + BLKIO_THROTTLE_READ_BPS_DEVICE)); + Map result = new HashMap(); + for (String string : strings) { + String[] strArgs = string.split(" "); + Device device = new Device(strArgs[0]); + Long bps = Long.valueOf(strArgs[1]); + result.put(device, bps); + } + return result; + } + + public void setWriteBps(Device device, long bps) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, BLKIO_THROTTLE_WRITE_BPS_DEVICE), + makeContext(device, bps)); + } + + public Map getWriteBps() throws IOException { + List strings = + CgroupUtils.readFileByLine(Constants.getDir(this.dir, + BLKIO_THROTTLE_WRITE_BPS_DEVICE)); + Map result = new HashMap(); + for (String string : strings) { + String[] strArgs = string.split(" "); + Device device = new Device(strArgs[0]); + Long bps = Long.valueOf(strArgs[1]); + result.put(device, bps); + } + return result; + } + + public void setReadIOps(Device device, long iops) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, BLKIO_THROTTLE_READ_IOPS_DEVICE), + makeContext(device, iops)); + } + + public Map getReadIOps() throws IOException { + List strings = + CgroupUtils.readFileByLine(Constants.getDir(this.dir, + BLKIO_THROTTLE_READ_IOPS_DEVICE)); + Map result = new HashMap(); + for (String string : strings) { + String[] strArgs = string.split(" "); + Device device = new Device(strArgs[0]); + Long iops = Long.valueOf(strArgs[1]); + result.put(device, iops); + } + return result; + } + + public void setWriteIOps(Device device, long iops) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, BLKIO_THROTTLE_WRITE_IOPS_DEVICE), + makeContext(device, iops)); + } + + public Map getWriteIOps() throws IOException { + List strings = + CgroupUtils.readFileByLine(Constants.getDir(this.dir, + BLKIO_THROTTLE_WRITE_IOPS_DEVICE)); + Map result = new HashMap(); + for (String string : strings) { + String[] strArgs = string.split(" "); + Device device = new Device(strArgs[0]); + Long iops = Long.valueOf(strArgs[1]); + result.put(device, iops); + } + return result; + } + + public Map> getThrottleIOServiced() + throws IOException { + return this.analyseRecord(CgroupUtils.readFileByLine(Constants.getDir( + this.dir, BLKIO_THROTTLE_IO_SERVICED))); + } + + public Map> getThrottleIOServiceByte() + throws IOException { + return this.analyseRecord(CgroupUtils.readFileByLine(Constants.getDir( + this.dir, BLKIO_THROTTLE_IO_SERVICE_BYTES))); + } + + public Map getBlkioTime() throws IOException { + Map result = new HashMap(); + List strs = + CgroupUtils.readFileByLine(Constants.getDir(this.dir, + BLKIO_TIME)); + for (String str : strs) { + String[] strArgs = str.split(" "); + result.put(new Device(strArgs[0]), Long.parseLong(strArgs[1])); + } + return result; + } + + public Map getBlkioSectors() throws IOException { + Map result = new HashMap(); + List strs = + CgroupUtils.readFileByLine(Constants.getDir(this.dir, + BLKIO_SECTORS)); + for (String str : strs) { + String[] strArgs = str.split(" "); + result.put(new Device(strArgs[0]), Long.parseLong(strArgs[1])); + } + return result; + } + + public Map> getIOServiced() + throws IOException { + return this.analyseRecord(CgroupUtils.readFileByLine(Constants.getDir( + this.dir, BLKIO_IO_SERVICED))); + } + + public Map> getIOServiceBytes() + throws IOException { + return this.analyseRecord(CgroupUtils.readFileByLine(Constants.getDir( + this.dir, BLKIO_IO_SERVICE_BYTES))); + } + + public Map> getIOServiceTime() + throws IOException { + return this.analyseRecord(CgroupUtils.readFileByLine(Constants.getDir( + this.dir, BLKIO_IO_SERVICE_TIME))); + } + + public Map> getIOWaitTime() + throws IOException { + return this.analyseRecord(CgroupUtils.readFileByLine(Constants.getDir( + this.dir, BLKIO_IO_WAIT_TIME))); + } + + public Map> getIOMerged() throws IOException { + return this.analyseRecord(CgroupUtils.readFileByLine(Constants.getDir( + this.dir, BLKIO_IO_MERGED))); + } + + public Map> getIOQueued() throws IOException { + return this.analyseRecord(CgroupUtils.readFileByLine(Constants.getDir( + this.dir, BLKIO_IO_QUEUED))); + } + + public void resetStats() throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, BLKIO_RESET_STATS), "1"); + } + + private String makeContext(Device device, Object data) { + StringBuilder sb = new StringBuilder(); + sb.append(device.toString()).append(" ").append(data); + return sb.toString(); + } + + private Map> analyseRecord(List strs) { + Map> result = + new HashMap>(); + for (String str : strs) { + String[] strArgs = str.split(" "); + if (strArgs.length != 3) + continue; + Device device = new Device(strArgs[0]); + RecordType key = RecordType.getType(strArgs[1]); + Long value = Long.parseLong(strArgs[2]); + Map record = result.get(device); + if (record == null) { + record = new HashMap(); + result.put(device, record); + } + record.put(key, value); + } + return result; + } + + public enum RecordType { + read, write, sync, async, total; + + public static RecordType getType(String type) { + if (type.equals("Read")) + return read; + else if (type.equals("Write")) + return write; + else if (type.equals("Sync")) + return sync; + else if (type.equals("Async")) + return async; + else if (type.equals("Total")) + return total; + else + return null; + } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/CgroupCore.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/CgroupCore.java new file mode 100755 index 000000000..eab5c51bf --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/CgroupCore.java @@ -0,0 +1,26 @@ +/** + * 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.container.cgroup.core; + +import com.alibaba.jstorm.container.SubSystemType; + +public interface CgroupCore { + + public SubSystemType getType(); + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/CpuCore.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/CpuCore.java new file mode 100755 index 000000000..609898e46 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/CpuCore.java @@ -0,0 +1,148 @@ +/** + * 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.container.cgroup.core; + +import java.io.IOException; +import java.util.List; + +import com.alibaba.jstorm.container.CgroupUtils; +import com.alibaba.jstorm.container.Constants; +import com.alibaba.jstorm.container.SubSystemType; + +public class CpuCore implements CgroupCore { + + public static final String CPU_SHARES = "/cpu.shares"; + public static final String CPU_RT_RUNTIME_US = "/cpu.rt_runtime_us"; + public static final String CPU_RT_PERIOD_US = "/cpu.rt_period_us"; + public static final String CPU_CFS_PERIOD_US = "/cpu.cfs_period_us"; + public static final String CPU_CFS_QUOTA_US = "/cpu.cfs_quota_us"; + public static final String CPU_STAT = "/cpu.stat"; + + private final String dir; + + public CpuCore(String dir) { + this.dir = dir; + } + + @Override + public SubSystemType getType() { + // TODO Auto-generated method stub + return SubSystemType.cpu; + } + + public void setCpuShares(int weight) throws IOException { + CgroupUtils.writeFileByLine(Constants.getDir(this.dir, CPU_SHARES), + String.valueOf(weight)); + } + + public int getCpuShares() throws IOException { + return Integer.parseInt(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, CPU_SHARES)).get(0)); + } + + public void setCpuRtRuntimeUs(long us) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, CPU_RT_RUNTIME_US), + String.valueOf(us)); + } + + public long getCpuRtRuntimeUs() throws IOException { + return Long.parseLong(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, CPU_RT_RUNTIME_US)).get(0)); + } + + public void setCpuRtPeriodUs(long us) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, CPU_RT_PERIOD_US), + String.valueOf(us)); + } + + public Long getCpuRtPeriodUs() throws IOException { + return Long.parseLong(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, CPU_RT_PERIOD_US)).get(0)); + } + + public void setCpuCfsPeriodUs(long us) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, CPU_CFS_PERIOD_US), + String.valueOf(us)); + } + + public Long getCpuCfsPeriodUs(long us) throws IOException { + return Long.parseLong(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, CPU_CFS_PERIOD_US)).get(0)); + } + + public void setCpuCfsQuotaUs(long us) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, CPU_CFS_QUOTA_US), + String.valueOf(us)); + } + + public Long getCpuCfsQuotaUs(long us) throws IOException { + return Long.parseLong(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, CPU_CFS_QUOTA_US)).get(0)); + } + + public Stat getCpuStat() throws IOException { + return new Stat(CgroupUtils.readFileByLine(Constants.getDir(this.dir, + CPU_STAT))); + } + + public static class Stat { + public final int nrPeriods; + public final int nrThrottled; + public final int throttledTime; + + public Stat(List statStr) { + this.nrPeriods = Integer.parseInt(statStr.get(0).split(" ")[1]); + this.nrThrottled = Integer.parseInt(statStr.get(1).split(" ")[1]); + this.throttledTime = Integer.parseInt(statStr.get(2).split(" ")[1]); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + nrPeriods; + result = prime * result + nrThrottled; + result = prime * result + throttledTime; + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + Stat other = (Stat) obj; + if (nrPeriods != other.nrPeriods) + return false; + if (nrThrottled != other.nrThrottled) + return false; + if (throttledTime != other.throttledTime) + return false; + return true; + } + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/CpuacctCore.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/CpuacctCore.java new file mode 100755 index 000000000..c54421b7a --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/CpuacctCore.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.container.cgroup.core; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import com.alibaba.jstorm.container.CgroupUtils; +import com.alibaba.jstorm.container.Constants; +import com.alibaba.jstorm.container.SubSystemType; + +public class CpuacctCore implements CgroupCore { + + public static final String CPUACCT_USAGE = "/cpuacct.usage"; + public static final String CPUACCT_STAT = "/cpuacct.stat"; + public static final String CPUACCT_USAGE_PERCPU = "/cpuacct.usage_percpu"; + + private final String dir; + + public CpuacctCore(String dir) { + this.dir = dir; + } + + @Override + public SubSystemType getType() { + // TODO Auto-generated method stub + return SubSystemType.cpuacct; + } + + public Long getCpuUsage() throws IOException { + return Long.parseLong(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, CPUACCT_USAGE)).get(0)); + } + + public Map getCpuStat() throws IOException { + List strs = + CgroupUtils.readFileByLine(Constants.getDir(this.dir, + CPUACCT_STAT)); + Map result = new HashMap(); + result.put(StatType.user, Long.parseLong(strs.get(0).split(" ")[1])); + result.put(StatType.system, Long.parseLong(strs.get(1).split(" ")[1])); + return result; + } + + public Long[] getPerCpuUsage() throws IOException { + String str = + CgroupUtils.readFileByLine( + Constants.getDir(this.dir, CPUACCT_USAGE_PERCPU)) + .get(0); + String[] strArgs = str.split(" "); + Long[] result = new Long[strArgs.length]; + for (int i = 0; i < result.length; i++) { + result[i] = Long.parseLong(strArgs[i]); + } + return result; + } + + public enum StatType { + user, system; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/CpusetCore.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/CpusetCore.java new file mode 100755 index 000000000..d693b6c0e --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/CpusetCore.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.container.cgroup.core; + +import java.io.IOException; +import java.util.LinkedList; + +import com.alibaba.jstorm.container.CgroupUtils; +import com.alibaba.jstorm.container.Constants; +import com.alibaba.jstorm.container.SubSystemType; + +public class CpusetCore implements CgroupCore { + + public static final String CPUSET_CPUS = "/cpuset.cpus"; + public static final String CPUSET_MEMS = "/cpuset.mems"; + public static final String CPUSET_MEMORY_MIGRATE = "/cpuset.memory_migrate"; + public static final String CPUSET_CPU_EXCLUSIVE = "/cpuset.cpu_exclusive"; + public static final String CPUSET_MEM_EXCLUSIVE = "/cpuset.mem_exclusive"; + public static final String CPUSET_MEM_HARDWALL = "/cpuset.mem_hardwall"; + public static final String CPUSET_MEMORY_PRESSURE = + "/cpuset.memory_pressure"; + public static final String CPUSET_MEMORY_PRESSURE_ENABLED = + "/cpuset.memory_pressure_enabled"; + public static final String CPUSET_MEMORY_SPREAD_PAGE = + "/cpuset.memory_spread_page"; + public static final String CPUSET_MEMORY_SPREAD_SLAB = + "/cpuset.memory_spread_slab"; + public static final String CPUSET_SCHED_LOAD_BALANCE = + "/cpuset.sched_load_balance"; + public static final String CPUSET_SCHED_RELAX_DOMAIN_LEVEL = + "/cpuset.sched_relax_domain_level"; + + private final String dir; + + public CpusetCore(String dir) { + this.dir = dir; + } + + @Override + public SubSystemType getType() { + // TODO Auto-generated method stub + return SubSystemType.cpuset; + } + + public void setCpus(int[] nums) throws IOException { + StringBuilder sb = new StringBuilder(); + for (int num : nums) { + sb.append(num); + sb.append(','); + } + sb.deleteCharAt(sb.length() - 1); + CgroupUtils.writeFileByLine(Constants.getDir(this.dir, CPUSET_CPUS), + sb.toString()); + } + + public int[] getCpus() throws IOException { + String output = + CgroupUtils.readFileByLine( + Constants.getDir(this.dir, CPUSET_CPUS)).get(0); + return parseNums(output); + } + + public void setMems(int[] nums) throws IOException { + StringBuilder sb = new StringBuilder(); + for (int num : nums) { + sb.append(num); + sb.append(','); + } + sb.deleteCharAt(sb.length() - 1); + CgroupUtils.writeFileByLine(Constants.getDir(this.dir, CPUSET_MEMS), + sb.toString()); + } + + public int[] getMems() throws IOException { + String output = + CgroupUtils.readFileByLine( + Constants.getDir(this.dir, CPUSET_MEMS)).get(0); + return parseNums(output); + } + + public void setMemMigrate(boolean flag) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, CPUSET_MEMORY_MIGRATE), + String.valueOf(flag ? 1 : 0)); + } + + public boolean isMemMigrate() throws IOException { + int output = + Integer.parseInt(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, CPUSET_MEMORY_MIGRATE)).get( + 0)); + return output > 0; + } + + public void setCpuExclusive(boolean flag) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, CPUSET_CPU_EXCLUSIVE), + String.valueOf(flag ? 1 : 0)); + } + + public boolean isCpuExclusive() throws IOException { + int output = + Integer.parseInt(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, CPUSET_CPU_EXCLUSIVE)) + .get(0)); + return output > 0; + } + + public void setMemExclusive(boolean flag) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, CPUSET_MEM_EXCLUSIVE), + String.valueOf(flag ? 1 : 0)); + } + + public boolean isMemExclusive() throws IOException { + int output = + Integer.parseInt(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, CPUSET_MEM_EXCLUSIVE)) + .get(0)); + return output > 0; + } + + public void setMemHardwall(boolean flag) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, CPUSET_MEM_HARDWALL), + String.valueOf(flag ? 1 : 0)); + } + + public boolean isMemHardwall() throws IOException { + int output = + Integer.parseInt(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, CPUSET_MEM_HARDWALL)).get(0)); + return output > 0; + } + + public int getMemPressure() throws IOException { + String output = + CgroupUtils.readFileByLine( + Constants.getDir(this.dir, CPUSET_MEMORY_PRESSURE)) + .get(0); + return Integer.parseInt(output); + } + + public void setMemPressureEnabled(boolean flag) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, CPUSET_MEMORY_PRESSURE_ENABLED), + String.valueOf(flag ? 1 : 0)); + } + + public boolean isMemPressureEnabled() throws IOException { + int output = + Integer.parseInt(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, + CPUSET_MEMORY_PRESSURE_ENABLED)).get(0)); + return output > 0; + } + + public void setMemSpreadPage(boolean flag) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, CPUSET_MEMORY_SPREAD_PAGE), + String.valueOf(flag ? 1 : 0)); + } + + public boolean isMemSpreadPage() throws IOException { + int output = + Integer.parseInt(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, CPUSET_MEMORY_SPREAD_PAGE)) + .get(0)); + return output > 0; + } + + public void setMemSpreadSlab(boolean flag) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, CPUSET_MEMORY_SPREAD_SLAB), + String.valueOf(flag ? 1 : 0)); + } + + public boolean isMemSpreadSlab() throws IOException { + int output = + Integer.parseInt(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, CPUSET_MEMORY_SPREAD_SLAB)) + .get(0)); + return output > 0; + } + + public void setSchedLoadBlance(boolean flag) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, CPUSET_SCHED_LOAD_BALANCE), + String.valueOf(flag ? 1 : 0)); + } + + public boolean isSchedLoadBlance() throws IOException { + int output = + Integer.parseInt(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, CPUSET_SCHED_LOAD_BALANCE)) + .get(0)); + return output > 0; + } + + public void setSchedRelaxDomainLevel(int value) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, CPUSET_SCHED_RELAX_DOMAIN_LEVEL), + String.valueOf(value)); + } + + public int getSchedRelaxDomainLevel() throws IOException { + String output = + CgroupUtils.readFileByLine( + Constants.getDir(this.dir, + CPUSET_SCHED_RELAX_DOMAIN_LEVEL)).get(0); + return Integer.parseInt(output); + } + + public static int[] parseNums(String outputStr) { + char[] output = outputStr.toCharArray(); + LinkedList numList = new LinkedList(); + int value = 0; + int start = 0; + boolean isHyphen = false; + for (char ch : output) { + if (ch == ',') { + if (isHyphen) { + for (; start <= value; start++) { + numList.add(start); + } + isHyphen = false; + } else { + numList.add(value); + } + value = 0; + } else if (ch == '-') { + isHyphen = true; + start = value; + value = 0; + } else { + value = value * 10 + (ch - '0'); + } + } + if (output[output.length - 1] != ',') { + if (isHyphen) { + for (; start <= value; start++) { + numList.add(start); + } + } else { + numList.add(value); + } + } + + int[] nums = new int[numList.size()]; + int index = 0; + for (int num : numList) { + nums[index] = num; + index++; + } + + return nums; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/DevicesCore.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/DevicesCore.java new file mode 100755 index 000000000..18326687f --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/DevicesCore.java @@ -0,0 +1,188 @@ +/** + * 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.container.cgroup.core; + +import java.io.IOException; +import java.util.List; + +import com.alibaba.jstorm.container.CgroupUtils; +import com.alibaba.jstorm.container.Constants; +import com.alibaba.jstorm.container.SubSystemType; +import com.alibaba.jstorm.container.cgroup.Device; + +public class DevicesCore implements CgroupCore { + + private final String dir; + + public static final String DEVICES_ALLOW = "/devices.allow"; + public static final String DEVICES_DENY = "/devices.deny"; + public static final String DEVICES_LIST = "/devices.list"; + + public static final char TYPE_ALL = 'a'; + public static final char TYPE_BLOCK = 'b'; + public static final char TYPE_CHAR = 'c'; + + public static final int ACCESS_READ = 1; + public static final int ACCESS_WRITE = 2; + public static final int ACCESS_CREATE = 4; + + public static final char ACCESS_READ_CH = 'r'; + public static final char ACCESS_WRITE_CH = 'w'; + public static final char ACCESS_CREATE_CH = 'm'; + + public DevicesCore(String dir) { + this.dir = dir; + } + + @Override + public SubSystemType getType() { + // TODO Auto-generated method stub + return SubSystemType.devices; + } + + public static class Record { + Device device; + char type; + int accesses; + + public Record(char type, Device device, int accesses) { + this.type = type; + this.device = device; + this.accesses = accesses; + } + + public Record(String output) { + if (output.contains("*")) { + System.out.println("Pre:" + output); + output = output.replaceAll("\\*", "-1"); + System.out.println("After:" + output); + } + String[] splits = output.split("[: ]"); + type = splits[0].charAt(0); + int major = Integer.parseInt(splits[1]); + int minor = Integer.parseInt(splits[2]); + device = new Device(major, minor); + accesses = 0; + for (char c : splits[3].toCharArray()) { + if (c == ACCESS_READ_CH) { + accesses |= ACCESS_READ; + } + if (c == ACCESS_CREATE_CH) { + accesses |= ACCESS_CREATE; + } + if (c == ACCESS_WRITE_CH) { + accesses |= ACCESS_WRITE; + } + } + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(type); + sb.append(' '); + sb.append(device.major); + sb.append(':'); + sb.append(device.minor); + sb.append(' '); + sb.append(getAccessesFlag(accesses)); + + return sb.toString(); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + accesses; + result = + prime * result + ((device == null) ? 0 : device.hashCode()); + result = prime * result + type; + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + Record other = (Record) obj; + if (accesses != other.accesses) + return false; + if (device == null) { + if (other.device != null) + return false; + } else if (!device.equals(other.device)) + return false; + if (type != other.type) + return false; + return true; + } + + public static Record[] parseRecordList(List output) { + Record[] records = new Record[output.size()]; + for (int i = 0, l = output.size(); i < l; i++) { + records[i] = new Record(output.get(i)); + } + + return records; + } + + public static StringBuilder getAccessesFlag(int accesses) { + StringBuilder sb = new StringBuilder(); + if ((accesses & ACCESS_READ) != 0) { + sb.append(ACCESS_READ_CH); + } + if ((accesses & ACCESS_WRITE) != 0) { + sb.append(ACCESS_WRITE_CH); + } + if ((accesses & ACCESS_CREATE) != 0) { + sb.append(ACCESS_CREATE_CH); + } + return sb; + } + } + + private void setPermission(String prop, char type, Device device, + int accesses) throws IOException { + Record record = new Record(type, device, accesses); + CgroupUtils.writeFileByLine(Constants.getDir(this.dir, prop), + record.toString()); + } + + public void setAllow(char type, Device device, int accesses) + throws IOException { + setPermission(DEVICES_ALLOW, type, device, accesses); + } + + public void setDeny(char type, Device device, int accesses) + throws IOException { + setPermission(DEVICES_DENY, type, device, accesses); + } + + public Record[] getList() throws IOException { + List output = + CgroupUtils.readFileByLine(Constants.getDir(this.dir, + DEVICES_LIST)); + return Record.parseRecordList(output); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/FreezerCore.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/FreezerCore.java new file mode 100755 index 000000000..c601c3e1e --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/FreezerCore.java @@ -0,0 +1,67 @@ +/** + * 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.container.cgroup.core; + +import java.io.IOException; + +import com.alibaba.jstorm.container.CgroupUtils; +import com.alibaba.jstorm.container.Constants; +import com.alibaba.jstorm.container.SubSystemType; + +public class FreezerCore implements CgroupCore { + + public static final String FREEZER_STATE = "/freezer.state"; + + private final String dir; + + public FreezerCore(String dir) { + this.dir = dir; + } + + @Override + public SubSystemType getType() { + // TODO Auto-generated method stub + return SubSystemType.freezer; + } + + public void setState(State state) throws IOException { + CgroupUtils.writeFileByLine(Constants.getDir(this.dir, FREEZER_STATE), + state.name().toUpperCase()); + } + + public State getState() throws IOException { + return State.getStateValue(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, FREEZER_STATE)).get(0)); + } + + public enum State { + frozen, freezing, thawed; + + public static State getStateValue(String state) { + if (state.equals("FROZEN")) + return frozen; + else if (state.equals("FREEZING")) + return freezing; + else if (state.equals("THAWED")) + return thawed; + else + return null; + } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/MemoryCore.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/MemoryCore.java new file mode 100755 index 000000000..a2db78c5c --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/MemoryCore.java @@ -0,0 +1,229 @@ +/** + * 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.container.cgroup.core; + +import java.io.IOException; + +import com.alibaba.jstorm.container.CgroupUtils; +import com.alibaba.jstorm.container.Constants; +import com.alibaba.jstorm.container.SubSystemType; + +public class MemoryCore implements CgroupCore { + + public static final String MEMORY_STAT = "/memory.stat"; + public static final String MEMORY_USAGE_IN_BYTES = "/memory.usage_in_bytes"; + public static final String MEMORY_MEMSW_USAGE_IN_BYTES = + "/memory.memsw.usage_in_bytes"; + public static final String MEMORY_MAX_USAGE_IN_BYTES = + "/memory.max_usage_in_bytes"; + public static final String MEMORY_MEMSW_MAX_USAGE_IN_BYTES = + "/memory.memsw.max_usage_in_bytes"; + public static final String MEMORY_LIMIT_IN_BYTES = "/memory.limit_in_bytes"; + public static final String MEMORY_MEMSW_LIMIT_IN_BYTES = + "/memory.memsw.limit_in_bytes"; + public static final String MEMORY_FAILCNT = "/memory.failcnt"; + public static final String MEMORY_MEMSW_FAILCNT = "/memory.memsw.failcnt"; + public static final String MEMORY_FORCE_EMPTY = "/memory.force_empty"; + public static final String MEMORY_SWAPPINESS = "/memory.swappiness"; + public static final String MEMORY_USE_HIERARCHY = "/memory.use_hierarchy"; + public static final String MEMORY_OOM_CONTROL = "/memory.oom_control"; + + private final String dir; + + public MemoryCore(String dir) { + this.dir = dir; + } + + @Override + public SubSystemType getType() { + // TODO Auto-generated method stub + return SubSystemType.memory; + } + + public static class Stat { + public final long cacheSize; + public final long rssSize; + public final long mappedFileSize; + public final long pgpginNum; + public final long pgpgoutNum; + public final long swapSize; + public final long activeAnonSize; + public final long inactiveAnonSize; + public final long activeFileSize; + public final long inactiveFileSize; + public final long unevictableSize; + public final long hierarchicalMemoryLimitSize; + public final long hierarchicalMemswLimitSize; + public final long totalCacheSize; + public final long totalRssSize; + public final long totalMappedFileSize; + public final long totalPgpginNum; + public final long totalPgpgoutNum; + public final long totalSwapSize; + public final long totalActiveAnonSize; + public final long totalInactiveAnonSize; + public final long totalActiveFileSize; + public final long totalInactiveFileSize; + public final long totalUnevictableSize; + public final long totalHierarchicalMemoryLimitSize; + public final long totalHierarchicalMemswLimitSize; + + public Stat(String output) { + String[] splits = output.split("\n"); + this.cacheSize = Long.parseLong(splits[0]); + this.rssSize = Long.parseLong(splits[1]); + this.mappedFileSize = Long.parseLong(splits[2]); + this.pgpginNum = Long.parseLong(splits[3]); + this.pgpgoutNum = Long.parseLong(splits[4]); + this.swapSize = Long.parseLong(splits[5]); + this.inactiveAnonSize = Long.parseLong(splits[6]); + this.activeAnonSize = Long.parseLong(splits[7]); + this.inactiveFileSize = Long.parseLong(splits[8]); + this.activeFileSize = Long.parseLong(splits[9]); + this.unevictableSize = Long.parseLong(splits[10]); + this.hierarchicalMemoryLimitSize = Long.parseLong(splits[11]); + this.hierarchicalMemswLimitSize = Long.parseLong(splits[12]); + this.totalCacheSize = Long.parseLong(splits[13]); + this.totalRssSize = Long.parseLong(splits[14]); + this.totalMappedFileSize = Long.parseLong(splits[15]); + this.totalPgpginNum = Long.parseLong(splits[16]); + this.totalPgpgoutNum = Long.parseLong(splits[17]); + this.totalSwapSize = Long.parseLong(splits[18]); + this.totalInactiveAnonSize = Long.parseLong(splits[19]); + this.totalActiveAnonSize = Long.parseLong(splits[20]); + this.totalInactiveFileSize = Long.parseLong(splits[21]); + this.totalActiveFileSize = Long.parseLong(splits[22]); + this.totalUnevictableSize = Long.parseLong(splits[23]); + this.totalHierarchicalMemoryLimitSize = Long.parseLong(splits[24]); + this.totalHierarchicalMemswLimitSize = Long.parseLong(splits[25]); + } + } + + public Stat getStat() throws IOException { + String output = + CgroupUtils.readFileByLine( + Constants.getDir(this.dir, MEMORY_STAT)).get(0); + Stat stat = new Stat(output); + return stat; + } + + public long getPhysicalUsage() throws IOException { + return Long.parseLong(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, MEMORY_USAGE_IN_BYTES)).get(0)); + } + + public long getWithSwapUsage() throws IOException { + return Long + .parseLong(CgroupUtils + .readFileByLine( + Constants.getDir(this.dir, + MEMORY_MEMSW_USAGE_IN_BYTES)).get(0)); + } + + public long getMaxPhysicalUsage() throws IOException { + return Long.parseLong(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, MEMORY_MAX_USAGE_IN_BYTES)).get(0)); + } + + public long getMaxWithSwapUsage() throws IOException { + return Long.parseLong(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, MEMORY_MEMSW_MAX_USAGE_IN_BYTES)) + .get(0)); + } + + public void setPhysicalUsageLimit(long value) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, MEMORY_LIMIT_IN_BYTES), + String.valueOf(value)); + } + + public long getPhysicalUsageLimit() throws IOException { + return Long.parseLong(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, MEMORY_LIMIT_IN_BYTES)).get(0)); + } + + public void setWithSwapUsageLimit(long value) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, MEMORY_MEMSW_LIMIT_IN_BYTES), + String.valueOf(value)); + } + + public long getWithSwapUsageLimit() throws IOException { + return Long + .parseLong(CgroupUtils + .readFileByLine( + Constants.getDir(this.dir, + MEMORY_MEMSW_LIMIT_IN_BYTES)).get(0)); + } + + public int getPhysicalFailCount() throws IOException { + return Integer.parseInt(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, MEMORY_FAILCNT)).get(0)); + } + + public int getWithSwapFailCount() throws IOException { + return Integer.parseInt(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, MEMORY_MEMSW_FAILCNT)).get(0)); + } + + public void clearForceEmpty() throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, MEMORY_FORCE_EMPTY), + String.valueOf(0)); + } + + public void setSwappiness(int value) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, MEMORY_SWAPPINESS), + String.valueOf(value)); + } + + public int getSwappiness() throws IOException { + return Integer.parseInt(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, MEMORY_SWAPPINESS)).get(0)); + } + + public void setUseHierarchy(boolean flag) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, MEMORY_USE_HIERARCHY), + String.valueOf(flag ? 1 : 0)); + } + + public boolean isUseHierarchy() throws IOException { + int output = + Integer.parseInt(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, MEMORY_USE_HIERARCHY)) + .get(0)); + return output > 0; + } + + public void setOomControl(boolean flag) throws IOException { + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, MEMORY_OOM_CONTROL), + String.valueOf(flag ? 1 : 0)); + } + + public boolean isOomControl() throws IOException { + String output = + CgroupUtils.readFileByLine( + Constants.getDir(this.dir, MEMORY_OOM_CONTROL)).get(0); + output = output.split("\n")[0].split("[\\s]")[1]; + int value = Integer.parseInt(output); + return value > 0; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/NetClsCore.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/NetClsCore.java new file mode 100755 index 000000000..dd80c0a60 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/NetClsCore.java @@ -0,0 +1,75 @@ +/** + * 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.container.cgroup.core; + +import java.io.IOException; + +import com.alibaba.jstorm.container.CgroupUtils; +import com.alibaba.jstorm.container.Constants; +import com.alibaba.jstorm.container.SubSystemType; +import com.alibaba.jstorm.container.cgroup.Device; + +public class NetClsCore implements CgroupCore { + + public static final String NET_CLS_CLASSID = "/net_cls.classid"; + + private final String dir; + + public NetClsCore(String dir) { + this.dir = dir; + } + + @Override + public SubSystemType getType() { + // TODO Auto-generated method stub + return SubSystemType.net_cls; + } + + private StringBuilder toHex(int num) { + String hex = num + ""; + StringBuilder sb = new StringBuilder(); + int l = hex.length(); + if (l > 4) { + hex = hex.substring(l - 4 - 1, l); + } + for (; l < 4; l++) { + sb.append('0'); + } + sb.append(hex); + return sb; + } + + public void setClassId(int major, int minor) throws IOException { + StringBuilder sb = new StringBuilder("0x"); + sb.append(toHex(major)); + sb.append(toHex(minor)); + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, NET_CLS_CLASSID), sb.toString()); + } + + public Device getClassId() throws IOException { + String output = + CgroupUtils.readFileByLine( + Constants.getDir(this.dir, NET_CLS_CLASSID)).get(0); + output = Integer.toHexString(Integer.parseInt(output)); + int major = Integer.parseInt(output.substring(0, output.length() - 4)); + int minor = Integer.parseInt(output.substring(output.length() - 4)); + return new Device(major, minor); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/NetPrioCore.java b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/NetPrioCore.java new file mode 100755 index 000000000..fd7e89915 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/container/cgroup/core/NetPrioCore.java @@ -0,0 +1,72 @@ +/** + * 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.container.cgroup.core; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import com.alibaba.jstorm.container.CgroupUtils; +import com.alibaba.jstorm.container.Constants; +import com.alibaba.jstorm.container.SubSystemType; + +public class NetPrioCore implements CgroupCore { + + public static final String NET_PRIO_PRIOIDX = "/net_prio.prioidx"; + public static final String NET_PRIO_IFPRIOMAP = "/net_prio.ifpriomap"; + + private final String dir; + + public NetPrioCore(String dir) { + this.dir = dir; + } + + @Override + public SubSystemType getType() { + // TODO Auto-generated method stub + return SubSystemType.net_prio; + } + + public int getPrioId() throws IOException { + return Integer.parseInt(CgroupUtils.readFileByLine( + Constants.getDir(this.dir, NET_PRIO_PRIOIDX)).get(0)); + } + + public void setIfPrioMap(String iface, int priority) throws IOException { + StringBuilder sb = new StringBuilder(); + sb.append(iface); + sb.append(' '); + sb.append(priority); + CgroupUtils.writeFileByLine( + Constants.getDir(this.dir, NET_PRIO_IFPRIOMAP), sb.toString()); + } + + public Map getIfPrioMap() throws IOException { + Map result = new HashMap(); + List strs = + CgroupUtils.readFileByLine(Constants.getDir(this.dir, + NET_PRIO_IFPRIOMAP)); + for (String str : strs) { + String[] strArgs = str.split(" "); + result.put(strArgs[0], Integer.valueOf(strArgs[1])); + } + return result; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/DefaultInimbus.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/DefaultInimbus.java new file mode 100755 index 000000000..6c2bd21b3 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/DefaultInimbus.java @@ -0,0 +1,72 @@ +/** + * 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.daemon.nimbus; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import backtype.storm.scheduler.INimbus; +import backtype.storm.scheduler.IScheduler; +import backtype.storm.scheduler.SupervisorDetails; +import backtype.storm.scheduler.Topologies; +import backtype.storm.scheduler.WorkerSlot; + +public class DefaultInimbus implements INimbus { + + @Override + public void prepare(Map stormConf, String schedulerLocalDir) { + // TODO Auto-generated method stub + + } + + @Override + public Collection allSlotsAvailableForScheduling( + Collection existingSupervisors, + Topologies topologies, Set topologiesMissingAssignments) { + // TODO Auto-generated method stub + Collection result = new HashSet(); + for (SupervisorDetails detail : existingSupervisors) { + for (Integer port : detail.getAllPorts()) + result.add(new WorkerSlot(detail.getId(), port)); + } + return result; + } + + @Override + public void assignSlots(Topologies topologies, + Map> newSlotsByTopologyId) { + // TODO Auto-generated method stub + + } + + @Override + public String getHostName( + Map existingSupervisors, String nodeId) { + // TODO Auto-generated method stub + return null; + } + + @Override + public IScheduler getForcedScheduler() { + // TODO Auto-generated method stub + return null; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusCache.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusCache.java new file mode 100644 index 000000000..38585953c --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusCache.java @@ -0,0 +1,146 @@ +/** + * 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.daemon.nimbus; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.TreeMap; + +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.cache.JStormCache; +import com.alibaba.jstorm.cache.RocksDBCache; +import com.alibaba.jstorm.cache.TimeoutMemCache; +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.Cluster; +import com.alibaba.jstorm.cluster.Common; +import com.alibaba.jstorm.cluster.StormBase; +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.task.TaskInfo; +import com.alibaba.jstorm.task.error.TaskError; +import com.alibaba.jstorm.utils.OSInfo; + +public class NimbusCache{ + private static final long serialVersionUID = 1685576554130463610L; + + private static final Logger LOG = LoggerFactory.getLogger(NimbusCache.class); + + + public static final String TIMEOUT_MEM_CACHE_CLASS = TimeoutMemCache.class.getName(); + public static final String ROCKS_DB_CACHE_CLASS = RocksDBCache.class.getName(); + + protected JStormCache memCache; + protected JStormCache dbCache; + protected StormClusterState zkCluster; + + public String getNimbusCacheClass(Map conf) { + boolean isLinux = OSInfo.isLinux(); + boolean isMac = OSInfo.isMac(); + boolean isLocal = StormConfig.local_mode(conf); + + if (isLocal == true) { + return TIMEOUT_MEM_CACHE_CLASS; + } + + if (isLinux == false && isMac == false) { + return TIMEOUT_MEM_CACHE_CLASS; + } + + String nimbusCacheClass = ConfigExtension.getNimbusCacheClass(conf); + if (StringUtils.isBlank(nimbusCacheClass) == false) { + return nimbusCacheClass; + } + + return ROCKS_DB_CACHE_CLASS; + + } + + public NimbusCache(Map conf, StormClusterState zkCluster) { + super(); + + String dbCacheClass = getNimbusCacheClass(conf); + LOG.info("NimbusCache db Cache will use {}", dbCacheClass); + + try { + dbCache = (JStormCache)Utils.newInstance(dbCacheClass); + + String dbDir = StormConfig.masterDbDir(conf); + conf.put(RocksDBCache.ROCKSDB_ROOT_DIR, dbDir); + + conf.put(RocksDBCache.ROCKSDB_RESET, ConfigExtension.getNimbusCacheReset(conf)); + + dbCache.init(conf); + + if (dbCache instanceof TimeoutMemCache) { + memCache = dbCache; + }else { + memCache = new TimeoutMemCache(); + memCache.init(conf); + } + }catch(java.lang.UnsupportedClassVersionError e) { + + if (e.getMessage().indexOf("Unsupported major.minor version") >= 0) { + LOG.error("!!!Please update jdk version to 7 or higher!!!"); + + } + LOG.error("Failed to create NimbusCache!", e); + throw new RuntimeException(e); + } catch (Exception e) { + LOG.error("Failed to create NimbusCache!", e); + throw new RuntimeException(e); + } + + this.zkCluster = zkCluster; + } + + public JStormCache getMemCache() { + return memCache; + } + + public JStormCache getDbCache() { + return dbCache; + } + + public void cleanup() { + dbCache.cleanup(); + + } + + + /** + * + * In the old design, + * DBCache will cache all taskInfo/taskErrors, this will be useful for huge topology + * + * But the latest zk design, taskInfo is only one znode, taskErros has few znode + * So remove them from DBCache + * Skip timely refresh taskInfo/taskErrors + * + */ + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusData.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusData.java new file mode 100644 index 000000000..02b574f6b --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusData.java @@ -0,0 +1,327 @@ +/** + * 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.daemon.nimbus; + +import java.io.IOException; +import java.nio.channels.Channel; +import java.util.HashMap; +import java.util.Map; +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 org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.cache.JStormCache; +import com.alibaba.jstorm.cluster.Cluster; +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.cluster.StormZkClusterState; +import com.alibaba.jstorm.task.TkHbCacheTime; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.TimeUtils; + +import backtype.storm.Config; +import backtype.storm.scheduler.INimbus; +import backtype.storm.utils.BufferFileInputStream; +import backtype.storm.utils.TimeCacheMap; + +/** + * All nimbus data + * + */ +public class NimbusData { + private static final Logger LOG = LoggerFactory.getLogger(NimbusData.class); + + private Map conf; + + private StormClusterState stormClusterState; + + // Map> + private ConcurrentHashMap> taskHeartbeatsCache; + + // TODO two kind of value:Channel/BufferFileInputStream + private TimeCacheMap downloaders; + private TimeCacheMap uploaders; + // cache thrift response to avoid scan zk too frequently + private NimbusCache cache; + + private int startTime; + + private final ScheduledExecutorService scheduExec; + + private AtomicInteger submittedCount; + + private StatusTransition statusTransition; + + private static final int SCHEDULE_THREAD_NUM = 8; + + private final INimbus inimubs; + + private final boolean localMode; + + private volatile boolean isLeader; + + private AtomicBoolean isShutdown = new AtomicBoolean(false); + + private final TopologyMetricsRunnable metricRunnable; + + // The topologys which has been submitted, but the assignment is not + // finished + private TimeCacheMap pendingSubmitTopologys; + + private Map topologyTaskTimeout; + + private TopologyNettyMgr topologyNettyMgr ; + + @SuppressWarnings({ "unchecked", "rawtypes" }) + public NimbusData(Map conf, INimbus inimbus) throws Exception { + this.conf = conf; + + createFileHandler(); + + this.submittedCount = new AtomicInteger(0); + + this.stormClusterState = Cluster.mk_storm_cluster_state(conf); + + createCache(); + + this.taskHeartbeatsCache = + new ConcurrentHashMap>(); + + this.scheduExec = Executors.newScheduledThreadPool(SCHEDULE_THREAD_NUM); + + this.statusTransition = new StatusTransition(this); + + this.startTime = TimeUtils.current_time_secs(); + + this.inimubs = inimbus; + + localMode = StormConfig.local_mode(conf); + + this.topologyNettyMgr = new TopologyNettyMgr(conf); + this.metricRunnable = new TopologyMetricsRunnable(this); + + pendingSubmitTopologys = + new TimeCacheMap(JStormUtils.MIN_30); + + topologyTaskTimeout = new ConcurrentHashMap(); + } + + /** + * Just for test + */ + public NimbusData() { + scheduExec = Executors.newScheduledThreadPool(6); + + inimubs = null; + conf = new HashMap(); + localMode = false; + this.metricRunnable = new TopologyMetricsRunnable(this); + } + + public void createFileHandler() { + TimeCacheMap.ExpiredCallback expiredCallback = + new TimeCacheMap.ExpiredCallback() { + @Override + public void expire(Object key, Object val) { + try { + LOG.info("Close file " + String.valueOf(key)); + if (val != null) { + if (val instanceof Channel) { + Channel channel = (Channel) val; + channel.close(); + } else if (val instanceof BufferFileInputStream) { + BufferFileInputStream is = + (BufferFileInputStream) val; + is.close(); + } + } + } catch (IOException e) { + LOG.error(e.getMessage(), e); + } + + } + }; + + int file_copy_expiration_secs = + JStormUtils.parseInt( + conf.get(Config.NIMBUS_FILE_COPY_EXPIRATION_SECS), 30); + uploaders = + new TimeCacheMap(file_copy_expiration_secs, + expiredCallback); + downloaders = + new TimeCacheMap(file_copy_expiration_secs, + expiredCallback); + } + + public void createCache() throws IOException { + cache = new NimbusCache(conf, stormClusterState); + + ((StormZkClusterState) stormClusterState).setCache(cache.getMemCache()); + } + + public int uptime() { + return (TimeUtils.current_time_secs() - startTime); + } + + public Map getConf() { + return conf; + } + + public void setConf(Map conf) { + this.conf = conf; + } + + public StormClusterState getStormClusterState() { + return stormClusterState; + } + + public void setStormClusterState(StormClusterState stormClusterState) { + this.stormClusterState = stormClusterState; + } + + public ConcurrentHashMap> getTaskHeartbeatsCache() { + return taskHeartbeatsCache; + } + + public Map getTaskHeartbeatsCache( + String topologyId, boolean createIfNotExist) { + Map ret = null; + ret = taskHeartbeatsCache.get(topologyId); + if (ret == null && createIfNotExist) { + ret = new ConcurrentHashMap(); + taskHeartbeatsCache.put(topologyId, ret); + } + return ret; + } + + public void setTaskHeartbeatsCache( + ConcurrentHashMap> taskHeartbeatsCache) { + this.taskHeartbeatsCache = taskHeartbeatsCache; + } + + public TimeCacheMap getDownloaders() { + return downloaders; + } + + public void setDownloaders(TimeCacheMap downloaders) { + this.downloaders = downloaders; + } + + public TimeCacheMap getUploaders() { + return uploaders; + } + + public void setUploaders(TimeCacheMap uploaders) { + this.uploaders = uploaders; + } + + public int getStartTime() { + return startTime; + } + + public void setStartTime(int startTime) { + this.startTime = startTime; + } + + public AtomicInteger getSubmittedCount() { + return submittedCount; + } + + public ScheduledExecutorService getScheduExec() { + return scheduExec; + } + + public StatusTransition getStatusTransition() { + return statusTransition; + } + + public void cleanup() { + cache.cleanup(); + LOG.info("Successfully shutdown Cache"); + try { + stormClusterState.disconnect(); + LOG.info("Successfully shutdown ZK Cluster Instance"); + } catch (Exception e) { + // TODO Auto-generated catch block + + } + try { + scheduExec.shutdown(); + LOG.info("Successfully shutdown threadpool"); + } catch (Exception e) { + } + + uploaders.cleanup(); + downloaders.cleanup(); + } + + public INimbus getInimubs() { + return inimubs; + } + + public boolean isLocalMode() { + return localMode; + } + + public boolean isLeader() { + return isLeader; + } + + public void setLeader(boolean isLeader) { + this.isLeader = isLeader; + } + + public AtomicBoolean getIsShutdown() { + return isShutdown; + } + + public JStormCache getMemCache() { + return cache.getMemCache(); + } + + public JStormCache getDbCache() { + return cache.getDbCache(); + } + + public NimbusCache getNimbusCache() { + return cache; + } + + public final TopologyMetricsRunnable getMetricRunnable() { + return metricRunnable; + } + + public TimeCacheMap getPendingSubmitTopoloygs() { + return pendingSubmitTopologys; + } + + public Map getTopologyTaskTimeout() { + return topologyTaskTimeout; + } + + public TopologyNettyMgr getTopologyNettyMgr() { + return topologyNettyMgr; + } + + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusServer.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusServer.java new file mode 100644 index 000000000..b22088e68 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusServer.java @@ -0,0 +1,417 @@ +/** + * 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.daemon.nimbus; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.server.THsHaServer; +import org.apache.thrift.transport.TNonblockingServerSocket; +import org.apache.thrift.transport.TTransportException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.generated.Nimbus; +import backtype.storm.generated.Nimbus.Iface; +import backtype.storm.scheduler.INimbus; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.callback.AsyncLoopRunnable; +import com.alibaba.jstorm.callback.AsyncLoopThread; +import com.alibaba.jstorm.client.ConfigExtension; +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.metric.SimpleJStormMetric; +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; + +/** + * + * NimbusServer work flow: 1. cleanup interrupted topology delete + * /storm-local-dir/nimbus/topologyid/stormdis delete + * /storm-zk-root/storms/topologyid + * + * 2. set /storm-zk-root/storms/topology stats as run + * + * 3. start one thread, every nimbus.monitor.reeq.secs set + * /storm-zk-root/storms/ all topology as monitor. when the topology's status is + * monitor, nimubs would reassign workers 4. start one threa, every + * nimubs.cleanup.inbox.freq.secs cleanup useless jar + * + * @author version 1: Nathan Marz version 2: Lixin/Chenjun version 3: Longda + * + */ +public class NimbusServer { + + private static final Logger LOG = LoggerFactory + .getLogger(NimbusServer.class); + + private NimbusData data; + + private ServiceHandler serviceHandler; + + private TopologyAssign topologyAssign; + + private THsHaServer thriftServer; + + private FollowerRunnable follower; + + private Httpserver hs; + + private List smartThreads = + new ArrayList(); + + public static void main(String[] args) throws Exception { + // read configuration files + @SuppressWarnings("rawtypes") + Map config = Utils.readStormConfig(); + + JStormServerUtils.startTaobaoJvmMonitor(); + + NimbusServer instance = new NimbusServer(); + + INimbus iNimbus = new DefaultInimbus(); + + instance.launchServer(config, iNimbus); + + } + + private void createPid(Map conf) throws Exception { + String pidDir = StormConfig.masterPids(conf); + + JStormServerUtils.createPid(pidDir); + } + + @SuppressWarnings("rawtypes") + private void launchServer(final Map conf, INimbus inimbus) { + LOG.info("Begin to start nimbus with conf " + conf); + + try { + // 1. check whether mode is distributed or not + StormConfig.validate_distributed_mode(conf); + + createPid(conf); + + initShutdownHook(); + + inimbus.prepare(conf, StormConfig.masterInimbus(conf)); + + data = createNimbusData(conf, inimbus); + + initFollowerThread(conf); + + int port = ConfigExtension.getNimbusDeamonHttpserverPort(conf); + hs = new Httpserver(port, conf); + hs.start(); + + initContainerHBThread(conf); + + while (!data.isLeader()) + Utils.sleep(5000); + + initUploadMetricThread(data); + + init(conf); + } catch (Throwable e) { + LOG.error("Fail to run nimbus ", e); + } finally { + cleanup(); + } + + LOG.info("Quit nimbus"); + } + + public ServiceHandler launcherLocalServer(final Map conf, INimbus inimbus) + throws Exception { + LOG.info("Begin to start nimbus on local model"); + + StormConfig.validate_local_mode(conf); + + inimbus.prepare(conf, StormConfig.masterInimbus(conf)); + + data = createNimbusData(conf, inimbus); + + // @@@ testing + initUploadMetricThread(data); + + init(conf); + + return serviceHandler; + } + + private void initContainerHBThread(Map conf) throws IOException { + AsyncLoopThread thread = SyncContainerHb.mkNimbusInstance(conf); + if (thread != null) { + smartThreads.add(thread); + } + } + + private void init(Map conf) throws Exception { + + NimbusUtils.cleanupCorruptTopologies(data); + + initTopologyAssign(); + + initTopologyStatus(); + + initCleaner(conf); + + serviceHandler = new ServiceHandler(data); + + if (!data.isLocalMode()) { + + initMonitor(conf); + + initThrift(conf); + + } + } + + @SuppressWarnings("rawtypes") + private NimbusData createNimbusData(Map conf, INimbus inimbus) + throws Exception { + + // Callback callback=new TimerCallBack(); + // StormTimer timer=Timer.mkTimerTimer(callback); + NimbusData data = new NimbusData(conf, inimbus); + + return data; + + } + + private void initTopologyAssign() { + topologyAssign = TopologyAssign.getInstance(); + topologyAssign.init(data); + } + + private void initTopologyStatus() throws Exception { + // get active topology in ZK + List active_ids = data.getStormClusterState().active_storms(); + + if (active_ids != null) { + + for (String topologyid : active_ids) { + // set the topology status as startup + // in fact, startup won't change anything + NimbusUtils.transition(data, topologyid, false, + StatusType.startup); + NimbusUtils.updateTopologyTaskTimeout(data, topologyid); + } + + } + + LOG.info("Successfully init topology status"); + } + + @SuppressWarnings("rawtypes") + private void initMonitor(Map conf) { + final ScheduledExecutorService scheduExec = data.getScheduExec(); + + // Schedule Nimbus monitor + MonitorRunnable r1 = new MonitorRunnable(data); + + int monitor_freq_secs = + JStormUtils.parseInt(conf.get(Config.NIMBUS_MONITOR_FREQ_SECS), + 10); + scheduExec.scheduleAtFixedRate(r1, 0, monitor_freq_secs, + TimeUnit.SECONDS); + + LOG.info("Successfully init Monitor thread"); + } + + /** + * Right now, every 600 seconds, nimbus will clean jar under + * /LOCAL-DIR/nimbus/inbox, which is the uploading topology directory + * + * @param conf + * @throws IOException + */ + @SuppressWarnings("rawtypes") + private void initCleaner(Map conf) throws IOException { + final ScheduledExecutorService scheduExec = data.getScheduExec(); + + // Schedule Nimbus inbox cleaner/nimbus/inbox jar + String dir_location = StormConfig.masterInbox(conf); + int inbox_jar_expiration_secs = + JStormUtils + .parseInt(conf + .get(Config.NIMBUS_INBOX_JAR_EXPIRATION_SECS), + 3600); + CleanRunnable r2 = + new CleanRunnable(dir_location, inbox_jar_expiration_secs); + + int cleanup_inbox_freq_secs = + JStormUtils.parseInt( + conf.get(Config.NIMBUS_CLEANUP_INBOX_FREQ_SECS), 600); + + scheduExec.scheduleAtFixedRate(r2, 0, cleanup_inbox_freq_secs, + TimeUnit.SECONDS); + + LOG.info("Successfully init " + dir_location + " cleaner"); + } + + @SuppressWarnings("rawtypes") + private void initThrift(Map conf) throws TTransportException { + Integer thrift_port = + JStormUtils.parseInt(conf.get(Config.NIMBUS_THRIFT_PORT)); + TNonblockingServerSocket socket = + new TNonblockingServerSocket(thrift_port); + + Integer maxReadBufSize = + JStormUtils.parseInt(conf + .get(Config.NIMBUS_THRIFT_MAX_BUFFER_SIZE)); + + THsHaServer.Args args = new THsHaServer.Args(socket); + args.workerThreads(ServiceHandler.THREAD_NUM); + args.protocolFactory(new TBinaryProtocol.Factory(false, true, + maxReadBufSize, -1)); + + args.processor(new Nimbus.Processor(serviceHandler)); + args.maxReadBufferBytes = maxReadBufSize; + + thriftServer = new THsHaServer(args); + + LOG.info("Successfully started nimbus: started Thrift server..."); + thriftServer.serve(); + } + + private void initFollowerThread(Map conf) { + follower = new FollowerRunnable(data, 5000); + Thread thread = new Thread(follower); + thread.setDaemon(true); + thread.start(); + LOG.info("Successfully init Follower thread"); + } + + private void initShutdownHook() { + Runtime.getRuntime().addShutdownHook(new Thread() { + public void run() { + NimbusServer.this.cleanup(); + } + + }); + } + + private void initUploadMetricThread(NimbusData data) { + final TopologyMetricsRunnable metricRunnable = data.getMetricRunnable(); + + int threadNum = ConfigExtension.getNimbusMetricThreadNum(data.getConf()); + + for (int i = 0; i < threadNum; i++) { + AsyncLoopThread thread = new AsyncLoopThread(metricRunnable); + smartThreads.add(thread); + } + + Runnable pusher = new Runnable() { + + @Override + public void run() { + // TODO Auto-generated method stub + TopologyMetricsRunnable.Upload event = + new TopologyMetricsRunnable.Upload(); + event.timeStamp = System.currentTimeMillis(); + + metricRunnable.pushEvent(event); + } + + }; + + ScheduledExecutorService scheduleService = data.getScheduExec(); + scheduleService.scheduleAtFixedRate(pusher, 120, 60, + TimeUnit.SECONDS); + + SimpleJStormMetric nimbusMetric = SimpleJStormMetric.mkInstance(); + scheduleService.scheduleAtFixedRate(nimbusMetric, 120, 60, + TimeUnit.SECONDS); + + //AsyncLoopThread nimbusCacheThread = new AsyncLoopThread(data.getNimbusCache().getCacheRunnable()); + //smartThreads.add(nimbusCacheThread); + + LOG.info("Successfully init metrics uploading thread"); + } + + public void cleanup() { + if (data.getIsShutdown().getAndSet(true) == true) { + LOG.info("Notify to quit nimbus"); + return; + } + + LOG.info("Begin to shutdown nimbus"); + AsyncLoopRunnable.getShutdown().set(true); + + data.getScheduExec().shutdownNow(); + + for (AsyncLoopThread t : smartThreads) { + + t.cleanup(); + JStormUtils.sleepMs(10); + t.interrupt(); + // try { + // t.join(); + // } catch (InterruptedException e) { + // LOG.error("join thread", e); + // } + LOG.info("Successfully cleanup " + t.getThread().getName()); + } + + if (serviceHandler != null) { + serviceHandler.shutdown(); + } + + if (topologyAssign != null) { + topologyAssign.cleanup(); + LOG.info("Successfully shutdown TopologyAssign thread"); + } + + if (follower != null) { + follower.clean(); + LOG.info("Successfully shutdown follower thread"); + } + + if (data != null) { + data.cleanup(); + LOG.info("Successfully shutdown NimbusData"); + } + + if (thriftServer != null) { + thriftServer.stop(); + LOG.info("Successfully shutdown thrift server"); + } + + if (hs != null) { + hs.shutdown(); + LOG.info("Successfully shutdown httpserver"); + } + + LOG.info("Successfully shutdown nimbus"); + // make sure shutdown nimbus + JStormUtils.halt_process(0, "!!!Shutdown!!!"); + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusUtils.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusUtils.java new file mode 100644 index 000000000..7181e77fe --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusUtils.java @@ -0,0 +1,723 @@ +/** + * 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.daemon.nimbus; + +import java.io.BufferedReader; +import java.io.Closeable; +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.security.InvalidParameterException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Date; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.generated.Bolt; +import backtype.storm.generated.ComponentCommon; +import backtype.storm.generated.NimbusStat; +import backtype.storm.generated.NimbusSummary; +import backtype.storm.generated.NotAliveException; +import backtype.storm.generated.SpoutSpec; +import backtype.storm.generated.StateSpoutSpec; +import backtype.storm.generated.StormTopology; +import backtype.storm.generated.SupervisorSummary; +import backtype.storm.generated.TopologySummary; +import backtype.storm.utils.ThriftTopologyUtils; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.Cluster; +import com.alibaba.jstorm.cluster.StormBase; +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; +import com.alibaba.jstorm.schedule.Assignment; +import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; +import com.alibaba.jstorm.task.TkHbCacheTime; +import com.alibaba.jstorm.task.heartbeat.TaskHeartbeat; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.PathUtils; +import com.alibaba.jstorm.utils.TimeUtils; + +public class NimbusUtils { + + private static Logger LOG = LoggerFactory.getLogger(NimbusUtils.class); + + /** + * add coustom KRYO serialization + * + */ + private static Map mapifySerializations(List sers) { + Map rtn = new HashMap(); + if (sers != null) { + int size = sers.size(); + for (int i = 0; i < size; i++) { + if (sers.get(i) instanceof Map) { + rtn.putAll((Map) sers.get(i)); + } else { + rtn.put(sers.get(i), null); + } + } + + } + return rtn; + } + + /** + * Normalize stormConf + * + * + * + * @param conf + * @param stormConf + * @param topology + * @return + * @throws Exception + */ + @SuppressWarnings("rawtypes") + public static Map normalizeConf(Map conf, Map stormConf, + StormTopology topology) throws Exception { + + List kryoRegisterList = new ArrayList(); + List kryoDecoratorList = new ArrayList(); + + Map totalConf = new HashMap(); + totalConf.putAll(conf); + totalConf.putAll(stormConf); + + Object totalRegister = totalConf.get(Config.TOPOLOGY_KRYO_REGISTER); + if (totalRegister != null) { + LOG.info("topology:" + stormConf.get(Config.TOPOLOGY_NAME) + + ", TOPOLOGY_KRYO_REGISTER" + + totalRegister.getClass().getName()); + + JStormUtils.mergeList(kryoRegisterList, totalRegister); + } + + Object totalDecorator = totalConf.get(Config.TOPOLOGY_KRYO_DECORATORS); + if (totalDecorator != null) { + LOG.info("topology:" + stormConf.get(Config.TOPOLOGY_NAME) + + ", TOPOLOGY_KRYO_DECORATOR" + + totalDecorator.getClass().getName()); + JStormUtils.mergeList(kryoDecoratorList, totalDecorator); + } + + Set cids = ThriftTopologyUtils.getComponentIds(topology); + for (Iterator it = cids.iterator(); it.hasNext();) { + String componentId = (String) it.next(); + + ComponentCommon common = + ThriftTopologyUtils.getComponentCommon(topology, + componentId); + String json = common.get_json_conf(); + if (json == null) { + continue; + } + Map mtmp = (Map) JStormUtils.from_json(json); + if (mtmp == null) { + StringBuilder sb = new StringBuilder(); + + sb.append("Failed to deserilaize " + componentId); + sb.append(" json configuration: "); + sb.append(json); + LOG.info(sb.toString()); + throw new Exception(sb.toString()); + } + + Object componentKryoRegister = + mtmp.get(Config.TOPOLOGY_KRYO_REGISTER); + + if (componentKryoRegister != null) { + LOG.info("topology:" + stormConf.get(Config.TOPOLOGY_NAME) + + ", componentId:" + componentId + + ", TOPOLOGY_KRYO_REGISTER" + + componentKryoRegister.getClass().getName()); + + JStormUtils.mergeList(kryoRegisterList, componentKryoRegister); + } + + Object componentDecorator = + mtmp.get(Config.TOPOLOGY_KRYO_DECORATORS); + if (componentDecorator != null) { + LOG.info("topology:" + stormConf.get(Config.TOPOLOGY_NAME) + + ", componentId:" + componentId + + ", TOPOLOGY_KRYO_DECORATOR" + + componentDecorator.getClass().getName()); + JStormUtils.mergeList(kryoDecoratorList, componentDecorator); + } + + } + + Map kryoRegisterMap = mapifySerializations(kryoRegisterList); + List decoratorList = JStormUtils.distinctList(kryoDecoratorList); + + Integer ackerNum = + JStormUtils.parseInt(totalConf + .get(Config.TOPOLOGY_ACKER_EXECUTORS)); + if (ackerNum == null) { + ackerNum = Integer.valueOf(1); + } + + Map rtn = new HashMap(); + rtn.putAll(stormConf); + rtn.put(Config.TOPOLOGY_KRYO_DECORATORS, decoratorList); + rtn.put(Config.TOPOLOGY_KRYO_REGISTER, kryoRegisterMap); + rtn.put(Config.TOPOLOGY_ACKER_EXECUTORS, ackerNum); + rtn.put(Config.TOPOLOGY_MAX_TASK_PARALLELISM, + totalConf.get(Config.TOPOLOGY_MAX_TASK_PARALLELISM)); + return rtn; + } + + public static Integer componentParalism(Map stormConf, + ComponentCommon common) { + Map mergeMap = new HashMap(); + mergeMap.putAll(stormConf); + + String jsonConfString = common.get_json_conf(); + if (jsonConfString != null) { + Map componentMap = (Map) JStormUtils.from_json(jsonConfString); + mergeMap.putAll(componentMap); + } + + Integer taskNum = common.get_parallelism_hint(); + if (taskNum == null) { + taskNum = Integer.valueOf(1); + } + + // don't get taskNum from component configuraiton + // skip .setTaskNum + // Integer taskNum = null; + // Object taskNumObject = mergeMap.get(Config.TOPOLOGY_TASKS); + // if (taskNumObject != null) { + // taskNum = JStormUtils.parseInt(taskNumObject); + // } else { + // taskNum = common.get_parallelism_hint(); + // if (taskNum == null) { + // taskNum = Integer.valueOf(1); + // } + // } + + Object maxTaskParalismObject = + mergeMap.get(Config.TOPOLOGY_MAX_TASK_PARALLELISM); + if (maxTaskParalismObject == null) { + return taskNum; + } else { + int maxTaskParalism = JStormUtils.parseInt(maxTaskParalismObject); + + return Math.min(maxTaskParalism, taskNum); + } + + } + + /** + * finalize component's task paralism + * + * @param topology + * @param fromConf means if the paralism is read from conf file instead of + * reading from topology code + * @return + */ + public static StormTopology normalizeTopology(Map stormConf, + 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(); + + ComponentCommon common = null; + if (component instanceof Bolt) { + common = ((Bolt) component).get_common(); + if (fromConf) { + Integer paraNum = + ConfigExtension.getBoltParallelism(stormConf, + componentName); + 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) { + 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) { + LOG.info("Set " + componentName + " as " + paraNum); + common.set_parallelism_hint(paraNum); + } + } + } + + Map componentMap = new HashMap(); + + String jsonConfString = common.get_json_conf(); + if (jsonConfString != null) { + componentMap + .putAll((Map) JStormUtils.from_json(jsonConfString)); + } + + Integer taskNum = componentParalism(stormConf, common); + + componentMap.put(Config.TOPOLOGY_TASKS, taskNum); + // change the executor's task number + common.set_parallelism_hint(taskNum); + LOG.info("Set " + componentName + " parallelism " + taskNum); + + common.set_json_conf(JStormUtils.to_json(componentMap)); + } + + return ret; + } + + /** + * clean the topology which is in ZK but not in local dir + * + * @throws Exception + * + */ + public static void cleanupCorruptTopologies(NimbusData data) + throws Exception { + + StormClusterState stormClusterState = data.getStormClusterState(); + + // get /local-storm-dir/nimbus/stormdist path + String master_stormdist_root = + StormConfig.masterStormdistRoot(data.getConf()); + + // listdir /local-storm-dir/nimbus/stormdist + List code_ids = + PathUtils.read_dir_contents(master_stormdist_root); + + // get topology in ZK /storms + List active_ids = data.getStormClusterState().active_storms(); + if (active_ids != null && active_ids.size() > 0) { + if (code_ids != null) { + // clean the topology which is in ZK but not in local dir + active_ids.removeAll(code_ids); + } + + for (String corrupt : active_ids) { + LOG.info("Corrupt topology " + + corrupt + + " has state on zookeeper but doesn't have a local dir on Nimbus. Cleaning up..."); + + /** + * Just removing the /STORMS is enough + * + */ + stormClusterState.remove_storm(corrupt); + } + } + + LOG.info("Successfully cleanup all old toplogies"); + + } + + public static boolean isTaskDead(NimbusData data, String topologyId, + Integer taskId) { + String idStr = " topology:" + topologyId + ",taskid:" + taskId; + + Integer zkReportTime = null; + + StormClusterState stormClusterState = data.getStormClusterState(); + TaskHeartbeat zkTaskHeartbeat = null; + try { + zkTaskHeartbeat = + stormClusterState.task_heartbeat(topologyId, taskId); + if (zkTaskHeartbeat != null) { + zkReportTime = zkTaskHeartbeat.getTimeSecs(); + } + } catch (Exception e) { + LOG.error("Failed to get ZK task hearbeat " + idStr, e); + } + + Map taskHBs = + data.getTaskHeartbeatsCache(topologyId, true); + + TkHbCacheTime taskHB = taskHBs.get(taskId); + if (taskHB == null) { + LOG.info("No task heartbeat cache " + idStr); + + if (zkTaskHeartbeat == null) { + LOG.info("No ZK task hearbeat " + idStr); + return true; + } + + taskHB = new TkHbCacheTime(); + taskHB.update(zkTaskHeartbeat); + + taskHBs.put(taskId, taskHB); + + return false; + } + + if (zkReportTime == null) { + LOG.debug("No ZK task heartbeat " + idStr); + // Task hasn't finish init + int nowSecs = TimeUtils.current_time_secs(); + int assignSecs = taskHB.getTaskAssignedTime(); + + int waitInitTimeout = + JStormUtils.parseInt(data.getConf().get( + Config.NIMBUS_TASK_LAUNCH_SECS)); + + if (nowSecs - assignSecs > waitInitTimeout) { + LOG.info(idStr + " failed to init "); + return true; + } else { + return false; + } + + } + + // the left is zkReportTime isn't null + // task has finished initialization + int nimbusTime = taskHB.getNimbusTime(); + int reportTime = taskHB.getTaskReportedTime(); + + int nowSecs = TimeUtils.current_time_secs(); + if (nimbusTime == 0) { + // taskHB no entry, first time + // update taskHB + taskHB.setNimbusTime(nowSecs); + taskHB.setTaskReportedTime(zkReportTime); + + LOG.info("Update taskheartbeat to nimbus cache " + idStr); + return false; + } + + if (reportTime != zkReportTime.intValue()) { + // zk has been updated the report time + taskHB.setNimbusTime(nowSecs); + taskHB.setTaskReportedTime(zkReportTime); + + LOG.debug(idStr + ",nimbusTime " + nowSecs + ",zkReport:" + + zkReportTime + ",report:" + reportTime); + return false; + } + + // the following is (zkReportTime == reportTime) + Integer taskHBTimeout = data.getTopologyTaskTimeout().get(topologyId); + if (taskHBTimeout == null) + taskHBTimeout = + JStormUtils.parseInt(data.getConf().get( + Config.NIMBUS_TASK_TIMEOUT_SECS)); + if (nowSecs - nimbusTime > taskHBTimeout) { + // task is dead + long ts = ((long) nimbusTime) * 1000; + Date lastTaskHBDate = new Date(ts); + StringBuilder sb = new StringBuilder(); + + sb.append(idStr); + sb.append(" last tasktime is "); + sb.append(nimbusTime); + sb.append(":").append(lastTaskHBDate); + sb.append(",current "); + sb.append(nowSecs); + sb.append(":").append(new Date(((long) nowSecs) * 1000)); + LOG.info(sb.toString()); + return true; + } + + return false; + + } + + public static void updateTaskHbStartTime(NimbusData data, + Assignment assignment, String topologyId) { + Map taskHBs = + data.getTaskHeartbeatsCache(topologyId, true); + + Map taskStartTimes = + assignment.getTaskStartTimeSecs(); + for (Entry entry : taskStartTimes.entrySet()) { + Integer taskId = entry.getKey(); + Integer taskStartTime = entry.getValue(); + + TkHbCacheTime taskHB = taskHBs.get(taskId); + if (taskHB == null) { + taskHB = new TkHbCacheTime(); + taskHBs.put(taskId, taskHB); + } + + taskHB.setTaskAssignedTime(taskStartTime); + } + + return; + } + + public static void transitionName(NimbusData data, String topologyName, + boolean errorOnNoTransition, StatusType transition_status, + T... args) throws Exception { + StormClusterState stormClusterState = data.getStormClusterState(); + String topologyId = + Cluster.get_topology_id(stormClusterState, topologyName); + if (topologyId == null) { + throw new NotAliveException(topologyName); + } + transition(data, topologyId, errorOnNoTransition, transition_status, + args); + } + + public static void transition(NimbusData data, String topologyid, + boolean errorOnNoTransition, StatusType transition_status, + T... args) { + try { + data.getStatusTransition().transition(topologyid, + errorOnNoTransition, transition_status, args); + } catch (Exception e) { + // TODO Auto-generated catch block + LOG.error("Failed to do status transition,", e); + } + } + + public static int getTopologyTaskNum(Assignment assignment) { + int numTasks = 0; + + for (ResourceWorkerSlot worker : assignment.getWorkers()) { + numTasks += worker.getTasks().size(); + } + return numTasks; + } + + public static List getTopologySummary( + StormClusterState stormClusterState, + Map assignments) throws Exception { + List topologySummaries = + new ArrayList(); + + // get all active topology's StormBase + Map bases = + Cluster.get_all_StormBase(stormClusterState); + for (Entry entry : bases.entrySet()) { + + String topologyId = entry.getKey(); + StormBase base = entry.getValue(); + + Assignment assignment = + stormClusterState.assignment_info(topologyId, null); + if (assignment == null) { + LOG.error("Failed to get assignment of " + topologyId); + continue; + } + assignments.put(topologyId, assignment); + + int num_workers = assignment.getWorkers().size(); + int num_tasks = getTopologyTaskNum(assignment); + + String errorString = null; + if (Cluster.is_topology_exist_error(stormClusterState, topologyId)) { + errorString = "Y"; + } else { + errorString = ""; + } + + TopologySummary topology = new TopologySummary(); + topology.set_id(topologyId); + topology.set_name(base.getStormName()); + topology.set_status(base.getStatusString()); + topology.set_uptime_secs(TimeUtils.time_delta(base + .getLanchTimeSecs())); + topology.set_num_workers(num_workers); + topology.set_num_tasks(num_tasks); + topology.set_error_info(errorString); + + topologySummaries.add(topology); + + } + + return topologySummaries; + } + + public static SupervisorSummary mkSupervisorSummary( + SupervisorInfo supervisorInfo, String supervisorId, + Map supervisorToUsedSlotNum) { + Integer usedNum = supervisorToUsedSlotNum.get(supervisorId); + + SupervisorSummary summary = + new SupervisorSummary(supervisorInfo.getHostName(), + supervisorId, supervisorInfo.getUptimeSecs(), + supervisorInfo.getWorkerPorts().size(), + usedNum == null ? 0 : usedNum); + + return summary; + } + + public static List mkSupervisorSummaries( + Map supervisorInfos, + Map assignments) { + + Map supervisorToLeftSlotNum = + new HashMap(); + for (Entry entry : assignments.entrySet()) { + Set workers = entry.getValue().getWorkers(); + + for (ResourceWorkerSlot worker : workers) { + + String supervisorId = worker.getNodeId(); + SupervisorInfo supervisorInfo = + supervisorInfos.get(supervisorId); + if (supervisorInfo == null) { + continue; + } + Integer slots = supervisorToLeftSlotNum.get(supervisorId); + if (slots == null) { + slots = 0; + supervisorToLeftSlotNum.put(supervisorId, slots); + } + supervisorToLeftSlotNum.put(supervisorId, ++slots); + } + } + + List ret = new ArrayList(); + for (Entry entry : supervisorInfos.entrySet()) { + String supervisorId = entry.getKey(); + SupervisorInfo supervisorInfo = entry.getValue(); + + SupervisorSummary summary = + mkSupervisorSummary(supervisorInfo, supervisorId, + supervisorToLeftSlotNum); + + ret.add(summary); + } + + Collections.sort(ret, new Comparator() { + + @Override + public int compare(SupervisorSummary o1, SupervisorSummary o2) { + + return o1.get_host().compareTo(o2.get_host()); + } + + }); + return ret; + } + + public static NimbusSummary getNimbusSummary( + StormClusterState stormClusterState, + List supervisorSummaries, NimbusData data) + throws Exception { + NimbusSummary ret = new NimbusSummary(); + + String master = stormClusterState.get_leader_host(); + NimbusStat nimbusMaster = new NimbusStat(); + nimbusMaster.set_host(master); + nimbusMaster.set_uptime_secs(String.valueOf(data.uptime())); + ret.set_nimbus_master(nimbusMaster); + + List nimbusSlaveList = new ArrayList(); + ret.set_nimbus_slaves(nimbusSlaveList); + Map nimbusSlaveMap = + Cluster.get_all_nimbus_slave(stormClusterState); + if (nimbusSlaveMap != null) { + for (Entry entry : nimbusSlaveMap.entrySet()) { + NimbusStat slave = new NimbusStat(); + slave.set_host(entry.getKey()); + slave.set_uptime_secs(entry.getValue()); + + nimbusSlaveList.add(slave); + } + } + + int totalPort = 0; + int usedPort = 0; + + for (SupervisorSummary supervisor : supervisorSummaries) { + totalPort += supervisor.get_num_workers(); + usedPort += supervisor.get_num_used_workers(); + } + + ret.set_supervisor_num(supervisorSummaries.size()); + ret.set_total_port_num(totalPort); + ret.set_used_port_num(usedPort); + ret.set_free_port_num(totalPort - usedPort); + ret.set_version(Utils.getVersion()); + + return ret; + + } + + public static void updateTopologyTaskTimeout(NimbusData data, + String topologyId) { + Map topologyConf = null; + try { + topologyConf = + StormConfig.read_nimbus_topology_conf(data.getConf(), + topologyId); + } catch (IOException e) { + LOG.warn("Failed to read configuration of " + topologyId + ", " + + e.getMessage()); + } + + Integer timeout = + JStormUtils.parseInt(topologyConf + .get(Config.NIMBUS_TASK_TIMEOUT_SECS)); + if (timeout == null) { + timeout = + JStormUtils.parseInt(data.getConf().get( + Config.NIMBUS_TASK_TIMEOUT_SECS)); + } + LOG.info("Setting taskTimeout:" + timeout + " for " + topologyId); + data.getTopologyTaskTimeout().put(topologyId, timeout); + } + + public static void removeTopologyTaskTimeout(NimbusData data, + String topologyId) { + data.getTopologyTaskTimeout().remove(topologyId); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/ServiceHandler.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/ServiceHandler.java new file mode 100644 index 000000000..513e83f0b --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/ServiceHandler.java @@ -0,0 +1,1523 @@ + +/** + * 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.daemon.nimbus; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; +import java.security.InvalidParameterException; +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.UUID; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang.StringUtils; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.daemon.Shutdownable; +import backtype.storm.generated.AlreadyAliveException; +import backtype.storm.generated.ClusterSummary; +import backtype.storm.generated.ComponentSummary; +import backtype.storm.generated.Credentials; +import backtype.storm.generated.ErrorInfo; +import backtype.storm.generated.InvalidTopologyException; +import backtype.storm.generated.KillOptions; +import backtype.storm.generated.MetricInfo; +import backtype.storm.generated.MonitorOptions; +import backtype.storm.generated.NettyMetric; +import backtype.storm.generated.Nimbus.Iface; +import backtype.storm.generated.NimbusSummary; +import backtype.storm.generated.NotAliveException; +import backtype.storm.generated.RebalanceOptions; +import backtype.storm.generated.StormTopology; +import backtype.storm.generated.SubmitOptions; +import backtype.storm.generated.SupervisorSummary; +import backtype.storm.generated.SupervisorWorkers; +import backtype.storm.generated.TaskComponent; +import backtype.storm.generated.TaskSummary; +import backtype.storm.generated.TopologyAssignException; +import backtype.storm.generated.TopologyInfo; +import backtype.storm.generated.TopologyInitialStatus; +import backtype.storm.generated.TopologyMetric; +import backtype.storm.generated.TopologySummary; +import backtype.storm.generated.WorkerSummary; +import backtype.storm.generated.WorkerUploadMetrics; +import backtype.storm.utils.BufferFileInputStream; +import backtype.storm.utils.TimeCacheMap; +import backtype.storm.utils.Utils; + +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; +import com.alibaba.jstorm.cluster.StormBase; +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; +import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.metric.SimpleJStormMetric; +import com.alibaba.jstorm.schedule.Assignment; +import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; +import com.alibaba.jstorm.task.TaskInfo; +import com.alibaba.jstorm.task.error.TaskError; +import com.alibaba.jstorm.task.heartbeat.TaskHeartbeat; +import com.alibaba.jstorm.utils.FailedAssignTopologyException; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.NetWorkUtils; +import com.alibaba.jstorm.utils.PathUtils; +import com.alibaba.jstorm.utils.Thrift; +import com.alibaba.jstorm.utils.TimeUtils; + +/** + * Thrift callback, all commands handling entrance + * + * @author version 1: lixin, version 2:Longda + * + */ +public class ServiceHandler implements Iface, Shutdownable, DaemonCommon { + private final static Logger LOG = LoggerFactory + .getLogger(ServiceHandler.class); + + public final static int THREAD_NUM = 64; + + private NimbusData data; + + private Map conf; + + public ServiceHandler(NimbusData data) { + this.data = data; + conf = data.getConf(); + } + + /** + * Shutdown the nimbus + */ + @Override + public void shutdown() { + LOG.info("Begin to shut down master"); + // Timer.cancelTimer(nimbus.getTimer()); + + LOG.info("Successfully shut down master"); + + } + + @Override + public boolean waiting() { + // @@@ TODO + return false; + } + + @Override + public void submitTopology(String name, String uploadedJarLocation, + String jsonConf, StormTopology topology) + throws AlreadyAliveException, InvalidTopologyException, + TopologyAssignException, TException { + SubmitOptions options = new SubmitOptions(TopologyInitialStatus.ACTIVE); + + submitTopologyWithOpts(name, uploadedJarLocation, jsonConf, topology, + options); + } + + private void makeAssignment(String topologyName, String topologyId, + TopologyInitialStatus status) throws FailedAssignTopologyException { + TopologyAssignEvent assignEvent = new TopologyAssignEvent(); + assignEvent.setTopologyId(topologyId); + assignEvent.setScratch(false); + assignEvent.setTopologyName(topologyName); + assignEvent.setOldStatus(Thrift + .topologyInitialStatusToStormStatus(status)); + + TopologyAssign.push(assignEvent); + + boolean isSuccess = assignEvent.waitFinish(); + if (isSuccess == true) { + LOG.info("Finish submit for " + topologyName); + } else { + throw new FailedAssignTopologyException(assignEvent.getErrorMsg()); + } + } + + /** + * Submit one Topology + * + * @param topologyname String: topology name + * @param uploadedJarLocation String: already uploaded jar path + * @param jsonConf String: jsonConf serialize all toplogy configuration to + * Json + * @param topology StormTopology: topology Object + */ + @SuppressWarnings("unchecked") + @Override + public void submitTopologyWithOpts(String topologyname, + String uploadedJarLocation, String jsonConf, + StormTopology topology, SubmitOptions options) + throws AlreadyAliveException, InvalidTopologyException, + TopologyAssignException, TException { + LOG.info("Receive " + topologyname + ", uploadedJarLocation:" + + uploadedJarLocation); + long start = System.nanoTime(); + try { + checkTopologyActive(data, topologyname, false); + } catch (AlreadyAliveException e) { + LOG.info(topologyname + " is already exist "); + throw e; + } 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 = Common.topologyNameToId(topologyname, counter); + data.getPendingSubmitTopoloygs().put(topologyId, null); + + try { + + Map serializedConf = + (Map) JStormUtils.from_json(jsonConf); + if (serializedConf == null) { + LOG.warn("Failed to serialized Configuration"); + throw new InvalidTopologyException( + "Failed to serilaze topology configuration"); + } + + serializedConf.put(Config.TOPOLOGY_ID, topologyId); + serializedConf.put(Config.TOPOLOGY_NAME, topologyname); + + Map stormConf; + + stormConf = + NimbusUtils.normalizeConf(conf, serializedConf, topology); + LOG.info("Normalized configuration:" + stormConf); + data.getTopologyNettyMgr().setTopology(stormConf); + + Map totalStormConf = + new HashMap(conf); + totalStormConf.putAll(stormConf); + + StormTopology normalizedTopology = + NimbusUtils.normalizeTopology(stormConf, topology, true); + + // this validates the structure of the topology + Common.validate_basic(normalizedTopology, totalStormConf, + topologyId); + + // don't need generate real topology, so skip Common.system_topology + // Common.system_topology(totalStormConf, topology); + + StormClusterState stormClusterState = data.getStormClusterState(); + + // create /local-dir/nimbus/topologyId/xxxx files + setupStormCode(conf, topologyId, uploadedJarLocation, stormConf, + normalizedTopology); + + // generate TaskInfo for every bolt or spout in ZK + // /ZK/tasks/topoologyId/xxx + setupZkTaskInfo(conf, topologyId, stormClusterState); + + // make assignments for a topology + LOG.info("Submit for " + topologyname + " with conf " + + serializedConf); + makeAssignment(topologyname, topologyId, + options.get_initial_status()); + + } 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); + sb.append(", uploadedJarLocation:" + uploadedJarLocation + "\n"); + LOG.error(sb.toString(), e); + data.getPendingSubmitTopoloygs().remove(topologyId); + 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); + sb.append(", uploadedJarLocation:" + uploadedJarLocation + "\n"); + LOG.error(sb.toString(), e); + data.getPendingSubmitTopoloygs().remove(topologyId); + throw new InvalidParameterException(sb.toString()); + } catch (InvalidTopologyException e) { + LOG.error("Topology is invalid. " + e.get_msg()); + data.getPendingSubmitTopoloygs().remove(topologyId); + throw e; + } 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); + sb.append(", uploadedJarLocation:" + uploadedJarLocation + "\n"); + LOG.error(sb.toString(), e); + data.getPendingSubmitTopoloygs().remove(topologyId); + throw new TopologyAssignException(sb.toString()); + }finally { + double spend = (System.nanoTime() - start)/1000000.0d; + SimpleJStormMetric.updateHistorgram("submitTopologyWithOpts", spend); + LOG.info("submitTopologyWithOpts {} spend {}ms", topologyname, spend); + } + + } + + /** + * kill topology + * + * @param topologyname String topology name + */ + @Override + public void killTopology(String name) throws NotAliveException, TException { + killTopologyWithOpts(name, new KillOptions()); + + } + + @Override + public void killTopologyWithOpts(String topologyName, KillOptions options) + throws NotAliveException, TException { + try { + checkTopologyActive(data, topologyName, true); + + String topologyId = getTopologyId(topologyName); + + Integer wait_amt = null; + if (options.is_set_wait_secs()) { + wait_amt = options.get_wait_secs(); + } + NimbusUtils.transitionName(data, topologyName, true, + StatusType.kill, wait_amt); + + TopologyMetricsRunnable.Remove event = + new TopologyMetricsRunnable.Remove(); + event.topologyId = topologyId; + + data.getMetricRunnable().pushEvent(event); + } catch (NotAliveException e) { + String errMsg = + "KillTopology Error, no this topology " + topologyName; + LOG.error(errMsg, e); + throw new NotAliveException(errMsg); + } catch (Exception e) { + String errMsg = "Failed to kill topology " + topologyName; + LOG.error(errMsg, e); + throw new TException(errMsg); + } + + } + + /** + * set topology status as active + * + * @param topologyname + * + */ + @Override + public void activate(String topologyName) throws NotAliveException, + TException { + try { + NimbusUtils.transitionName(data, topologyName, true, + StatusType.activate); + } catch (NotAliveException e) { + String errMsg = "Activate Error, no this topology " + topologyName; + LOG.error(errMsg, e); + throw new NotAliveException(errMsg); + } catch (Exception e) { + String errMsg = "Failed to active topology " + topologyName; + LOG.error(errMsg, e); + throw new TException(errMsg); + } + + } + + /** + * set topology stauts as deactive + * + * @param topologyname + * + */ + @Override + public void deactivate(String topologyName) throws NotAliveException, + TException { + + try { + NimbusUtils.transitionName(data, topologyName, true, + StatusType.inactivate); + } catch (NotAliveException e) { + String errMsg = + "Deactivate Error, no this topology " + topologyName; + LOG.error(errMsg, e); + throw new NotAliveException(errMsg); + } catch (Exception e) { + String errMsg = "Failed to deactivate topology " + topologyName; + LOG.error(errMsg, e); + throw new TException(errMsg); + } + + } + + /** + * rebalance one topology + * + * @@@ rebalance options hasn't implements + * + * It is used to let workers wait several seconds to finish jobs + * + * @param topologyname String + * @param options RebalanceOptions + */ + @Override + public void rebalance(String topologyName, RebalanceOptions options) + throws NotAliveException, TException, InvalidTopologyException { + + try { + + checkTopologyActive(data, topologyName, true); + Integer wait_amt = null; + String jsonConf = null; + Boolean reassign = false; + if (options != null) { + if (options.is_set_wait_secs()) + wait_amt = options.get_wait_secs(); + if (options.is_set_reassign()) + reassign = options.is_reassign(); + if (options.is_set_conf()) + jsonConf = options.get_conf(); + } + + LOG.info("Begin to rebalance " + topologyName + "wait_time:" + + wait_amt + ", reassign: " + reassign + + ", new worker/bolt configuration:" + jsonConf); + + Map conf = + (Map) JStormUtils.from_json(jsonConf); + + NimbusUtils.transitionName(data, topologyName, true, + StatusType.rebalance, wait_amt, reassign, conf); + } catch (NotAliveException e) { + String errMsg = "Rebalance Error, no this topology " + topologyName; + LOG.error(errMsg, e); + throw new NotAliveException(errMsg); + } catch (Exception e) { + String errMsg = "Failed to rebalance topology " + topologyName; + LOG.error(errMsg, e); + throw new TException(errMsg); + } + + } + + @Override + public void restart(String name, String jsonConf) throws NotAliveException, + InvalidTopologyException, TopologyAssignException, TException { + LOG.info("Begin to restart " + name + ", new configuration:" + jsonConf); + + // 1. get topologyId + StormClusterState stormClusterState = data.getStormClusterState(); + String topologyId = null; + try { + 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 + // 2. Deactivate + deactivate(name); + JStormUtils.sleepMs(5000); + 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 (Exception e) { + // TODO Auto-generated catch block + LOG.error("Fail to upload jar " + libName, e); + throw new TException(e); + } + } + + /** + * prepare to uploading topology jar, return the file location + * + * @throws + */ + @Override + public String beginFileUpload() throws TException { + + String fileLoc = null; + try { + String path = null; + String key = UUID.randomUUID().toString(); + path = StormConfig.masterInbox(conf) + "/" + key; + FileUtils.forceMkdir(new File(path)); + FileUtils.cleanDirectory(new File(path)); + fileLoc = path + "/stormjar-" + key + ".jar"; + + data.getUploaders().put(fileLoc, + Channels.newChannel(new FileOutputStream(fileLoc))); + LOG.info("Begin upload file from client to " + fileLoc); + return path; + } catch (FileNotFoundException e) { + LOG.error("File not found: " + fileLoc, e); + throw new TException(e); + } catch (IOException e) { + LOG.error("Upload file error: " + fileLoc, e); + throw new TException(e); + } + } + + /** + * uploading topology jar data + */ + @Override + public void uploadChunk(String location, ByteBuffer chunk) + throws TException { + TimeCacheMap uploaders = data.getUploaders(); + Object obj = uploaders.get(location); + if (obj == null) { + throw new TException( + "File for that location does not exist (or timed out) " + + location); + } + try { + if (obj instanceof WritableByteChannel) { + WritableByteChannel channel = (WritableByteChannel) obj; + channel.write(chunk); + uploaders.put(location, channel); + } else { + throw new TException("Object isn't WritableByteChannel for " + + location); + } + } catch (IOException e) { + String errMsg = + " WritableByteChannel write filed when uploadChunk " + + location; + LOG.error(errMsg); + throw new TException(e); + } + + } + + @Override + public void finishFileUpload(String location) throws TException { + + TimeCacheMap uploaders = data.getUploaders(); + Object obj = uploaders.get(location); + if (obj == null) { + throw new TException( + "File for that location does not exist (or timed out)"); + } + try { + if (obj instanceof WritableByteChannel) { + WritableByteChannel channel = (WritableByteChannel) obj; + channel.close(); + uploaders.remove(location); + LOG.info("Finished uploading file from client: " + location); + } else { + throw new TException("Object isn't WritableByteChannel for " + + location); + } + } catch (IOException e) { + LOG.error(" WritableByteChannel close failed when finishFileUpload " + + location); + } + + } + + @Override + public String beginFileDownload(String file) throws TException { + BufferFileInputStream is = null; + String id = null; + try { + int bufferSize = + JStormUtils.parseInt( + conf.get(Config.NIMBUS_THRIFT_MAX_BUFFER_SIZE), + 1024 * 1024) / 2; + + is = new BufferFileInputStream(file, bufferSize); + id = UUID.randomUUID().toString(); + data.getDownloaders().put(id, is); + } catch (FileNotFoundException e) { + LOG.error(e + "file:" + file + " not found"); + throw new TException(e); + } + + return id; + } + + @Override + public ByteBuffer downloadChunk(String id) throws TException { + TimeCacheMap downloaders = data.getDownloaders(); + Object obj = downloaders.get(id); + if (obj == null) { + throw new TException("Could not find input stream for that id"); + } + + try { + if (obj instanceof BufferFileInputStream) { + + BufferFileInputStream is = (BufferFileInputStream) obj; + byte[] ret = is.read(); + if (ret != null) { + downloaders.put(id, (BufferFileInputStream) is); + return ByteBuffer.wrap(ret); + } + } else { + throw new TException("Object isn't BufferFileInputStream for " + + id); + } + } catch (IOException e) { + LOG.error("BufferFileInputStream read failed when downloadChunk ", + e); + throw new TException(e); + } + byte[] empty = {}; + return ByteBuffer.wrap(empty); + } + + @Override + public void finishFileDownload(String id) throws TException { + data.getDownloaders().remove(id); + } + + /** + * get cluster's summary, it will contain SupervisorSummary and + * TopologySummary + * + * @return ClusterSummary + */ + @Override + public ClusterSummary getClusterInfo() throws TException { + long start = System.nanoTime(); + try { + + StormClusterState stormClusterState = data.getStormClusterState(); + + Map assignments = + new HashMap(); + + // get TopologySummary + List topologySummaries = + NimbusUtils.getTopologySummary(stormClusterState, + assignments); + + // all supervisors + Map supervisorInfos = + Cluster.get_all_SupervisorInfo(stormClusterState, null); + + // generate SupervisorSummaries + List supervisorSummaries = + NimbusUtils.mkSupervisorSummaries(supervisorInfos, + assignments); + + NimbusSummary nimbusSummary = + NimbusUtils.getNimbusSummary(stormClusterState, + supervisorSummaries, data); + + ClusterSummary ret = + new ClusterSummary(nimbusSummary, supervisorSummaries, + topologySummaries); + + return ret; + + } catch (TException e) { + LOG.info("Failed to get ClusterSummary ", e); + throw e; + } catch (Exception e) { + LOG.info("Failed to get ClusterSummary ", e); + throw new TException(e); + }finally { + double spend = (System.nanoTime() - start)/1000000.0d; + SimpleJStormMetric.updateHistorgram("getClusterInfo", spend); + LOG.info("getClusterInfo spend {}ms", spend); + } + } + + @Override + public String getVersion() throws TException { + try { + return Utils.getVersion(); + }catch(Exception e) { + String errMsg = "!!! Binary has been changed, please restart Nimbus !!! "; + LOG.error(errMsg, e); + throw new TException(errMsg, e); + } + } + + @Override + public SupervisorWorkers getSupervisorWorkers(String host) + throws NotAliveException, TException { + long start = System.nanoTime(); + try { + + StormClusterState stormClusterState = data.getStormClusterState(); + + String supervisorId = null; + SupervisorInfo supervisorInfo = null; + + String ip = NetWorkUtils.host2Ip(host); + String hostName = NetWorkUtils.ip2Host(host); + + // all supervisors + Map supervisorInfos = + Cluster.get_all_SupervisorInfo(stormClusterState, null); + + for (Entry entry : supervisorInfos + .entrySet()) { + + SupervisorInfo info = entry.getValue(); + if (info.getHostName().equals(hostName) + || info.getHostName().equals(ip)) { + supervisorId = entry.getKey(); + supervisorInfo = info; + break; + } + } + + if (supervisorId == null) { + throw new TException("No supervisor of " + host); + } + + Map assignments = + Cluster.get_all_assignment(stormClusterState, null); + + Map portWorkerSummarys = + new TreeMap(); + + Map metricInfoMap = + new HashMap(); + + int usedSlotNumber = 0; + + Map> topologyTaskToComponent = + new HashMap>(); + + for (Entry entry : assignments.entrySet()) { + String topologyId = entry.getKey(); + Assignment assignment = entry.getValue(); + + Set workers = assignment.getWorkers(); + + TopologyMetric topologyMetric = data.getMetricRunnable().getTopologyMetric(topologyId); + + + for (ResourceWorkerSlot worker : workers) { + if (supervisorId.equals(worker.getNodeId()) == false) { + continue; + } + usedSlotNumber++; + + Integer port = worker.getPort(); + WorkerSummary workerSummary = portWorkerSummarys.get(port); + if (workerSummary == null) { + workerSummary = new WorkerSummary(); + workerSummary.set_port(port); + workerSummary.set_topology(topologyId); + workerSummary.set_tasks(new ArrayList()); + + portWorkerSummarys.put(port, workerSummary); + } + + Map taskToComponent = + topologyTaskToComponent.get(topologyId); + if (taskToComponent == null) { + taskToComponent = Cluster.get_all_task_component(stormClusterState, topologyId, null); + topologyTaskToComponent + .put(topologyId, taskToComponent); + } + + int earliest = TimeUtils.current_time_secs(); + for (Integer taskId : worker.getTasks()) { + + TaskComponent taskComponent = new TaskComponent(); + taskComponent + .set_component(taskToComponent.get(taskId)); + taskComponent.set_taskId(taskId); + Integer startTime = + assignment.getTaskStartTimeSecs().get(taskId); + if (startTime != null && startTime < earliest) { + earliest = startTime; + } + + workerSummary.add_to_tasks(taskComponent); + + } + + workerSummary.set_uptime(TimeUtils.time_delta(earliest)); + + if (topologyMetric == null) { + LOG.warn("Failed to get topologyMetric of " + topologyId); + continue; + } + + String workerSlotName = + TopologyMetricsRunnable.getWorkerSlotName( + supervisorInfo.getHostName(), port); + if (topologyMetric.get_workerMetric() != null) { + MetricInfo workerMetricInfo = + topologyMetric.get_workerMetric().get( + workerSlotName); + + if (workerMetricInfo != null) { + metricInfoMap.put(workerSlotName, workerMetricInfo); + } + } + } + } + + List wokersList = new ArrayList(); + wokersList.addAll(portWorkerSummarys.values()); + + Map supervisorToUsedSlotNum = + new HashMap(); + supervisorToUsedSlotNum.put(supervisorId, usedSlotNumber); + SupervisorSummary supervisorSummary = + NimbusUtils.mkSupervisorSummary(supervisorInfo, + supervisorId, supervisorToUsedSlotNum); + + SupervisorWorkers ret = + new SupervisorWorkers(supervisorSummary, wokersList, + metricInfoMap); + return ret; + + } catch (TException e) { + LOG.info("Failed to get ClusterSummary ", e); + throw e; + } catch (Exception e) { + LOG.info("Failed to get ClusterSummary ", e); + throw new TException(e); + }finally { + double spend = (System.nanoTime() - start)/1000000.0d; + SimpleJStormMetric.updateHistorgram("getSupervisorWorkers", spend); + LOG.info("getSupervisorWorkers, {} spend {} ms", host, spend); + } + } + + /** + * Get TopologyInfo, it contain all data of the topology running status + * + * @return TopologyInfo + */ + @Override + public TopologyInfo getTopologyInfo(String topologyId) + throws NotAliveException, TException { + long start = System.nanoTime(); + StormClusterState stormClusterState = data.getStormClusterState(); + + try { + + // get topology's StormBase + StormBase base = stormClusterState.storm_base(topologyId, null); + if (base == null) { + throw new NotAliveException("No topology of " + topologyId); + } + + Assignment assignment = + stormClusterState.assignment_info(topologyId, null); + if (assignment == null) { + throw new NotAliveException("No topology of " + topologyId); + } + + + Map taskHBMap = + Cluster.get_all_task_heartbeat(stormClusterState, + topologyId); + + Map taskInfoMap = Cluster.get_all_taskInfo(stormClusterState, topologyId); + Map taskToComponent = Cluster.get_all_task_component(stormClusterState, topologyId, taskInfoMap); + Map taskToType = Cluster.get_all_task_type(stormClusterState, topologyId, taskInfoMap); + + + String errorString = null; + if (Cluster.is_topology_exist_error(stormClusterState, topologyId)) { + errorString = "Y"; + } else { + errorString = ""; + } + + TopologySummary topologySummary = new TopologySummary(); + topologySummary.set_id(topologyId); + topologySummary.set_name(base.getStormName()); + topologySummary.set_uptime_secs(TimeUtils.time_delta(base + .getLanchTimeSecs())); + ; + topologySummary.set_status(base.getStatusString()); + topologySummary.set_num_tasks(NimbusUtils + .getTopologyTaskNum(assignment)); + topologySummary.set_num_workers(assignment.getWorkers().size()); + topologySummary.set_error_info(errorString); + + Map componentSummaryMap = + new HashMap(); + + HashMap> componentToTasks = + JStormUtils.reverse_map(taskToComponent); + + for (Entry> entry : componentToTasks + .entrySet()) { + String name = entry.getKey(); + List taskIds = entry.getValue(); + if (taskIds == null || taskIds.size() == 0) { + LOG.warn("No task of component " + name); + continue; + } + + ComponentSummary componentSummary = new ComponentSummary(); + componentSummaryMap.put(name, componentSummary); + + componentSummary.set_name(name); + componentSummary.set_type(taskToType.get(taskIds.get(0))); + componentSummary.set_parallel(taskIds.size()); + componentSummary.set_task_ids(taskIds); + } + + Map taskSummaryMap = + new TreeMap(); + Map > taskErrors = Cluster.get_all_task_errors( + stormClusterState, topologyId); + + for (Integer taskId : taskInfoMap.keySet()) { + TaskSummary taskSummary = new TaskSummary(); + taskSummaryMap.put(taskId, taskSummary); + + taskSummary.set_task_id(taskId); + TaskHeartbeat hb = taskHBMap.get(String.valueOf(taskId)); + if (hb == null) { + taskSummary.set_status("Starting"); + taskSummary.set_uptime(0); + } else { + taskSummary.set_status("ACTIVE"); + taskSummary.set_uptime(hb.getUptimeSecs()); + } + + if (StringUtils.isBlank(errorString)) { + continue; + } + List taskErrorList = taskErrors.get(taskId); + if (taskErrorList != null && taskErrorList.size() != 0) { + for (TaskError taskError : taskErrorList) { + ErrorInfo errorInfo = + new ErrorInfo(taskError.getError(), + taskError.getTimSecs()); + + taskSummary.add_to_errors(errorInfo); + + String component = taskToComponent.get(taskId); + componentSummaryMap.get(component).add_to_errors( + errorInfo); + } + } + } + + for (ResourceWorkerSlot workerSlot : assignment.getWorkers()) { + String hostname = workerSlot.getHostname(); + int port = workerSlot.getPort(); + + for (Integer taskId : workerSlot.getTasks()) { + TaskSummary taskSummary = taskSummaryMap.get(taskId); + taskSummary.set_host(hostname); + taskSummary.set_port(port); + } + } + + TopologyInfo topologyInfo = new TopologyInfo(); + topologyInfo.set_topology(topologySummary); + topologyInfo.set_components(JStormUtils.mk_list(componentSummaryMap + .values())); + topologyInfo + .set_tasks(JStormUtils.mk_list(taskSummaryMap.values())); + topologyInfo.set_metrics(data.getMetricRunnable() + .getTopologyMetric(topologyId)); + + + return topologyInfo; + } catch (TException e) { + LOG.info("Failed to get topologyInfo " + topologyId, e); + throw e; + } catch (Exception e) { + LOG.info("Failed to get topologyInfo " + topologyId, e); + throw new TException("Failed to get topologyInfo" + topologyId); + }finally { + long end = System.nanoTime(); + double spend = (end - start)/1000000.0d; + SimpleJStormMetric.updateHistorgram("getTopologyInfo", spend); + LOG.info("Finish getTopologyInfo {}, spend {} ms", topologyId, spend); + } + + } + + @Override + public TopologyInfo getTopologyInfoByName(String topologyName) + throws NotAliveException, TException { + + String topologyId = getTopologyId(topologyName); + return getTopologyInfo(topologyId); + + } + + @Override + public String getNimbusConf() throws TException { + try { + String ret = JStormUtils.to_json(data.getConf()); + return ret; + } catch (Exception e) { + String err = "Failed to generate Nimbus configuration"; + LOG.error(err, e); + throw new TException(err); + } + } + + /** + * get topology configuration + * + * @param id String: topology id + * @return String + */ + @Override + public String getTopologyConf(String id) throws NotAliveException, + TException { + String rtn; + try { + Map topologyConf = + StormConfig.read_nimbus_topology_conf(conf, id); + rtn = JStormUtils.to_json(topologyConf); + } catch (IOException e) { + // TODO Auto-generated catch block + LOG.info("Failed to get configuration of " + id, e); + throw new TException(e); + } + return rtn; + } + + @Override + public String getTopologyId(String topologyName) throws NotAliveException, + TException { + // TODO Auto-generated method stub + StormClusterState stormClusterState = data.getStormClusterState(); + + try { + // get all active topology's StormBase + String topologyId = + Cluster.get_topology_id(stormClusterState, topologyName); + if (topologyId != null) { + return topologyId; + } + + } catch (Exception e) { + LOG.info("Failed to get getTopologyId " + topologyName, e); + throw new TException("Failed to get getTopologyId " + topologyName); + } + + // topologyId == null + throw new NotAliveException("No topology of " + topologyName); + } + + /** + * get StormTopology throw deserialize local files + * + * @param id String: topology id + * @return StormTopology + */ + @Override + public StormTopology getTopology(String id) throws NotAliveException, + TException { + StormTopology topology = null; + try { + StormTopology stormtopology = + StormConfig.read_nimbus_topology_code(conf, id); + if (stormtopology == null) { + throw new NotAliveException("No topology of " + id); + } + + Map topologyConf = + (Map) StormConfig + .read_nimbus_topology_conf(conf, id); + + topology = Common.system_topology(topologyConf, stormtopology); + } catch (Exception e) { + LOG.error("Failed to get topology " + id + ",", e); + throw new TException("Failed to get system_topology"); + } + return topology; + } + + @Override + public StormTopology getUserTopology(String id) throws NotAliveException, + TException { + StormTopology topology = null; + try { + StormTopology stormtopology = + StormConfig.read_nimbus_topology_code(conf, id); + if (stormtopology == null) { + throw new NotAliveException("No topology of " + id); + } + + return topology; + } catch (Exception e) { + LOG.error("Failed to get topology " + id + ",", e); + throw new TException("Failed to get system_topology"); + } + + } + + /** + * check whether the topology is bActive? + * + * @param nimbus + * @param topologyName + * @param bActive + * @throws Exception + */ + public void checkTopologyActive(NimbusData nimbus, String topologyName, + boolean bActive) throws Exception { + if (isTopologyActive(nimbus.getStormClusterState(), topologyName) != bActive) { + if (bActive) { + throw new NotAliveException(topologyName + " is not alive"); + } else { + throw new AlreadyAliveException(topologyName + + " is already active"); + } + } + } + + /** + * whether the topology is active by topology name + * + * @param stormClusterState see Cluster_clj + * @param topologyName + * @return boolean if the storm is active, return true, otherwise return + * false; + * @throws Exception + */ + public boolean isTopologyActive(StormClusterState stormClusterState, + String topologyName) throws Exception { + boolean rtn = false; + if (Cluster.get_topology_id(stormClusterState, topologyName) != null) { + rtn = true; + } + return rtn; + } + + /** + * create local topology files /local-dir/nimbus/topologyId/stormjar.jar + * /local-dir/nimbus/topologyId/stormcode.ser + * /local-dir/nimbus/topologyId/stormconf.ser + * + * @param conf + * @param topologyId + * @param tmpJarLocation + * @param stormConf + * @param topology + * @throws IOException + */ + private void setupStormCode(Map conf, String topologyId, + String tmpJarLocation, Map stormConf, + StormTopology topology) throws IOException { + // local-dir/nimbus/stormdist/topologyId + String stormroot = StormConfig.masterStormdistRoot(conf, topologyId); + + FileUtils.forceMkdir(new File(stormroot)); + FileUtils.cleanDirectory(new File(stormroot)); + + // copy jar to /local-dir/nimbus/topologyId/stormjar.jar + setupJar(conf, tmpJarLocation, stormroot); + + // serialize to file /local-dir/nimbus/topologyId/stormcode.ser + FileUtils.writeByteArrayToFile( + new File(StormConfig.stormcode_path(stormroot)), + Utils.serialize(topology)); + + // serialize to file /local-dir/nimbus/topologyId/stormconf.ser + FileUtils.writeByteArrayToFile( + 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 + * + * @param conf + * @param tmpJarLocation + * @param stormroot + * @throws IOException + */ + private void setupJar(Map conf, String tmpJarLocation, + String stormroot) throws IOException { + if (!StormConfig.local_mode(conf)) { + 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(); + + return; + } + } + + /** + * generate TaskInfo for every bolt or spout in ZK /ZK/tasks/topoologyId/xxx + * + * @param conf + * @param topologyId + * @param stormClusterState + * @throws Exception + */ + public void setupZkTaskInfo(Map conf, String topologyId, + StormClusterState stormClusterState) throws Exception { + + // mkdir /ZK/taskbeats/topoologyId + stormClusterState.setup_heartbeats(topologyId); + + Map taskToTaskInfo = + mkTaskComponentAssignments(conf, topologyId); + if (taskToTaskInfo == null || taskToTaskInfo.size() == 0) { + throw new InvalidTopologyException("Failed to generate TaskIDs map"); + } + // key is taskid, value is taskinfo + stormClusterState.set_task(topologyId, taskToTaskInfo); + } + + /** + * generate a taskid(Integer) for every task + * + * @param conf + * @param topologyid + * @return Map: from taskid to componentid + * @throws IOException + * @throws InvalidTopologyException + */ + public Map mkTaskComponentAssignments( + Map conf, String topologyid) throws IOException, + InvalidTopologyException { + + // @@@ here exist a little problem, + // we can directly pass stormConf from Submit method + Map stormConf = + StormConfig.read_nimbus_topology_conf(conf, topologyid); + + StormTopology stopology = + StormConfig.read_nimbus_topology_code(conf, topologyid); + + StormTopology topology = Common.system_topology(stormConf, stopology); + + + return Common.mkTaskInfo(stormConf, topology, topologyid); + } + + + + @Override + public void metricMonitor(String topologyName, MonitorOptions options) + throws NotAliveException, TException { + boolean isEnable = options.is_isEnable(); + StormClusterState clusterState = data.getStormClusterState(); + + try { + String topologyId = + Cluster.get_topology_id(clusterState, topologyName); + if (null != topologyId) { + clusterState.set_storm_monitor(topologyId, isEnable); + } else { + throw new NotAliveException( + "Failed to update metricsMonitor status as " + + topologyName + " is not alive"); + } + } catch (Exception e) { + String errMsg = "Failed to update metricsMonitor " + topologyName; + LOG.error(errMsg, e); + throw new TException(e); + } + + } + + @Override + public TopologyMetric getTopologyMetric(String topologyId) + throws NotAliveException, TException { + LOG.debug("Nimbus service handler, getTopologyMetric, topology ID: " + + topologyId); + long start = System.nanoTime(); + try { + TopologyMetric metric = + data.getMetricRunnable().getTopologyMetric(topologyId); + + return metric; + }finally { + double spend = ( System.nanoTime()- start)/1000000.0d;; + SimpleJStormMetric.updateHistorgram("getTopologyMetric", spend); + LOG.info("getTopologyMetric, {}:{}", topologyId, spend); + } + } + + @Override + public void workerUploadMetric(WorkerUploadMetrics uploadMetrics) + throws TException { + // TODO Auto-generated method stub + LOG.debug("!!!!!!! workerUploadMetric:{}:{}:{} ", uploadMetrics.get_topology_id(), + uploadMetrics.get_supervisor_id(), uploadMetrics.get_port()); + + TopologyMetricsRunnable.Update event = + new TopologyMetricsRunnable.Update(); + event.workerMetrics = uploadMetrics; + + data.getMetricRunnable().pushEvent(event); + } + + public void uploadNewCredentials(String topologyName, Credentials creds) { + // TODO Auto-generated method stub + + } + + @Override + public NettyMetric getNettyMetric(String topologyName, int pos) throws TException { + // TODO Auto-generated method stub + long start = System.nanoTime(); + try { + String topologyId = getTopologyId(topologyName); + + if (pos < 0) { + LOG.warn("Invalid pos {}, set it as 0", pos); + pos = 0; + } + SortedMap allConnections = data.getMetricRunnable().getNettyMetric(topologyId); + int mapSize = allConnections.size(); + + NettyMetric ret = new NettyMetric(); + + ret.set_connectionNum(mapSize); + + + Map selectConnections = new TreeMap(); + ret.set_connections(selectConnections); + int i = 0; + int selectMapSize = 0; + for (Entry entry: allConnections.entrySet()) { + i++; + if (i <= pos) { + continue; + } + + selectConnections.put(entry.getKey(), entry.getValue()); + selectMapSize++; + if (selectMapSize >= MetricDef.NETTY_METRICS_PACKAGE_SIZE) { + break; + } + } + + return ret; + }finally { + double spend = (System.nanoTime() - start)/1000000.0d; + SimpleJStormMetric.updateHistorgram("getNettyMetric", spend ); + LOG.info("getNettyMetric, {}:{} ms", topologyName, spend); + } + } + + + @Override + public NettyMetric getServerNettyMetric(String topologyName, String serverName) throws TException { + // TODO Auto-generated method stub + long start = System.nanoTime(); + try { + String topologyId = getTopologyId(topologyName); + + SortedMap allConnections = data.getMetricRunnable().getNettyMetric(topologyId); + int mapSize = allConnections.size(); + + NettyMetric ret = new NettyMetric(); + + String serverIp = NetWorkUtils.host2Ip(serverName); + Map selectConnections = new TreeMap(); + for (Entry entry: allConnections.entrySet()) { + if (entry.getKey().contains(serverIp)) { + selectConnections.put(entry.getKey(), entry.getValue()); + } + + } + ret.set_connectionNum(selectConnections.size()); + ret.set_connections(selectConnections); + return ret; + }finally { + double spend = (System.nanoTime()- start)/1000000.0d; + SimpleJStormMetric.updateHistorgram("getNettyMetric", spend); + LOG.info("getServerNettyMetric, {} : {}ms", topologyName, spend); + } + } + + @Override + public void updateConf(String name, String conf) throws NotAliveException, + InvalidTopologyException, TException { + try { + checkTopologyActive(data, name, true); + + Map config = + (Map) JStormUtils.from_json(conf); + + NimbusUtils.transitionName(data, name, true, + StatusType.update_conf, config); + } catch (NotAliveException e) { + String errMsg = "Rebalance Error, no this topology " + name; + LOG.error(errMsg, e); + throw new NotAliveException(errMsg); + } catch (Exception e) { + String errMsg = "Failed to rebalance topology " + name; + LOG.error(errMsg, e); + throw new TException(errMsg); + } + + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/StatusTransition.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/StatusTransition.java new file mode 100644 index 000000000..e98445587 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/StatusTransition.java @@ -0,0 +1,294 @@ +/** + * 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.daemon.nimbus; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.callback.Callback; +import com.alibaba.jstorm.callback.impl.ActiveTransitionCallback; +import com.alibaba.jstorm.callback.impl.DoRebalanceTransitionCallback; +import com.alibaba.jstorm.callback.impl.DoneRebalanceTransitionCallback; +import com.alibaba.jstorm.callback.impl.InactiveTransitionCallback; +import com.alibaba.jstorm.callback.impl.KillTransitionCallback; +import com.alibaba.jstorm.callback.impl.ReassignTransitionCallback; +import com.alibaba.jstorm.callback.impl.RebalanceTransitionCallback; +import com.alibaba.jstorm.callback.impl.RemoveTransitionCallback; +import com.alibaba.jstorm.callback.impl.UpdateConfTransitionCallback; +import com.alibaba.jstorm.cluster.StormBase; +import com.alibaba.jstorm.cluster.StormStatus; + +/** + * Status changing + * + * @author version1: lixin version2: Longda + * + * + * + */ +public class StatusTransition { + + private final static Logger LOG = LoggerFactory + .getLogger(StatusTransition.class); + + private NimbusData data; + + private Map topologyLocks = + new ConcurrentHashMap(); + + public StatusTransition(NimbusData data) { + this.data = data; + + } + + public void transition(String topologyid, boolean errorOnNoTransition, + StatusType changeStatus, T... args) throws Exception { + // lock outside + Object lock = topologyLocks.get(topologyid); + if (lock == null) { + 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); + + // update the lock times + topologyLocks.put(topologyid, lock); + } + } + + /** + * Changing status + * + * @param topologyId + * @param errorOnNTransition if it is true, failure will throw exception + * @param args -- will be used in the status changing callback + * + */ + public void transitionLock(String topologyid, + boolean errorOnNoTransition, StatusType changeStatus, T... args) + throws Exception { + + // get ZK's topology node's data, which is StormBase + StormBase stormbase = + data.getStormClusterState().storm_base(topologyid, null); + if (stormbase == null) { + + LOG.error("Cannot apply event changing status " + + changeStatus.getStatus() + " to " + topologyid + + " because failed to get StormBase from ZK"); + return; + } + + StormStatus currentStatus = stormbase.getStatus(); + if (currentStatus == null) { + LOG.error("Cannot apply event changing status " + + changeStatus.getStatus() + " to " + topologyid + + " because topologyStatus is null in ZK"); + return; + } + + // > + Map> callbackMap = + stateTransitions(topologyid, currentStatus); + + // get current changingCallbacks + Map changingCallbacks = + callbackMap.get(currentStatus.getStatusType()); + + if (changingCallbacks == null + || changingCallbacks.containsKey(changeStatus) == false + || changingCallbacks.get(changeStatus) == null) { + String msg = + "No transition for event: changing status:" + + changeStatus.getStatus() + ", current status: " + + currentStatus.getStatusType() + " topology-id: " + + topologyid; + LOG.info(msg); + if (errorOnNoTransition) { + throw new RuntimeException(msg); + } + return; + } + + Callback callback = changingCallbacks.get(changeStatus); + + Object obj = callback.execute(args); + if (obj != null && obj instanceof StormStatus) { + StormStatus newStatus = (StormStatus) obj; + // update status to ZK + data.getStormClusterState().update_storm(topologyid, newStatus); + LOG.info("Successfully updated " + topologyid + " as status " + + newStatus); + } + + LOG.info("Successfully apply event changing status " + + changeStatus.getStatus() + " to " + topologyid); + return; + + } + + /** + * generate status changing map + * + * + * + * @param topologyid + * @param status + * @return + * + * Map> means + * Map> + */ + + private Map> stateTransitions( + String topologyid, StormStatus currentStatus) { + + /** + * + * 1. Status: this status will be stored in ZK + * killed/inactive/active/rebalancing 2. action: + * + * monitor -- every Config.NIMBUS_MONITOR_FREQ_SECS seconds will trigger + * this only valid when current status is active inactivate -- client + * will trigger this action, only valid when current status is active + * activate -- client will trigger this action only valid when current + * status is inactive startup -- when nimbus startup, it will trigger + * this action only valid when current status is killed/rebalancing kill + * -- client kill topology will trigger this action, only valid when + * current status is active/inactive/killed remove -- 30 seconds after + * client submit kill command, it will do this action, only valid when + * current status is killed rebalance -- client submit rebalance + * command, only valid when current status is active/deactive + * do_rebalance -- 30 seconds after client submit rebalance command, it + * will do this action, only valid when current status is rebalance + */ + + Map> rtn = + new HashMap>(); + + // current status is active + Map activeMap = + new HashMap(); + activeMap.put(StatusType.monitor, new ReassignTransitionCallback(data, + topologyid)); + activeMap.put(StatusType.inactivate, new InactiveTransitionCallback()); + activeMap.put(StatusType.startup, null); + activeMap.put(StatusType.activate, null); + activeMap.put(StatusType.kill, new KillTransitionCallback(data, + topologyid)); + activeMap.put(StatusType.remove, null); + activeMap.put(StatusType.rebalance, new RebalanceTransitionCallback( + data, topologyid, currentStatus)); + activeMap.put(StatusType.do_rebalance, null); + activeMap.put(StatusType.done_rebalance, null); + activeMap.put(StatusType.update_conf, new UpdateConfTransitionCallback( + data, topologyid, currentStatus)); + + rtn.put(StatusType.active, activeMap); + + // current status is inactive + Map inactiveMap = + new HashMap(); + + inactiveMap.put(StatusType.monitor, new ReassignTransitionCallback( + data, topologyid, new StormStatus(StatusType.inactive))); + inactiveMap.put(StatusType.inactivate, null); + inactiveMap.put(StatusType.startup, null); + inactiveMap.put(StatusType.activate, new ActiveTransitionCallback()); + inactiveMap.put(StatusType.kill, new KillTransitionCallback(data, + topologyid)); + inactiveMap.put(StatusType.remove, null); + inactiveMap.put(StatusType.rebalance, new RebalanceTransitionCallback( + data, topologyid, currentStatus)); + inactiveMap.put(StatusType.do_rebalance, null); + inactiveMap.put(StatusType.done_rebalance, null); + inactiveMap.put(StatusType.update_conf, null); + + rtn.put(StatusType.inactive, inactiveMap); + + // current status is killed + Map killedMap = + new HashMap(); + + killedMap.put(StatusType.monitor, null); + killedMap.put(StatusType.inactivate, null); + killedMap.put(StatusType.startup, new KillTransitionCallback(data, + topologyid)); + killedMap.put(StatusType.activate, null); + killedMap.put(StatusType.kill, new KillTransitionCallback(data, + topologyid)); + killedMap.put(StatusType.remove, new RemoveTransitionCallback(data, + topologyid)); + killedMap.put(StatusType.rebalance, null); + killedMap.put(StatusType.do_rebalance, null); + killedMap.put(StatusType.done_rebalance, null); + killedMap.put(StatusType.update_conf, null); + rtn.put(StatusType.killed, killedMap); + + // current status is under rebalancing + Map rebalancingMap = + new HashMap(); + + StatusType rebalanceOldStatus = StatusType.active; + if (currentStatus.getOldStatus() != null) { + rebalanceOldStatus = currentStatus.getOldStatus().getStatusType(); + // fix double rebalance, make the status always as rebalacing + if (rebalanceOldStatus == StatusType.rebalancing) { + rebalanceOldStatus = StatusType.active; + } + } + + rebalancingMap.put(StatusType.monitor, null); + rebalancingMap.put(StatusType.inactivate, null); + rebalancingMap.put(StatusType.startup, new RebalanceTransitionCallback( + data, topologyid, new StormStatus(rebalanceOldStatus))); + rebalancingMap.put(StatusType.activate, null); + rebalancingMap.put(StatusType.kill, null); + rebalancingMap.put(StatusType.remove, null); + rebalancingMap + .put(StatusType.rebalance, new RebalanceTransitionCallback( + data, topologyid, currentStatus)); + rebalancingMap.put(StatusType.do_rebalance, + new DoRebalanceTransitionCallback(data, topologyid, + new StormStatus(rebalanceOldStatus))); + rebalancingMap.put(StatusType.done_rebalance, + new DoneRebalanceTransitionCallback(data, topologyid)); + rebalancingMap.put(StatusType.update_conf, null); + rtn.put(StatusType.rebalancing, rebalancingMap); + + /** + * @@@ just handling 4 kind of status, maybe add later + */ + + return rtn; + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/StatusType.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/StatusType.java new file mode 100644 index 000000000..cf785b7f6 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/StatusType.java @@ -0,0 +1,63 @@ +/** + * 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.daemon.nimbus; + +/** + * topology status: + * + * 1. Status: this status will be stored in ZK + * killed/inactive/active/rebalancing 2. action: + * + * monitor -- every Config.NIMBUS_MONITOR_FREQ_SECS seconds will trigger this + * only valid when current status is active inactivate -- client will trigger + * this action, only valid when current status is active activate -- client will + * trigger this action only valid when current status is inactive startup -- + * when nimbus startup, it will trigger this action only valid when current + * status is killed/rebalancing kill -- client kill topology will trigger this + * action, only valid when current status is active/inactive/killed remove -- 30 + * seconds after client submit kill command, it will do this action, only valid + * when current status is killed rebalance -- client submit rebalance command, + * only valid when current status is active/deactive do_rebalance -- 30 seconds + * after client submit rebalance command, it will do this action, only valid + * when current status is rebalance + * + * + * + */ + +public enum StatusType { + + // status + active("active"), inactive("inactive"), rebalancing("rebalancing"), killed( + "killed"), + + // actions + activate("activate"), inactivate("inactivate"), monitor("monitor"), startup( + "startup"), kill("kill"), remove("remove"), rebalance("rebalance"), do_rebalance( + "do-rebalance"), done_rebalance("done-rebalance"), update_conf("update-config"); + + private String status; + + StatusType(String status) { + this.status = status; + } + + public String getStatus() { + return status; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/TopologyAssign.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/TopologyAssign.java new file mode 100644 index 000000000..fd6f46115 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/TopologyAssign.java @@ -0,0 +1,996 @@ +/** + * 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.daemon.nimbus; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.LinkedBlockingQueue; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.generated.StormTopology; +import backtype.storm.scheduler.WorkerSlot; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.Cluster; +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.daemon.supervisor.SupervisorInfo; +import com.alibaba.jstorm.schedule.Assignment; +import com.alibaba.jstorm.schedule.AssignmentBak; +import com.alibaba.jstorm.schedule.IToplogyScheduler; +import com.alibaba.jstorm.schedule.TopologyAssignContext; +import com.alibaba.jstorm.schedule.default_assign.DefaultTopologyScheduler; +import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; +import com.alibaba.jstorm.utils.FailedAssignTopologyException; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.PathUtils; +import com.alibaba.jstorm.utils.TimeUtils; + +public class TopologyAssign implements Runnable { + + private final static Logger LOG = LoggerFactory + .getLogger(TopologyAssign.class); + + /** + * private constructor function to avoid multiple instance + */ + private TopologyAssign() { + + } + + private static TopologyAssign instance = null; + + public static TopologyAssign getInstance() { + synchronized (TopologyAssign.class) { + if (instance == null) { + instance = new TopologyAssign(); + } + return instance; + + } + } + + protected NimbusData nimbusData; + + protected Map schedulers; + + private Thread thread; + + private int cleanupTimeoutSec = 60; + + public static final String DEFAULT_SCHEDULER_NAME = "default"; + + public void init(NimbusData nimbusData) { + this.nimbusData = nimbusData; + + //this.cleanupTimeoutSec = 60; + + this.schedulers = new HashMap(); + + IToplogyScheduler defaultScheduler = new DefaultTopologyScheduler(); + defaultScheduler.prepare(nimbusData.getConf()); + + schedulers.put(DEFAULT_SCHEDULER_NAME, defaultScheduler); + + thread = new Thread(this); + thread.setName("TopologyAssign"); + thread.setDaemon(true); + thread.start(); + } + + public void cleanup() { + runFlag = false; + thread.interrupt(); + } + + protected static LinkedBlockingQueue queue = + new LinkedBlockingQueue(); + + public static void push(TopologyAssignEvent event) { + queue.offer(event); + } + + volatile boolean runFlag = false; + + public void run() { + LOG.info("TopologyAssign thread has been started"); + runFlag = true; + + while (runFlag) { + TopologyAssignEvent event; + try { + event = queue.take(); + } catch (InterruptedException e1) { + continue; + } + if (event == null) { + continue; + } + + boolean isSuccess = doTopologyAssignment(event); + + if (isSuccess == false) { + } else { + try { + cleanupDisappearedTopology(); + } catch (Exception e) { + LOG.error("Failed to do cleanup disappear topology ", e); + continue; + } + } + } + + } + + /** + * Create/Update topology assignment set topology status + * + * @param event + * @return + */ + protected boolean doTopologyAssignment(TopologyAssignEvent event) { + Assignment assignment = null; + try { + assignment = mkAssignment(event); + + if (!(event.isScratch())) + setTopologyStatus(event); + } catch (Throwable e) { + LOG.error("Failed to assign topology " + event.getTopologyId(), e); + event.fail(e.getMessage()); + return false; + } + + if (assignment != null) + backupAssignment(assignment, event); + event.done(); + return true; + } + + /** + * cleanup the topologies which are not in ZK /topology, but in other place + * + * @param nimbusData + * @param active_topologys + * @throws Exception + */ + public void cleanupDisappearedTopology() throws Exception { + StormClusterState clusterState = nimbusData.getStormClusterState(); + + List active_topologys = clusterState.active_storms(); + if (active_topologys == null) { + return; + } + + Set cleanupIds = + get_cleanup_ids(clusterState, active_topologys); + + for (String topologyId : cleanupIds) { + + LOG.info("Cleaning up " + topologyId); + + clusterState.try_remove_storm(topologyId); + // + nimbusData.getTaskHeartbeatsCache().remove(topologyId); + + NimbusUtils.removeTopologyTaskTimeout(nimbusData, topologyId); + + // get /nimbus/stormdist/topologyId + String master_stormdist_root = + StormConfig.masterStormdistRoot(nimbusData.getConf(), + topologyId); + try { + // delete topologyId local dir + PathUtils.rmr(master_stormdist_root); + } catch (IOException e) { + LOG.warn("Failed to delete " + master_stormdist_root + ",", e); + } + } + } + + private void get_code_ids(List code_ids, + HashSet latest_code_ids) throws IOException { + Map conf = nimbusData.getConf(); + + String master_stormdist_root = StormConfig.masterStormdistRoot(conf); + // listdir /local-dir/nimbus/stormdist + List all_code_ids = + PathUtils.read_dir_contents(master_stormdist_root); + code_ids.addAll(all_code_ids); + + long now = System.currentTimeMillis(); + for (String dir : code_ids) { + File file = new File(master_stormdist_root + File.separator + dir); + try { + if (file.exists() == false) { + continue; + } + + long modify = file.lastModified(); + + if (now - modify < cleanupTimeoutSec * 1000) { + latest_code_ids.add(dir); + } + } catch (Exception exception) { + LOG.error("Failed to get modify time of " + dir, exception); + } + + } + + } + + /** + * get topology ids which need to be cleanup + * + * @param clusterState + * @return + * @throws Exception + */ + private Set get_cleanup_ids(StormClusterState clusterState, + List active_topologys) throws Exception { + + 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 metric_ids = clusterState.get_metrics(); + + List code_ids = new ArrayList(); + HashSet latest_code_ids = new HashSet(); + get_code_ids(code_ids, latest_code_ids); + + // Set assigned_ids = + // JStormUtils.listToSet(clusterState.active_storms()); + Set to_cleanup_ids = new HashSet(); + Set pendingTopologys = + nimbusData.getPendingSubmitTopoloygs().keySet(); + + 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 (code_ids != null) { + to_cleanup_ids.addAll(code_ids); + } + + if (metric_ids != null) { + to_cleanup_ids.addAll(metric_ids); + } + + if (active_topologys != null) { + to_cleanup_ids.removeAll(active_topologys); + latest_code_ids.removeAll(active_topologys); + } + + if (pendingTopologys != null) { + to_cleanup_ids.removeAll(pendingTopologys); + } + + /** + * Why need to remove latest code. Due to competition between + * Thrift.threads and TopologyAssign thread + * + */ + to_cleanup_ids.removeAll(latest_code_ids); + LOG.info("Skip remove topology of " + latest_code_ids); + + return to_cleanup_ids; + } + + /** + * start a topology: set active status of the topology + * + * @param topologyName + * @param stormClusterState + * @param topologyId + * @throws Exception + */ + public void setTopologyStatus(TopologyAssignEvent event) throws Exception { + StormClusterState stormClusterState = nimbusData.getStormClusterState(); + + String topologyId = event.getTopologyId(); + String topologyName = event.getTopologyName(); + String group = event.getGroup(); + + StormStatus status = new StormStatus(StatusType.active); + if (event.getOldStatus() != null) { + status = event.getOldStatus(); + } + + boolean isEnable = + ConfigExtension + .isEnablePerformanceMetrics(nimbusData.getConf()); + + StormBase stormBase = stormClusterState.storm_base(topologyId, null); + if (stormBase == null) { + stormBase = + new StormBase(topologyName, TimeUtils.current_time_secs(), + status, group); + stormBase.setEnableMonitor(isEnable); + stormClusterState.activate_storm(topologyId, stormBase); + + } else { + + stormClusterState.update_storm(topologyId, status); + stormClusterState.set_storm_monitor(topologyId, isEnable); + + // here exist one hack operation + // when monitor/rebalance/startup topologyName is null + if (topologyName == null) { + event.setTopologyName(stormBase.getStormName()); + } + } + + LOG.info("Update " + topologyId + " " + status); + + } + + protected TopologyAssignContext prepareTopologyAssign( + TopologyAssignEvent event) throws Exception { + TopologyAssignContext ret = new TopologyAssignContext(); + + String topologyId = event.getTopologyId(); + + Map nimbusConf = nimbusData.getConf(); + Map topologyConf = + StormConfig.read_nimbus_topology_conf(nimbusConf, topologyId); + + StormTopology rawTopology = + StormConfig.read_nimbus_topology_code(nimbusConf, topologyId); + ret.setRawTopology(rawTopology); + + Map stormConf = new HashMap(); + stormConf.putAll(nimbusConf); + stormConf.putAll(topologyConf); + ret.setStormConf(stormConf); + + StormClusterState stormClusterState = nimbusData.getStormClusterState(); + + // get all running supervisor, don't need callback to watch supervisor + Map supInfos = + Cluster.get_all_SupervisorInfo(stormClusterState, null); + // init all AvailableWorkerPorts + for (Entry supInfo : supInfos.entrySet()) { + SupervisorInfo supervisor = supInfo.getValue(); + if (supervisor != null) + supervisor.setAvailableWorkerPorts(supervisor.getWorkerPorts()); + } + + getAliveSupervsByHb(supInfos, nimbusConf); + if (supInfos.size() == 0) { + throw new FailedAssignTopologyException( + "Failed to make assignment " + topologyId + + ", due to no alive supervisor"); + } + + Map taskToComponent = + Cluster.get_all_task_component(stormClusterState, topologyId, null); + ret.setTaskToComponent(taskToComponent); + + // get taskids /ZK/tasks/topologyId + Set allTaskIds = taskToComponent.keySet(); + if (allTaskIds == null || allTaskIds.size() == 0) { + String errMsg = + "Failed to get all task ID list from /ZK-dir/tasks/" + + topologyId; + LOG.warn(errMsg); + throw new IOException(errMsg); + } + ret.setAllTaskIds(allTaskIds); + + Set aliveTasks = new HashSet(); + // unstoppedTasks are tasks which are alive on no supervisor's(dead) + // machine + Set unstoppedTasks = new HashSet(); + Set deadTasks = new HashSet(); + Set unstoppedWorkers = + new HashSet(); + + Assignment existingAssignment = + stormClusterState.assignment_info(topologyId, null); + if (existingAssignment != null) { + aliveTasks = getAliveTasks(topologyId, allTaskIds); + unstoppedTasks = + getUnstoppedSlots(aliveTasks, supInfos, existingAssignment); + + deadTasks.addAll(allTaskIds); + deadTasks.removeAll(aliveTasks); + } + + ret.setDeadTaskIds(deadTasks); + ret.setUnstoppedTaskIds(unstoppedTasks); + + // Step 2: get all slots resource, free slots/ alive slots/ unstopped + // slots + getFreeSlots(supInfos, stormClusterState); + ret.setCluster(supInfos); + + if (existingAssignment == null) { + ret.setAssignType(TopologyAssignContext.ASSIGN_TYPE_NEW); + + try { + AssignmentBak lastAssignment = + stormClusterState.assignment_bak(event + .getTopologyName()); + if (lastAssignment != null) { + ret.setOldAssignment(lastAssignment.getAssignment()); + } + } catch (Exception e) { + LOG.warn("Fail to get old assignment", e); + } + } else { + ret.setOldAssignment(existingAssignment); + if (event.isScratch()) { + ret.setAssignType(TopologyAssignContext.ASSIGN_TYPE_REBALANCE); + ret.setIsReassign(event.isReassign()); + unstoppedWorkers = + getUnstoppedWorkers(unstoppedTasks, existingAssignment); + ret.setUnstoppedWorkers(unstoppedWorkers); + } else { + ret.setAssignType(TopologyAssignContext.ASSIGN_TYPE_MONITOR); + unstoppedWorkers = + getUnstoppedWorkers(aliveTasks, existingAssignment); + ret.setUnstoppedWorkers(unstoppedWorkers); + } + } + + return ret; + } + + /** + * make assignments for a topology The nimbus core function, this function + * has been totally rewrite + * + * @param nimbusData NimbusData + * @param topologyId String + * @param isScratch Boolean: isScratch is false unless rebalancing the + * topology + * @throws Exception + */ + public Assignment mkAssignment(TopologyAssignEvent event) throws Exception { + String topologyId = event.getTopologyId(); + + LOG.info("Determining assignment for " + topologyId); + + TopologyAssignContext context = prepareTopologyAssign(event); + + Set assignments = null; + + if (!StormConfig.local_mode(nimbusData.getConf())) { + + IToplogyScheduler scheduler = + schedulers.get(DEFAULT_SCHEDULER_NAME); + + assignments = scheduler.assignTasks(context); + + } else { + assignments = mkLocalAssignment(context); + } + + Assignment assignment = null; + if (assignments != null && assignments.size() > 0) { + Map nodeHost = + getTopologyNodeHost(context.getCluster(), + context.getOldAssignment(), assignments); + + Map startTimes = + getTaskStartTimes(context, nimbusData, topologyId, + context.getOldAssignment(), assignments); + + String codeDir = + StormConfig.masterStormdistRoot(nimbusData.getConf(), + topologyId); + + assignment = + new Assignment(codeDir, assignments, nodeHost, startTimes); + + StormClusterState stormClusterState = + nimbusData.getStormClusterState(); + + stormClusterState.set_assignment(topologyId, assignment); + + // update task heartbeat's start time + NimbusUtils.updateTaskHbStartTime(nimbusData, assignment, + topologyId); + + // @@@ TODO + + // Update metrics information in ZK when rebalance or reassignment + // Only update metrics monitor status when creating topology + // if (context.getAssignType() == + // TopologyAssignContext.ASSIGN_TYPE_REBALANCE + // || context.getAssignType() == + // TopologyAssignContext.ASSIGN_TYPE_MONITOR) + // NimbusUtils.updateMetricsInfo(nimbusData, topologyId, assignment); + + NimbusUtils.updateTopologyTaskTimeout(nimbusData, topologyId); + + LOG.info("Successfully make assignment for topology id " + + topologyId + ": " + assignment); + } + return assignment; + } + + private static Set mkLocalAssignment( + TopologyAssignContext context) { + Set result = new HashSet(); + Map cluster = context.getCluster(); + if (cluster.size() != 1) + throw new RuntimeException(); + SupervisorInfo localSupervisor = null; + String supervisorId = null; + for (Entry entry : cluster.entrySet()) { + supervisorId = entry.getKey(); + localSupervisor = entry.getValue(); + } + int port = localSupervisor.getAvailableWorkerPorts().iterator().next(); + ResourceWorkerSlot worker = new ResourceWorkerSlot(supervisorId, port); + worker.setTasks(new HashSet(context.getAllTaskIds())); + worker.setHostname(localSupervisor.getHostName()); + result.add(worker); + return result; + } + + /** + * @param existingAssignment + * @param taskWorkerSlot + * @return + * @throws Exception + */ + public static Map getTaskStartTimes( + TopologyAssignContext context, NimbusData nimbusData, + String topologyId, Assignment existingAssignment, + Set workers) throws Exception { + + Map startTimes = new TreeMap(); + + if (context.getAssignType() == TopologyAssignContext.ASSIGN_TYPE_NEW) { + int nowSecs = TimeUtils.current_time_secs(); + for (ResourceWorkerSlot worker : workers) { + for (Integer changedTaskId : worker.getTasks()) { + startTimes.put(changedTaskId, nowSecs); + } + } + + return startTimes; + } + + Set oldWorkers = new HashSet(); + + if (existingAssignment != null) { + Map taskStartTimeSecs = + existingAssignment.getTaskStartTimeSecs(); + if (taskStartTimeSecs != null) { + startTimes.putAll(taskStartTimeSecs); + } + + if (existingAssignment.getWorkers() != null) { + oldWorkers = existingAssignment.getWorkers(); + } + } + + StormClusterState zkClusterState = nimbusData.getStormClusterState(); + Set changedTaskIds = getNewOrChangedTaskIds(oldWorkers, workers); + int nowSecs = TimeUtils.current_time_secs(); + for (Integer changedTaskId : changedTaskIds) { + startTimes.put(changedTaskId, nowSecs); + zkClusterState.remove_task_heartbeat(topologyId, changedTaskId); + } + + Set removedTaskIds = getRemovedTaskIds(oldWorkers, workers); + for (Integer removedTaskId : removedTaskIds) { + startTimes.remove(removedTaskId); + zkClusterState.remove_task_heartbeat(topologyId, removedTaskId); + } + + LOG.info("Task assignment has been changed: " + changedTaskIds + + ", removed tasks " + removedTaskIds); + return startTimes; + } + + public static Map getTopologyNodeHost( + Map supervisorMap, + Assignment existingAssignment, Set workers) { + + // the following is that remove unused node from allNodeHost + Set usedNodes = new HashSet(); + for (ResourceWorkerSlot worker : workers) { + + usedNodes.add(worker.getNodeId()); + } + + // map + Map allNodeHost = new HashMap(); + + if (existingAssignment != null) { + allNodeHost.putAll(existingAssignment.getNodeHost()); + } + + // get alive supervisorMap Map + Map nodeHost = + SupervisorInfo.getNodeHost(supervisorMap); + if (nodeHost != null) { + allNodeHost.putAll(nodeHost); + } + + Map ret = new HashMap(); + + for (String supervisorId : usedNodes) { + if (allNodeHost.containsKey(supervisorId)) { + ret.put(supervisorId, allNodeHost.get(supervisorId)); + } else { + LOG.warn("Node " + supervisorId + + " doesn't in the supervisor list"); + } + } + + return ret; + } + + /** + * get all taskids which are assigned newly or reassigned + * + * @param taskToWorkerSlot + * @param newtaskToWorkerSlot + * @return Set taskid which is assigned newly or reassigned + */ + public static Set getNewOrChangedTaskIds( + Set oldWorkers, Set workers) { + + Set rtn = new HashSet(); + HashMap workerPortMap = + HostPortToWorkerMap(oldWorkers); + for (ResourceWorkerSlot worker : workers) { + ResourceWorkerSlot oldWorker = workerPortMap.get(worker.getHostPort()); + if (oldWorker != null) { + Set oldTasks = oldWorker.getTasks(); + for (Integer task : worker.getTasks()) { + if (!(oldTasks.contains(task))) + rtn.add(task); + } + } else { + rtn.addAll(worker.getTasks()); + } + } + return rtn; + } + + public static Set getRemovedTaskIds( + Set oldWorkers, Set workers) { + + Set rtn = new HashSet(); + Set oldTasks = getTaskSetFromWorkerSet(oldWorkers); + Set newTasks = getTaskSetFromWorkerSet(workers); + for (Integer taskId : oldTasks) { + if (!(newTasks.contains(taskId))) { + rtn.add(taskId); + } + } + return rtn; + } + + private static Set getTaskSetFromWorkerSet( + Set workers) { + Set rtn = new HashSet(); + for (ResourceWorkerSlot worker : workers) { + rtn.addAll(worker.getTasks()); + } + return rtn; + } + + private static HashMap HostPortToWorkerMap( + Set workers) { + HashMap rtn = + new HashMap(); + for (ResourceWorkerSlot worker : workers) { + rtn.put(worker.getHostPort(), worker); + } + return rtn; + } + + /** + * sort slots, the purpose is to ensure that the tasks are assigned in + * balancing + * + * @param allSlots + * @return List + */ + public static List sortSlots(Set allSlots, + int needSlotNum) { + + Map> nodeMap = + new HashMap>(); + + // group by first + for (WorkerSlot np : allSlots) { + String node = np.getNodeId(); + + List list = nodeMap.get(node); + if (list == null) { + list = new ArrayList(); + nodeMap.put(node, list); + } + + list.add(np); + + } + + for (Entry> entry : nodeMap.entrySet()) { + List ports = entry.getValue(); + + Collections.sort(ports, new Comparator() { + + @Override + public int compare(WorkerSlot first, WorkerSlot second) { + String firstNode = first.getNodeId(); + String secondNode = second.getNodeId(); + if (firstNode.equals(secondNode) == false) { + return firstNode.compareTo(secondNode); + } else { + return first.getPort() - second.getPort(); + } + + } + + }); + } + + // interleave + List> splitup = + new ArrayList>(nodeMap.values()); + + Collections.sort(splitup, new Comparator>() { + + public int compare(List o1, List o2) { + return o2.size() - o1.size(); + } + }); + + List sortedFreeSlots = JStormUtils.interleave_all(splitup); + + if (sortedFreeSlots.size() <= needSlotNum) { + return sortedFreeSlots; + + } + + // sortedFreeSlots > needSlotNum + return sortedFreeSlots.subList(0, needSlotNum); + } + + /** + * Get unstopped slots from alive task list + * + * @param aliveAssigned + * @param supInfos + * @return + */ + public Set getUnstoppedSlots(Set aliveTasks, + Map supInfos, Assignment existAssignment) { + Set ret = new HashSet(); + + Set oldWorkers = existAssignment.getWorkers(); + + Set aliveSupervisors = supInfos.keySet(); + + for (ResourceWorkerSlot worker : oldWorkers) { + for (Integer taskId : worker.getTasks()) { + if (aliveTasks.contains(taskId) == false) { + // task is dead + continue; + } + + String oldTaskSupervisorId = worker.getNodeId(); + + if (aliveSupervisors.contains(oldTaskSupervisorId) == false) { + // supervisor is dead + ret.add(taskId); + continue; + } + } + } + + return ret; + + } + + private Set getUnstoppedWorkers( + Set aliveTasks, Assignment existAssignment) { + Set ret = new HashSet(); + for (ResourceWorkerSlot worker : existAssignment.getWorkers()) { + boolean alive = true; + for (Integer task : worker.getTasks()) { + if (!aliveTasks.contains(task)) { + alive = false; + break; + } + } + if (alive) { + ret.add(worker); + } + } + return ret; + } + + /** + * Get free resources + * + * @param supervisorInfos + * @param stormClusterState + * @throws Exception + */ + public static void getFreeSlots( + Map supervisorInfos, + StormClusterState stormClusterState) throws Exception { + + Map assignments = + Cluster.get_all_assignment(stormClusterState, null); + + for (Entry entry : assignments.entrySet()) { + String topologyId = entry.getKey(); + Assignment assignment = entry.getValue(); + + Set workers = assignment.getWorkers(); + + for (ResourceWorkerSlot worker : workers) { + + SupervisorInfo supervisorInfo = + supervisorInfos.get(worker.getNodeId()); + if (supervisorInfo == null) { + // the supervisor is dead + continue; + } + supervisorInfo.getAvailableWorkerPorts().remove(worker.getPort()); + } + } + + } + + /** + * find all alived taskid Does not assume that clocks are synchronized. Task + * heartbeat is only used so that nimbus knows when it's received a new + * heartbeat. All timing is done by nimbus and tracked through + * task-heartbeat-cache + * + * @param conf + * @param topologyId + * @param stormClusterState + * @param taskIds + * @param taskStartTimes + * @param taskHeartbeatsCache --Map>> + * @return Set : taskid + * @throws Exception + */ + public Set getAliveTasks(String topologyId, Set taskIds) + throws Exception { + + Set aliveTasks = new HashSet(); + + // taskIds is the list from ZK /ZK-DIR/tasks/topologyId + for (int taskId : taskIds) { + + boolean isDead = + NimbusUtils.isTaskDead(nimbusData, topologyId, taskId); + if (isDead == false) { + aliveTasks.add(taskId); + } + + } + + return aliveTasks; + } + + /** + * Backup the toplogy's Assignment to ZK + * + * @@@ Question Do we need to do backup operation every time? + * @param assignment + * @param event + */ + public void backupAssignment(Assignment assignment, + TopologyAssignEvent event) { + String topologyId = event.getTopologyId(); + String topologyName = event.getTopologyName(); + try { + + StormClusterState zkClusterState = + nimbusData.getStormClusterState(); + // one little problem, get tasks twice when assign one topology + Map tasks = + Cluster.get_all_task_component(zkClusterState, topologyId, null); + + Map> componentTasks = + JStormUtils.reverse_map(tasks); + + for (Entry> entry : componentTasks.entrySet()) { + List keys = entry.getValue(); + + Collections.sort(keys); + + } + + AssignmentBak assignmentBak = + new AssignmentBak(componentTasks, assignment); + zkClusterState.backup_assignment(topologyName, assignmentBak); + + } catch (Exception e) { + LOG.warn("Failed to backup " + topologyId + " assignment " + + assignment, e); + } + } + + private void getAliveSupervsByHb( + Map supervisorInfos, Map conf) { + int currentTime = TimeUtils.current_time_secs(); + int hbTimeout = + JStormUtils.parseInt( + conf.get(Config.NIMBUS_SUPERVISOR_TIMEOUT_SECS), + (JStormUtils.MIN_1 * 3)); + Set supervisorTobeRemoved = new HashSet(); + + for (Entry entry : supervisorInfos.entrySet()) { + SupervisorInfo supInfo = entry.getValue(); + int lastReportTime = supInfo.getTimeSecs(); + if ((currentTime - lastReportTime) > hbTimeout) { + LOG.warn("Supervisor-" + supInfo.getHostName() + + " is dead. lastReportTime=" + lastReportTime); + supervisorTobeRemoved.add(entry.getKey()); + } + } + + for (String name : supervisorTobeRemoved) { + supervisorInfos.remove(name); + } + } + + /** + * @param args + */ + public static void main(String[] args) { + // TODO Auto-generated method stub + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/TopologyAssignEvent.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/TopologyAssignEvent.java new file mode 100755 index 000000000..8725918b4 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/TopologyAssignEvent.java @@ -0,0 +1,117 @@ +/** + * 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.daemon.nimbus; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import com.alibaba.jstorm.cluster.StormStatus; + +public class TopologyAssignEvent { + + // unit is minutes + private static final int DEFAULT_WAIT_TIME = 2; + private String topologyId; + private String topologyName; // if this field has been set, it is create + private String group; + // topology + private boolean isScratch; + private boolean isReassign; + private StormStatus oldStatus; // if this field has been set, it is + // rebalance + private CountDownLatch latch = new CountDownLatch(1); + private boolean isSuccess = false; + private String errorMsg; + + public String getTopologyId() { + return topologyId; + } + + public void setTopologyId(String topologyId) { + this.topologyId = topologyId; + } + + public boolean isScratch() { + return isScratch; + } + + public void setScratch(boolean isScratch) { + this.isScratch = isScratch; + } + + public boolean isReassign() { + return isReassign; + } + + public void setReassign(boolean isReassign) { + this.isReassign = isReassign; + } + + public StormStatus getOldStatus() { + return oldStatus; + } + + public void setOldStatus(StormStatus oldStatus) { + this.oldStatus = oldStatus; + } + + public String getTopologyName() { + return topologyName; + } + + public void setTopologyName(String topologyName) { + this.topologyName = topologyName; + } + + public boolean waitFinish() { + try { + latch.await(DEFAULT_WAIT_TIME, TimeUnit.MINUTES); + } catch (InterruptedException e) { + + } + return isSuccess; + } + + public boolean isFinish() { + return latch.getCount() == 0; + } + + public void done() { + isSuccess = true; + latch.countDown(); + } + + public void fail(String errorMsg) { + isSuccess = false; + this.errorMsg = errorMsg; + latch.countDown(); + } + + public String getErrorMsg() { + return errorMsg; + } + + public String getGroup() { + return group; + } + + public void setGroup(String group) { + this.group = group; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/TopologyMetricsRunnable.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/TopologyMetricsRunnable.java new file mode 100755 index 000000000..b8eeb20c4 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/TopologyMetricsRunnable.java @@ -0,0 +1,727 @@ +/** + * 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.daemon.nimbus; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.BlockingDeque; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.cache.JStormCache; +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.Cluster; +import com.alibaba.jstorm.cluster.Common; +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.common.metric.Histogram; +import com.alibaba.jstorm.metric.AlimonitorClient; +import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.metric.MetricSendClient; +import com.alibaba.jstorm.metric.MetricThrift; +import com.alibaba.jstorm.metric.SimpleJStormMetric; +import com.alibaba.jstorm.schedule.Assignment; +import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; +import com.alibaba.jstorm.utils.TimeCacheMap; +import com.codahale.metrics.Gauge; + +import backtype.storm.generated.MetricInfo; +import backtype.storm.generated.MetricWindow; +import backtype.storm.generated.TopologyMetric; +import backtype.storm.generated.WorkerUploadMetrics; + +public class TopologyMetricsRunnable extends RunnableCallback { + private static final Logger LOG = LoggerFactory.getLogger(TopologyMetricsRunnable.class); + private static final String DEAD_SUPERVISOR_HEAD = "DeadSupervisor-"; + + public static interface Event { + + } + + public static class Update implements Event { + public WorkerUploadMetrics workerMetrics; + } + + public static class Remove implements Event { + public String topologyId; + } + + public static class Upload implements Event { + public long timeStamp; + } + + public static final String CACHE_NAMESPACE_METRIC = "cache_namespace_metric"; + public static final String CACHE_NAMESPACE_NETTY = "cache_namespace_netty"; + protected NimbusCache nimbusCache; + protected JStormCache dbCache; + + /** + * cache all worker metrics will waste a little memory + * + */ + protected Map> topologyWorkers; + protected TimeCacheMap removing; + + protected BlockingDeque queue; + protected StormClusterState stormClusterState; + + protected MetricSendClient metricSendClient; + protected TopologyMetric emptyTopologyMetric = mkTopologyMetric(); + protected TreeMap emptyNettyMetric = new TreeMap(); + protected AtomicBoolean isShutdown; + protected boolean localMode; + protected TopologyNettyMgr topologyNettyMgr; + + protected Histogram updateHistogram; + protected AtomicBoolean isUploading = new AtomicBoolean(false); + protected Histogram uploadHistogram; + + public TopologyMetricsRunnable(NimbusData nimbusData) { + + this.nimbusCache = nimbusData.getNimbusCache(); + this.dbCache = nimbusCache.getDbCache(); + this.topologyWorkers = new ConcurrentHashMap>(); + this.removing = new TimeCacheMap(600); + this.queue = new LinkedBlockingDeque(); + this.stormClusterState = nimbusData.getStormClusterState(); + this.isShutdown = nimbusData.getIsShutdown(); + this.topologyNettyMgr = nimbusData.getTopologyNettyMgr(); + + if (ConfigExtension.isAlimonitorMetricsPost(nimbusData.getConf())) { + metricSendClient = new AlimonitorClient(AlimonitorClient.DEFAUT_ADDR, AlimonitorClient.DEFAULT_PORT, true); + } else { + metricSendClient = new MetricSendClient(); + } + localMode = StormConfig.local_mode(nimbusData.getConf()); + + updateHistogram = SimpleJStormMetric.registerHistorgram("TopologyMetricsRunnable_Update"); + uploadHistogram = SimpleJStormMetric.registerHistorgram("TopologyMetricsRunnable_Upload"); + + SimpleJStormMetric.registerWorkerGauge(new Gauge() { + + @Override + public Double getValue() { + // TODO Auto-generated method stub + return (double) queue.size(); + } + }, "TopologyMetricsRunnable_Queue"); + } + + public void pushEvent(TopologyMetricsRunnable.Event cmd) { + queue.offer(cmd); + } + + public TopologyMetric mkTopologyMetric() { + TopologyMetric emptyTopologyMetric = new TopologyMetric(); + + MetricInfo topologyMetricInfo = MetricThrift.mkMetricInfo(); + emptyTopologyMetric.set_topologyMetric(topologyMetricInfo); + + emptyTopologyMetric.set_componentMetric(new HashMap()); + emptyTopologyMetric.set_workerMetric(new HashMap()); + emptyTopologyMetric.set_taskMetric(new HashMap()); + return emptyTopologyMetric; + } + + @Override + public void run() { + try { + TopologyMetricsRunnable.Event event = queue.take(); + + if (event instanceof Remove) { + + handleRemoveEvent((Remove) event); + return; + } else if (event instanceof Update) { + handleUpdateEvent((Update) event); + return; + } else if (event instanceof Upload) { + handleUploadEvent((Upload) event); + return; + } else { + LOG.error("Unknow event type"); + return; + } + + } catch (Exception e) { + if (isShutdown.get() == false) { + LOG.error(e.getMessage(), e); + } + } + } + + public void handleRemoveEvent(Remove event) { + String topologyId = event.topologyId; + TopologyMetric topologyMetric = (TopologyMetric) dbCache.get(getTopologyKey(topologyId)); + if (topologyMetric == null) { + LOG.warn("No TopologyMetric of " + topologyId); + return; + } + + removing.put(topologyId, System.currentTimeMillis()); + dbCache.remove(getTopologyKey(topologyId)); + dbCache.remove(getNettyTopologyKey(topologyId)); + topologyNettyMgr.rmTopology(topologyId); + LOG.info("Successfully remove TopologyMetric of " + topologyId); + return; + + } + + public void cleanDeadSupervisorWorker(TopologyMetric metric) { + List removeList = new ArrayList(); + + Map workerMetric = metric.get_workerMetric(); + if (workerMetric == null) { + return; + } + for (String hostPort : workerMetric.keySet()) { + if (hostPort.startsWith(DEAD_SUPERVISOR_HEAD)) { + removeList.add(hostPort); + } + } + + for (String removed : removeList) { + workerMetric.remove(removed); + } + } + + public void cleanTopology() { + Map removingMap = removing.buildMap(); + + Map assignMap = null; + try { + assignMap = Cluster.get_all_assignment(stormClusterState, null); + } catch (Exception e1) { + // TODO Auto-generated catch block + LOG.info("Failed to get Assignments"); + } + + for (String topologyId : topologyWorkers.keySet()) { + if (assignMap.containsKey(topologyId) == false) { + removingMap.put(topologyId, System.currentTimeMillis()); + } + } + + for (String topologyId : removingMap.keySet()) { + dbCache.remove(getTopologyKey(topologyId)); + + Set workers = topologyWorkers.get(topologyId); + if (workers != null) { + for (String workerSlot : workers) { + dbCache.remove(getWorkerKey(topologyId, workerSlot)); + } + topologyWorkers.remove(topologyId); + } + + } + + for (Entry> entry : topologyWorkers.entrySet()) { + String topologyId = entry.getKey(); + Set metricWorkers = entry.getValue(); + + Set workerSlots = new HashSet(); + + Assignment assignment = assignMap.get(topologyId); + if (assignment == null) { + LOG.error("Assignment disappear of " + topologyId); + continue; + } + + for (ResourceWorkerSlot worker : assignment.getWorkers()) { + String slot = getWorkerSlotName(worker.getNodeId(), worker.getPort()); + workerSlots.add(slot); + } + + Set removes = new HashSet(); + for (String slot : metricWorkers) { + if (workerSlots.contains(slot) == false) { + LOG.info("Remove worker metrics of {}:{}", topologyId, slot); + removes.add(slot); + } + } + + for (String slot : removes) { + metricWorkers.remove(slot); + dbCache.remove(getWorkerKey(topologyId, slot)); + } + } + } + + /** + * Upload metric to ZK + * + * @param event + */ + public void handleUploadEvent(Upload event) { + if (isUploading.getAndSet(true) == true) { + LOG.info("Nimbus is alread uploading"); + return ; + } + + long start = System.currentTimeMillis(); + + cleanTopology(); + + render(); + + isUploading.set(false); + + long end = System.currentTimeMillis(); + uploadHistogram.update(end - start); + + + } + + public String getWorkerHostname(WorkerUploadMetrics workerMetrics) { + + String hostname = null; + String supervisorId = workerMetrics.get_supervisor_id(); + try { + hostname = Cluster.get_supervisor_hostname(stormClusterState, supervisorId); + } catch (Exception e) { + // TODO Auto-generated catch block + LOG.warn("Failed to get hostname of " + supervisorId); + } + if (hostname == null) { + hostname = DEAD_SUPERVISOR_HEAD + supervisorId; + } + + return hostname; + } + + public void avgMetricWindow(MetricWindow metric, int parallel) { + if (parallel == 0) { + return; + } + Map map = metric.get_metricWindow(); + Map newMap = new HashMap(); + if (map != null) { + for (Entry entry : map.entrySet()) { + newMap.put(entry.getKey(), entry.getValue() / parallel); + } + } + + metric.set_metricWindow(newMap); + } + + public MetricInfo mergeMetricInfo(MetricInfo from, MetricInfo to, Set tags) { + if (to == null) { + to = MetricThrift.mkMetricInfo(); + } + + if (from.get_baseMetric() == null) { + LOG.warn("No base Metric "); + return to; + } + + for (String tag : tags) { + + MetricWindow fromMetric = from.get_baseMetric().get(tag); + Map toMetricMap = to.get_baseMetric(); + if (toMetricMap == null) { + toMetricMap = new HashMap(); + to.set_baseMetric(toMetricMap); + } + + MetricWindow toMetric = toMetricMap.get(tag); + + toMetric = MetricThrift.mergeMetricWindow(fromMetric, toMetric); + + toMetricMap.put(tag, toMetric); + + } + + return to; + } + + public Map> mergeTaskStreams( + Map> componentStreams, + Map> taskStreams, + Map> componentStreamParallel) { + + if (taskStreams == null || taskStreams.size() == 0) { + return componentStreams; + } + + if (componentStreams == null) { + componentStreams = new HashMap>(); + } + + for (Entry> entry : taskStreams.entrySet()) { + String metricName = entry.getKey(); + Map streamMetricWindows = entry.getValue(); + + if (streamMetricWindows == null) { + continue; + } + + Map streamCounters = componentStreamParallel.get(metricName); + if (streamCounters == null) { + streamCounters = new HashMap(); + componentStreamParallel.put(metricName, streamCounters); + } + + Map componentStreamMetricWindows = componentStreams.get(metricName); + if (componentStreamMetricWindows == null) { + componentStreamMetricWindows = new HashMap(); + componentStreams.put(metricName, componentStreamMetricWindows); + } + + for (Entry streamEntry : streamMetricWindows.entrySet()) { + String streamName = streamEntry.getKey(); + MetricWindow taskMetricWindow = streamEntry.getValue(); + + MetricWindow componentMetricWindow = componentStreamMetricWindows.get(streamName); + + componentMetricWindow = MetricThrift.mergeMetricWindow(taskMetricWindow, componentMetricWindow); + + componentStreamMetricWindows.put(streamName, componentMetricWindow); + + AtomicInteger counter = streamCounters.get(streamName); + if (counter == null) { + counter = new AtomicInteger(0); + streamCounters.put(streamName, counter); + } + counter.incrementAndGet(); + } + } + + return componentStreams; + } + + public void avgStreams(Map> tagStreamsMetrics, Map> counters, String tag) { + if (tagStreamsMetrics == null) { + return; + } + + Map streamMetrics = tagStreamsMetrics.get(tag); + if (streamMetrics == null) { + return; + } + + for (Entry entry : streamMetrics.entrySet()) { + String streamName = entry.getKey(); + MetricWindow metric = entry.getValue(); + + AtomicInteger counter = counters.get(tag).get(streamName); + if (counter == null) { + continue; + + } + + avgMetricWindow(metric, counter.get()); + } + } + + public void mergeTasks(TopologyMetric topologyMetric, String topologyId) { + Map taskMetrics = topologyMetric.get_taskMetric(); + + Map taskToComponent = null; + try { + taskToComponent = Cluster.get_all_task_component(stormClusterState, topologyId, null); + } catch (Exception e) { + // TODO Auto-generated catch block + LOG.error("Failed to get taskToComponent"); + return ; + } + if (taskToComponent == null) { + LOG.error("Failed to get taskToComponent"); + return ; + } + + Map componentMetrics = topologyMetric.get_componentMetric(); + if (componentMetrics == null) { + componentMetrics = new HashMap(); + topologyMetric.set_componentMetric(componentMetrics); + } + + Map componentTaskParallel = new HashMap(); + Map> componentStreamParallel = new HashMap>(); + + for (Entry entry : taskMetrics.entrySet()) { + Integer taskId = entry.getKey(); + MetricInfo taskMetric = entry.getValue(); + + String component = taskToComponent.get(taskId); + if (component == null) { + LOG.error("Failed to get component of task " + taskId); + continue; + } + + MetricInfo componentMetric = componentMetrics.get(component); + + componentMetric = mergeMetricInfo(taskMetric, componentMetric, MetricDef.MERGE_SUM_TAG); + componentMetric = mergeMetricInfo(taskMetric, componentMetric, MetricDef.MERGE_AVG_TAG); + + Map> input = mergeTaskStreams(componentMetric.get_inputMetric(), taskMetric.get_inputMetric(), componentStreamParallel); + componentMetric.set_inputMetric(input); + + Map> output = mergeTaskStreams(componentMetric.get_outputMetric(), taskMetric.get_outputMetric(), componentStreamParallel); + componentMetric.set_outputMetric(output); + + componentMetrics.put(component, componentMetric); + + AtomicInteger counter = componentTaskParallel.get(component); + if (counter == null) { + counter = new AtomicInteger(0); + componentTaskParallel.put(component, counter); + } + + counter.incrementAndGet(); + } + + for (Entry entry : componentMetrics.entrySet()) { + String componentName = entry.getKey(); + MetricInfo metricInfo = entry.getValue(); + + AtomicInteger counter = componentTaskParallel.get(componentName); + + for (String tag : MetricDef.MERGE_AVG_TAG) { + MetricWindow metricWindow = metricInfo.get_baseMetric().get(tag); + + avgMetricWindow(metricWindow, counter.get()); + + avgStreams(metricInfo.get_inputMetric(), componentStreamParallel, tag); + avgStreams(metricInfo.get_outputMetric(), componentStreamParallel, tag); + } + } + } + + public void mergeComponent(TopologyMetric topologyMetric) { + MetricInfo topologyMetricInfo = MetricThrift.mkMetricInfo(); + topologyMetric.set_topologyMetric(topologyMetricInfo); + Map componentMetrics = topologyMetric.get_componentMetric(); + if (componentMetrics == null) { + return; + } + + for (MetricInfo componentMetric : componentMetrics.values()) { + topologyMetricInfo = mergeMetricInfo(componentMetric, topologyMetricInfo, MetricDef.MERGE_SUM_TAG); + } + + topologyMetric.set_topologyMetric(topologyMetricInfo); + } + + public void mergeTopology(TopologyMetric topologyMetric, WorkerUploadMetrics workerMetrics) { + String topologyId = workerMetrics.get_topology_id(); + + Map taskMetrics = topologyMetric.get_taskMetric(); + if (taskMetrics == null) { + taskMetrics = new HashMap(); + topologyMetric.set_taskMetric(taskMetrics); + } + taskMetrics.putAll(workerMetrics.get_taskMetric()); + + String hostname = getWorkerHostname(workerMetrics); + topologyMetric.put_to_workerMetric(getWorkerSlotName(hostname, workerMetrics.get_port()), workerMetrics.get_workerMetric()); + + } + + public void mergeNetty(WorkerUploadMetrics workerMetric, String topologyId, Set connections) { + + if (topologyNettyMgr.getTopology(topologyId) == false) { + return ; + } + Map connectionMetrics = workerMetric.get_nettyMetric().get_connections(); + for (Entry entry : connectionMetrics.entrySet()) { + String connectionName = entry.getKey(); + MetricInfo metric = entry.getValue(); + + MetricInfo cacheMetric = (MetricInfo)dbCache.get(getNettyConnectionKey(topologyId, connectionName)); + cacheMetric = MetricThrift.mergeMetricInfo(metric, cacheMetric); + + connections.add(connectionName); + + dbCache.put(getNettyConnectionKey(topologyId, connectionName), cacheMetric); + } + } + + public void mergeNetty(String topologyId, Set connections) { + if (topologyNettyMgr.getTopology(topologyId) == false) { + LOG.info("Skip merge netty detail metrics"); + return ; + } + // @@@ + // this function will cost much memory when worker number is more than 200 + Map metricMap = new TreeMap(); + + for (String connection : connections) { + MetricInfo cacheMetric = (MetricInfo)dbCache.get(getNettyConnectionKey(topologyId, connection)); + if (cacheMetric == null) { + LOG.warn("Failed to get cacheMetric of {}:{}", topologyId, connection ); + continue; + } + + metricMap.put(connection, cacheMetric); + dbCache.remove(getNettyConnectionKey(topologyId, connection)); + } + + dbCache.put(getNettyTopologyKey(topologyId), metricMap); + // accelerate free memory + metricMap.clear(); + } + + public void render() { + for (Entry> entry : topologyWorkers.entrySet()) { + String topologyId = entry.getKey(); + Set workers = entry.getValue(); + Set connections = new TreeSet(); + + TopologyMetric topologyMetric = new TopologyMetric(); + + boolean isExistWorker = false; + for (String workerId : workers) { + WorkerUploadMetrics workerMetric = (WorkerUploadMetrics) dbCache.get(getWorkerKey(topologyId, workerId)); + if (workerMetric == null) { + LOG.warn("Failed to get WorkerUploadMetrics of " + getWorkerKey(topologyId, workerId)); + continue; + } + isExistWorker = true; + mergeTopology(topologyMetric, workerMetric); + + mergeNetty(workerMetric, topologyId, connections); + } + if (isExistWorker == false) { + LOG.info("No worker metrics of {}", topologyId); + continue; + } + + mergeTasks(topologyMetric, topologyId); + + mergeComponent(topologyMetric); + + + dbCache.put(getTopologyKey(topologyId), topologyMetric); + + mergeNetty(topologyId, connections); + + LOG.info("Successfully render topologyId of " + topologyId); + + uploadToAlimonitor(topologyMetric, topologyId); + + cleanDeadSupervisorWorker(topologyMetric); + + + try { + + //LOG.info(topologyId + " metrics is :\n" + Utils.toPrettyJsonString(topologyMetric)); + LOG.info(topologyId + " finish metric"); + stormClusterState.set_topology_metric(topologyId, topologyMetric); + LOG.info("Successfully uploaded toplogy metrics: " + topologyId); + } catch (Exception e) { + // TODO Auto-generated catch block + LOG.info("Failed to upload toplogy metrics: " + topologyId, e); + continue; + } + + } + } + + public void handleUpdateEvent(Update event) { + long start = System.currentTimeMillis(); + + WorkerUploadMetrics workerMetrics = event.workerMetrics; + + String topologyId = workerMetrics.get_topology_id(); + if (removing.containsKey(topologyId) == true) { + LOG.info("Topology " + topologyId + " has been removed, skip update"); + return; + } + + Set workers = topologyWorkers.get(topologyId); + if (workers == null) { + workers = new HashSet(); + topologyWorkers.put(topologyId, workers); + } + + String workerSlot = getWorkerSlotName(workerMetrics.get_supervisor_id(), workerMetrics.get_port()); + + workers.add(workerSlot); + dbCache.put(getWorkerKey(topologyId, workerSlot), workerMetrics); + + long end = System.currentTimeMillis(); + + updateHistogram.update((end - start)); + } + + public void uploadToAlimonitor(TopologyMetric topologyMetric, String topologyId) { + // @@@ TODO + } + + + public TopologyMetric getTopologyMetric(String topologyId) { + long start = System.nanoTime(); + try { + TopologyMetric ret = (TopologyMetric) dbCache.get(getTopologyKey(topologyId)); + if (ret == null) { + return emptyTopologyMetric; + } else { + return ret; + } + }finally { + long end = System.nanoTime(); + + SimpleJStormMetric.updateHistorgram("getTopologyMetric", (end - start)/1000000.0d); + } + } + + public SortedMap getNettyMetric(String topologyId) { + TreeMap ret = (TreeMap)dbCache.get(getNettyTopologyKey(topologyId)); + if (ret == null) { + return emptyNettyMetric; + }else { + return ret; + } + } + + public static String getWorkerSlotName(String hostname, Integer port) { + return hostname + ":" + port; + } + + public static String getWorkerKey(String topologyId, String workerSlot) { + return CACHE_NAMESPACE_METRIC + "@" + topologyId + "@" + workerSlot; + } + + public static String getTopologyKey(String topologyId) { + return CACHE_NAMESPACE_METRIC + "@" + topologyId; + } + + public static String getNettyConnectionKey(String topologyId, String connection) { + return CACHE_NAMESPACE_NETTY + "@" + topologyId + "@" + connection; + } + + public static String getNettyTopologyKey(String topologyId) { + return CACHE_NAMESPACE_NETTY + "@" + topologyId; + } + + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/TopologyNettyMgr.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/TopologyNettyMgr.java new file mode 100644 index 000000000..7eaccab5d --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/nimbus/TopologyNettyMgr.java @@ -0,0 +1,105 @@ +package com.alibaba.jstorm.daemon.nimbus; + +import java.util.Map; + +import org.jboss.netty.util.internal.ConcurrentHashMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.Common; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.utils.JStormUtils; + +import backtype.storm.Config; +import backtype.storm.generated.InvalidTopologyException; + +public class TopologyNettyMgr { + private static Logger LOG = LoggerFactory.getLogger(TopologyNettyMgr.class); + private boolean defaultEnable = true; + private Map nimbusConf; + private ConcurrentHashMap setting = new ConcurrentHashMap(); + private static final int WORKER_DISABLE_THREADHOLD = 200; + + public TopologyNettyMgr(Map conf) { + nimbusConf = conf; + + Boolean isEnable = ConfigExtension.isEnableTopologyNettyMetrics(conf); + if (isEnable != null) { + defaultEnable = isEnable; + } + + LOG.info("Default netty metrics setting is " + defaultEnable); + } + + protected boolean getTopology(Map conf) { + Boolean isEnable = ConfigExtension.isEnableTopologyNettyMetrics(conf); + if (isEnable != null) { + return isEnable; + } + + int workerNum = JStormUtils.parseInt(conf.get(Config.TOPOLOGY_WORKERS), 1); + if (workerNum <= WORKER_DISABLE_THREADHOLD) { + isEnable = Boolean.TRUE; + }else { + isEnable = Boolean.FALSE; + } + + return isEnable; + } + + public boolean getTopology(String topologyId) { + try { + String topologyName = Common.topologyIdToName(topologyId); + + Boolean isEnable = setting.get(topologyName); + if (isEnable != null) { + return isEnable; + } + + Map topologyConf = + StormConfig.read_nimbus_topology_conf(nimbusConf, topologyId); + + isEnable = getTopology(topologyConf); + setting.put(topologyName, isEnable); + LOG.info("{} netty metrics setting is {}", topologyName, isEnable); + return isEnable; + + }catch(Exception e) { + LOG.info("Failed to get {} netty metrics setting ", topologyId); + return defaultEnable; + } + + + } + + public void setTopology(Map conf) { + String topologyName = (String)conf.get(Config.TOPOLOGY_NAME); + if (topologyName == null) { + LOG.info("No topologyName setting"); + return ; + } + + boolean isEnable = getTopology(conf); + + setting.put(topologyName, isEnable); + + LOG.info("{} netty metrics setting is {}", topologyName, isEnable); + return ; + + } + + public void rmTopology(String topologyId) { + String topologyName; + try { + topologyName = Common.topologyIdToName(topologyId); + setting.remove(topologyName); + LOG.info("Remove {} netty metrics setting ", topologyName); + } catch (InvalidTopologyException e) { + // TODO Auto-generated catch block + + } + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/CgroupManager.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/CgroupManager.java new file mode 100644 index 000000000..8d2ba2427 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/CgroupManager.java @@ -0,0 +1,160 @@ +/** + * 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.daemon.supervisor; + +import java.io.File; +import java.io.IOException; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.container.CgroupCenter; +import com.alibaba.jstorm.container.Hierarchy; +import com.alibaba.jstorm.container.SubSystemType; +import com.alibaba.jstorm.utils.SystemOperation; +import com.alibaba.jstorm.container.cgroup.CgroupCommon; +import com.alibaba.jstorm.container.cgroup.core.CgroupCore; +import com.alibaba.jstorm.container.cgroup.core.CpuCore; +import com.alibaba.jstorm.utils.JStormUtils; + +public class CgroupManager { + + public static final Logger LOG = LoggerFactory + .getLogger(CgroupManager.class); + + public static final String JSTORM_HIERARCHY_NAME = "jstorm_cpu"; + + public static final int ONE_CPU_SLOT = 1024; + + private CgroupCenter center; + + private Hierarchy h; + + private CgroupCommon rootCgroup; + + private static final String JSTORM_CPU_HIERARCHY_DIR = "/cgroup/cpu"; + private static String rootDir; + + public CgroupManager(Map conf) { + LOG.info("running on cgroup mode"); + + // Cgconfig service is used to create the corresponding cpu hierarchy + // "/cgroup/cpu" + rootDir = ConfigExtension.getCgroupRootDir(conf); + if (rootDir == null) + throw new RuntimeException( + "Check configuration file. The supervisor.cgroup.rootdir is missing."); + + File file = new File(JSTORM_CPU_HIERARCHY_DIR + "/" + rootDir); + if (!file.exists()) { + LOG.error(JSTORM_CPU_HIERARCHY_DIR + "/" + rootDir + + " is not existing."); + throw new RuntimeException( + "Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file."); + } + center = CgroupCenter.getInstance(); + if (center == null) + throw new RuntimeException( + "Cgroup error, please check /proc/cgroups"); + this.prepareSubSystem(); + } + + private int validateCpuUpperLimitValue(int value) { + /* + * Valid value is -1 or 1~10 -1 means no control + */ + if (value > 10) + value = 10; + else if (value < 1 && value != -1) + value = 1; + + return value; + } + + private void setCpuUsageUpperLimit(CpuCore cpuCore, int cpuCoreUpperLimit) + throws IOException { + /* + * User cfs_period & cfs_quota to control the upper limit use of cpu + * core e.g. If making a process to fully use two cpu cores, set + * cfs_period_us to 100000 and set cfs_quota_us to 200000 The highest + * value of "cpu core upper limit" is 10 + */ + cpuCoreUpperLimit = validateCpuUpperLimitValue(cpuCoreUpperLimit); + + if (cpuCoreUpperLimit == -1) { + // No control of cpu usage + cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit); + } else { + cpuCore.setCpuCfsPeriodUs(100000); + cpuCore.setCpuCfsQuotaUs(cpuCoreUpperLimit * 100000); + } + } + + public String startNewWorker(Map conf, int cpuNum, String workerId) + throws SecurityException, IOException { + CgroupCommon workerGroup = + new CgroupCommon(workerId, h, this.rootCgroup); + this.center.create(workerGroup); + CgroupCore cpu = workerGroup.getCores().get(SubSystemType.cpu); + CpuCore cpuCore = (CpuCore) cpu; + cpuCore.setCpuShares(cpuNum * ONE_CPU_SLOT); + setCpuUsageUpperLimit(cpuCore, + ConfigExtension.getWorkerCpuCoreUpperLimit(conf)); + + StringBuilder sb = new StringBuilder(); + sb.append("cgexec -g cpu:").append(workerGroup.getName()).append(" "); + return sb.toString(); + } + + public void shutDownWorker(String workerId, boolean isKilled) { + CgroupCommon workerGroup = + new CgroupCommon(workerId, h, this.rootCgroup); + try { + if (isKilled == false) { + for (Integer pid : workerGroup.getTasks()) { + JStormUtils.kill(pid); + } + JStormUtils.sleepMs(1500); + } + center.delete(workerGroup); + } catch (Exception e) { + LOG.info("No task of " + workerId); + } + + } + + public void close() throws IOException { + this.center.delete(this.rootCgroup); + } + + private void prepareSubSystem() { + h = center.busy(SubSystemType.cpu); + if (h == null) { + Set types = new HashSet(); + types.add(SubSystemType.cpu); + h = + new Hierarchy(JSTORM_HIERARCHY_NAME, types, + JSTORM_CPU_HIERARCHY_DIR); + } + rootCgroup = new CgroupCommon(rootDir, h, h.getRootCgroups()); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/Heartbeat.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/Heartbeat.java new file mode 100644 index 000000000..e55aabe1a --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/Heartbeat.java @@ -0,0 +1,155 @@ +/** + * 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.daemon.supervisor; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; + +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.utils.JStormServerUtils; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.TimeUtils; + +/** + * supervisor Heartbeat, just write SupervisorInfo to ZK + */ +class Heartbeat extends RunnableCallback { + + private static Logger LOG = LoggerFactory.getLogger(Heartbeat.class); + + private static final int CPU_THREADHOLD = 4; + private static final long MEM_THREADHOLD = 8 * JStormUtils.SIZE_1_G; + + private Map conf; + + private StormClusterState stormClusterState; + + private String supervisorId; + + private String myHostName; + + private final int startTime; + + private final int frequence; + + private SupervisorInfo supervisorInfo; + + private AtomicBoolean hbUpdateTrigger; + + /** + * @param conf + * @param stormClusterState + * @param supervisorId + * @param myHostName + */ + @SuppressWarnings({ "rawtypes", "unchecked" }) + public Heartbeat(Map conf, StormClusterState stormClusterState, + String supervisorId) { + + String myHostName = JStormServerUtils.getHostName(conf); + + this.stormClusterState = stormClusterState; + this.supervisorId = supervisorId; + this.conf = conf; + this.myHostName = myHostName; + this.startTime = TimeUtils.current_time_secs(); + this.frequence = + JStormUtils.parseInt(conf + .get(Config.SUPERVISOR_HEARTBEAT_FREQUENCY_SECS)); + this.hbUpdateTrigger = new AtomicBoolean(true); + + initSupervisorInfo(conf); + + LOG.info("Successfully init supervisor heartbeat thread, " + + supervisorInfo); + } + + private void initSupervisorInfo(Map conf) { + List portList = JStormUtils.getSupervisorPortList(conf); + + if (!StormConfig.local_mode(conf)) { + try { + + boolean isLocaliP = false; + isLocaliP = myHostName.equals("127.0.0.1"); + if(isLocaliP){ + throw new Exception("the hostname which supervisor get is localhost"); + } + }catch(Exception e1){ + LOG.error("get supervisor host error!", e1); + throw new RuntimeException(e1); + } + Set ports = JStormUtils.listToSet(portList); + supervisorInfo = + new SupervisorInfo(myHostName, supervisorId, ports); + } else { + Set ports = JStormUtils.listToSet(portList.subList(0, 1)); + supervisorInfo = + new SupervisorInfo(myHostName, supervisorId, ports); + } + } + + @SuppressWarnings("unchecked") + public void update() { + supervisorInfo.setTimeSecs(TimeUtils.current_time_secs()); + supervisorInfo + .setUptimeSecs((int) (TimeUtils.current_time_secs() - startTime)); + + try { + stormClusterState + .supervisor_heartbeat(supervisorId, supervisorInfo); + } catch (Exception e) { + LOG.error("Failed to update SupervisorInfo to ZK"); + + } + } + + @Override + public Object getResult() { + return frequence; + } + + @Override + public void run() { + boolean updateHb = hbUpdateTrigger.getAndSet(false); + if (updateHb) { + update(); + } + } + + public int getStartTime() { + return startTime; + } + + public SupervisorInfo getSupervisorInfo() { + return supervisorInfo; + } + + public void updateHbTrigger(boolean update) { + hbUpdateTrigger.set(update); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/Httpserver.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/Httpserver.java new file mode 100755 index 000000000..fad1346f5 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/Httpserver.java @@ -0,0 +1,482 @@ +/** + * 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.daemon.supervisor; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.OutputStream; +import java.io.RandomAccessFile; +import java.net.BindException; +import java.net.HttpURLConnection; +import java.net.InetSocketAddress; +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; +import java.util.Map; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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; +import com.alibaba.jstorm.utils.FileAttribute; +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; +import com.sun.net.httpserver.HttpExchange; +import com.sun.net.httpserver.HttpHandler; +import com.sun.net.httpserver.HttpServer; + +public class Httpserver implements Shutdownable { + + private static Logger LOG = LoggerFactory.getLogger(Httpserver.class); + + private HttpServer hs; + private int port; + private Map conf; + + public Httpserver(int port, Map conf) { + this.port = port; + this.conf = conf; + + } + + static class LogHandler implements HttpHandler { + + private String logDir; + private String stormHome; + private ArrayList accessDirs = new ArrayList(); + Map conf; + private final int pageSize; + + public LogHandler(Map conf) { + + this.pageSize = ConfigExtension.getLogPageSize(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); // +++ + + } + + public void handlFailure(HttpExchange t, String errorMsg) + throws IOException { + LOG.error(errorMsg); + + byte[] data = errorMsg.getBytes(); + t.sendResponseHeaders(HttpURLConnection.HTTP_BAD_REQUEST, + data.length); + OutputStream os = t.getResponseBody(); + os.write(data); + os.close(); + } + + 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); + if (StringUtils.isBlank(cmd) == true) { + handlFailure(t, "Bad Request, Not set command type"); + return; + } + + if (HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD_SHOW.equals(cmd)) { + handleShowLog(t, paramMap); + return; + } else if (HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD_LIST + .equals(cmd)) { + handleListDir(t, paramMap); + return; + } 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(); + + for (String param : StringUtils.split(uriRawQuery, "&")) { + String[] pair = StringUtils.split(param, "="); + if (pair.length == 2) { + paramMap.put(pair[0], pair[1]); + } + } + + return paramMap; + } + + private void handleShowLog(HttpExchange t, Map paramMap) + throws IOException { + Pair logPair = queryLog(t, paramMap); + if (logPair == null) { + return; + } + + String size = + String.format( + HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_SIZE_FORMAT, + logPair.getFirst()); + byte[] sizeByts = size.getBytes(); + + byte[] logData = logPair.getSecond(); + + t.sendResponseHeaders(HttpURLConnection.HTTP_OK, sizeByts.length + + logData.length); + OutputStream os = t.getResponseBody(); + os.write(sizeByts); + os.write(logData); + os.close(); + } + + private Pair queryLog(HttpExchange t, + Map paramMap) throws IOException { + + String fileParam = + paramMap.get(HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_LOGFILE); + if (StringUtils.isBlank(fileParam)) { + handlFailure(t, "Bad Request, Params Error, no log file name."); + return null; + } + + String logFile = Joiner.on(File.separator).join(logDir, fileParam); + accessCheck(logFile); + FileChannel fc = null; + MappedByteBuffer fout = null; + long fileSize = 0; + byte[] ret = "Failed to get data".getBytes(); + try { + fc = new RandomAccessFile(logFile, "r").getChannel(); + + fileSize = fc.size(); + + long position = fileSize - pageSize; + try { + String posStr = + paramMap.get(HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_POS); + if (StringUtils.isBlank(posStr) == false) { + long pos = Long.valueOf(posStr); + + position = pos; + } + } catch (Exception e) { + LOG.warn("Invalide position " + position); + } + if (position < 0) { + position = 0L; + } + + long size = Math.min(fileSize - position, pageSize); + + LOG.info("logview " + logFile + ", position=" + position + + ", size=" + size); + fout = fc.map(FileChannel.MapMode.READ_ONLY, position, size); + + ret = new byte[(int) size]; + fout.get(ret); + String str = + new String(ret, + ConfigExtension.getLogViewEncoding(conf)); + return new Pair(fileSize, str.getBytes()); + + } catch (FileNotFoundException e) { + LOG.warn(e.getMessage(), e); + handlFailure(t, "Bad Request, Failed to find " + fileParam); + return null; + + } catch (IOException e) { + LOG.warn(e.getMessage(), e); + handlFailure(t, "Bad Request, Failed to open " + fileParam); + return null; + } finally { + fout = null; + if (fc != null) { + IOUtils.closeQuietly(fc); + } + } + + } + + byte[] getJSonFiles(String dir) throws Exception { + Map fileMap = + new HashMap(); + + String path = logDir; + if (dir != null) { + path = path + File.separator + dir; + } + accessCheck(path); + + LOG.info("List dir " + path); + + File file = new File(path); + + String[] files = file.list(); + + for (String fileName : files) { + String logFile = Joiner.on(File.separator).join(path, fileName); + + FileAttribute fileAttribute = new FileAttribute(); + fileAttribute.setFileName(fileName); + + File subFile = new File(logFile); + + Date modify = new Date(subFile.lastModified()); + fileAttribute.setModifyTime(TimeFormat.getSecond(modify)); + + if (subFile.isFile()) { + fileAttribute.setIsDir(String.valueOf(false)); + fileAttribute.setSize(String.valueOf(subFile.length())); + + fileMap.put(logFile, fileAttribute); + } else if (subFile.isDirectory()) { + fileAttribute.setIsDir(String.valueOf(true)); + fileAttribute.setSize(String.valueOf(4096)); + + fileMap.put(logFile, fileAttribute); + } + + } + + String fileJsonStr = JStormUtils.to_json(fileMap); + return fileJsonStr.getBytes(); + } + + void handleListDir(HttpExchange t, Map paramMap) + throws IOException { + byte[] filesJson = "Failed to get file list".getBytes(); + + try { + 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; + } + + t.sendResponseHeaders(HttpURLConnection.HTTP_OK, filesJson.length); + OutputStream os = t.getResponseBody(); + os.write(filesJson); + os.close(); + } + + void handleJstack(StringBuffer sb, Integer pid) { + + String cmd = "jstack " + pid; + + try { + LOG.info("Begin to execute " + cmd); + Process process = + JStormUtils.launch_process(cmd, + new HashMap(), false); + + // Process process = Runtime.getRuntime().exec(sb.toString()); + + InputStream stdin = process.getInputStream(); + BufferedReader reader = + new BufferedReader(new InputStreamReader(stdin)); + + JStormUtils.sleepMs(1000); + + // if (process.exitValue() != 0) { + // LOG.info("Failed to execute " + sb.toString()); + // return null; + // } + + String str; + while ((str = reader.readLine()) != null) { + if (StringUtils.isBlank(str)) { + // LOG.info(str + " is Blank"); + continue; + } + + sb.append(str).append("\r\n"); + } + + LOG.info("Successfully get output of " + cmd); + return; + } catch (IOException e) { + LOG.info("Failed to execute " + cmd, e); + sb.append("Failed to execute " + cmd); + return; + } catch (Exception e) { + LOG.error(e.getMessage(), e); + sb.append("Failed to execute " + cmd + ", " + e.getCause()); + return; + } + } + + void handleJstack(HttpExchange t, Map paramMap) + throws IOException { + String workerPort = + paramMap.get(HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_WORKER_PORT); + if (workerPort == null) { + handlFailure(t, "Not set worker's port"); + return; + } + + LOG.info("Begin to get jstack of " + workerPort); + StringBuffer sb = new StringBuffer(); + List pids = Worker.getOldPortPids(workerPort); + for (Integer pid : pids) { + sb.append("!!!!!!!!!!!!!!!!!!\r\n"); + sb.append("WorkerPort:" + workerPort + ", pid:" + pid); + sb.append("\r\n!!!!!!!!!!!!!!!!!!\r\n"); + + handleJstack(sb, pid); + } + + byte[] data = sb.toString().getBytes(); + t.sendResponseHeaders(HttpURLConnection.HTTP_OK, data.length); + OutputStream os = t.getResponseBody(); + 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() { + int numHandler = 3; + InetSocketAddress socketAddr = new InetSocketAddress(port); + Executor executor = Executors.newFixedThreadPool(numHandler); + + try { + hs = HttpServer.create(socketAddr, 0); + hs.createContext(HttpserverUtils.HTTPSERVER_CONTEXT_PATH_LOGVIEW, + new LogHandler(conf)); + hs.setExecutor(executor); + hs.start(); + + } catch (BindException e) { + LOG.info("Httpserver Already start!"); + hs = null; + return; + } catch (IOException e) { + LOG.error("Httpserver Start Failed", e); + hs = null; + return; + } + LOG.info("Success start httpserver at port:" + port); + + } + + @Override + public void shutdown() { + if (hs != null) { + hs.stop(0); + LOG.info("Successfully stop http server"); + } + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/SandBoxMaker.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/SandBoxMaker.java new file mode 100755 index 000000000..dfee522f6 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/SandBoxMaker.java @@ -0,0 +1,221 @@ +/** + * 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.daemon.supervisor; + +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.LineNumberReader; +import java.io.PrintWriter; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.UUID; + +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.StormConfig; + +/** + * Right now generate java sandbox policy through template file + * + * In the future, generating java sandbox policy will through hardcode + * + * @author longda + * @version + */ +public class SandBoxMaker { + private static final Logger LOG = LoggerFactory + .getLogger(SandBoxMaker.class); + + public static final String SANBOX_TEMPLATE_NAME = "sandbox.policy"; + + public static final String JSTORM_HOME_KEY = "%JSTORM_HOME%"; + + public static final String CLASS_PATH_KEY = "%CLASS_PATH%"; + + public static final String LOCAL_DIR_KEY = "%JSTORM_LOCAL_DIR%"; + + // this conf should only be Supervisor Conf + private final Map conf; + + private final boolean isEnable; + + private final Map replaceBaseMap = + new HashMap(); + + public SandBoxMaker(Map conf) { + this.conf = conf; + + isEnable = ConfigExtension.isJavaSandBoxEnable(conf); + + LOG.info("Java Sandbox Policy :" + String.valueOf(isEnable)); + + String jstormHome = System.getProperty("jstorm.home"); + if (jstormHome == null) { + jstormHome = "./"; + } + + replaceBaseMap.put(JSTORM_HOME_KEY, jstormHome); + + replaceBaseMap.put(LOCAL_DIR_KEY, + (String) conf.get(Config.STORM_LOCAL_DIR)); + + LOG.info("JSTORM_HOME is " + jstormHome); + } + + private String genClassPath(String classPathLine) { + StringBuilder sb = new StringBuilder(); + + String[] classPathes = classPathLine.split(":"); + for (String classpath : classPathes) { + if (StringUtils.isBlank(classpath)) { + continue; + } + + File file = new File(classpath); + if (file.isDirectory()) { + sb.append(" permission java.io.FilePermission \""); + sb.append(classpath).append(File.separator).append("**"); + sb.append("\", \"read\";\n"); + } else { + sb.append(" permission java.io.FilePermission \""); + sb.append(classpath); + sb.append("\", \"read\";\n"); + } + + } + + return sb.toString(); + } + + private String replaceLine(String line, Map replaceMap) { + + for (Entry entry : replaceMap.entrySet()) { + if (line.contains(CLASS_PATH_KEY)) { + return genClassPath(entry.getValue()); + } else if (line.contains(entry.getKey())) { + return line.replace(entry.getKey(), entry.getValue()); + } + } + + return line; + } + + public String generatePolicyFile(Map replaceMap) + throws IOException { + // dynamic generate policy file, no static file + String tmpPolicy = + StormConfig.supervisorTmpDir(conf) + File.separator + + UUID.randomUUID().toString(); + + InputStream inputStream = + SandBoxMaker.class.getClassLoader().getResourceAsStream( + SANBOX_TEMPLATE_NAME); + + PrintWriter writer = + new PrintWriter(new BufferedWriter(new FileWriter(tmpPolicy))); + + try { + + InputStreamReader inputReader = new InputStreamReader(inputStream); + + BufferedReader reader = + new BufferedReader(new LineNumberReader(inputReader)); + + String line = null; + while ((line = reader.readLine()) != null) { + String replaced = replaceLine(line, replaceMap); + + writer.println(replaced); + } + + return tmpPolicy; + } catch (Exception e) { + LOG.error("Failed to generate policy file\n", e); + throw new IOException(e); + } finally { + if (inputStream != null) { + inputStream.close(); + } + if (writer != null) { + writer.close(); + } + + } + } + + /** + * Generate command string + * + * @param workerId + * @return + * @throws IOException + */ + public String sandboxPolicy(String workerId, Map replaceMap) + throws IOException { + if (isEnable == false) { + return ""; + } + + replaceMap.putAll(replaceBaseMap); + + String tmpPolicy = generatePolicyFile(replaceMap); + + File file = new File(tmpPolicy); + String policyPath = + StormConfig.worker_root(conf, workerId) + File.separator + + SANBOX_TEMPLATE_NAME; + File dest = new File(policyPath); + file.renameTo(dest); + + StringBuilder sb = new StringBuilder(); + sb.append(" -Djava.security.manager -Djava.security.policy="); + sb.append(policyPath); + + return sb.toString(); + + } + + public static void main(String[] args) { + Map conf = Utils.readStormConfig(); + + conf.put("java.sandbox.enable", Boolean.valueOf(true)); + + SandBoxMaker maker = new SandBoxMaker(conf); + + try { + System.out.println("sandboxPolicy:" + + maker.sandboxPolicy("simple", + new HashMap())); + } catch (IOException e) { + e.printStackTrace(); + } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/ShutdownWork.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/ShutdownWork.java new file mode 100644 index 000000000..0b906e34c --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/ShutdownWork.java @@ -0,0 +1,193 @@ +/** + * 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.daemon.supervisor; + +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.daemon.worker.ProcessSimulator; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.PathUtils; +import com.alibaba.jstorm.utils.TimeUtils; + +public class ShutdownWork extends RunnableCallback { + + private static Logger LOG = LoggerFactory.getLogger(ShutdownWork.class); + + /** + * shutdown all workers + * + * @param conf + * @param supervisorId + * @param removed + * @param workerThreadPids + * @param cgroupManager + * @param block + * @param killingWorkers + * + * @return the topologys whose workers are shutdown successfully + */ + public void shutWorker(Map conf, String supervisorId, + Map removed, + ConcurrentHashMap workerThreadPids, + CgroupManager cgroupManager, boolean block, + Map killingWorkers, + Map taskCleanupTimeoutMap) { + Map> workerId2Pids = + new HashMap>(); + + boolean localMode = false; + + int maxWaitTime = 0; + + if (killingWorkers == null) + killingWorkers = new HashMap(); + + for (Entry entry : removed.entrySet()) { + String workerId = entry.getKey(); + String topologyId = entry.getValue(); + + List pids = null; + try { + pids = getPid(conf, workerId); + } catch (IOException e1) { + LOG.error("Failed to get pid for " + workerId + " of " + + topologyId); + } + workerId2Pids.put(workerId, pids); + + if (killingWorkers.get(workerId) == null) { + killingWorkers.put(workerId, TimeUtils.current_time_secs()); + LOG.info("Begin to shut down " + topologyId + ":" + workerId); + try { + String threadPid = workerThreadPids.get(workerId); + + // local mode + if (threadPid != null) { + ProcessSimulator.killProcess(threadPid); + localMode = true; + continue; + } + + for (String pid : pids) { + JStormUtils.process_killed(Integer.parseInt(pid)); + } + + if (taskCleanupTimeoutMap != null + && taskCleanupTimeoutMap.get(topologyId) != null) { + maxWaitTime = + Math.max(maxWaitTime, + taskCleanupTimeoutMap.get(topologyId)); + } else { + maxWaitTime = + Math.max(maxWaitTime, ConfigExtension + .getTaskCleanupTimeoutSec(conf)); + } + } catch (Exception e) { + LOG.info("Failed to shutdown ", e); + } + } + } + + if (block) { + JStormUtils.sleepMs(maxWaitTime); + } + + for (Entry entry : removed.entrySet()) { + String workerId = entry.getKey(); + String topologyId = entry.getValue(); + List pids = workerId2Pids.get(workerId); + + int cleanupTimeout; + if (taskCleanupTimeoutMap != null + && taskCleanupTimeoutMap.get(topologyId) != null) { + cleanupTimeout = taskCleanupTimeoutMap.get(topologyId); + } else { + cleanupTimeout = ConfigExtension.getTaskCleanupTimeoutSec(conf); + } + + int initCleaupTime = killingWorkers.get(workerId); + if (TimeUtils.current_time_secs() - initCleaupTime > cleanupTimeout) { + if (localMode == false) { + for (String pid : pids) { + JStormUtils + .ensure_process_killed(Integer.parseInt(pid)); + if (cgroupManager != null) { + cgroupManager.shutDownWorker(workerId, true); + } + } + } + + tryCleanupWorkerDir(conf, workerId); + LOG.info("Successfully shut down " + workerId); + killingWorkers.remove(workerId); + } + } + } + + /** + * clean the directory , subdirectories of STORM-LOCAL-DIR/workers/workerId + * + * + * @param conf + * @param workerId + * @throws IOException + */ + public void tryCleanupWorkerDir(Map conf, String workerId) { + try { + // delete heartbeat dir LOCAL_DIR/workers/workid/heartbeats + PathUtils.rmr(StormConfig.worker_heartbeats_root(conf, workerId)); + // delete pid dir, LOCAL_DIR/workers/workerid/pids + PathUtils.rmr(StormConfig.worker_pids_root(conf, workerId)); + // delete workerid dir, LOCAL_DIR/worker/workerid + PathUtils.rmr(StormConfig.worker_root(conf, workerId)); + } catch (Exception e) { + LOG.warn(e + "Failed to cleanup worker " + 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-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/StateHeartbeat.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/StateHeartbeat.java new file mode 100755 index 000000000..c159f4bd9 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/StateHeartbeat.java @@ -0,0 +1,53 @@ +/** + * 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.daemon.supervisor; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; + +import com.alibaba.jstorm.daemon.worker.State; +import com.alibaba.jstorm.daemon.worker.WorkerHeartbeat; + +/** + * Worker's state and Hearbeat + * + * @author Xin.Zhou + */ +public class StateHeartbeat { + private State state; + private WorkerHeartbeat hb; + + public StateHeartbeat(State state, WorkerHeartbeat hb) { + this.state = state; + this.hb = hb; + } + + public State getState() { + return this.state; + } + + public WorkerHeartbeat getHeartbeat() { + return this.hb; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/Supervisor.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/Supervisor.java new file mode 100755 index 000000000..abc24480b --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/Supervisor.java @@ -0,0 +1,247 @@ +/** + * 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.daemon.supervisor; + +import java.io.File; +import java.util.Map; +import java.util.UUID; +import java.util.Vector; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.commons.io.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.messaging.IContext; +import backtype.storm.utils.LocalState; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.callback.AsyncLoopRunnable; +import com.alibaba.jstorm.callback.AsyncLoopThread; +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.Cluster; +import com.alibaba.jstorm.cluster.Common; +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.daemon.worker.hearbeat.SyncContainerHb; +import com.alibaba.jstorm.event.EventManagerImp; +import com.alibaba.jstorm.event.EventManagerPusher; +import com.alibaba.jstorm.utils.JStormServerUtils; +import com.alibaba.jstorm.utils.JStormUtils; + +/** + * + * + * Supevisor workflow 1. write SupervisorInfo to ZK + * + * 2. Every 10 seconds run SynchronizeSupervisor 2.1 download new topology 2.2 + * release useless worker 2.3 assgin new task to + * /local-dir/supervisor/localstate 2.4 add one syncProcesses event + * + * 3. Every supervisor.monitor.frequency.secs run SyncProcesses 3.1 kill useless + * worker 3.2 start new worker + * + * 4. create heartbeat thread every supervisor.heartbeat.frequency.secs, write + * SupervisorInfo to ZK + */ + +public class Supervisor { + + private static Logger LOG = LoggerFactory.getLogger(Supervisor.class); + + + /** + * create and start one supervisor + * + * @param conf : configurationdefault.yaml storm.yaml + * @param sharedContext : null (right now) + * @return SupervisorManger: which is used to shutdown all workers and + * supervisor + */ + @SuppressWarnings("rawtypes") + public SupervisorManger mkSupervisor(Map conf, IContext sharedContext) + throws Exception { + + LOG.info("Starting Supervisor with conf " + conf); + + /** + * Step 1: cleanup all files in /storm-local-dir/supervisor/tmp + */ + String path = StormConfig.supervisorTmpDir(conf); + FileUtils.cleanDirectory(new File(path)); + + /* + * Step 2: create ZK operation instance StromClusterState + */ + + StormClusterState stormClusterState = + Cluster.mk_storm_cluster_state(conf); + + /* + * Step 3, create LocalStat LocalStat is one KV database 4.1 create + * LocalState instance; 4.2 get supervisorId, if no supervisorId, create + * one + */ + + LocalState localState = StormConfig.supervisorState(conf); + + String supervisorId = (String) localState.get(Common.LS_ID); + if (supervisorId == null) { + supervisorId = UUID.randomUUID().toString(); + localState.put(Common.LS_ID, supervisorId); + } + + Vector threads = new Vector(); + + // Step 5 create HeartBeat + // every supervisor.heartbeat.frequency.secs, write SupervisorInfo to ZK + // sync hearbeat to nimbus + Heartbeat hb = new Heartbeat(conf, stormClusterState, supervisorId); + hb.update(); + AsyncLoopThread heartbeat = + new AsyncLoopThread(hb, false, null, Thread.MIN_PRIORITY, true); + threads.add(heartbeat); + + // Sync heartbeat to Apsara Container + AsyncLoopThread syncContainerHbThread = + SyncContainerHb.mkSupervisorInstance(conf); + if (syncContainerHbThread != null) { + threads.add(syncContainerHbThread); + } + + // Step 6 create and start sync Supervisor thread + // every supervisor.monitor.frequency.secs second run SyncSupervisor + EventManagerImp processEventManager = new EventManagerImp(); + AsyncLoopThread processEventThread = + new AsyncLoopThread(processEventManager); + threads.add(processEventThread); + + ConcurrentHashMap workerThreadPids = + new ConcurrentHashMap(); + SyncProcessEvent syncProcessEvent = + new SyncProcessEvent(supervisorId, conf, localState, + workerThreadPids, sharedContext); + + EventManagerImp syncSupEventManager = new EventManagerImp(); + AsyncLoopThread syncSupEventThread = + new AsyncLoopThread(syncSupEventManager); + threads.add(syncSupEventThread); + + SyncSupervisorEvent syncSupervisorEvent = + new SyncSupervisorEvent(supervisorId, conf, + processEventManager, syncSupEventManager, + stormClusterState, localState, syncProcessEvent, hb); + + int syncFrequence = + JStormUtils.parseInt(conf + .get(Config.SUPERVISOR_MONITOR_FREQUENCY_SECS)); + EventManagerPusher syncSupervisorPusher = + new EventManagerPusher(syncSupEventManager, + syncSupervisorEvent, syncFrequence); + AsyncLoopThread syncSupervisorThread = + new AsyncLoopThread(syncSupervisorPusher); + threads.add(syncSupervisorThread); + + 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(); + } + + // SupervisorManger which can shutdown all supervisor and workers + return new SupervisorManger(conf, supervisorId, threads, + syncSupEventManager, processEventManager, httpserver, + stormClusterState, workerThreadPids); + } + + /** + * shutdown + * + * @param supervisor + */ + 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); + + JStormServerUtils.createPid(pidDir); + } + + /** + * start supervisor + */ + public void run() { + + SupervisorManger supervisorManager = null; + try { + Map conf = Utils.readStormConfig(); + + StormConfig.validate_distributed_mode(conf); + + createPid(conf); + + supervisorManager = mkSupervisor(conf, null); + + JStormUtils.redirectOutput("/dev/null"); + + initShutdownHook(supervisorManager); + + while (supervisorManager.isFinishShutdown() == false) { + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + + } + } + + } catch (Exception e) { + LOG.error("Failed to start supervisor\n", e); + System.exit(1); + }finally { + LOG.info("Shutdown supervisor!!!"); + } + + + } + + /** + * supervisor daemon enter entrance + * + * @param args + */ + public static void main(String[] args) { + + JStormServerUtils.startTaobaoJvmMonitor(); + + Supervisor instance = new Supervisor(); + + instance.run(); + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/SupervisorDaemon.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/SupervisorDaemon.java new file mode 100755 index 000000000..f1db7419a --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/SupervisorDaemon.java @@ -0,0 +1,30 @@ +/** + * 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.daemon.supervisor; + +import java.util.Map; + +public interface SupervisorDaemon { + + public String getId(); + + public Map getConf(); + + public void ShutdownAllWorkers(); + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/SupervisorInfo.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/SupervisorInfo.java new file mode 100644 index 000000000..f53ef728e --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/SupervisorInfo.java @@ -0,0 +1,186 @@ +/** + * 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.daemon.supervisor; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; + +/** + * Object stored in ZK /ZK-DIR/supervisors + * + * @author Xin.Zhou/Longda + */ +public class SupervisorInfo implements Serializable { + + private static final long serialVersionUID = -8384417078907518922L; + + private final String hostName; + private final String supervisorId; + + private Integer timeSecs; + private Integer uptimeSecs; + + private Set workerPorts; + + private transient Set availableWorkerPorts; + + public SupervisorInfo(String hostName, String supervisorId, + Set workerPorts) { + this.hostName = hostName; + this.supervisorId = supervisorId; + this.workerPorts = workerPorts; + } + + public String getHostName() { + return hostName; + } + + public String getSupervisorId() { + return supervisorId; + } + + public int getTimeSecs() { + return timeSecs; + } + + public void setTimeSecs(int timeSecs) { + this.timeSecs = timeSecs; + } + + public int getUptimeSecs() { + return uptimeSecs; + } + + public void setUptimeSecs(int uptimeSecs) { + this.uptimeSecs = uptimeSecs; + } + + public Set getWorkerPorts() { + return workerPorts; + } + public void setAvailableWorkerPorts(Set workerPorts){ + if (availableWorkerPorts == null) + availableWorkerPorts = new HashSet(); + availableWorkerPorts.addAll(workerPorts); + } + public Set getAvailableWorkerPorts() { + if (availableWorkerPorts == null) + availableWorkerPorts = new HashSet(); + return availableWorkerPorts; + } + public void setWorkerPorts(Set workerPorts) { + this.workerPorts = workerPorts; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = + prime * result + ((hostName == null) ? 0 : hostName.hashCode()); + result = + prime + * result + + ((supervisorId == null) ? 0 : supervisorId.hashCode()); + result = + prime * result + ((timeSecs == null) ? 0 : timeSecs.hashCode()); + result = + prime * result + + ((uptimeSecs == null) ? 0 : uptimeSecs.hashCode()); + result = + prime * result + + ((workerPorts == null) ? 0 : workerPorts.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + SupervisorInfo other = (SupervisorInfo) obj; + if (hostName == null) { + if (other.hostName != null) + return false; + } else if (!hostName.equals(other.hostName)) + return false; + if (supervisorId == null) { + if (other.supervisorId != null) + return false; + } else if (!supervisorId.equals(other.supervisorId)) + return false; + if (timeSecs == null) { + if (other.timeSecs != null) + return false; + } else if (!timeSecs.equals(other.timeSecs)) + return false; + if (uptimeSecs == null) { + if (other.uptimeSecs != null) + return false; + } else if (!uptimeSecs.equals(other.uptimeSecs)) + return false; + if (workerPorts == null) { + if (other.workerPorts != null) + return false; + } else if (!workerPorts.equals(other.workerPorts)) + return false; + return true; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } + + /** + * get Map + * + * @param stormClusterState + * @param callback + * @return + */ + public static Map getNodeHost( + Map supInfos) { + + Map rtn = new HashMap(); + + for (Entry entry : supInfos.entrySet()) { + + SupervisorInfo superinfo = entry.getValue(); + + String supervisorid = entry.getKey(); + + rtn.put(supervisorid, superinfo.getHostName()); + + } + + return rtn; + } + +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/SupervisorManger.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/SupervisorManger.java new file mode 100644 index 000000000..a2806de91 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/SupervisorManger.java @@ -0,0 +1,197 @@ +/** + * 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.daemon.supervisor; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Vector; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.callback.AsyncLoopRunnable; +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; + +/** + * supervisor shutdown manager which can shutdown supervisor + */ +public class SupervisorManger extends ShutdownWork implements SupervisorDaemon, + DaemonCommon, Runnable { + + private static Logger LOG = LoggerFactory.getLogger(SupervisorManger.class); + + // private Supervisor supervisor; + + private Map conf; + + private String supervisorId; + + private AtomicBoolean shutdown; + + private Vector threads; + + private EventManager processesEventManager; + + private EventManager eventManager; + + private Httpserver httpserver; + + private StormClusterState stormClusterState; + + private ConcurrentHashMap workerThreadPidsAtom; + + private volatile boolean isFinishShutdown = false; + + public SupervisorManger(Map conf, String supervisorId, + Vector threads, + EventManager processesEventManager, EventManager eventManager, + Httpserver httpserver, StormClusterState stormClusterState, + ConcurrentHashMap workerThreadPidsAtom) { + this.conf = conf; + this.supervisorId = supervisorId; + this.shutdown = new AtomicBoolean(false); + this.threads = threads; + this.processesEventManager = processesEventManager; + this.eventManager = eventManager; + this.httpserver = httpserver; + this.stormClusterState = stormClusterState; + this.workerThreadPidsAtom = workerThreadPidsAtom; + + Runtime.getRuntime().addShutdownHook(new Thread(this)); + } + + @Override + public void shutdown() { + if (shutdown.getAndSet(true) == true) { + LOG.info("Supervisor has been shutdown before " + supervisorId); + return; + } + LOG.info("Shutting down supervisor " + supervisorId); + AsyncLoopRunnable.getShutdown().set(true); + + int size = threads.size(); + for (AsyncLoopThread thread : threads) { + thread.cleanup(); + JStormUtils.sleepMs(10); + thread.interrupt(); + // try { + // thread.join(); + // } catch (InterruptedException e) { + // LOG.error(e.getMessage(), e); + // } + LOG.info("Successfully shutdown thread:" + + thread.getThread().getName()); + } + eventManager.shutdown(); + processesEventManager.shutdown(); + try { + stormClusterState.disconnect(); + } catch (Exception e) { + // TODO Auto-generated catch block + LOG.error("Failed to shutdown ZK client", e); + } + if (httpserver != null) { + httpserver.shutdown(); + } + + // if (this.cgroupManager != null) + // try { + // this.cgroupManager.close(); + // } catch (IOException e) { + // // TODO Auto-generated catch block + // LOG.error("Fail to close cgroup", e); + // } + + isFinishShutdown = true; + + JStormUtils.halt_process(0, "!!!Shutdown!!!"); + } + + @Override + public void ShutdownAllWorkers() { + LOG.info("Begin to shutdown all workers"); + String path; + try { + path = StormConfig.worker_root(conf); + } catch (IOException e1) { + // TODO Auto-generated catch block + LOG.error("Failed to get Local worker dir", e1); + return; + } + List myWorkerIds = PathUtils.read_dir_contents(path); + HashMap workerId2topologyIds = + new HashMap(); + + for (String workerId : myWorkerIds) { + workerId2topologyIds.put(workerId, null); + } + + shutWorker(conf, supervisorId, workerId2topologyIds, + workerThreadPidsAtom, null, true, null, null); + } + + @Override + public Map getConf() { + return conf; + } + + @Override + public String getId() { + return supervisorId; + } + + @Override + public boolean waiting() { + if (shutdown.get()) { + return true; + } + + Boolean bThread = true; + int size = threads.size(); + for (int i = 0; i < size; i++) { + if (!(Boolean) threads.elementAt(i).isSleeping()) { + bThread = false; + return false; + } + } + boolean bManagers = true; + if (eventManager.waiting() && processesEventManager.waiting()) { + bManagers = false; + return false; + } + return true; + } + + public void run() { + shutdown(); + } + + public boolean isFinishShutdown() { + return isFinishShutdown; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/SyncProcessEvent.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/SyncProcessEvent.java new file mode 100644 index 000000000..d90eb2947 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/SyncProcessEvent.java @@ -0,0 +1,1057 @@ +/** + * 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.daemon.supervisor; + +import java.io.File; +import java.io.IOException; +import java.util.Collection; +import java.util.Date; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicReference; + +import java.util.regex.Pattern; + +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.GenericOptionsParser; +import backtype.storm.messaging.IContext; +import backtype.storm.utils.LocalState; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.Common; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.daemon.worker.LocalAssignment; +import com.alibaba.jstorm.daemon.worker.ProcessSimulator; +import com.alibaba.jstorm.daemon.worker.State; +import com.alibaba.jstorm.daemon.worker.Worker; +import com.alibaba.jstorm.daemon.worker.WorkerHeartbeat; +import com.alibaba.jstorm.daemon.worker.WorkerShutdown; +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; + +/** + * SyncProcesses (1) kill bad worker (2) start new worker + */ +class SyncProcessEvent extends ShutdownWork { + private static Logger LOG = LoggerFactory.getLogger(SyncProcessEvent.class); + + private LocalState localState; + + private Map conf; + + private ConcurrentHashMap workerThreadPids; + + private String supervisorId; + + private IContext sharedContext; + + private CgroupManager cgroupManager; + + 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; + /** + * workerIdToStartTimeAndPort ensure workId is unique, but don't ensure workId for workerPort + */ + private Map portToWorkerId = new HashMap(); + + private AtomicReference needDownloadTopologys; + + // workers under killing, Map; + private Map killingWorkers; + + // private Supervisor supervisor; + private int lastTime; + + /** + * @param conf + * @param localState + * @param workerThreadPids + * @param supervisorId + * @param sharedContext + * @param workerThreadPidsReadLock + * @param workerThreadPidsWriteLock + */ + public SyncProcessEvent(String supervisorId, Map conf, + LocalState localState, + ConcurrentHashMap workerThreadPids, + IContext sharedContext) { + + this.supervisorId = supervisorId; + + this.conf = conf; + + this.localState = localState; + + this.workerThreadPids = workerThreadPids; + + // right now, sharedContext is null + this.sharedContext = sharedContext; + + this.sandBoxMaker = new SandBoxMaker(conf); + + this.workerIdToStartTimeAndPort = + new HashMap>(); + + this.needDownloadTopologys = new AtomicReference(); + + if (ConfigExtension.isEnableCgroup(conf)) { + cgroupManager = new CgroupManager(conf); + } + + killingWorkers = new HashMap(); + } + + /** + * @@@ Change the old logic In the old logic, it will store + * LS_LOCAL_ASSIGNMENTS Map into LocalState + * + * But I don't think LS_LOCAL_ASSIGNMENTS is useful, so remove this + * logic + */ + @SuppressWarnings("unchecked") + @Override + public void run() { + + } + + public void run(Map localAssignments) { + LOG.debug("Syncing processes, interval seconds:" + + TimeUtils.time_delta(lastTime)); + lastTime = TimeUtils.current_time_secs(); + try { + + /** + * Step 1: get assigned tasks from localstat Map + */ + if (localAssignments == null) { + localAssignments = new HashMap(); + } + LOG.debug("Assigned tasks: " + localAssignments); + + /** + * Step 2: get local WorkerStats from local_dir/worker/ids/heartbeat + * Map + */ + Map localWorkerStats = null; + try { + localWorkerStats = + getLocalWorkerStats(conf, localState, localAssignments); + } catch (Exception e) { + LOG.error("Failed to get Local worker stats"); + throw e; + } + LOG.debug("Allocated: " + localWorkerStats); + + /** + * Step 3: kill Invalid Workers and remove killed worker from + * localWorkerStats + */ + Map taskCleaupTimeoutMap = null; + Set keepPorts = null; + try { + taskCleaupTimeoutMap = + (Map) localState + .get(Common.LS_TASK_CLEANUP_TIMEOUT); + keepPorts = + killUselessWorkers(localWorkerStats, localAssignments, + taskCleaupTimeoutMap); + localState.put(Common.LS_TASK_CLEANUP_TIMEOUT, + taskCleaupTimeoutMap); + } catch (IOException e) { + LOG.error("Failed to kill workers", e); + } + + // check new workers + checkNewWorkers(conf); + + // check which topology need update + checkNeedUpdateTopologys(localWorkerStats, localAssignments); + + // start new workers + startNewWorkers(keepPorts, localAssignments); + + } catch (Exception e) { + LOG.error("Failed Sync Process", e); + // throw e + } + + } + + /** + * check all workers is failed or not + */ + @SuppressWarnings("unchecked") + public void checkNeedUpdateTopologys( + Map localWorkerStats, + Map localAssignments) throws Exception { + Set topologys = new HashSet(); + + for (Map.Entry entry : localAssignments + .entrySet()) { + topologys.add(entry.getValue().getTopologyId()); + } + + for (StateHeartbeat stateHb : localWorkerStats.values()) { + State state = stateHb.getState(); + if (!state.equals(State.notStarted)) { + String topologyId = stateHb.getHeartbeat().getTopologyId(); + topologys.remove(topologyId); + } + } + long currTime = System.currentTimeMillis(); + Set needRemoveTopologys = new HashSet(); + for (String topologyId : topologys) { + try { + long lastModifytime = + StormConfig.get_supervisor_topology_Bianrymodify_time( + conf, topologyId); + if ((currTime - lastModifytime) / 1000 < (JStormUtils.MIN_1 * 2)) { + LOG.debug("less 2 minite ,so removed " + topologyId); + needRemoveTopologys.add(topologyId); + } + } catch (Exception e) { + LOG.error( + "Failed to get the time of file last modification for topology" + + topologyId, e); + needRemoveTopologys.add(topologyId); + } + } + topologys.removeAll(needRemoveTopologys); + + if (topologys.size() > 0) { + LOG.debug("Following topologys is going to re-download the jars, " + + topologys); + } + needDownloadTopologys.set(topologys); + } + + /** + * mark all new Workers + * + * @param workerIds + * @pdOid 52b11418-7474-446d-bff5-0ecd68f4954f + */ + public void markAllNewWorkers(Map workerIds) { + + int startTime = TimeUtils.current_time_secs(); + + for (Entry entry : workerIds.entrySet()) { + String oldWorkerIds = portToWorkerId.get(entry.getKey()); + if(oldWorkerIds != null){ + workerIdToStartTimeAndPort.remove(oldWorkerIds); + // update portToWorkerId + LOG.info("exit port is still occupied by old wokerId, so remove unuseful " + + oldWorkerIds+ " form workerIdToStartTimeAndPort"); + } + portToWorkerId.put(entry.getKey(), entry.getValue()); + workerIdToStartTimeAndPort.put(entry.getValue(), + new Pair(startTime, entry.getKey())); + } + } + + /** + * check new workers if the time is not > * + * SUPERVISOR_WORKER_START_TIMEOUT_SECS, otherwise info failed + * + * @param conf + * @pdOid f0a6ab43-8cd3-44e1-8fd3-015a2ec51c6a + */ + 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) { + 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 { + LOG.info("Successfully start worker " + workerId); + workers.add(workerId); + } + } + for (String workerId : workers) { + Integer port = this.workerIdToStartTimeAndPort.get(workerId).getSecond(); + this.workerIdToStartTimeAndPort.remove(workerId); + this.portToWorkerId.remove(port); + } + } + public Map getPortToWorkerId(){ + return portToWorkerId; + } + + /** + * get localstat approved workerId's map + * + * @return Map [workerheart, state] is also a + * map, key is "workheartbeat" and "state" + * @param conf + * @param localState + * @param assignedTasks + * @throws IOException + * @pdOid 11c9bebb-d082-4c51-b323-dd3d5522a649 + */ + @SuppressWarnings("unchecked") + public Map getLocalWorkerStats(Map conf, + LocalState localState, Map assignedTasks) + throws Exception { + + Map workeridHbstate = + new HashMap(); + + int now = TimeUtils.current_time_secs(); + + /** + * Get Map from + * local_dir/worker/ids/heartbeat + */ + Map idToHeartbeat = readWorkerHeartbeats(conf); + for (Map.Entry entry : idToHeartbeat + .entrySet()) { + + String workerid = entry.getKey().toString(); + + WorkerHeartbeat whb = entry.getValue(); + + State state = null; + + if (whb == null) { + state = State.notStarted; + Pair timeToPort = this.workerIdToStartTimeAndPort.get(workerid); + if (timeToPort != null) { + LocalAssignment localAssignment = assignedTasks.get(timeToPort.getSecond()); + if (localAssignment == null) { + LOG.info("Following worker don't exit assignment, so remove this port=" + + timeToPort.getSecond()); + state = State.disallowed; + //workerId is disallowed ,so remove it from workerIdToStartTimeAndPort + Integer port = this.workerIdToStartTimeAndPort.get(workerid).getSecond(); + this.workerIdToStartTimeAndPort.remove(workerid); + this.portToWorkerId.remove(port); + } + } + } else if (matchesAssignment(whb, assignedTasks) == false) { + + // workerId isn't approved or + // isn't assigned task + state = State.disallowed; + + } else if ((now - whb.getTimeSecs()) > JStormUtils.parseInt(conf + .get(Config.SUPERVISOR_WORKER_TIMEOUT_SECS))) {// + + state = State.timedOut; + } else { + if (isWorkerDead(workerid)) { + state = State.timedOut; + } else { + state = State.valid; + } + } + + if (state != State.valid) { + if (killingWorkers.containsKey(workerid) == false) + LOG.info("Worker:" + workerid + " state:" + state + + " WorkerHeartbeat:" + whb + " assignedTasks:" + + assignedTasks + " at supervisor time-secs " + now); + } else { + LOG.debug("Worker:" + workerid + " state:" + state + + " WorkerHeartbeat: " + whb + + " at supervisor time-secs " + now); + } + + workeridHbstate.put(workerid, new StateHeartbeat(state, whb)); + } + + return workeridHbstate; + } + + /** + * check whether the workerheartbeat is allowed in the assignedTasks + * + * @param whb : WorkerHeartbeat + * @param assignedTasks + * @return boolean if true, the assignments(LS-LOCAL-ASSIGNMENTS) is match + * with workerheart if fasle, is not matched + */ + public boolean matchesAssignment(WorkerHeartbeat whb, + Map assignedTasks) { + + boolean isMatch = true; + LocalAssignment localAssignment = assignedTasks.get(whb.getPort()); + + if (localAssignment == null) { + LOG.debug("Following worker has been removed, port=" + + whb.getPort() + ", assignedTasks=" + assignedTasks); + isMatch = false; + } else if (!whb.getTopologyId().equals(localAssignment.getTopologyId())) { + // topology id not equal + LOG.info("topology id not equal whb=" + whb.getTopologyId() + + ",localAssignment=" + localAssignment.getTopologyId()); + isMatch = false; + }/* + * else if (!(whb.getTaskIds().equals(localAssignment.getTaskIds()))) { + * // task-id isn't equal LOG.info("task-id isn't equal whb=" + + * whb.getTaskIds() + ",localAssignment=" + + * localAssignment.getTaskIds()); isMatch = false; } + */ + + return isMatch; + } + + /** + * get all workers heartbeats of the supervisor + * + * @param conf + * @return Map + * @throws IOException + * @throws IOException + */ + public Map readWorkerHeartbeats(Map conf) + throws Exception { + + Map workerHeartbeats = + new HashMap(); + + // get the path: STORM-LOCAL-DIR/workers + String path = StormConfig.worker_root(conf); + + List workerIds = PathUtils.read_dir_contents(path); + + if (workerIds == null) { + LOG.info("No worker dir under " + path); + return workerHeartbeats; + + } + + for (String workerId : workerIds) { + + WorkerHeartbeat whb = readWorkerHeartbeat(conf, workerId); + + // ATTENTION: whb can be null + workerHeartbeats.put(workerId, whb); + } + return workerHeartbeats; + } + + /** + * get worker heartbeat by workerid + * + * @param conf + * @param workerId + * @returns WorkerHeartbeat + * @throws IOException + */ + public WorkerHeartbeat readWorkerHeartbeat(Map conf, String workerId) + throws Exception { + + try { + LocalState ls = StormConfig.worker_state(conf, workerId); + + return (WorkerHeartbeat) ls.get(Common.LS_WORKER_HEARTBEAT); + } catch (IOException e) { + LOG.error("Failed to get worker Heartbeat", e); + return null; + } + + } + + /** + * launch a worker in local mode + * + * @param conf + * @param sharedcontext + * @param topologyId + * @param supervisorId + * @param port + * @param workerId + * @param workerThreadPidsAtom + * @throws Exception + */ + public void launchWorker(Map conf, IContext sharedcontext, + String topologyId, String supervisorId, Integer port, + String workerId, + ConcurrentHashMap workerThreadPidsAtom) + throws Exception { + + String pid = UUID.randomUUID().toString(); + + WorkerShutdown worker = + Worker.mk_worker(conf, sharedcontext, topologyId, supervisorId, + port, workerId, null); + + ProcessSimulator.registerProcess(pid, worker); + + 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 enableLog4j = false; + String userDefLog4jConf = + ConfigExtension.getUserDefinedLog4jConf(totalConf); + if (StringUtils.isBlank(userDefLog4jConf) == false) { + enableLog4j = true; + } + + if (enableLog4j == true) { + filterJars.add("log4j-over-slf4j"); + filterJars.add("logback-core"); + filterJars.add("logback-classic"); + + } else { + filterJars.add("slf4j-log4j"); + filterJars.add("log4j"); + } + } + + 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) { + + String regex = item + "[-._0-9]*.jar"; + Pattern p = Pattern.compile(regex); + + if (p.matcher(fileName).matches() == true) { + return true; + } + } + return false; + } + + public AtomicReference getTopologyIdNeedDownload() { + return needDownloadTopologys; + } + + private String getClassPath(String stormjar, String stormHome, Map totalConf) { + + // String classpath = JStormUtils.current_classpath() + ":" + stormjar; + // return classpath; + + String classpath = JStormUtils.current_classpath(); + + String[] classpathes = classpath.split(":"); + + Set classSet = new HashSet(); + + for (String classJar : classpathes) { + if (StringUtils.isBlank(classJar) == true) { + continue; + } + classSet.add(classJar); + } + + if (stormHome != null) { + List stormHomeFiles = + PathUtils.read_dir_contents(stormHome); + + for (String file : stormHomeFiles) { + if (file.endsWith(".jar")) { + classSet.add(stormHome + File.separator + file); + } + } + + List stormLibFiles = + PathUtils.read_dir_contents(stormHome + File.separator + + "lib"); + for (String file : stormLibFiles) { + if (file.endsWith(".jar")) { + classSet.add(stormHome + File.separator + "lib" + + File.separator + file); + } + } + + } + + Set filterJars = setFilterJars(totalConf); + + StringBuilder sb = new StringBuilder(); + for (String jar : classSet) { + if (isKeyContain(filterJars, jar)) { + LOG.info("Remove " + jar); + continue; + } + sb.append(jar + ":"); + } + + if (ConfigExtension.isEnableTopologyClassLoader(totalConf)) { + return sb.toString().substring(0, sb.length() - 1); + } else { + sb.append(stormjar); + return sb.toString(); + } + + } + + public String getChildOpts(Map stormConf) { + String childopts = " "; + + if (stormConf.get(Config.TOPOLOGY_WORKER_CHILDOPTS) != null) { + childopts += + (String) stormConf.get(Config.TOPOLOGY_WORKER_CHILDOPTS); + } else if (ConfigExtension.getWorkerGc(stormConf) != null) { + childopts += ConfigExtension.getWorkerGc(stormConf); + } + + return childopts; + } + + public String getLogParameter(Map conf, String stormHome, + String topologyName, int port) { + final String LOGBACK_CONF_TAG = "logback.configurationFile"; + final String LOGBACK_CONF_TAG_CMD = " -D" + LOGBACK_CONF_TAG + "="; + final String DEFAULT_LOG_CONF = "jstorm.logback.xml"; + + 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 userDefLogbackConf = + ConfigExtension.getUserDefinedLogbackConf(conf); + String logConf = System.getProperty(LOGBACK_CONF_TAG); + + if (StringUtils.isBlank(userDefLogbackConf) == false) { + LOG.info("Use user fined back conf " + userDefLogbackConf); + commandSB.append(LOGBACK_CONF_TAG_CMD).append(userDefLogbackConf); + } else if (StringUtils.isBlank(logConf) == false) { + commandSB.append(LOGBACK_CONF_TAG_CMD).append(logConf); + } else if (StringUtils.isBlank(stormHome) == false) { + commandSB.append(LOGBACK_CONF_TAG_CMD).append(stormHome) + .append(File.separator).append("conf") + .append(File.separator).append(DEFAULT_LOG_CONF); + } else { + commandSB.append(LOGBACK_CONF_TAG_CMD + DEFAULT_LOG_CONF); + } + + final String LOG4J_CONF_TAG = "log4j.configuration"; + String userDefLog4jConf = ConfigExtension.getUserDefinedLog4jConf(conf); + if (StringUtils.isBlank(userDefLog4jConf) == false) { + LOG.info("Use user fined log4j conf " + userDefLog4jConf); + commandSB.append(" -D" + LOG4J_CONF_TAG + "=").append( + userDefLog4jConf); + } + + return commandSB.toString(); + } + + private String getGcDumpParam(Map totalConf) { + // String gcPath = ConfigExtension.getWorkerGcPath(totalConf); + String gcPath = JStormUtils.getLogDir(); + + Date now = new Date(); + String nowStr = TimeFormat.getSecond(now); + + StringBuilder gc = new StringBuilder(); + + gc.append(" -Xloggc:"); + gc.append(gcPath); + gc.append(File.separator); + gc.append("%TOPOLOGYID%-worker-%ID%-"); + gc.append(nowStr); + gc.append("-gc.log -verbose:gc -XX:HeapDumpPath="); + gc.append(gcPath).append(File.separator).append("java-%TOPOLOGYID%-") + .append(nowStr).append(".hprof"); + gc.append(" "); + + return gc.toString(); + } + + /** + * launch a worker in distributed mode + * + * @param conf + * @param sharedcontext + * @param topologyId + * @param supervisorId + * @param port + * @param workerId + * @throws IOException + * @pdOid 6ea369dd-5ce2-4212-864b-1f8b2ed94abb + */ + public void launchWorker(Map conf, IContext sharedcontext, + String topologyId, String supervisorId, Integer port, + String workerId, LocalAssignment assignment) throws IOException { + + // STORM-LOCAL-DIR/supervisor/stormdist/topologyId + String stormroot = + StormConfig.supervisor_stormdist_root(conf, topologyId); + + // STORM-LOCAL-DIR/supervisor/stormdist/topologyId/stormjar.jar + String stormjar = StormConfig.stormjar_path(stormroot); + + // get supervisor conf + Map stormConf = + StormConfig.read_supervisor_topology_conf(conf, topologyId); + + Map totalConf = new HashMap(); + totalConf.putAll(conf); + totalConf.putAll(stormConf); + + // get classpath + // String[] param = new String[1]; + // param[0] = stormjar; + // String classpath = JStormUtils.add_to_classpath( + // JStormUtils.current_classpath(), param); + + // get child process parameter + + String stormhome = System.getProperty("jstorm.home"); + + long memSize = assignment.getMem(); + int cpuNum = assignment.getCpu(); + long memGsize = memSize / JStormUtils.SIZE_1_G; + int gcThreadsNum = memGsize > 4 ? (int) (memGsize * 1.5) : 4; + String childopts = getChildOpts(totalConf); + + childopts += getGcDumpParam(totalConf); + + Map environment = new HashMap(); + + if (ConfigExtension.getWorkerRedirectOutput(totalConf)) { + environment.put("REDIRECT", "true"); + } else { + environment.put("REDIRECT", "false"); + } + + environment.put("LD_LIBRARY_PATH", + (String) totalConf.get(Config.JAVA_LIBRARY_PATH)); + + StringBuilder commandSB = new StringBuilder(); + + try { + if (this.cgroupManager != null) { + commandSB.append(cgroupManager.startNewWorker(totalConf, + cpuNum, workerId)); + } + } catch (Exception e) { + LOG.error("fail to prepare cgroup to workerId: " + workerId, e); + return; + } + + // commandSB.append("java -server -Xdebug -Xrunjdwp:transport=dt_socket,address=8000,server=y,suspend=n "); + commandSB.append("java -server "); + commandSB.append(" -Xms" + memSize); + commandSB.append(" -Xmx" + memSize + " "); + commandSB.append(" -Xmn" + memSize / 3 + " "); + commandSB.append(" -XX:PermSize=" + memSize / 16); + commandSB.append(" -XX:MaxPermSize=" + memSize / 8); + commandSB.append(" -XX:ParallelGCThreads=" + gcThreadsNum); + commandSB.append(" " + childopts); + commandSB.append(" " + + (assignment.getJvm() == null ? "" : assignment.getJvm())); + + commandSB.append(" -Djava.library.path="); + commandSB.append((String) totalConf.get(Config.JAVA_LIBRARY_PATH)); + + if (stormhome != null) { + commandSB.append(" -Djstorm.home="); + commandSB.append(stormhome); + } + + commandSB.append(getLogParameter(totalConf, stormhome, + assignment.getTopologyName(), port)); + + String classpath = getClassPath(stormjar, stormhome, totalConf); + String workerClassPath = + (String) totalConf.get(Config.TOPOLOGY_CLASSPATH); + List otherLibs = + (List) stormConf + .get(GenericOptionsParser.TOPOLOGY_LIB_NAME); + StringBuilder sb = new StringBuilder(); + if (otherLibs != null) { + for (String libName : otherLibs) { + sb.append(StormConfig.stormlib_path(stormroot, libName)) + .append(":"); + } + } + workerClassPath = workerClassPath + ":" + sb.toString(); + + Map policyReplaceMap = new HashMap(); + String realClassPath = classpath + ":" + workerClassPath; + policyReplaceMap.put(SandBoxMaker.CLASS_PATH_KEY, realClassPath); + commandSB + .append(sandBoxMaker.sandboxPolicy(workerId, policyReplaceMap)); + + commandSB.append(" -cp "); + // commandSB.append(workerClassPath + ":"); + commandSB.append(classpath); + if (!ConfigExtension.isEnableTopologyClassLoader(totalConf)) + commandSB.append(":").append(workerClassPath); + + commandSB.append(" com.alibaba.jstorm.daemon.worker.Worker "); + commandSB.append(topologyId); + + commandSB.append(" "); + commandSB.append(supervisorId); + + commandSB.append(" "); + commandSB.append(port); + + commandSB.append(" "); + commandSB.append(workerId); + + commandSB.append(" "); + commandSB.append(workerClassPath + ":" + stormjar); + + 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(cmd, environment, true); + } + + private Set killUselessWorkers( + Map localWorkerStats, + Map localAssignments, + Map taskCleanupTimeoutMap) { + Map removed = new HashMap(); + Set keepPorts = new HashSet(); + + for (Entry entry : localWorkerStats.entrySet()) { + + String workerid = entry.getKey(); + StateHeartbeat hbstate = entry.getValue(); + if (workerIdToStartTimeAndPort.containsKey(workerid) + && hbstate.getState().equals(State.notStarted)) + 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, null); + } + + if (killingWorkers.containsKey(workerid) == false) { + + StringBuilder sb = new StringBuilder(); + sb.append("Shutting down and clearing state for id "); + sb.append(workerid); + sb.append(";State:"); + sb.append(hbstate); + + LOG.info(sb.toString()); + } + } + } + + shutWorker(conf, supervisorId, removed, workerThreadPids, + cgroupManager, false, killingWorkers, taskCleanupTimeoutMap); + Set activeTopologys = new HashSet(); + if (killingWorkers.size() == 0) { + // When all workers under killing are killed successfully, + // clean the task cleanup timeout map correspondingly. + for (Entry entry : localAssignments + .entrySet()) { + activeTopologys.add(entry.getValue().getTopologyId()); + } + + Set obsoleteTopologys = new HashSet(); + for (String topologyId : taskCleanupTimeoutMap.keySet()) { + if (activeTopologys.contains(topologyId) == false) { + obsoleteTopologys.add(topologyId); + } + } + for (String topologyId : obsoleteTopologys) { + taskCleanupTimeoutMap.remove(topologyId); + } + } + + for (String removedWorkerId : removed.keySet()) { + localWorkerStats.remove(removedWorkerId); + } + // Keep the workers which are still under starting + for (Entry> entry : workerIdToStartTimeAndPort + .entrySet()) { + String workerId = entry.getKey(); + StateHeartbeat hbstate = localWorkerStats.get(workerId); + if (hbstate != null) + if (hbstate.getState().equals(State.notStarted)) { + keepPorts.add(entry.getValue().getSecond()); + } + } + return keepPorts; + } + + private void startNewWorkers(Set keepPorts, + Map localAssignments) throws Exception { + /** + * Step 4: get reassigned tasks, which is in assignedTasks, but not in + * keeperPorts Map + */ + Map newWorkers = + JStormUtils.select_keys_pred(keepPorts, localAssignments); + + /** + * Step 5: generate new work ids + */ + Map newWorkerIds = new HashMap(); + + for (Entry entry : newWorkers.entrySet()) { + Integer port = entry.getKey(); + LocalAssignment assignment = entry.getValue(); + + String workerId = UUID.randomUUID().toString(); + + newWorkerIds.put(port, workerId); + + // create new worker Id directory + // LOCALDIR/workers/newworkid/pids + try { + StormConfig.worker_pids_root(conf, workerId); + } catch (IOException e1) { + LOG.error("Failed to create " + workerId + " localdir", e1); + throw e1; + } + + StringBuilder sb = new StringBuilder(); + sb.append("Launching worker with assiangment "); + sb.append(assignment.toString()); + sb.append(" for the supervisor "); + sb.append(supervisorId); + sb.append(" on port "); + sb.append(port); + sb.append(" with id "); + sb.append(workerId); + LOG.info(sb.toString()); + + try { + String clusterMode = StormConfig.cluster_mode(conf); + + if (clusterMode.equals("distributed")) { + launchWorker(conf, sharedContext, + assignment.getTopologyId(), supervisorId, port, + workerId, assignment); + } else if (clusterMode.equals("local")) { + launchWorker(conf, sharedContext, + assignment.getTopologyId(), supervisorId, port, + workerId, workerThreadPids); + } + } catch (Exception e) { + String errorMsg = + "Failed to launchWorker workerId:" + workerId + ":" + + port; + LOG.error(errorMsg, e); + throw e; + } + + } + + /** + * FIXME, workerIds should be Set, not Collection, but here simplify the + * logic + */ + 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 { + 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.info( + "Failed to check whether worker is dead through /proc/pid", + e); + return false; + } + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/SyncSupervisorEvent.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/SyncSupervisorEvent.java new file mode 100644 index 000000000..32aa0f1e2 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor/SyncSupervisorEvent.java @@ -0,0 +1,618 @@ +/** + * 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.daemon.supervisor; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.commons.io.FileExistsException; +import org.apache.commons.io.FileUtils; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.utils.LocalState; + +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.Cluster; +import com.alibaba.jstorm.cluster.Common; +import com.alibaba.jstorm.cluster.StormBase; +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.daemon.worker.LocalAssignment; +import com.alibaba.jstorm.event.EventManager; +import com.alibaba.jstorm.event.EventManagerZkPusher; +import com.alibaba.jstorm.schedule.Assignment; +import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; +import com.alibaba.jstorm.utils.JStormServerUtils; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.PathUtils; +import com.alibaba.jstorm.utils.TimeUtils; + +/** + * supervisor SynchronizeSupervisor workflow (1) writer local assignment to + * LocalState (2) download new Assignment's topology (3) remove useless Topology + * (4) push one SyncProcessEvent to SyncProcessEvent's EventManager + */ +class SyncSupervisorEvent extends RunnableCallback { + + private static final Logger LOG = LoggerFactory + .getLogger(SyncSupervisorEvent.class); + + // private Supervisor supervisor; + + private String supervisorId; + + private EventManager processEventManager; + + private EventManager syncSupEventManager; + + private StormClusterState stormClusterState; + + private LocalState localState; + + private Map conf; + + private SyncProcessEvent syncProcesses; + + private int lastTime; + + private Heartbeat heartbeat; + + /** + * @param conf + * @param processEventManager + * @param syncSupEventManager + * @param stormClusterState + * @param supervisorId + * @param localState + * @param syncProcesses + */ + public SyncSupervisorEvent(String supervisorId, Map conf, + EventManager processEventManager, EventManager syncSupEventManager, + StormClusterState stormClusterState, LocalState localState, + SyncProcessEvent syncProcesses, Heartbeat heartbeat) { + + this.syncProcesses = syncProcesses; + this.processEventManager = processEventManager; + this.syncSupEventManager = syncSupEventManager; + this.stormClusterState = stormClusterState; + this.conf = conf; + this.supervisorId = supervisorId; + this.localState = localState; + this.heartbeat = heartbeat; + } + + @Override + public void run() { + LOG.debug("Synchronizing supervisor, interval seconds:" + + TimeUtils.time_delta(lastTime)); + lastTime = TimeUtils.current_time_secs(); + + try { + + RunnableCallback syncCallback = + new EventManagerZkPusher(this, syncSupEventManager); + + /** + * Step 1: get all assignments and register /ZK-dir/assignment and + * every assignment watch + * + */ + Map assignments = + Cluster.get_all_assignment(stormClusterState, syncCallback); + LOG.debug("Get all assignments " + assignments); + + /** + * Step 2: get topologyIds list from + * STORM-LOCAL-DIR/supervisor/stormdist/ + */ + List downloadedTopologyIds = + StormConfig.get_supervisor_toplogy_list(conf); + LOG.debug("Downloaded storm ids: " + downloadedTopologyIds); + + /** + * Step 3: get from ZK local node's + * assignment + */ + Map zkAssignment = + getLocalAssign(stormClusterState, supervisorId, assignments); + Map localAssignment; + Set updateTopologys; + + /** + * Step 4: writer local assignment to LocalState + */ + try { + LOG.debug("Writing local assignment " + zkAssignment); + localAssignment = + (Map) localState + .get(Common.LS_LOCAL_ASSIGNMENTS); + if (localAssignment == null) { + localAssignment = new HashMap(); + } + localState.put(Common.LS_LOCAL_ASSIGNMENTS, zkAssignment); + + updateTopologys = + getUpdateTopologys(localAssignment, zkAssignment); + Set reDownloadTopologys = + getNeedReDownloadTopologys(localAssignment); + if (reDownloadTopologys != null) { + updateTopologys.addAll(reDownloadTopologys); + } + } catch (IOException e) { + LOG.error("put LS_LOCAL_ASSIGNMENTS " + zkAssignment + + " of localState failed"); + throw e; + } + + /** + * Step 5: download code from ZK + */ + Map topologyCodes = + getTopologyCodeLocations(assignments, supervisorId); + + downloadTopology(topologyCodes, downloadedTopologyIds, + updateTopologys, assignments); + + /** + * Step 6: remove any downloaded useless topology + */ + removeUselessTopology(topologyCodes, downloadedTopologyIds); + + /** + * Step 7: push syncProcesses Event + */ + // processEventManager.add(syncProcesses); + syncProcesses.run(zkAssignment); + + // If everything is OK, set the trigger to update heartbeat of + // supervisor + heartbeat.updateHbTrigger(true); + } catch (Exception e) { + LOG.error("Failed to Sync Supervisor", e); + // throw new RuntimeException(e); + } + + } + + /** + * download code ; two cluster mode: local and distributed + * + * @param conf + * @param topologyId + * @param masterCodeDir + * @param clusterMode + * @throws IOException + */ + private void downloadStormCode(Map conf, String topologyId, + String masterCodeDir) throws IOException, TException { + String clusterMode = StormConfig.cluster_mode(conf); + + if (clusterMode.endsWith("distributed")) { + downloadDistributeStormCode(conf, topologyId, masterCodeDir); + } else if (clusterMode.endsWith("local")) { + downloadLocalStormCode(conf, topologyId, masterCodeDir); + + } + } + + private void downloadLocalStormCode(Map conf, String topologyId, + String masterCodeDir) throws IOException, TException { + + // STORM-LOCAL-DIR/supervisor/stormdist/storm-id + String stormroot = + StormConfig.supervisor_stormdist_root(conf, topologyId); + + FileUtils.copyDirectory(new File(masterCodeDir), new File(stormroot)); + + ClassLoader classloader = + Thread.currentThread().getContextClassLoader(); + + String resourcesJar = resourcesJar(); + + URL url = classloader.getResource(StormConfig.RESOURCES_SUBDIR); + + String targetDir = stormroot + '/' + StormConfig.RESOURCES_SUBDIR; + + if (resourcesJar != null) { + + LOG.info("Extracting resources from jar at " + resourcesJar + + " to " + targetDir); + + JStormUtils.extract_dir_from_jar(resourcesJar, + StormConfig.RESOURCES_SUBDIR, stormroot);// extract dir + // from jar;; + // util.clj + } else if (url != null) { + + LOG.info("Copying resources at " + url.toString() + " to " + + targetDir); + + FileUtils.copyDirectory(new File(url.getFile()), (new File( + targetDir))); + + } + } + + /** + * Don't need synchronize, due to EventManager will execute serially + * + * @param conf + * @param topologyId + * @param masterCodeDir + * @throws IOException + * @throws TException + */ + private void downloadDistributeStormCode(Map conf, String topologyId, + String masterCodeDir) throws IOException, TException { + + // STORM_LOCAL_DIR/supervisor/tmp/(UUID) + String tmproot = + StormConfig.supervisorTmpDir(conf) + File.separator + + UUID.randomUUID().toString(); + + // STORM_LOCAL_DIR/supervisor/stormdist/topologyId + String stormroot = + StormConfig.supervisor_stormdist_root(conf, topologyId); + + JStormServerUtils.downloadCodeFromMaster(conf, tmproot, masterCodeDir, + topologyId, true); + + // tmproot/stormjar.jar + String localFileJarTmp = StormConfig.stormjar_path(tmproot); + + // extract dir from jar + JStormUtils.extract_dir_from_jar(localFileJarTmp, + StormConfig.RESOURCES_SUBDIR, tmproot); + + File srcDir = new File(tmproot); + File destDir = new File(stormroot); + try { + FileUtils.moveDirectory(srcDir, destDir); + } catch (FileExistsException e) { + FileUtils.copyDirectory(srcDir, destDir); + FileUtils.deleteQuietly(srcDir); + } + } + + private String resourcesJar() { + + String path = System.getProperty("java.class.path"); + if (path == null) { + return null; + } + + String[] paths = path.split(File.pathSeparator); + + List jarPaths = new ArrayList(); + for (String s : paths) { + if (s.endsWith(".jar")) { + jarPaths.add(s); + } + } + + /** + * FIXME, this place seems exist problem + */ + List rtn = new ArrayList(); + int size = jarPaths.size(); + for (int i = 0; i < size; i++) { + if (JStormUtils.zipContainsDir(jarPaths.get(i), + StormConfig.RESOURCES_SUBDIR)) { + rtn.add(jarPaths.get(i)); + } + } + + if (rtn.size() == 0) + return null; + + return rtn.get(0); + } + + /** + * a port must be assigned one topology + * + * @param stormClusterState + * @param supervisorId + * @param callback + * @throws Exception + * @returns map: {port,LocalAssignment} + */ + private Map getLocalAssign( + StormClusterState stormClusterState, String supervisorId, + Map assignments) throws Exception { + + Map portLA = + new HashMap(); + + for (Entry assignEntry : assignments.entrySet()) { + String topologyId = assignEntry.getKey(); + Assignment assignment = assignEntry.getValue(); + + Map portTasks = + readMyTasks(stormClusterState, topologyId, supervisorId, + assignment); + if (portTasks == null) { + continue; + } + + // a port must be assigned one storm + for (Entry entry : portTasks.entrySet()) { + + Integer port = entry.getKey(); + + LocalAssignment la = entry.getValue(); + + if (!portLA.containsKey(port)) { + portLA.put(port, la); + } else { + throw new RuntimeException( + "Should not have multiple topologys assigned to one port"); + } + } + } + + return portLA; + } + + /** + * get local node's tasks + * + * @param stormClusterState + * @param topologyId + * @param supervisorId + * @param callback + * @return Map: {port, LocalAssignment} + * @throws Exception + */ + private Map readMyTasks( + StormClusterState stormClusterState, String topologyId, + String supervisorId, Assignment assignmenInfo) throws Exception { + + Map portTasks = + new HashMap(); + + Set workers = assignmenInfo.getWorkers(); + if (workers == null) { + LOG.error("No worker of assignement's " + assignmenInfo); + return portTasks; + } + + for (ResourceWorkerSlot worker : workers) { + if (!supervisorId.equals(worker.getNodeId())) + continue; + portTasks.put(worker.getPort(), new LocalAssignment(topologyId, + worker.getTasks(), Common.topologyIdToName(topologyId), + worker.getMemSize(), worker.getCpu(), worker.getJvm(), + assignmenInfo.getTimeStamp())); + } + + return portTasks; + } + + /** + * get mastercodedir for every topology + * + * @param stormClusterState + * @param callback + * @throws Exception + * @returns Map: from zookeeper + */ + public static Map getTopologyCodeLocations( + Map assignments, String supervisorId) + throws Exception { + + Map rtn = new HashMap(); + for (Entry entry : assignments.entrySet()) { + String topologyid = entry.getKey(); + Assignment assignmenInfo = entry.getValue(); + + Set workers = assignmenInfo.getWorkers(); + for (ResourceWorkerSlot worker : workers) { + String node = worker.getNodeId(); + if (supervisorId.equals(node)) { + rtn.put(topologyid, assignmenInfo.getMasterCodeDir()); + break; + } + } + + } + return rtn; + } + + public void downloadTopology(Map topologyCodes, + List downloadedTopologyIds, Set updateTopologys, + Map assignments) throws Exception { + + Set downloadTopologys = new HashSet(); + + for (Entry entry : topologyCodes.entrySet()) { + + String topologyId = entry.getKey(); + String masterCodeDir = entry.getValue(); + + if (!downloadedTopologyIds.contains(topologyId) + || updateTopologys.contains(topologyId)) { + + LOG.info("Downloading code for storm id " + topologyId + + " from " + masterCodeDir); + + try { + downloadStormCode(conf, topologyId, masterCodeDir); + // Update assignment timeStamp + StormConfig.write_supervisor_topology_timestamp(conf, + topologyId, assignments.get(topologyId) + .getTimeStamp()); + } catch (IOException e) { + LOG.error(e + " downloadStormCode failed " + "topologyId:" + + topologyId + "masterCodeDir:" + masterCodeDir); + + } catch (TException e) { + LOG.error(e + " downloadStormCode failed " + "topologyId:" + + topologyId + "masterCodeDir:" + masterCodeDir); + } + LOG.info("Finished downloading code for storm id " + topologyId + + " from " + masterCodeDir); + + downloadTopologys.add(topologyId); + } + } + + updateTaskCleanupTimeout(downloadTopologys); + } + + public void removeUselessTopology(Map topologyCodes, + List downloadedTopologyIds) { + for (String topologyId : downloadedTopologyIds) { + + if (!topologyCodes.containsKey(topologyId)) { + + LOG.info("Removing code for storm id " + topologyId); + + String path = null; + try { + path = + StormConfig.supervisor_stormdist_root(conf, + topologyId); + PathUtils.rmr(path); + } catch (IOException e) { + String errMsg = "rmr the path:" + path + "failed\n"; + LOG.error(errMsg, e); + } + } + } + } + + private Set getUpdateTopologys( + Map localAssignments, + Map zkAssignments) { + Set ret = new HashSet(); + if (localAssignments != null && zkAssignments != null) { + for (Entry entry : localAssignments + .entrySet()) { + Integer port = entry.getKey(); + LocalAssignment localAssignment = entry.getValue(); + + LocalAssignment zkAssignment = zkAssignments.get(port); + + if (localAssignment == null || zkAssignment == null) + continue; + + if (localAssignment.getTopologyId().equals( + zkAssignment.getTopologyId()) + && localAssignment.getTimeStamp() < zkAssignment + .getTimeStamp()) + if (ret.add(localAssignment.getTopologyId())) { + LOG.info("Topology-" + localAssignment.getTopologyId() + + " has been updated. LocalTs=" + + localAssignment.getTimeStamp() + ", ZkTs=" + + zkAssignment.getTimeStamp()); + } + } + } + + return ret; + } + + private Set getNeedReDownloadTopologys( + Map localAssignment) { + Set reDownloadTopologys = + syncProcesses.getTopologyIdNeedDownload().getAndSet(null); + if (reDownloadTopologys == null || reDownloadTopologys.size() == 0) + return null; + Set needRemoveTopologys = new HashSet(); + Map portToStartWorkerId = + syncProcesses.getPortToWorkerId(); + for (Entry entry : localAssignment + .entrySet()) { + if (portToStartWorkerId.containsKey(entry.getKey())) + needRemoveTopologys.add(entry.getValue().getTopologyId()); + } + LOG.debug( + "worker is starting on these topology, so delay download topology binary: " + + needRemoveTopologys); + reDownloadTopologys.removeAll(needRemoveTopologys); + if (reDownloadTopologys.size() > 0) + LOG.info("Following topologys is going to re-download the jars, " + + reDownloadTopologys); + return reDownloadTopologys; + } + + private void updateTaskCleanupTimeout(Set topologys) { + Map topologyConf = null; + Map taskCleanupTimeouts = + new HashMap(); + + for (String topologyId : topologys) { + try { + topologyConf = + StormConfig.read_supervisor_topology_conf(conf, + topologyId); + } catch (IOException e) { + LOG.info("Failed to read conf for " + topologyId); + } + + Integer cleanupTimeout = null; + if (topologyConf != null) { + cleanupTimeout = + JStormUtils.parseInt(topologyConf + .get(ConfigExtension.TASK_CLEANUP_TIMEOUT_SEC)); + } + + if (cleanupTimeout == null) { + cleanupTimeout = ConfigExtension.getTaskCleanupTimeoutSec(conf); + } + + taskCleanupTimeouts.put(topologyId, cleanupTimeout); + } + + Map localTaskCleanupTimeouts = null; + try { + localTaskCleanupTimeouts = + (Map) localState + .get(Common.LS_TASK_CLEANUP_TIMEOUT); + } catch (IOException e) { + LOG.error("Failed to read local task cleanup timeout map", e); + } + + if (localTaskCleanupTimeouts == null) + localTaskCleanupTimeouts = taskCleanupTimeouts; + else + localTaskCleanupTimeouts.putAll(taskCleanupTimeouts); + + try { + localState.put(Common.LS_TASK_CLEANUP_TIMEOUT, + localTaskCleanupTimeouts); + } catch (IOException e) { + LOG.error("Failed to write local task cleanup timeout map", e); + } + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/BatchDrainerRunable.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/BatchDrainerRunable.java new file mode 100755 index 000000000..81e437402 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/BatchDrainerRunable.java @@ -0,0 +1,60 @@ +/** + * 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.daemon.worker; + +import java.util.List; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.messaging.IConnection; +import backtype.storm.messaging.TaskMessage; + +import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.utils.DisruptorRunable; +import com.alibaba.jstorm.utils.Pair; + +//import com.alibaba.jstorm.message.zeroMq.ISendConnection; + +/** + * + * Tuple sender + * + * @author yannian + * + */ +public class BatchDrainerRunable extends DisruptorRunable { + private final static Logger LOG = LoggerFactory + .getLogger(BatchDrainerRunable.class); + + public BatchDrainerRunable(WorkerData workerData) { + super(workerData.getSendingQueue(), MetricDef.BATCH_DRAINER_THREAD); + + } + + @Override + public void handleEvent(Object event, boolean endOfBatch) throws Exception { + + Pair> pair = + (Pair>) event; + + pair.getFirst().send(pair.getSecond()); + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/ContextMaker.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/ContextMaker.java new file mode 100755 index 000000000..a26032310 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/ContextMaker.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.daemon.worker; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.generated.StormTopology; +import backtype.storm.generated.StreamInfo; +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Fields; +import backtype.storm.utils.ThriftTopologyUtils; + +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.PathUtils; + +/** + * ContextMaker This class is used to create TopologyContext + * + * @author yannian/Longda + * + */ +public class ContextMaker { + private static Logger LOG = LoggerFactory.getLogger(ContextMaker.class); + + private WorkerData workerData; + + private String resourcePath; + private String pidDir; + private String codeDir; + private List workerTasks; + + @SuppressWarnings("rawtypes") + public ContextMaker(WorkerData workerData) { + /* + * Map stormConf, String topologyId, String workerId, HashMap tasksToComponent, Integer port, List workerTasks + */ + this.workerData = workerData; + this.workerTasks = JStormUtils.mk_list(workerData.getTaskids()); + + try { + Map stormConf = workerData.getStormConf(); + String topologyId = workerData.getTopologyId(); + String workerId = workerData.getWorkerId(); + + String distroot = + StormConfig + .supervisor_stormdist_root(stormConf, topologyId); + + resourcePath = + StormConfig.supervisor_storm_resources_path(distroot); + + pidDir = StormConfig.worker_pids_root(stormConf, workerId); + + String codePath = StormConfig.stormcode_path(distroot); + codeDir = PathUtils.parent_path(codePath); + + } catch (IOException e) { + LOG.error("Failed to create ContextMaker", e); + throw new RuntimeException(e); + } + } + + public TopologyContext makeTopologyContext(StormTopology topology, + Integer taskId, clojure.lang.Atom openOrPrepareWasCalled) { + + Map stormConf = workerData.getStormConf(); + String topologyId = workerData.getTopologyId(); + + HashMap> componentToStreamToFields = + new HashMap>(); + + Set components = ThriftTopologyUtils.getComponentIds(topology); + for (String component : components) { + + Map streamToFieldsMap = + new HashMap(); + + Map streamInfoMap = + ThriftTopologyUtils.getComponentCommon(topology, component) + .get_streams(); + for (Entry entry : streamInfoMap.entrySet()) { + String streamId = entry.getKey(); + StreamInfo streamInfo = entry.getValue(); + + streamToFieldsMap.put(streamId, + new Fields(streamInfo.get_output_fields())); + } + + componentToStreamToFields.put(component, streamToFieldsMap); + } + + return new TopologyContext(topology, stormConf, + workerData.getTasksToComponent(), + workerData.getComponentToSortedTasks(), + componentToStreamToFields, topologyId, resourcePath, pidDir, + taskId, workerData.getPort(), workerTasks, + workerData.getDefaultResources(), + workerData.getUserResources(), workerData.getExecutorData(), + workerData.getRegisteredMetrics(), openOrPrepareWasCalled); + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/DrainerRunable.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/DrainerRunable.java new file mode 100755 index 000000000..3477cc498 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/DrainerRunable.java @@ -0,0 +1,139 @@ +/** + * 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.daemon.worker; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.ConcurrentHashMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.messaging.IConnection; +import backtype.storm.messaging.TaskMessage; +import backtype.storm.scheduler.WorkerSlot; +import backtype.storm.utils.DisruptorQueue; + +import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.utils.DisruptorRunable; +import com.alibaba.jstorm.utils.Pair; + +//import com.alibaba.jstorm.message.zeroMq.ISendConnection; + +/** + * + * Tuple sender + * + * @author yannian + * + */ +public class DrainerRunable extends DisruptorRunable { + private final static Logger LOG = LoggerFactory + .getLogger(DrainerRunable.class); + + private DisruptorQueue transferQueue; + private ConcurrentHashMap nodeportSocket; + private ConcurrentHashMap taskNodeport; + + private Map> dispatchMap; + private DisruptorQueue sendingQueue; + + private final boolean isDirectSend = true; + + private DisruptorQueue queue; + + public DrainerRunable(WorkerData workerData) { + super(workerData.getTransferQueue(), MetricDef.DRAINER_THREAD); + this.sendingQueue = workerData.getSendingQueue(); + this.nodeportSocket = workerData.getNodeportSocket(); + this.taskNodeport = workerData.getTaskNodeport(); + this.dispatchMap = new HashMap>(); + + this.queue = workerData.getTransferQueue(); + + this.queue.consumerStarted(); + } + + public void handleOneEvent(TaskMessage felem) { + + int taskId = felem.task(); + byte[] tuple = felem.message(); + + WorkerSlot nodePort = taskNodeport.get(taskId); + if (nodePort == null) { + String errormsg = "can`t not found IConnection to " + taskId; + LOG.warn("DrainerRunable warn", new Exception(errormsg)); + return; + } + IConnection conn = nodeportSocket.get(nodePort); + if (conn == null) { + String errormsg = "can`t not found nodePort " + nodePort; + LOG.warn("DrainerRunable warn", new Exception(errormsg)); + return; + } + + if (conn.isClosed() == true) { + // if connection has been closed, just skip the package + LOG.debug("Skip one tuple of " + taskId + + ", due to close connection of " + nodePort); + return; + } + + if (isDirectSend) { + conn.send(felem); + return; + } + + List list = dispatchMap.get(conn); + if (list == null) { + list = new ArrayList(); + dispatchMap.put(conn, list); + } + list.add(felem); + return; + + } + + public void handleFinish() { + for (Entry> entry : dispatchMap + .entrySet()) { + Pair> pair = + new Pair>(entry.getKey(), + entry.getValue()); + + sendingQueue.publish(pair); + } + + dispatchMap.clear(); + } + + @Override + public void handleEvent(Object event, boolean endOfBatch) throws Exception { + + handleOneEvent((TaskMessage) event); + + if (endOfBatch == true && isDirectSend == false) { + handleFinish(); + } + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/LocalAssignment.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/LocalAssignment.java new file mode 100755 index 000000000..312c57f36 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/LocalAssignment.java @@ -0,0 +1,162 @@ +/** + * 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.daemon.worker; + +import java.io.Serializable; +import java.util.HashSet; +import java.util.Set; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; + +/** + * Supervisor LocalAssignment + * + */ +public class LocalAssignment implements Serializable { + public static final long serialVersionUID = 4054639727225043554L; + private final String topologyId; + private final String topologyName; + private Set taskIds; + private long mem; + private int cpu; + private String jvm; + private long timeStamp; + + public LocalAssignment(String topologyId, Set taskIds, + String topologyName, long mem, int cpu, String jvm, long timeStamp) { + this.topologyId = topologyId; + this.taskIds = new HashSet(taskIds); + this.topologyName = topologyName; + this.mem = mem; + this.cpu = cpu; + this.jvm = jvm; + this.timeStamp = timeStamp; + } + + public String getTopologyId() { + return topologyId; + } + + public Set getTaskIds() { + return taskIds; + } + + public void setTaskIds(Set taskIds) { + this.taskIds = new HashSet(taskIds); + } + + public String getTopologyName() { + return topologyName; + } + + public String getJvm() { + return jvm; + } + + public void setJvm(String jvm) { + this.jvm = jvm; + } + + public long getMem() { + return mem; + } + + public void setMem(long mem) { + this.mem = mem; + } + + public int getCpu() { + return cpu; + } + + public void setCpu(int cpu) { + this.cpu = cpu; + } + + public long getTimeStamp() { + return timeStamp; + } + + public void setTimeStamp(long timeStamp) { + this.timeStamp = timeStamp; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + cpu; + result = prime * result + ((jvm == null) ? 0 : jvm.hashCode()); + result = prime * result + (int) (mem ^ (mem >>> 32)); + result = prime * result + ((taskIds == null) ? 0 : taskIds.hashCode()); + result = + prime * result + + ((topologyId == null) ? 0 : topologyId.hashCode()); + result = + prime + * result + + ((topologyName == null) ? 0 : topologyName.hashCode()); + result = prime * result + (int) (timeStamp & 0xffffffff); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + LocalAssignment other = (LocalAssignment) obj; + if (cpu != other.cpu) + return false; + if (jvm == null) { + if (other.jvm != null) + return false; + } else if (!jvm.equals(other.jvm)) + return false; + if (mem != other.mem) + return false; + if (taskIds == null) { + if (other.taskIds != null) + return false; + } else if (!taskIds.equals(other.taskIds)) + return false; + if (topologyId == null) { + if (other.topologyId != null) + return false; + } else if (!topologyId.equals(other.topologyId)) + return false; + if (topologyName == null) { + if (other.topologyName != null) + return false; + } else if (!topologyName.equals(other.topologyName)) + return false; + if (timeStamp != other.getTimeStamp()) + return false; + return true; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/ProcessSimulator.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/ProcessSimulator.java new file mode 100755 index 000000000..628e0f52a --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/ProcessSimulator.java @@ -0,0 +1,100 @@ +/** + * 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.daemon.worker; + +import java.util.Collection; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ProcessSimulator { + private static Logger LOG = LoggerFactory.getLogger(ProcessSimulator.class); + + protected static Object lock = new Object(); + + /** + * skip old function name: pid-counter + */ + + protected static ConcurrentHashMap processMap = + new ConcurrentHashMap(); + + /** + * Register process handler old function name: register-process + * + * @param pid + * @param shutdownable + */ + public static void registerProcess(String pid, WorkerShutdown shutdownable) { + processMap.put(pid, shutdownable); + } + + /** + * Get process handle old function name: process-handle + * + * @param pid + * @return + */ + protected static WorkerShutdown getProcessHandle(String pid) { + return processMap.get(pid); + } + + /** + * Get all process handles old function name:all-processes + * + * @return + */ + protected static Collection GetAllProcessHandles() { + return processMap.values(); + } + + /** + * Kill pid handle old function name: KillProcess + * + * @param pid + */ + public static void killProcess(String pid) { + synchronized (lock) { + LOG.info("Begin killing process " + pid); + + WorkerShutdown shutdownHandle = getProcessHandle(pid); + + if (shutdownHandle != null) { + shutdownHandle.shutdown(); + } + + processMap.remove(pid); + + LOG.info("Successfully killing process " + pid); + } + } + + /** + * kill all handle old function name: kill-all-processes + */ + public static void killAllProcesses() { + Set pids = processMap.keySet(); + for (String pid : pids) { + killProcess(pid); + } + + LOG.info("Successfully kill all processes"); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/RefreshActive.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/RefreshActive.java new file mode 100755 index 000000000..3f8acfcf2 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/RefreshActive.java @@ -0,0 +1,139 @@ +/** + * 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.daemon.worker; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; + +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.cluster.StormBase; +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.daemon.nimbus.StatusType; +import com.alibaba.jstorm.task.TaskShutdownDameon; +import com.alibaba.jstorm.utils.JStormUtils; + +/** + * Timely check whether topology is active or not and whether the metrics + * monitor is enable or disable from ZK + * + * @author yannian/Longda + * + */ +public class RefreshActive extends RunnableCallback { + private static Logger LOG = LoggerFactory.getLogger(RefreshActive.class); + + private WorkerData workerData; + + private AtomicBoolean shutdown; + private AtomicBoolean monitorEnable; + private Map conf; + private StormClusterState zkCluster; + private String topologyId; + private Integer frequence; + + // private Object lock = new Object(); + + @SuppressWarnings("rawtypes") + public RefreshActive(WorkerData workerData) { + this.workerData = workerData; + + this.shutdown = workerData.getShutdown(); + this.monitorEnable = workerData.getMonitorEnable(); + this.conf = workerData.getStormConf(); + this.zkCluster = workerData.getZkCluster(); + this.topologyId = workerData.getTopologyId(); + this.frequence = + JStormUtils.parseInt(conf.get(Config.TASK_REFRESH_POLL_SECS), + 10); + } + + @Override + public void run() { + + try { + StatusType newTopologyStatus = StatusType.activate; + // /ZK-DIR/topology + StormBase base = zkCluster.storm_base(topologyId, this); + if (base == null) { + // @@@ normally the topology has been removed + LOG.warn("Failed to get StromBase from ZK of " + topologyId); + newTopologyStatus = StatusType.killed; + } else { + + newTopologyStatus = base.getStatus().getStatusType(); + } + + // Process the topology status change + StatusType oldTopologyStatus = workerData.getTopologyStatus(); + + if (newTopologyStatus.equals(oldTopologyStatus)) { + return; + } + + LOG.info("Old TopologyStatus:" + oldTopologyStatus + + ", new TopologyStatus:" + newTopologyStatus); + + List tasks = workerData.getShutdownTasks(); + if (tasks == null) { + LOG.info("Tasks aren't ready or begin to shutdown"); + return; + } + + if (newTopologyStatus.equals(StatusType.active)) { + for (TaskShutdownDameon task : tasks) { + task.active(); + } + } else if (newTopologyStatus.equals(StatusType.rebalancing)) { + // TODO + // But this may be updated in the future. + for (TaskShutdownDameon task : tasks) { + task.deactive(); + } + } else { + for (TaskShutdownDameon task : tasks) { + task.deactive(); + } + } + workerData.setTopologyStatus(newTopologyStatus); + + boolean newMonitorEnable = base.isEnableMonitor(); + boolean oldMonitorEnable = monitorEnable.get(); + if (newMonitorEnable != oldMonitorEnable) { + LOG.info("Change MonitorEnable from " + oldMonitorEnable + + " to " + newMonitorEnable); + monitorEnable.set(newMonitorEnable); + } + } catch (Exception e) { + // TODO Auto-generated catch block + LOG.error("Failed to get topology from ZK ", e); + return; + } + + } + + @Override + public Object getResult() { + return frequence; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/RefreshConnections.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/RefreshConnections.java new file mode 100644 index 000000000..48cc945c5 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/RefreshConnections.java @@ -0,0 +1,368 @@ +/** + * 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.daemon.worker; + +import java.io.FileNotFoundException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.messaging.IConnection; +import backtype.storm.messaging.IContext; +import backtype.storm.scheduler.WorkerSlot; + +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.schedule.Assignment; +import com.alibaba.jstorm.schedule.Assignment.AssignmentType; +import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; +import com.alibaba.jstorm.task.Task; +import com.alibaba.jstorm.task.TaskShutdownDameon; +import com.alibaba.jstorm.task.heartbeat.TaskHeartbeat; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.TimeUtils; + +/** + * + * Update current worker and other workers' zeroMQ connection. + * + * When worker shutdown/create, need update these connection + * + * @author yannian/Longda + * + */ +public class RefreshConnections extends RunnableCallback { + private static Logger LOG = LoggerFactory + .getLogger(RefreshConnections.class); + + private WorkerData workerData; + + private AtomicBoolean shutdown; + + @SuppressWarnings("rawtypes") + private Map conf; + + private StormClusterState zkCluster; + + private String topologyId; + + private Set outboundTasks; + + private ConcurrentHashMap nodeportSocket; + + private IContext context; + + private ConcurrentHashMap taskNodeport; + + private Integer frequence; + + private String supervisorId; + + private int taskTimeoutSecs; + + // private ReentrantReadWriteLock endpoint_socket_lock; + + @SuppressWarnings("rawtypes") + public RefreshConnections(WorkerData workerData) { + + this.workerData = workerData; + + this.shutdown = workerData.getShutdown(); + this.conf = workerData.getStormConf(); + this.zkCluster = workerData.getZkCluster(); + this.topologyId = workerData.getTopologyId(); + this.outboundTasks = workerData.getOutboundTasks(); + this.nodeportSocket = workerData.getNodeportSocket(); + this.context = workerData.getContext(); + this.taskNodeport = workerData.getTaskNodeport(); + this.supervisorId = workerData.getSupervisorId(); + + // 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 + public void run() { + + try { + // + // @@@ does lock need? + // endpoint_socket_lock.writeLock().lock(); + // + + synchronized (this) { + Assignment assignment = + zkCluster.assignment_info(topologyId, this); + if (assignment == null) { + String errMsg = "Failed to get Assignment of " + topologyId; + LOG.error(errMsg); + // throw new RuntimeException(errMsg); + return; + } + + // Compare the assignment timestamp of + // "jstorm_home/data/supervisor/topo-id/timestamp" + // with one in workerData to check if the topology code is + // updated. If so, the outbound + // task map should be updated accordingly. + try { + Long localAssignmentTS = + StormConfig.read_supervisor_topology_timestamp( + conf, topologyId); + if (localAssignmentTS.longValue() > workerData + .getAssignmentTs().longValue()) { + try { + if (assignment.getAssignmentType() == AssignmentType.Config) { + LOG.info("Get config reload request for " + topologyId); + // If config was updated, notify all tasks + List taskShutdowns = workerData.getShutdownTasks(); + Map newConf = StormConfig.read_supervisor_topology_conf(conf, topologyId); + workerData.getStormConf().putAll(newConf); + for (TaskShutdownDameon taskSD : taskShutdowns) { + taskSD.updateConf(newConf); + } + workerData.setAssignmentType(AssignmentType.Config); + } else { + Set addedTasks = getAddedTasks(assignment); + Set removedTasks = + getRemovedTasks(assignment); + + workerData.updateWorkerData(assignment); + + if (removedTasks.size() > 0) + shutdownTasks(removedTasks); + if (addedTasks.size() > 0) + createTasks(addedTasks); + + Set tmpOutboundTasks = + Worker.worker_output_tasks(workerData); + if (outboundTasks.equals(tmpOutboundTasks) == false) { + for (int taskId : tmpOutboundTasks) { + if (outboundTasks.contains(taskId) == false) + workerData + .addOutboundTaskStatusIfAbsent(taskId); + } + for (int taskId : workerData + .getOutboundTaskStatus().keySet()) { + if (tmpOutboundTasks.contains(taskId) == false) { + workerData + .removeOutboundTaskStatus(taskId); + } + } + workerData.setOutboundTasks(tmpOutboundTasks); + outboundTasks = tmpOutboundTasks; + } + workerData.setAssignmentType(AssignmentType.Assign); + } + + // If everything is OK, update the assignment TS. + // Then the tasks is + // going to update the related data. + workerData.setAssignmentTs(localAssignmentTS); + } catch (Exception e) { + LOG.warn("Failed to update worker data", e); + } + } + + } catch (FileNotFoundException e) { + LOG.warn( + "Failed to read supervisor topology timeStamp for " + + topologyId + " port=" + + workerData.getPort(), e); + } + + Set workers = assignment.getWorkers(); + if (workers == null) { + String errMsg = + "Failed to get taskToResource of " + topologyId; + LOG.error(errMsg); + return; + } + workerData.getWorkerToResource().addAll(workers); + + Map my_assignment = + new HashMap(); + + Map node = assignment.getNodeHost(); + + // only reserve outboundTasks + Set need_connections = new HashSet(); + + Set localTasks = new HashSet(); + + if (workers != null && outboundTasks != null) { + for (ResourceWorkerSlot worker : workers) { + if (supervisorId.equals(worker.getNodeId()) + && worker.getPort() == workerData.getPort()) + localTasks.addAll(worker.getTasks()); + for (Integer id : worker.getTasks()) { + if (outboundTasks.contains(id)) { + my_assignment.put(id, worker); + need_connections.add(worker); + } + } + } + } + taskNodeport.putAll(my_assignment); + workerData.setLocalTasks(localTasks); + + // get which connection need to be remove or add + Set current_connections = nodeportSocket.keySet(); + Set new_connections = new HashSet(); + Set remove_connections = new HashSet(); + + for (WorkerSlot node_port : need_connections) { + if (!current_connections.contains(node_port)) { + new_connections.add(node_port); + } + } + + for (WorkerSlot node_port : current_connections) { + if (!need_connections.contains(node_port)) { + remove_connections.add(node_port); + } + } + + // create new connection + for (WorkerSlot nodePort : new_connections) { + + String host = node.get(nodePort.getNodeId()); + + int port = nodePort.getPort(); + + IConnection conn = context.connect(topologyId, host, port); + + nodeportSocket.put(nodePort, conn); + + LOG.info("Add connection to " + nodePort); + } + + // close useless connection + for (WorkerSlot node_port : remove_connections) { + 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); + throw new RuntimeException(e); + } + + // finally { + // endpoint_socket_lock.writeLock().unlock(); + // } + + } + + @Override + public Object getResult() { + return frequence; + } + + private Set getAddedTasks(Assignment assignment) { + Set ret = new HashSet(); + try { + Set taskIds = + assignment.getCurrentWorkerTasks( + workerData.getSupervisorId(), workerData.getPort()); + for (Integer taskId : taskIds) { + if (!(workerData.getTaskids().contains(taskId))) + ret.add(taskId); + } + } catch (Exception e) { + LOG.warn("Failed to get added task list for" + + workerData.getTopologyId()); + ; + } + return ret; + } + + private Set getRemovedTasks(Assignment assignment) { + Set ret = new HashSet(); + try { + Set taskIds = + assignment.getCurrentWorkerTasks( + workerData.getSupervisorId(), workerData.getPort()); + for (Integer taskId : workerData.getTaskids()) { + if (!(taskIds.contains(taskId))) + ret.add(taskId); + } + } catch (Exception e) { + LOG.warn("Failed to get removed task list for" + + workerData.getTopologyId()); + ; + } + return ret; + } + + private void createTasks(Set tasks) { + for (Integer taskId : tasks) { + try { + TaskShutdownDameon shutdown = Task.mk_task(workerData, taskId); + workerData.addShutdownTask(shutdown); + } catch (Exception e) { + LOG.error("Failed to create task-" + taskId, e); + } + } + } + + private void shutdownTasks(Set tasks) { + for (Integer taskId : tasks) { + try { + List shutdowns = + workerData.getShutdownDaemonbyTaskIds(tasks); + for (TaskShutdownDameon shutdown : shutdowns) { + shutdown.shutdown(); + } + } catch (Exception e) { + LOG.error("Failed to shutdown task-" + taskId, e); + } + } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/ShutdownableDameon.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/ShutdownableDameon.java new file mode 100755 index 000000000..2006b05a6 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/ShutdownableDameon.java @@ -0,0 +1,27 @@ +/** + * 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.daemon.worker; + +import backtype.storm.daemon.Shutdownable; + +import com.alibaba.jstorm.cluster.DaemonCommon; + +public interface ShutdownableDameon extends Shutdownable, DaemonCommon, + Runnable { + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/State.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/State.java new file mode 100755 index 000000000..5e7a3bbe4 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/State.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.daemon.worker; + +/** + * Worker's status + * + * @author chenjun + * + */ +public enum State { + valid, disallowed, notStarted, timedOut; +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/VirtualPortDispatch.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/VirtualPortDispatch.java new file mode 100755 index 000000000..21dc37c31 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/VirtualPortDispatch.java @@ -0,0 +1,99 @@ +/** + * 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.daemon.worker; + +import java.util.concurrent.ConcurrentHashMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.messaging.IConnection; +import backtype.storm.messaging.TaskMessage; +import backtype.storm.utils.DisruptorQueue; + +import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.utils.DisruptorRunable; + +//import com.alibaba.jstorm.message.zeroMq.ISendConnection; + +/** + * Message dispatcher + * + * @author yannian/Longda + * + */ +public class VirtualPortDispatch extends DisruptorRunable { + private final static Logger LOG = LoggerFactory + .getLogger(VirtualPortDispatch.class); + + private ConcurrentHashMap deserializeQueues; + private IConnection recvConnection; + + public VirtualPortDispatch(WorkerData workerData, + IConnection recvConnection, DisruptorQueue recvQueue) { + super(recvQueue, MetricDef.DISPATCH_THREAD); + + this.recvConnection = recvConnection; + this.deserializeQueues = workerData.getDeserializeQueues(); + + } + + public void shutdownRecv() { + // don't need send shutdown command to every task + // due to every task has been shutdown by workerData.active + // at the same time queue has been fulll + // byte shutdownCmd[] = { TaskStatus.SHUTDOWN }; + // for (DisruptorQueue queue : deserializeQueues.values()) { + // + // queue.publish(shutdownCmd); + // } + + try { + recvConnection.close(); + } catch (Exception e) { + + } + recvConnection = null; + } + + @Override + public void shutdown() { + super.shutdown(); + LOG.info("Begin to shutdown VirtualPortDispatch"); + shutdownRecv(); + LOG.info("Successfully shudown VirtualPortDispatch"); + } + + @Override + public void handleEvent(Object event, boolean endOfBatch) throws Exception { + TaskMessage message = (TaskMessage) event; + + int task = message.task(); + + DisruptorQueue queue = deserializeQueues.get(task); + if (queue == null) { + LOG.warn("Received invalid message directed at port " + task + + ". Dropping..."); + return; + } + + queue.publish(message.message()); + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/Worker.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/Worker.java new file mode 100644 index 000000000..d5cf9c886 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/Worker.java @@ -0,0 +1,492 @@ +/** + * 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.daemon.worker; + +import java.io.BufferedReader; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +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 org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.generated.Grouping; +import backtype.storm.generated.StormTopology; +import backtype.storm.messaging.IConnection; +import backtype.storm.messaging.IContext; +import backtype.storm.task.TopologyContext; +import backtype.storm.utils.DisruptorQueue; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.callback.AsyncLoopThread; +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.Common; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.daemon.worker.hearbeat.SyncContainerHb; +import com.alibaba.jstorm.daemon.worker.hearbeat.WorkerHeartbeatRunable; +import com.alibaba.jstorm.metric.JStormMetricsReporter; +import com.alibaba.jstorm.task.Task; +import com.alibaba.jstorm.task.TaskShutdownDameon; +import com.alibaba.jstorm.task.heartbeat.TaskHeartbeatRunable; +import com.alibaba.jstorm.utils.JStormServerUtils; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.NetWorkUtils; +import com.alibaba.jstorm.utils.PathUtils; +import com.lmax.disruptor.WaitStrategy; +import com.lmax.disruptor.dsl.ProducerType; + +/** + * worker entrance + * + * @author yannian/Longda + * + */ +public class Worker { + + private static Logger LOG = LoggerFactory.getLogger(Worker.class); + + /** + * Why need workerData, it is for thread comeptition + */ + private WorkerData workerData; + + @SuppressWarnings({ "rawtypes", "unchecked" }) + public Worker(Map conf, IContext context, String topology_id, + String supervisor_id, int port, String worker_id, String jar_path) + throws Exception { + + workerData = + new WorkerData(conf, context, topology_id, supervisor_id, port, + worker_id, jar_path); + + } + + /** + * get current task's output task list + * + * @param tasks_component + * @param mk_topology_context + * @param task_ids + * @throws Exception + */ + public static Set worker_output_tasks(WorkerData workerData) { + + ContextMaker context_maker = workerData.getContextMaker(); + Set task_ids = workerData.getTaskids(); + StormTopology topology = workerData.getSysTopology(); + + Set rtn = new HashSet(); + + for (Integer taskid : task_ids) { + TopologyContext context = + context_maker.makeTopologyContext(topology, taskid, null); + + // > + Map> targets = + context.getThisTargets(); + for (Map e : targets.values()) { + for (String componentId : e.keySet()) { + List tasks = + context.getComponentTasks(componentId); + rtn.addAll(tasks); + } + } + } + + return rtn; + } + + private RefreshConnections makeRefreshConnections() { + + // get output streams of every task + Set outboundTasks = worker_output_tasks(workerData); + + workerData.initOutboundTaskStatus(outboundTasks); + workerData.setOutboundTasks(outboundTasks); + + RefreshConnections refresh_connections = new RefreshConnections(workerData); + + return refresh_connections; + } + + private List createTasks() throws Exception { + List shutdowntasks = + new ArrayList(); + + Set taskids = workerData.getTaskids(); + + for (int taskid : taskids) { + + TaskShutdownDameon t = Task.mk_task(workerData, taskid); + + shutdowntasks.add(t); + } + + return shutdowntasks; + } + + @Deprecated + private DisruptorQueue startDispatchDisruptor() { + Map stormConf = workerData.getStormConf(); + + int queue_size = + Utils.getInt( + stormConf.get(Config.TOPOLOGY_TRANSFER_BUFFER_SIZE), + 1024); + WaitStrategy waitStrategy = + (WaitStrategy) JStormUtils.createDisruptorWaitStrategy(stormConf); + DisruptorQueue recvQueue = + DisruptorQueue.mkInstance("Dispatch", ProducerType.MULTI, + queue_size, waitStrategy); + // stop consumerStarted + recvQueue.consumerStarted(); + + return recvQueue; + } + + private void startDispatchThread() { + // remove dispatch thread, send tuple directly from nettyserver + //startDispatchDisruptor(); + + IContext context = workerData.getContext(); + String topologyId = workerData.getTopologyId(); + + IConnection recvConnection = + context.bind(topologyId, workerData.getPort(), workerData.getDeserializeQueues()); + + workerData.setRecvConnection(recvConnection); + } + + public WorkerShutdown execute() throws Exception { + List threads = new ArrayList(); + + startDispatchThread(); + + // create client before create task + // so create client connection before create task + // refresh connection + RefreshConnections refreshConn = makeRefreshConnections(); + AsyncLoopThread refreshconn = + new AsyncLoopThread(refreshConn, false, Thread.MIN_PRIORITY, + true); + threads.add(refreshconn); + + // refresh ZK active status + RefreshActive refreshZkActive = new RefreshActive(workerData); + AsyncLoopThread refreshzk = + new AsyncLoopThread(refreshZkActive, false, + Thread.MIN_PRIORITY, true); + threads.add(refreshzk); + + // Sync heartbeat to Apsara Container + AsyncLoopThread syncContainerHbThread = + SyncContainerHb.mkWorkerInstance(workerData.getStormConf()); + if (syncContainerHbThread != null) { + threads.add(syncContainerHbThread); + } + + JStormMetricsReporter metricReporter = + new JStormMetricsReporter(workerData); + AsyncLoopThread metricThread = new AsyncLoopThread(metricReporter); + threads.add(metricThread); + + // create task heartbeat + TaskHeartbeatRunable taskHB = new TaskHeartbeatRunable(workerData); + AsyncLoopThread taskHBThread = new AsyncLoopThread(taskHB); + threads.add(taskHBThread); + + // refresh hearbeat to Local dir + RunnableCallback heartbeat_fn = new WorkerHeartbeatRunable(workerData); + AsyncLoopThread hb = + new AsyncLoopThread(heartbeat_fn, false, null, + Thread.NORM_PRIORITY, true); + threads.add(hb); + + // shutdown task callbacks + List shutdowntasks = createTasks(); + workerData.setShutdownTasks(shutdowntasks); + + return new WorkerShutdown(workerData, threads); + + } + + /** + * create worker instance and run it + * + * @param conf + * @param mq_context + * @param topology_id + * @param supervisor_id + * @param port + * @param worker_id + * @return + * @throws Exception + */ + @SuppressWarnings("rawtypes") + public static WorkerShutdown mk_worker(Map conf, IContext context, + String topology_id, String supervisor_id, int port, + String worker_id, String jar_path) throws Exception { + + StringBuilder sb = new StringBuilder(); + sb.append("topologyId:" + topology_id + ", "); + sb.append("port:" + port + ", "); + sb.append("workerId:" + worker_id + ", "); + sb.append("jarPath:" + jar_path + "\n"); + + LOG.info("Begin to run worker:" + sb.toString()); + + Worker w = + new Worker(conf, context, topology_id, supervisor_id, port, + worker_id, jar_path); + + w.redirectOutput(); + + return w.execute(); + } + + public void redirectOutput() { + + if (System.getenv("REDIRECT") == null + || !System.getenv("REDIRECT").equals("true")) { + return; + } + + String DEFAULT_OUT_TARGET_FILE = JStormUtils.getLogFileName(); + if (DEFAULT_OUT_TARGET_FILE == null) { + DEFAULT_OUT_TARGET_FILE = "/dev/null"; + } else { + DEFAULT_OUT_TARGET_FILE += ".out"; + } + + String outputFile = + ConfigExtension.getWorkerRedirectOutputFile(workerData + .getStormConf()); + if (outputFile == null) { + outputFile = DEFAULT_OUT_TARGET_FILE; + } else { + try { + File file = new File(outputFile); + if (file.exists() == false) { + PathUtils.touch(outputFile); + } else { + if (file.isDirectory() == true) { + LOG.warn("Failed to write " + outputFile); + outputFile = DEFAULT_OUT_TARGET_FILE; + } else if (file.canWrite() == false) { + LOG.warn("Failed to write " + outputFile); + outputFile = DEFAULT_OUT_TARGET_FILE; + } + } + + } catch (Exception e) { + LOG.warn("Failed to touch " + outputFile, e); + outputFile = DEFAULT_OUT_TARGET_FILE; + } + } + + try { + JStormUtils.redirectOutput(outputFile); + } catch (Exception e) { + LOG.warn("Failed to redirect to " + outputFile, e); + } + + } + + /** + * Have one problem if the worker's start parameter length is longer than + * 4096, ps -ef|grep com.alibaba.jstorm.daemon.worker.Worker can't find + * worker + * + * @param port + */ + + public static List getOldPortPids(String port) { + String currPid = JStormUtils.process_pid(); + + List ret = new ArrayList(); + + StringBuilder sb = new StringBuilder(); + + sb.append("ps -Af "); + // sb.append(" | grep "); + // sb.append(Worker.class.getName()); + // sb.append(" |grep "); + // sb.append(port); + // sb.append(" |grep -v grep"); + + try { + LOG.info("Begin to execute " + sb.toString()); + Process process = + JStormUtils.launch_process(sb.toString(), + new HashMap(), false); + + // Process process = Runtime.getRuntime().exec(sb.toString()); + + InputStream stdin = process.getInputStream(); + BufferedReader reader = + new BufferedReader(new InputStreamReader(stdin)); + + JStormUtils.sleepMs(1000); + + // if (process.exitValue() != 0) { + // LOG.info("Failed to execute " + sb.toString()); + // return null; + // } + + String str; + while ((str = reader.readLine()) != null) { + if (StringUtils.isBlank(str)) { + // LOG.info(str + " is Blank"); + continue; + } + + // LOG.info("Output:" + str); + if (str.contains(Worker.class.getName()) == false) { + continue; + } else if (str.contains(port) == false) { + continue; + } + + LOG.info("Find :" + str); + + String[] fields = StringUtils.split(str); + + boolean find = false; + int i = 0; + for (; i < fields.length; i++) { + String field = fields[i]; + LOG.debug("Filed, " + i + ":" + field); + + if (field.contains(Worker.class.getName()) == true) { + if (i + 3 >= fields.length) { + LOG.info("Failed to find port "); + + } else if (fields[i + 3].equals(String.valueOf(port))) { + find = true; + } + + break; + } + } + + if (find == false) { + LOG.info("No old port worker"); + continue; + } + + if (fields.length >= 2) { + try { + if (currPid.equals(fields[1])) { + LOG.info("Skip kill myself"); + continue; + } + + Integer pid = Integer.valueOf(fields[1]); + + LOG.info("Find one process :" + pid.toString()); + ret.add(pid); + } catch (Exception e) { + LOG.error(e.getMessage(), e); + continue; + } + } + + } + + return ret; + } catch (IOException e) { + LOG.info("Failed to execute " + sb.toString()); + return ret; + } catch (Exception e) { + LOG.info(e.getMessage(), e); + return ret; + } + } + + public static void killOldWorker(String port) { + + List oldPids = getOldPortPids(port); + for (Integer pid : oldPids) { + + JStormUtils.kill(pid); + } + + } + + /** + * worker entrance + * + * @param args + */ + @SuppressWarnings("rawtypes") + public static void main(String[] args) { + if (args.length < 5) { + StringBuilder sb = new StringBuilder(); + sb.append("The length of args is less than 5 "); + for (String arg : args) { + sb.append(arg + " "); + } + LOG.error(sb.toString()); + System.exit(-1); + } + + StringBuilder sb = new StringBuilder(); + + 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]; + + killOldWorker(port_str); + + Map conf = Utils.readStormConfig(); + StormConfig.validate_distributed_mode(conf); + + 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, jar_path); + sd.join(); + + LOG.info("Successfully shutdown worker " + sb.toString()); + } catch (Throwable e) { + String errMsg = "Failed to create worker, " + sb.toString(); + LOG.error(errMsg, e); + JStormUtils.halt_process(-1, errMsg); + } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerData.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerData.java new file mode 100644 index 000000000..da6907012 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerData.java @@ -0,0 +1,615 @@ +/** + * 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.daemon.worker; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.URL; +import java.security.InvalidParameterException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.generated.InvalidTopologyException; +import backtype.storm.generated.StormTopology; +import backtype.storm.messaging.IConnection; +import backtype.storm.messaging.IContext; +import backtype.storm.messaging.TransportFactory; +import backtype.storm.scheduler.WorkerSlot; +import backtype.storm.utils.DisruptorQueue; +import backtype.storm.utils.Utils; +import backtype.storm.utils.WorkerClassLoader; + +import com.alibaba.jstorm.callback.AsyncLoopDefaultKill; +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.Cluster; +import com.alibaba.jstorm.cluster.ClusterState; +import com.alibaba.jstorm.cluster.Common; +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.common.metric.window.Metric; +import com.alibaba.jstorm.daemon.nimbus.StatusType; +import com.alibaba.jstorm.daemon.worker.timer.TimerTrigger; +import com.alibaba.jstorm.message.netty.ControlMessage; +import com.alibaba.jstorm.schedule.Assignment; +import com.alibaba.jstorm.schedule.Assignment.AssignmentType; +import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; +import com.alibaba.jstorm.task.TaskShutdownDameon; +import com.alibaba.jstorm.utils.JStormServerUtils; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.zk.ZkTool; +import com.lmax.disruptor.WaitStrategy; +import com.lmax.disruptor.dsl.ProducerType; + + +public class WorkerData { + + private static Logger LOG = LoggerFactory.getLogger(WorkerData.class); + + // system configuration + + private Map conf; + // worker configuration + + private Map stormConf; + + // message queue + private IContext context; + + private final String topologyId; + private final String supervisorId; + private final Integer port; + private final String workerId; + // worker status :active/shutdown + private AtomicBoolean shutdown; + private AtomicBoolean monitorEnable; + + // Topology status + private StatusType topologyStatus; + + // ZK interface + private ClusterState zkClusterstate; + private StormClusterState zkCluster; + + // running taskId list in current worker + private Set taskids; + + // connection to other workers + private ConcurrentHashMap nodeportSocket; + // + private ConcurrentHashMap taskNodeport; + + private ConcurrentSkipListSet workerToResource; + + private volatile Set outboundTasks; + private Set localTasks; + + private ConcurrentHashMap innerTaskTransfer; + private ConcurrentHashMap deserializeQueues; + + // + private ConcurrentHashMap tasksToComponent; + + private ConcurrentHashMap> componentToSortedTasks; + + private Map defaultResources; + private Map userResources; + private Map executorData; + private Map registeredMetrics; + + // raw topology is deserialized from local jar + // it doesn't contain acker + private StormTopology rawTopology; + // sys topology is the running topology in the worker + // it contain ackers + private StormTopology sysTopology; + + private ContextMaker contextMaker; + + // shutdown woker entrance + private final AsyncLoopDefaultKill workHalt = new AsyncLoopDefaultKill(); + + // sending tuple's queue + // private LinkedBlockingQueue transferQueue; + private DisruptorQueue transferQueue; + + private DisruptorQueue sendingQueue; + + private List shutdownTasks; + + private ConcurrentHashMap outTaskStatus; // true => active + + public static final int THREAD_POOL_NUM = 4; + private ScheduledExecutorService threadPool; + + private volatile Long assignmentTS; // Assignment timeStamp. The time of + // last update of assignment + private volatile AssignmentType assignmentType; + + private IConnection recvConnection; + + @SuppressWarnings({ "rawtypes", "unchecked" }) + public WorkerData(Map conf, IContext context, String topology_id, + String supervisor_id, int port, String worker_id, String jar_path) + throws Exception { + + this.conf = conf; + this.context = context; + this.topologyId = topology_id; + this.supervisorId = supervisor_id; + this.port = port; + this.workerId = worker_id; + + this.shutdown = new AtomicBoolean(false); + + this.monitorEnable = new AtomicBoolean(true); + this.topologyStatus = StatusType.active; + + if (StormConfig.cluster_mode(conf).equals("distributed")) { + String pidDir = StormConfig.worker_pids_root(conf, worker_id); + JStormServerUtils.createPid(pidDir); + } + + // create zk interface + this.zkClusterstate = ZkTool.mk_distributed_cluster_state(conf); + this.zkCluster = Cluster.mk_storm_cluster_state(zkClusterstate); + + Map rawConf = + StormConfig.read_supervisor_topology_conf(conf, topology_id); + this.stormConf = new HashMap(); + this.stormConf.putAll(conf); + this.stormConf.putAll(rawConf); + + ConfigExtension.setLocalSupervisorId(stormConf, supervisorId); + ConfigExtension.setLocalWorkerId(stormConf, workerId); + ConfigExtension.setLocalWorkerPort(stormConf, port); + ControlMessage.setPort(port); + Metric.setEnable(ConfigExtension.isEnablePerformanceMetrics(stormConf)); + + LOG.info("Worker Configuration " + stormConf); + + try { + + boolean enableClassloader = + ConfigExtension.isEnableTopologyClassLoader(stormConf); + boolean enableDebugClassloader = + ConfigExtension.isEnableClassloaderDebug(stormConf); + + if (jar_path == null && enableClassloader == true + && !conf.get(Config.STORM_CLUSTER_MODE).equals("local")) { + 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(); + for (String path : paths) { + if (StringUtils.isBlank(path)) + continue; + URL url = new URL("File:" + path); + urls.add(url); + } + 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); + throw new InvalidParameterException(); + } + + if (this.context == null) { + this.context = TransportFactory.makeContext(stormConf); + } + + boolean disruptorUseSleep = + ConfigExtension.isDisruptorUseSleep(stormConf); + DisruptorQueue.setUseSleep(disruptorUseSleep); + boolean isLimited = + ConfigExtension.getTopologyBufferSizeLimited(stormConf); + DisruptorQueue.setLimited(isLimited); + LOG.info("Disruptor use sleep:" + disruptorUseSleep + ", limited size:" + + isLimited); + + // this.transferQueue = new LinkedBlockingQueue(); + int buffer_size = + Utils.getInt(conf.get(Config.TOPOLOGY_TRANSFER_BUFFER_SIZE)); + WaitStrategy waitStrategy = + (WaitStrategy) JStormUtils.createDisruptorWaitStrategy(conf); + this.transferQueue = + DisruptorQueue.mkInstance("TotalTransfer", ProducerType.MULTI, + buffer_size, waitStrategy); + this.transferQueue.consumerStarted(); + this.sendingQueue = + DisruptorQueue.mkInstance("TotalSending", ProducerType.MULTI, + buffer_size, waitStrategy); + this.sendingQueue.consumerStarted(); + + this.nodeportSocket = new ConcurrentHashMap(); + this.taskNodeport = new ConcurrentHashMap(); + this.workerToResource = new ConcurrentSkipListSet(); + this.innerTaskTransfer = + new ConcurrentHashMap(); + this.deserializeQueues = + new ConcurrentHashMap(); + this.tasksToComponent = new ConcurrentHashMap(); + this.componentToSortedTasks = + new ConcurrentHashMap>(); + + Assignment assignment = zkCluster.assignment_info(topologyId, null); + if (assignment == null) { + String errMsg = "Failed to get Assignment of " + topologyId; + LOG.error(errMsg); + throw new RuntimeException(errMsg); + } + workerToResource.addAll(assignment.getWorkers()); + + // get current worker's task list + + this.taskids = assignment.getCurrentWorkerTasks(supervisorId, port); + if (taskids.size() == 0) { + throw new RuntimeException("No tasks running current workers"); + } + LOG.info("Current worker taskList:" + taskids); + + // deserialize topology code from local dir + rawTopology = + StormConfig.read_supervisor_topology_code(conf, topology_id); + sysTopology = Common.system_topology(stormConf, rawTopology); + + generateMaps(); + + contextMaker = new ContextMaker(this); + + outTaskStatus = new ConcurrentHashMap(); + + threadPool = Executors.newScheduledThreadPool(THREAD_POOL_NUM); + TimerTrigger.setScheduledExecutorService(threadPool); + + try { + Long tmp = + StormConfig.read_supervisor_topology_timestamp(conf, + topology_id); + assignmentTS = (tmp == null ? System.currentTimeMillis() : tmp); + } catch (FileNotFoundException e) { + assignmentTS = System.currentTimeMillis(); + } + + outboundTasks = new HashSet(); + + LOG.info("Successfully create WorkerData"); + + } + + /** + * private ConcurrentHashMap taskNodeport; private + * HashMap tasksToComponent; private Map> componentToSortedTasks; private Map> componentToStreamToFields; private Map + * defaultResources; private Map userResources; private + * Map executorData; private Map registeredMetrics; + * + * @throws Exception + */ + private void generateMaps() throws Exception { + updateTaskComponentMap(); + + this.defaultResources = new HashMap(); + this.userResources = new HashMap(); + this.executorData = new HashMap(); + this.registeredMetrics = new HashMap(); + } + + public Map getConf() { + return conf; + } + + public AtomicBoolean getShutdown() { + return shutdown; + } + + public StatusType getTopologyStatus() { + return topologyStatus; + } + + public void setTopologyStatus(StatusType topologyStatus) { + this.topologyStatus = topologyStatus; + } + + public Map getStormConf() { + return stormConf; + } + + public IContext getContext() { + return context; + } + + public String getTopologyId() { + return topologyId; + } + + public String getSupervisorId() { + return supervisorId; + } + + public Integer getPort() { + return port; + } + + public String getWorkerId() { + return workerId; + } + + public ClusterState getZkClusterstate() { + return zkClusterstate; + } + + public StormClusterState getZkCluster() { + return zkCluster; + } + + public Set getTaskids() { + return taskids; + } + + public ConcurrentHashMap getNodeportSocket() { + return nodeportSocket; + } + + public ConcurrentHashMap getTaskNodeport() { + return taskNodeport; + } + + public ConcurrentSkipListSet getWorkerToResource() { + return workerToResource; + } + + public ConcurrentHashMap getInnerTaskTransfer() { + return innerTaskTransfer; + } + + public ConcurrentHashMap getDeserializeQueues() { + return deserializeQueues; + } + + public ConcurrentHashMap getTasksToComponent() { + return tasksToComponent; + } + + public StormTopology getRawTopology() { + return rawTopology; + } + + public StormTopology getSysTopology() { + return sysTopology; + } + + public ContextMaker getContextMaker() { + return contextMaker; + } + + public AsyncLoopDefaultKill getWorkHalt() { + return workHalt; + } + + public DisruptorQueue getTransferQueue() { + return transferQueue; + } + + // public LinkedBlockingQueue getTransferQueue() { + // return transferQueue; + // } + + public DisruptorQueue getSendingQueue() { + return sendingQueue; + } + + public Map> getComponentToSortedTasks() { + return componentToSortedTasks; + } + + public Map getDefaultResources() { + return defaultResources; + } + + public Map getUserResources() { + return userResources; + } + + public Map getExecutorData() { + return executorData; + } + + public Map getRegisteredMetrics() { + return registeredMetrics; + } + + public List getShutdownTasks() { + return shutdownTasks; + } + + public void setShutdownTasks(List shutdownTasks) { + this.shutdownTasks = shutdownTasks; + } + + public void addShutdownTask(TaskShutdownDameon shutdownTask) { + this.shutdownTasks.add(shutdownTask); + } + + public List getShutdownDaemonbyTaskIds( + Set taskIds) { + List ret = new ArrayList(); + for (TaskShutdownDameon shutdown : shutdownTasks) { + if (taskIds.contains(shutdown.getTaskId())) + ret.add(shutdown); + } + return ret; + } + + public Set getLocalTasks() { + return localTasks; + } + + public void setLocalTasks(Set localTasks) { + this.localTasks = localTasks; + } + + public void initOutboundTaskStatus(Set outboundTasks) { + for (Integer taskId : outboundTasks) { + outTaskStatus.put(taskId, false); + } + } + + public Map getOutboundTaskStatus() { + return outTaskStatus; + } + + public void addOutboundTaskStatusIfAbsent(Integer taskId) { + outTaskStatus.putIfAbsent(taskId, false); + } + + public void removeOutboundTaskStatus(Integer taskId) { + outTaskStatus.remove(taskId); + } + + 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; + } + + public void setAssignmentTs(Long time) { + assignmentTS = time; + } + + public Long getAssignmentTs() { + return assignmentTS; + } + + public void setAssignmentType(AssignmentType type) { + this.assignmentType = type; + } + + public AssignmentType getAssignmentType() { + return assignmentType; + } + + public void updateWorkerData(Assignment assignment) throws Exception { + updateTaskIds(assignment); + updateTaskComponentMap(); + updateStormTopology(); + } + + public void updateTaskIds(Assignment assignment) { + this.taskids.clear(); + this.taskids.addAll(assignment + .getCurrentWorkerTasks(supervisorId, port)); + } + + public void setOutboundTasks(Set outboundTasks) { + this.outboundTasks = outboundTasks; + } + + public Set getOutboundTasks() { + return outboundTasks; + } + + private void updateTaskComponentMap() throws Exception { + Map tmp = Common.getTaskToComponent( + Cluster.get_all_taskInfo(zkCluster, topologyId)); + + this.tasksToComponent.putAll(tmp); + LOG.info("Updated tasksToComponentMap:" + tasksToComponent); + + this.componentToSortedTasks.putAll(JStormUtils.reverse_map(tmp)); + for (java.util.Map.Entry> entry : componentToSortedTasks + .entrySet()) { + List tasks = entry.getValue(); + + Collections.sort(tasks); + } + } + + private void updateStormTopology() { + StormTopology rawTmp = null; + StormTopology sysTmp = null; + + try { + rawTmp = + StormConfig.read_supervisor_topology_code(conf, topologyId); + sysTmp = Common.system_topology(stormConf, rawTopology); + } catch (IOException e) { + LOG.error("Failed to read supervisor topology code for " + + topologyId, e); + return; + } catch (InvalidTopologyException e) { + LOG.error("Failed to update sysTopology for " + topologyId, e); + return; + } + + updateTopology(rawTopology, rawTmp); + updateTopology(sysTopology, sysTmp); + } + + private void updateTopology(StormTopology oldTopology, + StormTopology newTopology) { + oldTopology.set_bolts(newTopology.get_bolts()); + oldTopology.set_spouts(newTopology.get_spouts()); + oldTopology.set_state_spouts(newTopology.get_state_spouts()); + } + + public AtomicBoolean getMonitorEnable() { + return monitorEnable; + } + + public IConnection getRecvConnection() { + return recvConnection; + } + + public void setRecvConnection(IConnection recvConnection) { + this.recvConnection = recvConnection; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerHeartbeat.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerHeartbeat.java new file mode 100755 index 000000000..d8ec622e9 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerHeartbeat.java @@ -0,0 +1,89 @@ +/** + * 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.daemon.worker; + +import java.io.Serializable; +import java.util.HashSet; +import java.util.Set; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; + +/** + * Worker's Heartbeat data woker will update the object to + * /LOCAL-DIR/workers/${woker-id}/heartbeats + * + * @author yannian/Longda + * + */ +public class WorkerHeartbeat implements Serializable { + + private static final long serialVersionUID = -914166726205534892L; + private int timeSecs; + private String topologyId; + private Set taskIds; + private Integer port; + + public WorkerHeartbeat(int timeSecs, String topologyId, + Set taskIds, Integer port) { + + this.timeSecs = timeSecs; + this.topologyId = topologyId; + this.taskIds = new HashSet(taskIds); + this.port = port; + + } + + public int getTimeSecs() { + return timeSecs; + } + + public void setTimeSecs(int timeSecs) { + this.timeSecs = timeSecs; + } + + public String getTopologyId() { + return topologyId; + } + + public void setTopologyId(String topologyId) { + this.topologyId = topologyId; + } + + public Set getTaskIds() { + return taskIds; + } + + public void setTaskIds(Set taskIds) { + this.taskIds = new HashSet(taskIds); + } + + public Integer getPort() { + return port; + } + + public void setPort(Integer port) { + this.port = port; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerShutdown.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerShutdown.java new file mode 100755 index 000000000..0691feee5 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerShutdown.java @@ -0,0 +1,186 @@ +/** + * 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.daemon.worker; + +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.messaging.IConnection; +import backtype.storm.messaging.IContext; +import backtype.storm.scheduler.WorkerSlot; + +import com.alibaba.jstorm.callback.AsyncLoopRunnable; +import com.alibaba.jstorm.callback.AsyncLoopThread; +import com.alibaba.jstorm.cluster.ClusterState; +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.task.TaskShutdownDameon; +import com.alibaba.jstorm.utils.JStormUtils; + +/** + * Shutdown worker + * + * @author yannian/Longda + * + */ +public class WorkerShutdown implements ShutdownableDameon { + private static Logger LOG = LoggerFactory.getLogger(WorkerShutdown.class); + + public static final String HOOK_SIGNAL = "USR2"; + + private List shutdowntasks; + private AtomicBoolean shutdown; + private ConcurrentHashMap nodeportSocket; + private IContext context; + private List threads; + private StormClusterState zkCluster; + private ClusterState cluster_state; + private ScheduledExecutorService threadPool; + private IConnection recvConnection; + + // active nodeportSocket context zkCluster zkClusterstate + public WorkerShutdown(WorkerData workerData, List _threads) { + + this.shutdowntasks = workerData.getShutdownTasks(); + this.threads = _threads; + + this.shutdown = workerData.getShutdown(); + this.nodeportSocket = workerData.getNodeportSocket(); + this.context = workerData.getContext(); + this.zkCluster = workerData.getZkCluster(); + this.cluster_state = workerData.getZkClusterstate(); + this.threadPool = workerData.getThreadPool(); + this.recvConnection = workerData.getRecvConnection(); + + Runtime.getRuntime().addShutdownHook(new Thread(this)); + + // PreCleanupTasks preCleanupTasks = new PreCleanupTasks(); + // // install signals + // Signal sig = new Signal(HOOK_SIGNAL); + // Signal.handle(sig, preCleanupTasks); + } + + @Override + public void shutdown() { + + if (shutdown.getAndSet(true) == true) { + LOG.info("Worker has been shutdown already"); + return; + } + + if(recvConnection != null) { + recvConnection.close(); + } + + AsyncLoopRunnable.getShutdown().set(true); + threadPool.shutdown(); + + + // shutdown tasks + for (ShutdownableDameon task : shutdowntasks) { + task.shutdown(); + } + + // shutdown worker's demon thread + // refreshconn, refreshzk, hb, drainer + for (AsyncLoopThread t : threads) { + LOG.info("Begin to shutdown " + t.getThread().getName()); + t.cleanup(); + JStormUtils.sleepMs(100); + t.interrupt(); + // try { + // t.join(); + // } catch (InterruptedException e) { + // LOG.error("join thread", e); + // } + LOG.info("Successfully " + t.getThread().getName()); + } + + // send data to close connection + for (WorkerSlot k : nodeportSocket.keySet()) { + IConnection value = nodeportSocket.get(k); + value.close(); + } + + context.term(); + + // close ZK client + try { + zkCluster.disconnect(); + cluster_state.close(); + } catch (Exception e) { + // TODO Auto-generated catch block + LOG.info("Shutdown error,", e); + } + + JStormUtils.halt_process(0, "!!!Shutdown!!!"); + } + + public void join() throws InterruptedException { + for (TaskShutdownDameon task : shutdowntasks) { + task.join(); + } + for (AsyncLoopThread t : threads) { + t.join(); + } + + } + + public boolean waiting() { + Boolean isExistsWait = false; + for (ShutdownableDameon task : shutdowntasks) { + if (task.waiting()) { + isExistsWait = true; + break; + } + } + for (AsyncLoopThread thr : threads) { + if (thr.isSleeping()) { + isExistsWait = true; + break; + } + } + return isExistsWait; + } + + @Override + public void run() { + // TODO Auto-generated method stub + shutdown(); + } + + // class PreCleanupTasks implements SignalHandler { + // + // @Override + // public void handle(Signal arg0) { + // LOG.info("Receive " + arg0.getName() + ", begin to do pre_cleanup job"); + // + // for (ShutdownableDameon task : shutdowntasks) { + // task.shutdown(); + // } + // + // LOG.info("Successfully do pre_cleanup job"); + // } + // + // } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/hearbeat/SyncContainerHb.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/hearbeat/SyncContainerHb.java new file mode 100755 index 000000000..783f584a7 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/hearbeat/SyncContainerHb.java @@ -0,0 +1,386 @@ +/** + * 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.daemon.worker.hearbeat; + +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.callback.AsyncLoopThread; +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.PathUtils; + +public class SyncContainerHb extends RunnableCallback { + private final static Logger LOG = LoggerFactory + .getLogger(SyncContainerHb.class); + + private String readDir; + private String writeDir; + private int timeoutSeconds = 60; + private int frequence = 10; + private int reserverNum = 10; + private int noContainerHbTimes = 0; + private boolean isFirstRead = true; + private static final int SECOND_MILLISCOND = 1000; + private static final int MAX_NO_CONTAINER_HB_TIMES = 30; + + public void removeOld(List fileList, String dir) { + if (fileList.size() <= reserverNum) { + // don't need remove old files + return; + } + + int removeNum = fileList.size() - reserverNum; + for (int i = 0; i < removeNum; i++) { + String fileName = fileList.get(i); + + String filePath = dir + File.separator + fileName; + try { + PathUtils.rmpath(filePath); + } catch (Exception e) { + LOG.error("Failed to delete " + filePath, e); + } + LOG.info("Remove hearbeat file " + filePath); + } + } + + public void checkNoContainerHbTimes() { + noContainerHbTimes++; + if (noContainerHbTimes >= MAX_NO_CONTAINER_HB_TIMES) { + LOG.info("It's long time no container hearbeat"); + throw new RuntimeException("It's long time no container hearbeat"); + } + } + + public void handlReadDir() { + if (StringUtils.isBlank(readDir) == true) { + return; + } + + File file = new File(readDir); + if (file.exists() == false) { + LOG.info(readDir + " doesn't exist right now"); + checkNoContainerHbTimes(); + return; + } else if (file.isDirectory() == false) { + String msg = readDir + " isn't dir"; + LOG.error(msg); + throw new RuntimeException(msg); + } + + String[] files = file.list(); + if (files.length == 0) { + LOG.info(readDir + " doesn't contain hearbeat files right now"); + checkNoContainerHbTimes(); + return; + } + + noContainerHbTimes = 0; + List fileList = JStormUtils.mk_list(files); + Collections.sort(fileList); + + // removeOld(fileList); + + String biggest = fileList.get(fileList.size() - 1); + + long now = System.currentTimeMillis() / SECOND_MILLISCOND; + long hb = 0; + + try { + hb = Long.valueOf(biggest); + } catch (Exception e) { + LOG.info("Heartbeat file " + biggest + + " isn't a valid file, remove it"); + + String path = readDir + File.separator + biggest; + try { + PathUtils.rmpath(path); + } catch (Exception e1) { + LOG.error("Failed to delete " + path, e1); + } + + } + + if (now - hb > timeoutSeconds) { + if (isFirstRead == true) { + checkNoContainerHbTimes(); + return; + } + + StringBuilder sb = new StringBuilder(); + + sb.append("It's long time no container's hearbeat, "); + sb.append("ContainerDir:").append(readDir); + sb.append(",last hearbeat:").append(biggest); + LOG.error(sb.toString()); + + throw new RuntimeException(sb.toString()); + } else { + isFirstRead = false; + LOG.info("Receive container hearbeat " + biggest); + } + + } + + public void handleWriteDir() { + if (StringUtils.isBlank(writeDir) == true) { + return; + } + + String seconds = + String.valueOf(System.currentTimeMillis() / SECOND_MILLISCOND); + + String path = writeDir + File.separator + seconds; + + try { + PathUtils.touch(path); + LOG.info("Successfully touch " + path); + } catch (IOException e) { + LOG.error("Failed to touch " + path, e); + throw new RuntimeException("Failed to touch " + path); + } + + File file = new File(writeDir); + String[] files = file.list(); + if (files.length == 0) { + LOG.info(readDir + " doesn't contain hearbeat files right now"); + return; + } + + List fileList = JStormUtils.mk_list(files); + Collections.sort(fileList); + + removeOld(fileList, writeDir); + + } + + @Override + public void run() { + + handleWriteDir(); + + handlReadDir(); + + } + + @Override + public void shutdown() { + frequence = -1; + LOG.info("Shutdown sync container thread"); + } + + public Object getResult() { + return frequence; + } + + public String getReadDir() { + return readDir; + } + + public void resetReadHeatbeats() { + File file = new File(readDir); + + if (file.exists() == false) { + LOG.info("Read hearbeat directory hasn't been created " + readDir); + return; + } else if (file.isDirectory() == false) { + LOG.error(readDir + " isn't a directory "); + throw new RuntimeException(readDir + " isn't a directory "); + } + + String[] files = file.list(); + for (String fileName : files) { + String path = readDir + File.separator + fileName; + + try { + PathUtils.rmr(path); + } catch (IOException e) { + // TODO Auto-generated catch block + LOG.error("Failed to remove " + path, e); + } + } + + LOG.info("Successfully reset read heatbeats " + readDir); + } + + public void setReadDir(String readDir) { + this.readDir = readDir; + if (StringUtils.isBlank(readDir) == true) { + LOG.warn("ReadDir is empty"); + return; + } else { + LOG.info("ReadDir is " + readDir); + } + + } + + public int getTimeoutSeconds() { + return timeoutSeconds; + } + + public void setTimeoutSeconds(int timeoutSeconds) { + this.timeoutSeconds = timeoutSeconds; + } + + public int getFrequence() { + return frequence; + } + + public void setFrequence(int frequence) { + this.frequence = frequence; + } + + public String getWriteDir() { + return writeDir; + } + + public void setWriteDir(String writeDir) { + this.writeDir = writeDir; + if (StringUtils.isBlank(writeDir) == true) { + LOG.warn("writeDir is empty"); + return; + } else { + LOG.info("writeDir is " + writeDir); + } + + File file = new File(writeDir); + + if (file.exists() == false) { + file.mkdirs(); + LOG.info("Create Directory " + writeDir); + return; + } else if (file.isDirectory() == false) { + LOG.error(writeDir + " isn't a directory "); + throw new RuntimeException(writeDir + " isn't a directory "); + } + + return; + } + + public int getReserverNum() { + return reserverNum; + } + + public void setReserverNum(int reserverNum) { + this.reserverNum = reserverNum; + } + + public static AsyncLoopThread mkInstance(String containerHbDir, + String hbDir, int timeout, int frequence) { + SyncContainerHb syncContainerHbThread = new SyncContainerHb(); + + syncContainerHbThread.setReadDir(containerHbDir); + syncContainerHbThread.setWriteDir(hbDir); + syncContainerHbThread.setTimeoutSeconds(timeout); + syncContainerHbThread.setFrequence(frequence); + + StringBuilder sb = new StringBuilder(); + sb.append("Run process under Apsara/Yarn container"); + sb.append("ContainerDir:").append(containerHbDir); + sb.append("MyDir:").append(hbDir); + sb.append(", timeout:").append(timeout); + sb.append(",frequence:").append(frequence); + LOG.info(sb.toString()); + + AsyncLoopThread thread = + new AsyncLoopThread(syncContainerHbThread, true, + Thread.NORM_PRIORITY, true); + + return thread; + } + + public static AsyncLoopThread mkNimbusInstance(Map conf) throws IOException { + boolean isEnable = ConfigExtension.isEnableContainerNimbus(); + if (isEnable == false) { + LOG.info("Run nimbus without Apsara/Yarn container"); + return null; + } + + String containerHbDir = ConfigExtension.getContainerNimbusHearbeat(); + String hbDir = StormConfig.masterHearbeatForContainer(conf); + int timeout = ConfigExtension.getContainerHeartbeatTimeoutSeconds(conf); + int frequence = ConfigExtension.getContainerHeartbeatFrequence(conf); + + return mkInstance(containerHbDir, hbDir, timeout, frequence); + + } + + public static AsyncLoopThread mkSupervisorInstance(Map conf) + throws IOException { + boolean isEnableContainer = + ConfigExtension.isEnableContainerSupervisor(); + if (isEnableContainer) { + String containerHbDir = + ConfigExtension.getContainerSupervisorHearbeat(); + String hbDir = StormConfig.supervisorHearbeatForContainer(conf); + int timeout = + ConfigExtension.getContainerHeartbeatTimeoutSeconds(conf); + int frequence = + ConfigExtension.getContainerHeartbeatFrequence(conf); + + return mkInstance(containerHbDir, hbDir, timeout, frequence); + } + + boolean isWorkerAutomaticStop = + ConfigExtension.isWorkerStopWithoutSupervisor(conf); + if (isWorkerAutomaticStop) { + String containerHbDir = null; + String hbDir = StormConfig.supervisorHearbeatForContainer(conf); + int timeout = + ConfigExtension.getContainerHeartbeatTimeoutSeconds(conf); + int frequence = + ConfigExtension.getContainerHeartbeatFrequence(conf); + + return mkInstance(containerHbDir, hbDir, timeout, frequence); + } + + LOG.info("Run Supervisor without Apsara/Yarn container"); + return null; + + } + + public static AsyncLoopThread mkWorkerInstance(Map conf) throws IOException { + boolean isEnableContainer = + ConfigExtension.isEnableContainerSupervisor(); + boolean isWorkerAutomaticStop = + ConfigExtension.isWorkerStopWithoutSupervisor(conf); + if (isEnableContainer == false && isWorkerAutomaticStop == false) { + LOG.info("Run worker without Apsara/Yarn container"); + return null; + } + + String containerHbDir = + StormConfig.supervisorHearbeatForContainer(conf); + String hbDir = null; + int timeout = ConfigExtension.getContainerHeartbeatTimeoutSeconds(conf); + int frequence = ConfigExtension.getContainerHeartbeatFrequence(conf); + + return mkInstance(containerHbDir, hbDir, timeout, frequence); + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/hearbeat/WorkerHeartbeatRunable.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/hearbeat/WorkerHeartbeatRunable.java new file mode 100755 index 000000000..74aca0501 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/hearbeat/WorkerHeartbeatRunable.java @@ -0,0 +1,127 @@ +/** + * 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.daemon.worker.hearbeat; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CopyOnWriteArraySet; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.utils.LocalState; + +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.cluster.Common; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.daemon.worker.WorkerData; +import com.alibaba.jstorm.daemon.worker.WorkerHeartbeat; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.TimeUtils; + +/** + * worker Heartbeat + * + * @author yannian/Longda + * + */ +public class WorkerHeartbeatRunable extends RunnableCallback { + private static Logger LOG = LoggerFactory + .getLogger(WorkerHeartbeatRunable.class); + + private WorkerData workerData; + + private AtomicBoolean shutdown; + private Map conf; + private String worker_id; + private Integer port; + private String topologyId; + private CopyOnWriteArraySet task_ids; + // private Object lock = new Object(); + + private Integer frequence; + + private Map workerStates; + + public WorkerHeartbeatRunable(WorkerData workerData) { + + this.workerData = workerData; + + this.conf = workerData.getStormConf(); + this.worker_id = workerData.getWorkerId(); + this.port = workerData.getPort(); + this.topologyId = workerData.getTopologyId(); + this.task_ids = + new CopyOnWriteArraySet(workerData.getTaskids()); + this.shutdown = workerData.getShutdown(); + + String key = Config.WORKER_HEARTBEAT_FREQUENCY_SECS; + frequence = JStormUtils.parseInt(conf.get(key), 10); + + this.workerStates = new HashMap(); + } + + private LocalState getWorkerState() throws IOException { + LocalState state = workerStates.get(worker_id); + if (state == null) { + state = StormConfig.worker_state(conf, worker_id); + workerStates.put(worker_id, state); + } + return state; + } + + /** + * do hearbeat, update LocalState + * + * @throws IOException + */ + + public void doHeartbeat() throws IOException { + + int currtime = TimeUtils.current_time_secs(); + WorkerHeartbeat hb = + new WorkerHeartbeat(currtime, topologyId, task_ids, port); + + LOG.debug("Doing heartbeat:" + worker_id + ",port:" + port + ",hb" + + hb.toString()); + + LocalState state = getWorkerState(); + state.put(Common.LS_WORKER_HEARTBEAT, hb); + + } + + @Override + public void run() { + + try { + doHeartbeat(); + } catch (IOException e) { + LOG.error("work_heart_beat_fn fail", e); + throw new RuntimeException(e); + } + + } + + @Override + public Object getResult() { + return frequence; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/timer/RotatingMapTrigger.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/timer/RotatingMapTrigger.java new file mode 100755 index 000000000..5a59e6fe6 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/timer/RotatingMapTrigger.java @@ -0,0 +1,86 @@ +/** + * 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.daemon.worker.timer; + +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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 = LoggerFactory + .getLogger(RotatingMapTrigger.class); + + public RotatingMapTrigger(Map conf, String name, DisruptorQueue queue) { + this.name = name; + this.queue = queue; + this.opCode = TimerConstants.ROTATING_MAP; + + 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-core/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TaskBatchFlushTrigger.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TaskBatchFlushTrigger.java new file mode 100644 index 000000000..3a5353f4e --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TaskBatchFlushTrigger.java @@ -0,0 +1,50 @@ +/** + * 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.daemon.worker.timer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.task.TaskBatchTransfer; + +public class TaskBatchFlushTrigger extends TimerTrigger{ + private static final Logger LOG = LoggerFactory.getLogger(TickTupleTrigger.class); + + private TaskBatchTransfer batchTransfer; + + public TaskBatchFlushTrigger(int frequence, String name, TaskBatchTransfer transfer) { + if (frequence <= 0) { + LOG.warn(" The frequence of " + name + " is invalid"); + frequence = 1; + } + this.firstTime = frequence; + this.frequence = frequence; + this.batchTransfer = transfer; + } + + @Override + public void run() { + try { + batchTransfer.flush(); + } catch (Exception e) { + LOG.warn("Failed to public timer event to " + name, e); + return; + } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TaskHeartbeatTrigger.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TaskHeartbeatTrigger.java new file mode 100644 index 000000000..0b67776e2 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TaskHeartbeatTrigger.java @@ -0,0 +1,84 @@ +/** + * 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.daemon.worker.timer; + +import java.util.Map; +import java.util.concurrent.BlockingQueue; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.utils.DisruptorQueue; + +import com.alibaba.jstorm.daemon.worker.timer.TimerTrigger.TimerEvent; +import com.alibaba.jstorm.utils.JStormUtils; + +public class TaskHeartbeatTrigger extends TimerTrigger { + private static final Logger LOG = LoggerFactory + .getLogger(TaskHeartbeatTrigger.class); + + private int taskId; + + private BlockingQueue controlQueue; + + public TaskHeartbeatTrigger(Map conf, String name, DisruptorQueue queue, + BlockingQueue controlQueue, int taskId) { + this.name = name; + this.queue = queue; + this.controlQueue = controlQueue; + this.opCode = TimerConstants.TASK_HEARTBEAT; + + this.taskId = taskId; + + frequence = + JStormUtils.parseInt( + conf.get(Config.TASK_HEARTBEAT_FREQUENCY_SECS), 10); + + firstTime = frequence; + } + + @Override + public void updateObject() { + this.object = Integer.valueOf(taskId); + } + + @Override + public void run() { + + try { + updateObject(); + + if (object == null) { + LOG.info("Timer " + name + " 's object is null "); + return; + } + + TimerEvent event = new TimerEvent(opCode, object); + + controlQueue.offer(event); + LOG.debug("Offer task HB event to controlQueue, taskId=" + taskId); + } catch (Exception e) { + LOG.warn("Failed to public timer event to " + name, e); + return; + } + + LOG.debug(" Trigger timer event to " + name); + + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TickTupleTrigger.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TickTupleTrigger.java new file mode 100755 index 000000000..ecf01c587 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TickTupleTrigger.java @@ -0,0 +1,62 @@ +/** + * 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.daemon.worker.timer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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 = LoggerFactory + .getLogger(TickTupleTrigger.class); + + TopologyContext topologyContext; + + public TickTupleTrigger(TopologyContext topologyContext, int frequence, + String name, DisruptorQueue queue) { + this.name = name; + this.queue = queue; + this.opCode = TimerConstants.TICK_TUPLE; + + 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-core/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TimerConstants.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TimerConstants.java new file mode 100644 index 000000000..daf0cff03 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TimerConstants.java @@ -0,0 +1,26 @@ +/** + * 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.daemon.worker.timer; + +public class TimerConstants { + public static final int DEFAULT = -1; + public static final int ROTATING_MAP = 1; + public static final int TICK_TUPLE = 2; + public static final int TASK_HEARTBEAT = 3; + public static final int BATCH_FLUSH = 4; +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TimerTrigger.java b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TimerTrigger.java new file mode 100644 index 000000000..4cecbea92 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TimerTrigger.java @@ -0,0 +1,169 @@ +/** + * 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.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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.utils.DisruptorQueue; + +import com.lmax.disruptor.InsufficientCapacityException; + +public class TimerTrigger implements Runnable { + private static final Logger LOG = LoggerFactory + .getLogger(TimerTrigger.class); + + private static ScheduledExecutorService threadPool; + + public static void setScheduledExecutorService( + ScheduledExecutorService scheduledExecutorService) { + threadPool = scheduledExecutorService; + } + + protected String name; + protected int opCode; + protected int firstTime; + protected int frequence; + protected DisruptorQueue queue; + protected Object object; + protected boolean block = true; + + public void register() { + register(TimeUnit.SECONDS); + } + + public void register(TimeUnit timeUnit) { + threadPool.scheduleAtFixedRate(this, firstTime, frequence, + timeUnit); + 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; + } + + TimerEvent event = new TimerEvent(opCode, object); + queue.publish(event, 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 getOpCode() { + return opCode; + } + + public void setOpCode(int opCode) { + this.opCode = opCode; + } + + 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); + } + + public class TimerEvent { + private int opCode; + private Object msg; + + public TimerEvent(int opCode, Object msg) { + this.opCode = opCode; + this.msg = msg; + } + + public int getOpCode() { + return opCode; + } + + public Object getMsg() { + return msg; + } + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/drpc/ClearThread.java b/jstorm-core/src/main/java/com/alibaba/jstorm/drpc/ClearThread.java new file mode 100755 index 000000000..5f16b0b16 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/drpc/ClearThread.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.drpc; + +import java.util.Map.Entry; +import java.util.concurrent.Semaphore; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.generated.DRPCExecutionException; + +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.TimeUtils; + +public class ClearThread extends RunnableCallback { + private static final Logger LOG = LoggerFactory + .getLogger(ClearThread.class); + + private final int REQUEST_TIMEOUT_SECS; + private static final int TIMEOUT_CHECK_SECS = 5; + + private Drpc drpcService; + + public ClearThread(Drpc drpc) { + drpcService = drpc; + + REQUEST_TIMEOUT_SECS = + JStormUtils.parseInt( + drpcService.getConf().get( + Config.DRPC_REQUEST_TIMEOUT_SECS), 60); + LOG.info("Drpc timeout seconds is " + REQUEST_TIMEOUT_SECS); + } + + @Override + public void run() { + + for (Entry e : drpcService.getIdtoStart().entrySet()) { + if (TimeUtils.time_delta(e.getValue()) > REQUEST_TIMEOUT_SECS) { + String id = e.getKey(); + + drpcService.getIdtoResult().put(id, + new DRPCExecutionException("Request timed out")); + Semaphore s = drpcService.getIdtoSem().get(id); + if (s != null) { + s.release(); + } + drpcService.cleanup(id); + LOG.info("Clear request " + id); + } + } + + JStormUtils.sleepMs(10); + + } + + public Object getResult() { + return TIMEOUT_CHECK_SECS; + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/drpc/Drpc.java b/jstorm-core/src/main/java/com/alibaba/jstorm/drpc/Drpc.java new file mode 100755 index 000000000..09b4885d8 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/drpc/Drpc.java @@ -0,0 +1,319 @@ +/** + * 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.drpc; + +import java.util.Map; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.Semaphore; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.thrift.TException; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.server.THsHaServer; +import org.apache.thrift.transport.TNonblockingServerSocket; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.daemon.Shutdownable; +import backtype.storm.generated.DRPCExecutionException; +import backtype.storm.generated.DRPCRequest; +import backtype.storm.generated.DistributedRPC; +import backtype.storm.generated.DistributedRPCInvocations; + +import com.alibaba.jstorm.callback.AsyncLoopRunnable; +import com.alibaba.jstorm.callback.AsyncLoopThread; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.TimeUtils; + +/** + * Drpc + * + * @author yannian + * + */ +public class Drpc implements DistributedRPC.Iface, + DistributedRPCInvocations.Iface, Shutdownable { + + private static final Logger LOG = LoggerFactory.getLogger(Drpc.class); + + public static void main(String[] args) throws Exception { + LOG.info("Begin to start Drpc server"); + + final Drpc service = new Drpc(); + + service.init(); + } + + private Map conf; + + private THsHaServer handlerServer; + + private THsHaServer invokeServer; + + private AsyncLoopThread clearThread; + + private AtomicBoolean shutdown = new AtomicBoolean(false); + + private THsHaServer initHandlerServer(Map conf, final Drpc service) + throws Exception { + int port = JStormUtils.parseInt(conf.get(Config.DRPC_PORT)); + int workerThreadNum = + JStormUtils.parseInt(conf.get(Config.DRPC_WORKER_THREADS)); + int queueSize = JStormUtils.parseInt(conf.get(Config.DRPC_QUEUE_SIZE)); + + TNonblockingServerSocket socket = new TNonblockingServerSocket(port); + THsHaServer.Args targs = new THsHaServer.Args(socket); + targs.workerThreads(64); + targs.protocolFactory(new TBinaryProtocol.Factory()); + targs.processor(new DistributedRPC.Processor( + service)); + + ThreadPoolExecutor executor = + new ThreadPoolExecutor(workerThreadNum, workerThreadNum, 60, + TimeUnit.SECONDS, new ArrayBlockingQueue(queueSize)); + targs.executorService(executor); + + THsHaServer handlerServer = new THsHaServer(targs); + LOG.info("Successfully init Handler Server " + port); + + return handlerServer; + } + + private THsHaServer initInvokeServer(Map conf, final Drpc service) + throws Exception { + int port = JStormUtils.parseInt(conf.get(Config.DRPC_INVOCATIONS_PORT)); + + TNonblockingServerSocket socket = new TNonblockingServerSocket(port); + THsHaServer.Args targsInvoke = new THsHaServer.Args(socket); + targsInvoke.workerThreads(64); + targsInvoke.protocolFactory(new TBinaryProtocol.Factory()); + targsInvoke + .processor(new DistributedRPCInvocations.Processor( + service)); + + THsHaServer invokeServer = new THsHaServer(targsInvoke); + + LOG.info("Successfully init Invoke Server " + port); + return invokeServer; + } + + private void initThrift() throws Exception { + + handlerServer = initHandlerServer(conf, this); + + invokeServer = initInvokeServer(conf, this); + + Runtime.getRuntime().addShutdownHook(new Thread() { + public void run() { + Drpc.this.shutdown(); + handlerServer.stop(); + invokeServer.stop(); + } + + }); + + LOG.info("Starting Distributed RPC servers..."); + new Thread(new Runnable() { + @Override + public void run() { + invokeServer.serve(); + } + }).start(); + handlerServer.serve(); + } + + private void initClearThread() { + clearThread = new AsyncLoopThread(new ClearThread(this)); + LOG.info("Successfully start clear thread"); + } + + public void init() throws Exception { + conf = StormConfig.read_storm_config(); + LOG.info("Configuration is \n" + conf); + + initClearThread(); + + initThrift(); + } + + public Drpc() { + } + + @Override + public void shutdown() { + if (shutdown.getAndSet(true) == true) { + LOG.info("Notify to quit drpc"); + return; + } + + LOG.info("Begin to shutdown drpc"); + AsyncLoopRunnable.getShutdown().set(true); + + clearThread.interrupt(); + + try { + clearThread.join(); + } catch (InterruptedException e) { + } + LOG.info("Successfully cleanup clear thread"); + + invokeServer.stop(); + LOG.info("Successfully stop invokeServer"); + + handlerServer.stop(); + LOG.info("Successfully stop handlerServer"); + + } + + private AtomicInteger ctr = new AtomicInteger(0); + private ConcurrentHashMap idtoSem = + new ConcurrentHashMap(); + private ConcurrentHashMap idtoResult = + new ConcurrentHashMap(); + private ConcurrentHashMap idtoStart = + new ConcurrentHashMap(); + private ConcurrentHashMap> requestQueues = + new ConcurrentHashMap>(); + + public void cleanup(String id) { + LOG.info("clean id " + id + " @ " + (System.currentTimeMillis())); + + idtoSem.remove(id); + idtoResult.remove(id); + idtoStart.remove(id); + } + + @Override + public String execute(String function, String args) + throws DRPCExecutionException, TException { + LOG.info("Received DRPC request for " + function + " " + args + " at " + + (System.currentTimeMillis())); + int idinc = this.ctr.incrementAndGet(); + int maxvalue = 1000000000; + int newid = idinc % maxvalue; + if (idinc != newid) { + this.ctr.compareAndSet(idinc, newid); + } + + String strid = String.valueOf(newid); + Semaphore sem = new Semaphore(0); + + DRPCRequest req = new DRPCRequest(args, strid); + this.idtoStart.put(strid, TimeUtils.current_time_secs()); + this.idtoSem.put(strid, sem); + ConcurrentLinkedQueue queue = acquireQueue(function); + queue.add(req); + LOG.info("Waiting for DRPC request for " + function + " " + args + + " at " + (System.currentTimeMillis())); + try { + sem.acquire(); + } catch (InterruptedException e) { + LOG.error("acquire fail ", e); + } + LOG.info("Acquired for DRPC request for " + function + " " + args + + " at " + (System.currentTimeMillis())); + + Object result = this.idtoResult.get(strid); + LOG.info("Returning for DRPC request for " + function + " " + args + + " at " + (System.currentTimeMillis())); + + this.cleanup(strid); + + if (result instanceof DRPCExecutionException) { + throw (DRPCExecutionException) result; + } + return String.valueOf(result); + } + + @Override + public void result(String id, String result) throws TException { + Semaphore sem = this.idtoSem.get(id); + LOG.info("Received result " + result + " for id " + id + " at " + + (System.currentTimeMillis())); + if (sem != null) { + this.idtoResult.put(id, result); + sem.release(); + } + + } + + @Override + public DRPCRequest fetchRequest(String functionName) throws TException { + + ConcurrentLinkedQueue queue = acquireQueue(functionName); + DRPCRequest req = queue.poll(); + if (req != null) { + LOG.info("Fetched request for " + functionName + " at " + + (System.currentTimeMillis())); + return req; + } else { + return new DRPCRequest("", ""); + } + + } + + @Override + public void failRequest(String id) throws TException { + Semaphore sem = this.idtoSem.get(id); + LOG.info("failRequest result for id " + id + " at " + + (System.currentTimeMillis())); + if (sem != null) { + this.idtoResult.put(id, + new DRPCExecutionException("Request failed")); + sem.release(); + } + } + + private ConcurrentLinkedQueue acquireQueue(String function) { + ConcurrentLinkedQueue reqQueue = + requestQueues.get(function); + if (reqQueue == null) { + reqQueue = new ConcurrentLinkedQueue(); + requestQueues.put(function, reqQueue); + } + return reqQueue; + } + + public ConcurrentHashMap getIdtoSem() { + return idtoSem; + } + + public ConcurrentHashMap getIdtoResult() { + return idtoResult; + } + + public ConcurrentHashMap getIdtoStart() { + return idtoStart; + } + + public AtomicBoolean isShutdown() { + return shutdown; + } + + public Map getConf() { + return conf; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/event/EventManager.java b/jstorm-core/src/main/java/com/alibaba/jstorm/event/EventManager.java new file mode 100755 index 000000000..152febc12 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/event/EventManager.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.event; + +import com.alibaba.jstorm.callback.RunnableCallback; + +public interface EventManager { + public void add(RunnableCallback event_fn); + + public boolean waiting(); + + public void shutdown(); +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/event/EventManagerImp.java b/jstorm-core/src/main/java/com/alibaba/jstorm/event/EventManagerImp.java new file mode 100755 index 000000000..cdfa1cc07 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/event/EventManagerImp.java @@ -0,0 +1,84 @@ +/** + * 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.event; + +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicInteger; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.callback.RunnableCallback; + +/** + * Event Manager, drop one event from queue, then execute the event. + */ +public class EventManagerImp extends RunnableCallback implements EventManager { + private static final Logger LOG = LoggerFactory + .getLogger(EventManagerImp.class); + + private AtomicInteger added = new AtomicInteger(); + private AtomicInteger processed = new AtomicInteger(); + + private LinkedBlockingQueue queue = + new LinkedBlockingQueue(); + + private Exception e; + + public void proccessinc() { + processed.incrementAndGet(); + } + + @Override + public void add(RunnableCallback event_fn) { + added.incrementAndGet(); + queue.add(event_fn); + } + + @Override + public boolean waiting() { + return (processed.get() == added.get()); + + } + + @Override + public Exception error() { + return e; + } + + @Override + public void run() { + try { + RunnableCallback r = queue.take(); + if (r == null) { + + return; + } + + r.run(); + + e = r.error(); + + proccessinc(); + + } catch (InterruptedException e) { + LOG.info("InterruptedException when processing event "); + } + + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/event/EventManagerPusher.java b/jstorm-core/src/main/java/com/alibaba/jstorm/event/EventManagerPusher.java new file mode 100755 index 000000000..7f5e9efa2 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/event/EventManagerPusher.java @@ -0,0 +1,50 @@ +/** + * 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.event; + +import com.alibaba.jstorm.callback.RunnableCallback; + +/** + * put event to eventManager queue + */ +public class EventManagerPusher extends RunnableCallback { + + private EventManager eventManager; + + private RunnableCallback event; + + private int frequence; + + public EventManagerPusher(EventManager eventManager, + RunnableCallback event, int frequence) { + this.eventManager = eventManager; + this.event = event; + this.frequence = frequence; + } + + @Override + public Object getResult() { + return frequence; + } + + @Override + public void run() { + eventManager.add(event); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/event/EventManagerZkPusher.java b/jstorm-core/src/main/java/com/alibaba/jstorm/event/EventManagerZkPusher.java new file mode 100755 index 000000000..01f20ac4f --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/event/EventManagerZkPusher.java @@ -0,0 +1,42 @@ +/** + * 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.event; + +import com.alibaba.jstorm.callback.RunnableCallback; + +public class EventManagerZkPusher extends RunnableCallback { + + private EventManager eventManager; + + private RunnableCallback cb; + + /** + * @param cb + * @param eventManager + */ + public EventManagerZkPusher(RunnableCallback cb, EventManager eventManager) { + this.eventManager = eventManager; + this.cb = cb; + } + + @Override + public void run() { + eventManager.add(cb); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/ControlMessage.java b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/ControlMessage.java new file mode 100755 index 000000000..d68347ab7 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/ControlMessage.java @@ -0,0 +1,86 @@ +/** + * 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.message.netty; + +import org.jboss.netty.buffer.ChannelBuffer; +import org.jboss.netty.buffer.ChannelBufferOutputStream; +import org.jboss.netty.buffer.ChannelBuffers; + +public enum ControlMessage { + EOB_MESSAGE((short) -201), OK_RESPONSE((short) -200); + + + private short code; + private long timeStamp; + protected static int port; + + static public void setPort(int port) { + ControlMessage.port = port; + } + + // private constructor + private ControlMessage(short code) { + this.code = code; + } + + /** + * Return a control message per an encoded status code + * + * @param encoded + * @return + */ + static ControlMessage mkMessage(short encoded) { + for (ControlMessage cm : ControlMessage.values()) { + if (encoded == cm.code) + return cm; + } + return null; + } + + static int encodeLength() { + return 14; // short + long + int + } + + /** + * encode the current Control Message into a channel buffer + * + * @throws Exception + */ + ChannelBuffer buffer() throws Exception { + ChannelBufferOutputStream bout = + new ChannelBufferOutputStream( + ChannelBuffers.directBuffer(encodeLength())); + write(bout); + bout.close(); + return bout.buffer(); + } + + void write(ChannelBufferOutputStream bout) throws Exception { + bout.writeShort(code); + bout.writeLong(System.currentTimeMillis()); + bout.writeInt(port); + } + + long getTimeStamp() { + return timeStamp; + } + + void setTimeStamp(long timeStamp) { + this.timeStamp = timeStamp; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/MessageBatch.java b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/MessageBatch.java new file mode 100755 index 000000000..172822d83 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/MessageBatch.java @@ -0,0 +1,194 @@ +/** + * 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.message.netty; + +import java.util.ArrayList; + +import org.jboss.netty.buffer.ChannelBuffer; +import org.jboss.netty.buffer.ChannelBufferOutputStream; +import org.jboss.netty.buffer.ChannelBuffers; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.messaging.TaskMessage; + +class MessageBatch { + private static final Logger LOG = LoggerFactory + .getLogger(MessageBatch.class); + private int buffer_size; + private ArrayList msgs; + private int encoded_length; + + MessageBatch(int buffer_size) { + this.buffer_size = buffer_size; + msgs = new ArrayList(); + encoded_length = ControlMessage.EOB_MESSAGE.encodeLength(); + } + + void add(Object obj) { + if (obj == null) + throw new RuntimeException("null object forbidded in message batch"); + + if (obj instanceof TaskMessage) { + TaskMessage msg = (TaskMessage) obj; + msgs.add(msg); + encoded_length += msgEncodeLength(msg); + return; + } + + if (obj instanceof ControlMessage) { + ControlMessage msg = (ControlMessage) obj; + msgs.add(msg); + encoded_length += msg.encodeLength(); + return; + } + + throw new RuntimeException("Unsuppoted object type " + + obj.getClass().getName()); + } + + void remove(Object obj) { + if (obj == null) + return; + + if (obj instanceof TaskMessage) { + TaskMessage msg = (TaskMessage) obj; + msgs.remove(msg); + encoded_length -= msgEncodeLength(msg); + return; + } + + if (obj instanceof ControlMessage) { + ControlMessage msg = (ControlMessage) obj; + msgs.remove(msg); + encoded_length -= msg.encodeLength(); + return; + } + } + + Object get(int index) { + return msgs.get(index); + } + + /** + * try to add a TaskMessage to a batch + * + * @param taskMsg + * @return false if the msg could not be added due to buffer size limit; + * true otherwise + */ + boolean tryAdd(TaskMessage taskMsg) { + if ((encoded_length + msgEncodeLength(taskMsg)) > buffer_size) + return false; + add(taskMsg); + return true; + } + + private int msgEncodeLength(TaskMessage taskMsg) { + if (taskMsg == null) + return 0; + + int size = 6; // INT + SHORT + if (taskMsg.message() != null) + size += taskMsg.message().length; + return size; + } + + /** + * Has this batch used up allowed buffer size + * + * @return + */ + boolean isFull() { + return encoded_length >= buffer_size; + } + + /** + * true if this batch doesn't have any messages + * + * @return + */ + boolean isEmpty() { + return msgs.isEmpty(); + } + + /** + * # of msgs in this batch + * + * @return + */ + int size() { + return msgs.size(); + } + + public int getEncoded_length() { + return encoded_length; + } + + /** + * create a buffer containing the encoding of this batch + */ + ChannelBuffer buffer() throws Exception { + ChannelBufferOutputStream bout = + new ChannelBufferOutputStream( + ChannelBuffers.directBuffer(encoded_length)); + + for (Object msg : msgs) + if (msg instanceof TaskMessage) + writeTaskMessage(bout, (TaskMessage) msg); + else { + // LOG.debug("Write one non-TaskMessage {}", msg ); + ((ControlMessage) msg).write(bout); + } + + // add a END_OF_BATCH indicator + ControlMessage.EOB_MESSAGE.write(bout); + // LOG.debug("ControlMessage.EOB_MESSAGE " ); + + bout.close(); + + return bout.buffer(); + } + + /** + * write a TaskMessage into a stream + * + * Each TaskMessage is encoded as: task ... short(2) len ... int(4) payload + * ... byte[] * + */ + private void writeTaskMessage(ChannelBufferOutputStream bout, + TaskMessage message) throws Exception { + int payload_len = 0; + if (message.message() != null) + payload_len = message.message().length; + + int task_id = message.task(); + if (task_id > Short.MAX_VALUE) + throw new RuntimeException("Task ID should not exceed " + + Short.MAX_VALUE); + + bout.writeShort((short) task_id); + bout.writeInt(payload_len); + if (payload_len > 0) + bout.write(message.message()); + + // @@@ TESTING CODE + // LOG.info("Write one message taskid:{}, len:{}, data:{}", task_id + // , payload_len, JStormUtils.toPrintableString(message.message()) ); + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/MessageDecoder.java b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/MessageDecoder.java new file mode 100755 index 000000000..b147092d3 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/MessageDecoder.java @@ -0,0 +1,236 @@ +/** + * 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.message.netty; + +import java.net.InetSocketAddress; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.jboss.netty.buffer.ChannelBuffer; +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelHandlerContext; +import org.jboss.netty.handler.codec.frame.FrameDecoder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.common.metric.Histogram; +import com.alibaba.jstorm.common.metric.Meter; +import com.alibaba.jstorm.metric.JStormMetrics; +import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.utils.NetWorkUtils; + +import backtype.storm.messaging.TaskMessage; + +public class MessageDecoder extends FrameDecoder { + private static final Logger LOG = LoggerFactory + .getLogger(MessageDecoder.class); + + // here doesn't use Timer due to competition + private static Histogram timer = JStormMetrics + .registerWorkerHistogram(MetricDef.NETWORK_MSG_DECODE_TIME); + private static Meter recvSpeed = JStormMetrics + .registerWorkerMeter(MetricDef.NETTY_SRV_RECV_SPEED); + private static Map networkTransmitTimeMap = + new HashMap(); + private static Map transmitNameMap = + new HashMap(); + private boolean isServer; + private String localIp; + private int localPort; + + public MessageDecoder(boolean isServer, Map conf) { + this.isServer = isServer; + this.localPort = ConfigExtension.getLocalWorkerPort(conf); + this.localIp = NetWorkUtils.ip(); + + } + + /* + * Each ControlMessage is encoded as: code (<0) ... short(2) Each + * TaskMessage is encoded as: task (>=0) ... short(2) len ... int(4) payload + * ... byte[] * + */ + protected Object decode(ChannelHandlerContext ctx, Channel channel, + ChannelBuffer buf) throws Exception { + // Make sure that we have received at least a short + long available = buf.readableBytes(); + // Length of control message is 10. + // Minimum length of a task message is 6(short taskId, int length). + if (available < 6) { + // need more data + return null; + } + + Long startTime = null; + if (isServer) { + startTime = System.nanoTime(); + } + try { + // Mark the current buffer position before reading task/len field + // because the whole frame might not be in the buffer yet. + // We will reset the buffer position to the marked position if + // there's not enough bytes in the buffer. + buf.markReaderIndex(); + + // read the short field + short code = buf.readShort(); + available -= 2; + + // case 1: Control message + ControlMessage ctrl_msg = ControlMessage.mkMessage(code); + if (ctrl_msg != null) { + if (available < 12) { + // The time stamp bytes were not received yet - return null. + buf.resetReaderIndex(); + return null; + } + long timeStamp = buf.readLong(); + int clientPort = buf.readInt(); + available -= 12; + if (ctrl_msg == ControlMessage.EOB_MESSAGE) { + + long interval = System.currentTimeMillis() - timeStamp; + if (interval > 0) { + + Histogram netTransTime = + getTransmitHistogram(channel, clientPort); + if (netTransTime != null) { + netTransTime.update(interval ); + + } + } + + recvSpeed.update(Double.valueOf(ControlMessage + .encodeLength())); + } + + return ctrl_msg; + } + + // case 2: task Message + short task = code; + + // Make sure that we have received at least an integer (length) + if (available < 4) { + // need more data + buf.resetReaderIndex(); + + return null; + } + + // Read the length field. + int length = buf.readInt(); + if (length <= 0) { + LOG.info( + "Receive one message whose TaskMessage's message length is {}", + length); + return new TaskMessage(task, null); + } + + // Make sure if there's enough bytes in the buffer. + available -= 4; + if (available < length) { + // The whole bytes were not received yet - return null. + buf.resetReaderIndex(); + + return null; + } + + // There's enough bytes in the buffer. Read it. + ChannelBuffer payload = buf.readBytes(length); + + // Successfully decoded a frame. + // Return a TaskMessage object + + byte[] rawBytes = payload.array(); + // @@@ TESTING CODE + // LOG.info("Receive task:{}, length: {}, data:{}", + // task, length, JStormUtils.toPrintableString(rawBytes)); + + TaskMessage ret = new TaskMessage(task, rawBytes); + recvSpeed.update(Double.valueOf(rawBytes.length + 6)); + return ret; + } finally { + if (isServer) { + Long endTime = System.nanoTime(); + timer.update((endTime - startTime) / 1000000.0d); + } + } + + } + + public Histogram getTransmitHistogram(Channel channel, int clientPort) { + Histogram netTransTime = networkTransmitTimeMap.get(channel); + if (netTransTime == null) { + + InetSocketAddress sockAddr = + (InetSocketAddress) (channel.getRemoteAddress()); + + String nettyConnection = + NettyConnection.mkString(sockAddr.getAddress() + .getHostAddress(), clientPort, localIp, localPort); + try { + netTransTime = + JStormMetrics.registerWorkerHistogram( + MetricDef.NETTY_SRV_MSG_TRANS_TIME, + nettyConnection); + } catch (Exception e) { + LOG.error("{}.{} has been register", + MetricDef.NETTY_SRV_MSG_TRANS_TIME, nettyConnection); + removeTransmitHistogram(nettyConnection); + return null; + } + networkTransmitTimeMap.put(channel, netTransTime); + transmitNameMap.put(channel, nettyConnection); + LOG.info("Register Transmit Histogram of {}, channel {}", + nettyConnection, channel); + } + + return netTransTime; + } + + public static void removeTransmitHistogram(Channel channel) { + Histogram netTransTime = networkTransmitTimeMap.remove(channel); + if (netTransTime != null) { + + String nettyConnection = transmitNameMap.remove(channel); + + JStormMetrics.unregisterWorkerMetric( + MetricDef.NETTY_SRV_MSG_TRANS_TIME, nettyConnection); + + LOG.info("Remove Transmit Histogram of {}, channel {}", + nettyConnection, channel); + } + } + + public static void removeTransmitHistogram(String nettyConnection) { + Channel oldChannel = null; + + for (Entry entry: transmitNameMap.entrySet()) { + if (nettyConnection.equals(entry.getValue())) { + oldChannel = entry.getKey(); + } + } + + removeTransmitHistogram(oldChannel); + } + + +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/MessageEncoder.java b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/MessageEncoder.java new file mode 100755 index 000000000..61e918765 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/MessageEncoder.java @@ -0,0 +1,40 @@ +/** + * 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.message.netty; + +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelHandlerContext; +import org.jboss.netty.handler.codec.oneone.OneToOneEncoder; + +public class MessageEncoder extends OneToOneEncoder { + @Override + protected Object encode(ChannelHandlerContext ctx, Channel channel, + Object obj) throws Exception { + if (obj instanceof ControlMessage) { + return ((ControlMessage) obj).buffer(); + } + + if (obj instanceof MessageBatch) { + return ((MessageBatch) obj).buffer(); + } + + throw new RuntimeException("Unsupported encoding of object of class " + + obj.getClass().getName()); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyClient.java b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyClient.java new file mode 100644 index 000000000..6b79fb4e7 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyClient.java @@ -0,0 +1,614 @@ +/** + * 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.message.netty; + +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +import org.jboss.netty.bootstrap.ClientBootstrap; +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelFactory; +import org.jboss.netty.channel.ChannelFuture; +import org.jboss.netty.channel.ChannelFutureListener; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.messaging.IConnection; +import backtype.storm.messaging.TaskMessage; +import backtype.storm.utils.DisruptorQueue; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.common.metric.Histogram; +import com.alibaba.jstorm.common.metric.Meter; +import com.alibaba.jstorm.common.metric.QueueGauge; +import com.alibaba.jstorm.metric.JStormHealthCheck; +import com.alibaba.jstorm.metric.JStormMetrics; +import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.utils.JStormServerUtils; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.NetWorkUtils; +import com.codahale.metrics.health.HealthCheck; + +class NettyClient implements IConnection { + private static final Logger LOG = LoggerFactory + .getLogger(NettyClient.class); + protected String name; + + protected final int max_retries; + protected final int base_sleep_ms; + protected final int max_sleep_ms; + protected final long timeoutMs; + protected final int MAX_SEND_PENDING; + + protected AtomicInteger retries; + + protected AtomicReference channelRef; + protected ClientBootstrap bootstrap; + protected final InetSocketAddress remote_addr; + protected final ChannelFactory factory; + + protected final int buffer_size; + protected final AtomicBoolean being_closed; + + protected AtomicLong pendings; + protected int messageBatchSize; + protected AtomicReference messageBatchRef; + + protected ScheduledExecutorService scheduler; + + protected String address; + // doesn't use timer, due to competition + protected Histogram sendTimer; + protected Histogram batchSizeHistogram; + protected Meter sendSpeed; + + protected ReconnectRunnable reconnector; + protected ChannelFactory clientChannelFactory; + + protected Set closingChannel; + + protected AtomicBoolean isConnecting = new AtomicBoolean(false); + + protected NettyConnection nettyConnection; + + protected Map stormConf; + + protected boolean connectMyself; + + protected Object channelClosing = new Object(); + + @SuppressWarnings("rawtypes") + NettyClient(Map storm_conf, ChannelFactory factory, + ScheduledExecutorService scheduler, String host, int port, + ReconnectRunnable reconnector) { + this.stormConf = storm_conf; + this.factory = factory; + this.scheduler = scheduler; + this.reconnector = reconnector; + + retries = new AtomicInteger(0); + channelRef = new AtomicReference(null); + being_closed = new AtomicBoolean(false); + pendings = new AtomicLong(0); + + nettyConnection = new NettyConnection(); + nettyConnection.setClientPort(NetWorkUtils.ip(), + ConfigExtension.getLocalWorkerPort(storm_conf)); + nettyConnection.setServerPort(host, port); + + // Configure + buffer_size = + Utils.getInt(storm_conf + .get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE)); + max_retries = + Math.min(30, Utils.getInt(storm_conf + .get(Config.STORM_MESSAGING_NETTY_MAX_RETRIES))); + base_sleep_ms = + Utils.getInt(storm_conf + .get(Config.STORM_MESSAGING_NETTY_MIN_SLEEP_MS)); + max_sleep_ms = + Utils.getInt(storm_conf + .get(Config.STORM_MESSAGING_NETTY_MAX_SLEEP_MS)); + + timeoutMs = ConfigExtension.getNettyPendingBufferTimeout(storm_conf); + MAX_SEND_PENDING = + (int) ConfigExtension.getNettyMaxSendPending(storm_conf); + + this.messageBatchSize = + Utils.getInt( + storm_conf.get(Config.STORM_NETTY_MESSAGE_BATCH_SIZE), + 262144); + messageBatchRef = new AtomicReference(); + + // Start the connection attempt. + remote_addr = new InetSocketAddress(host, port); + name = remote_addr.toString(); + connectMyself = isConnectMyself(stormConf, host, port); + + address = JStormServerUtils.getName(host, port); + + if (connectMyself == false) { + registerMetrics(); + } + closingChannel = new HashSet(); + } + + public void registerMetrics() { + sendTimer = + JStormMetrics.registerWorkerHistogram( + MetricDef.NETTY_CLI_SEND_TIME, nettyConnection.toString()); + batchSizeHistogram = + JStormMetrics.registerWorkerHistogram( + MetricDef.NETTY_CLI_BATCH_SIZE, nettyConnection.toString()); + sendSpeed = JStormMetrics.registerWorkerMeter(MetricDef.NETTY_CLI_SEND_SPEED, + nettyConnection.toString()); + + CacheGaugeHealthCheck cacheGauge = + new CacheGaugeHealthCheck(messageBatchRef, + MetricDef.NETTY_CLI_CACHE_SIZE + ":" + nettyConnection.toString()); + JStormMetrics.registerWorkerGauge(cacheGauge, + MetricDef.NETTY_CLI_CACHE_SIZE, nettyConnection.toString()); + JStormHealthCheck.registerWorkerHealthCheck( + MetricDef.NETTY_CLI_CACHE_SIZE + ":" + nettyConnection.toString(), cacheGauge); + + JStormMetrics.registerWorkerGauge( + new com.codahale.metrics.Gauge() { + + @Override + public Double getValue() { + return ((Long) pendings.get()).doubleValue(); + } + }, MetricDef.NETTY_CLI_SEND_PENDING, nettyConnection.toString()); + + JStormHealthCheck.registerWorkerHealthCheck( + MetricDef.NETTY_CLI_CONNECTION + ":" + nettyConnection.toString(), + new HealthCheck() { + HealthCheck.Result healthy = HealthCheck.Result.healthy(); + HealthCheck.Result unhealthy = HealthCheck.Result.unhealthy + ("NettyConnection " + nettyConnection.toString() + " is broken."); + @Override + protected Result check() throws Exception { + // TODO Auto-generated method stub + if (isChannelReady() == null) { + return unhealthy; + }else { + return healthy; + } + } + + }); + } + + public void start() { + bootstrap = new ClientBootstrap(clientChannelFactory); + bootstrap.setOption("tcpNoDelay", true); + bootstrap.setOption("reuserAddress", true); + bootstrap.setOption("sendBufferSize", buffer_size); + bootstrap.setOption("keepAlive", true); + + // Set up the pipeline factory. + bootstrap.setPipelineFactory(new StormClientPipelineFactory(this, stormConf)); + reconnect(); + } + + public boolean isConnectMyself(Map conf, String host, int port) { + String localIp = NetWorkUtils.ip(); + String remoteIp = NetWorkUtils.host2Ip(host); + int localPort = ConfigExtension.getLocalWorkerPort(conf); + + if (localPort == port && + localIp.equals(remoteIp)) { + return true; + } + + return false; + } + + /** + * The function can't be synchronized, otherwise it will be deadlock + * + */ + public void doReconnect() { + if (channelRef.get() != null) { + + // if (channelRef.get().isWritable()) { + // LOG.info("already exist a writable channel, give up reconnect, {}", + // channelRef.get()); + // return; + // } + return; + } + + if (isClosed() == true) { + return; + } + + if (isConnecting.getAndSet(true)) { + LOG.info("Connect twice {}", name()); + return; + } + + long sleepMs = getSleepTimeMs(); + LOG.info("Reconnect ... [{}], {}, sleep {}ms", retries.get(), name, + sleepMs); + ChannelFuture future = bootstrap.connect(remote_addr); + future.addListener(new ChannelFutureListener() { + public void operationComplete(ChannelFuture future) + throws Exception { + isConnecting.set(false); + Channel channel = future.getChannel(); + if (future.isSuccess()) { + // do something else + LOG.info("Connection established, channel = :{}", channel); + setChannel(channel); + // handleResponse(); + } else { + LOG.info( + "Failed to reconnect ... [{}], {}, channel = {}, cause = {}", + retries.get(), name, channel, future.getCause()); + reconnect(); + } + } + }); + JStormUtils.sleepMs(sleepMs); + + return; + + } + + public void reconnect() { + reconnector.pushEvent(this); + } + + /** + * # of milliseconds to wait per exponential back-off policy + */ + private int getSleepTimeMs() { + + int sleepMs = base_sleep_ms * retries.incrementAndGet(); + if (sleepMs > 1000) { + sleepMs = 1000; + } + return sleepMs; + } + + /** + * Enqueue a task message to be sent to server + */ + @Override + public void send(List messages) { + LOG.warn("Should be overload"); + } + + @Override + public void send(TaskMessage message) { + LOG.warn("Should be overload"); + } + + Channel isChannelReady() { + Channel channel = channelRef.get(); + if (channel == null) { + return null; + } + + // improve performance skill check + if (channel.isWritable() == false) { + return null; + } + + return channel; + } + + protected synchronized void flushRequest(Channel channel, + final MessageBatch requests) { + if (requests == null || requests.isEmpty()) + return; + + Double batchSize = Double.valueOf(requests.getEncoded_length()); + batchSizeHistogram.update(batchSize); + pendings.incrementAndGet(); + sendSpeed.update(batchSize); + ChannelFuture future = channel.write(requests); + future.addListener(new ChannelFutureListener() { + public void operationComplete(ChannelFuture future) + throws Exception { + + pendings.decrementAndGet(); + if (!future.isSuccess()) { + Channel channel = future.getChannel(); + if (isClosed() == false) { + LOG.info("Failed to send requests to " + name + ": " + + channel.toString() + ":", future.getCause()); + } + + if (null != channel) { + + exceptionChannel(channel); + } + } else { + // LOG.debug("{} request(s) sent", requests.size()); + } + } + }); + } + + public void unregisterMetrics() { + JStormMetrics.unregisterWorkerMetric(MetricDef.NETTY_CLI_SEND_TIME, + nettyConnection.toString()); + JStormMetrics.unregisterWorkerMetric(MetricDef.NETTY_CLI_BATCH_SIZE, + nettyConnection.toString()); + JStormMetrics.unregisterWorkerMetric(MetricDef.NETTY_CLI_SEND_PENDING, + nettyConnection.toString()); + JStormMetrics.unregisterWorkerMetric(MetricDef.NETTY_CLI_CACHE_SIZE, + nettyConnection.toString()); + JStormMetrics.unregisterWorkerMetric(MetricDef.NETTY_CLI_SEND_SPEED, + nettyConnection.toString()); + + JStormHealthCheck + .unregisterWorkerHealthCheck(MetricDef.NETTY_CLI_CACHE_SIZE + + ":" + nettyConnection.toString()); + + JStormHealthCheck.unregisterWorkerHealthCheck( + MetricDef.NETTY_CLI_CONNECTION + ":" + nettyConnection.toString()); + } + + /** + * gracefully close this client. + * + * We will send all existing requests, and then invoke close_n_release() + * method + */ + public void close() { + LOG.info("Close netty connection to {}", name()); + if (being_closed.compareAndSet(false, true) == false) { + LOG.info("Netty client has been closed."); + return; + } + + if (connectMyself == false) { + unregisterMetrics(); + } + + Channel channel = channelRef.get(); + if (channel == null) { + LOG.info("Channel {} has been closed before", name()); + return; + } + + if (channel.isWritable()) { + MessageBatch toBeFlushed = messageBatchRef.getAndSet(null); + flushRequest(channel, toBeFlushed); + } + + // wait for pendings to exit + final long timeoutMilliSeconds = 10 * 1000; + final long start = System.currentTimeMillis(); + + LOG.info("Waiting for pending batchs to be sent with " + name() + + "..., timeout: {}ms, pendings: {}", timeoutMilliSeconds, + pendings.get()); + + while (pendings.get() != 0) { + try { + long delta = System.currentTimeMillis() - start; + if (delta > timeoutMilliSeconds) { + LOG.error( + "Timeout when sending pending batchs with {}..., there are still {} pending batchs not sent", + name(), pendings.get()); + break; + } + Thread.sleep(1000); // sleep 1s + } catch (InterruptedException e) { + break; + } + } + + close_n_release(); + + } + + /** + * close_n_release() is invoked after all messages have been sent. + */ + void close_n_release() { + if (channelRef.get() != null) { + setChannel(null); + } + + } + + /** + * Avoid channel double close + * + * @param channel + */ + void closeChannel(final Channel channel) { + synchronized (channelClosing) { + if (closingChannel.contains(channel)) { + LOG.info(channel.toString() + " is already closed"); + return; + } + + closingChannel.add(channel); + } + + LOG.debug(channel.toString() + " begin to closed"); + ChannelFuture closeFuture = channel.close(); + closeFuture.addListener(new ChannelFutureListener() { + public void operationComplete(ChannelFuture future) + throws Exception { + + synchronized (channelClosing) { + closingChannel.remove(channel); + } + LOG.debug(channel.toString() + " finish closed"); + } + }); + } + + void disconnectChannel(Channel channel) { + if (isClosed()) { + return; + } + + if (channel == channelRef.get()) { + setChannel(null); + reconnect(); + } else { + closeChannel(channel); + } + + } + + void exceptionChannel(Channel channel) { + if (channel == channelRef.get()) { + setChannel(null); + } else { + closeChannel(channel); + } + } + + void setChannel(Channel newChannel) { + final Channel oldChannel = channelRef.getAndSet(newChannel); + + if (newChannel != null) { + retries.set(0); + } + + final String oldLocalAddres = + (oldChannel == null) ? "null" : oldChannel.getLocalAddress() + .toString(); + String newLocalAddress = + (newChannel == null) ? "null" : newChannel.getLocalAddress() + .toString(); + LOG.info("Use new channel {} replace old channel {}", newLocalAddress, + oldLocalAddres); + + // avoid one netty client use too much connection, close old one + if (oldChannel != newChannel && oldChannel != null) { + + closeChannel(oldChannel); + LOG.info("Successfully close old channel " + oldLocalAddres); + // scheduler.schedule(new Runnable() { + // + // @Override + // public void run() { + // + // } + // }, 10, TimeUnit.SECONDS); + + // @@@ todo + // pendings.set(0); + } + } + + @Override + public boolean isClosed() { + return being_closed.get(); + } + + public AtomicBoolean getBeing_closed() { + return being_closed; + } + + public int getBuffer_size() { + return buffer_size; + } + + public SocketAddress getRemoteAddr() { + return remote_addr; + } + + public String name() { + return name; + } + + public void handleResponse() { + LOG.warn("Should be overload"); + } + + @Override + public Object recv(Integer taskId, int flags) { + throw new UnsupportedOperationException( + "recvTask: Client connection should not receive any messages"); + } + + @Override + public void registerQueue(Integer taskId, DisruptorQueue recvQueu) { + throw new UnsupportedOperationException( + "recvTask: Client connection should not receive any messages"); + } + + @Override + public void enqueue(TaskMessage message) { + throw new UnsupportedOperationException( + "recvTask: Client connection should not receive any messages"); + } + + public static class CacheGaugeHealthCheck extends HealthCheck implements + com.codahale.metrics.Gauge { + + AtomicReference messageBatchRef; + String name; + Result healthy; + + public CacheGaugeHealthCheck( + AtomicReference messageBatchRef, String name) { + this.messageBatchRef = messageBatchRef; + this.name = name; + this.healthy = HealthCheck.Result.healthy(); + } + + @Override + public Double getValue() { + // TODO Auto-generated method stub + MessageBatch messageBatch = messageBatchRef.get(); + if (messageBatch == null) { + return 0.0; + } else { + Double ret = (double) messageBatch.getEncoded_length(); + return ret; + } + + } + + @Override + protected Result check() throws Exception { + // TODO Auto-generated method stub + Double size = getValue(); + if (size > 8 * JStormUtils.SIZE_1_M) { + return HealthCheck.Result.unhealthy(name + + QueueGauge.QUEUE_IS_FULL); + } else { + return healthy; + } + } + + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyClientAsync.java b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyClientAsync.java new file mode 100644 index 000000000..1d582baba --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyClientAsync.java @@ -0,0 +1,350 @@ +/** + * 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.message.netty; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.messaging.TaskMessage; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.utils.IntervalCheck; +import com.alibaba.jstorm.utils.JStormServerUtils; +import com.alibaba.jstorm.utils.JStormUtils; + +class NettyClientAsync extends NettyClient { + private static final Logger LOG = LoggerFactory + .getLogger(NettyClientAsync.class); + public static final String PREFIX = "Netty-Client-"; + + // when batch buffer size is more than BATCH_THREASHOLD_WARN + // it will block Drainer thread + protected long BATCH_THREASHOLD_WARN; + protected final boolean directlySend; + + protected AtomicBoolean flush_later; + protected int flushCheckInterval; + protected final boolean blockSend; + + boolean isDirectSend(Map conf) { + + if (JStormServerUtils.isOnePending(conf) == true) { + return true; + } + + return !ConfigExtension.isNettyTransferAsyncBatch(conf); + } + + boolean isBlockSend(Map storm_conf) { + if (ConfigExtension.isTopologyContainAcker(storm_conf) == false) { + return false; + } + + return ConfigExtension.isNettyASyncBlock(storm_conf); + } + + @SuppressWarnings("rawtypes") + NettyClientAsync(Map storm_conf, ChannelFactory factory, + ScheduledExecutorService scheduler, String host, int port, + ReconnectRunnable reconnector) { + super(storm_conf, factory, scheduler, host, port, reconnector); + + BATCH_THREASHOLD_WARN = + ConfigExtension.getNettyBufferThresholdSize(storm_conf); + + blockSend = isBlockSend(storm_conf); + + directlySend = isDirectSend(storm_conf); + + flush_later = new AtomicBoolean(false); + flushCheckInterval = + Utils.getInt(storm_conf + .get(Config.STORM_NETTY_FLUSH_CHECK_INTERVAL_MS), 10); + + Runnable flusher = new Runnable() { + @Override + public void run() { + flush(); + } + }; + long initialDelay = Math.min(1000, max_sleep_ms * max_retries); + scheduler.scheduleAtFixedRate(flusher, initialDelay, + flushCheckInterval, TimeUnit.MILLISECONDS); + + clientChannelFactory = factory; + + start(); + + LOG.info(this.toString()); + } + + /** + * Enqueue a task message to be sent to server + */ + @Override + synchronized public void send(List messages) { + // throw exception if the client is being closed + if (isClosed()) { + LOG.warn("Client is being closed, and does not take requests any more"); + return; + } + + long start = System.nanoTime(); + try { + pushBatch(messages); + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + long end = System.nanoTime(); + sendTimer.update((end - start)/1000000.0d); + + } + } + + @Override + synchronized public void send(TaskMessage message) { + // throw exception if the client is being closed + if (isClosed()) { + LOG.warn("Client is being closed, and does not take requests any more"); + return; + } + + long start = System.nanoTime(); + try { + pushBatch(message); + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + long end = System.nanoTime(); + sendTimer.update((end - start)/1000000.0d); + } + } + + void waitChannelReady(long cachedSize, long sleepMs) { + long begin = System.currentTimeMillis(); + boolean changeThreadhold = false; + IntervalCheck oneSecond = new IntervalCheck(); + IntervalCheck timeoutIntervalCheck = new IntervalCheck(); + timeoutIntervalCheck.setIntervalMs(timeoutMs); + sleepMs = (sleepMs < timeoutMs) ? sleepMs : timeoutMs; + while (isChannelReady() == null) { + + long now = System.currentTimeMillis(); + long delt = now - begin; + if (oneSecond.check() == true) { + LOG.warn( + "Target server {} is unavailable, pending {}, bufferSize {}, block sending {}ms", + name, pendings.get(), cachedSize, delt); + } + + if (timeoutIntervalCheck.check() == true) { + if (messageBatchRef.get() != null) { + LOG.warn( + "Target server {} is unavailable, wait too much time, throw timeout message", + name); + messageBatchRef.set(null); + } + setChannel(null); + LOG.warn("Reset channel as null"); + + if (blockSend == false) { + reconnect(); + break; + } + } + + reconnect(); + JStormUtils.sleepMs(sleepMs); + + if (delt > 2 * timeoutMs * 1000L && changeThreadhold == false) { + if (channelRef.get() != null + && BATCH_THREASHOLD_WARN >= 2 * messageBatchSize) { + // it is just channel isn't writable; + BATCH_THREASHOLD_WARN = BATCH_THREASHOLD_WARN / 2; + LOG.info("Reduce BATCH_THREASHOLD_WARN to {}", + BATCH_THREASHOLD_WARN); + + changeThreadhold = true; + } + } + + if (isClosed()) { + LOG.info("Channel has been closed " + name()); + break; + } + } + } + + long getDelaySec(long cachedSize) { + long count = cachedSize / BATCH_THREASHOLD_WARN; + long sleepMs = (long) (Math.pow(2, count) * 10); + + if (sleepMs > 1000) { + sleepMs = 1000; + } + + return sleepMs; + } + + void handleFailedChannel(MessageBatch messageBatch) { + + messageBatchRef.set(messageBatch); + flush_later.set(true); + + long cachedSize = messageBatch.getEncoded_length(); + if (cachedSize > BATCH_THREASHOLD_WARN) { + long sleepMs = getDelaySec(cachedSize); + waitChannelReady(cachedSize, sleepMs); + } + return; + } + + void pushBatch(List messages) { + + if (messages.isEmpty()) { + return; + } + + MessageBatch messageBatch = messageBatchRef.getAndSet(null); + if (null == messageBatch) { + messageBatch = new MessageBatch(messageBatchSize); + } + + for (TaskMessage message : messages) { + if (TaskMessage.isEmpty(message)) { + continue; + } + + messageBatch.add(message); + + if (messageBatch.isFull()) { + Channel channel = isChannelReady(); + if (channel != null) { + flushRequest(channel, messageBatch); + + messageBatch = new MessageBatch(messageBatchSize); + } + + } + } + + Channel channel = isChannelReady(); + if (channel == null) { + handleFailedChannel(messageBatch); + return; + } else if (messageBatch.isEmpty() == false) { + flushRequest(channel, messageBatch); + } + + return; + } + + void pushBatch(TaskMessage message) { + + if (TaskMessage.isEmpty(message)) { + return; + } + + MessageBatch messageBatch = messageBatchRef.getAndSet(null); + if (null == messageBatch) { + messageBatch = new MessageBatch(messageBatchSize); + } + + messageBatch.add(message); + + Channel channel = isChannelReady(); + if (channel == null) { + handleFailedChannel(messageBatch); + return; + } + + if (messageBatch.isFull()) { + flushRequest(channel, messageBatch); + + return; + } + + if (directlySend) { + flushRequest(channel, messageBatch); + } else { + if (messageBatchRef.compareAndSet(null, messageBatch)) { + flush_later.set(true); + } + else + LOG.error("MessageBatch will be lost. This should not happen."); + } + + return; + } + + void flush() { + if (isClosed() == true) { + return; + } + + if (flush_later.get() == false) { + return; + } + + Channel channel = isChannelReady(); + if (channel == null) { + return; + } + + flush_later.set(false); + MessageBatch toBeFlushed = messageBatchRef.getAndSet(null); + flushRequest(channel, toBeFlushed); + } + + @Override + Channel isChannelReady() { + Channel channel = super.isChannelReady(); + if (channel == null) { + return null; + } + + if (blockSend == true && pendings.get() >= MAX_SEND_PENDING) { + return null; + } + return channel; + } + + @Override + public void handleResponse() { + // do nothing + return; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyClientSync.java b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyClientSync.java new file mode 100644 index 000000000..c239dd10e --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyClientSync.java @@ -0,0 +1,332 @@ +/** + * 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.message.netty; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelFactory; +import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.messaging.TaskMessage; +import backtype.storm.utils.DisruptorQueue; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.common.metric.QueueGauge; +import com.alibaba.jstorm.metric.JStormHealthCheck; +import com.alibaba.jstorm.metric.JStormMetrics; +import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.utils.JStormServerUtils; +import com.alibaba.jstorm.utils.JStormUtils; +import com.codahale.metrics.Gauge; +import com.lmax.disruptor.EventHandler; +import com.lmax.disruptor.WaitStrategy; +import com.lmax.disruptor.dsl.ProducerType; + +class NettyClientSync extends NettyClient implements EventHandler { + private static final Logger LOG = LoggerFactory + .getLogger(NettyClientSync.class); + + private ConcurrentLinkedQueue batchQueue; + private DisruptorQueue disruptorQueue; + private ExecutorService bossExecutor; + private ExecutorService workerExecutor; + + private AtomicLong emitTs = new AtomicLong(0); + + @SuppressWarnings("rawtypes") + NettyClientSync(Map storm_conf, ChannelFactory factory, + ScheduledExecutorService scheduler, String host, int port, + ReconnectRunnable reconnector) { + super(storm_conf, factory, scheduler, host, port, reconnector); + + batchQueue = new ConcurrentLinkedQueue(); + + WaitStrategy waitStrategy = + (WaitStrategy) JStormUtils + .createDisruptorWaitStrategy(storm_conf); + + disruptorQueue = + DisruptorQueue.mkInstance(name, ProducerType.MULTI, + MAX_SEND_PENDING * 8, waitStrategy); + disruptorQueue.consumerStarted(); + + if (connectMyself == false) { + registerSyncMetrics(); + } + + Runnable trigger = new Runnable() { + @Override + public void run() { + trigger(); + } + }; + + scheduler.scheduleAtFixedRate(trigger, 10, 1, TimeUnit.SECONDS); + + /** + * In sync mode, it can't directly use common factory, it will occur + * problem when client close and restart + */ + ThreadFactory bossFactory = + new NettyRenameThreadFactory(MetricDef.NETTY_CLI + + JStormServerUtils.getName(host, port) + "-boss"); + bossExecutor = Executors.newCachedThreadPool(bossFactory); + ThreadFactory workerFactory = + new NettyRenameThreadFactory(MetricDef.NETTY_CLI + + JStormServerUtils.getName(host, port) + "-worker"); + workerExecutor = Executors.newCachedThreadPool(workerFactory); + + clientChannelFactory = + new NioClientSocketChannelFactory(bossExecutor, workerExecutor, + 1); + + start(); + + LOG.info(this.toString()); + } + + public void registerSyncMetrics() { + JStormMetrics.registerWorkerGauge(new Gauge() { + @Override + public Double getValue() { + return Double.valueOf(batchQueue.size()); + } + }, MetricDef.NETTY_CLI_SYNC_BATCH_QUEUE, nettyConnection.toString()); + + QueueGauge cacheQueueGauge = + new QueueGauge(MetricDef.NETTY_CLI_SYNC_DISR_QUEUE + + nettyConnection.toString(), disruptorQueue); + + JStormMetrics + .registerWorkerGauge(cacheQueueGauge, + MetricDef.NETTY_CLI_SYNC_DISR_QUEUE, + nettyConnection.toString()); + JStormHealthCheck.registerWorkerHealthCheck( + MetricDef.NETTY_CLI_SYNC_DISR_QUEUE + ":" + + nettyConnection.toString(), cacheQueueGauge); + } + + /** + * Enqueue a task message to be sent to server + */ + @Override + public void send(List messages) { + for (TaskMessage msg : messages) { + disruptorQueue.publish(msg); + } + } + + @Override + public void send(TaskMessage message) { + disruptorQueue.publish(message); + } + + public void flushBatch(MessageBatch batch, Channel channel) { + emitTs.set(System.currentTimeMillis()); + if (batch == null) { + LOG.warn("Handle no data to {}, this shouldn't occur", name); + + } else if (channel == null || channel.isWritable() == false) { + LOG.warn("Channel occur exception, during batch messages {}", name); + batchQueue.offer(batch); + } else { + + flushRequest(channel, batch); + } + } + + /** + * Don't take care of competition + * + * @param blocked + */ + public void sendData() { + long start = System.nanoTime(); + try { + MessageBatch batch = batchQueue.poll(); + if (batch == null) { + + disruptorQueue.consumeBatchWhenAvailable(this); + + batch = batchQueue.poll(); + } + + Channel channel = channelRef.get(); + flushBatch(batch, channel); + } catch (Throwable e) { + LOG.error("Occur e", e); + String err = name + " nettyclient occur unknow exception"; + JStormUtils.halt_process(-1, err); + } finally { + long end = System.nanoTime(); + sendTimer.update((end - start) / 1000000.0d); + } + } + + public void sendAllData() { + + long start = System.nanoTime(); + try { + disruptorQueue.consumeBatch(this); + MessageBatch batch = batchQueue.poll(); + while (batch != null) { + Channel channel = channelRef.get(); + if (channel == null) { + LOG.info("No channel {} to flush all data", name); + return; + } else if (channel.isWritable() == false) { + LOG.info("Channel {} is no writable", name); + return; + } + flushBatch(batch, channel); + batch = batchQueue.poll(); + } + } catch (Throwable e) { + LOG.error("Occur e", e); + String err = name + " nettyclient occur unknow exception"; + JStormUtils.halt_process(-1, err); + } finally { + long end = System.nanoTime(); + sendTimer.update((end - start) / 1000000.0d); + } + } + + @Override + public void handleResponse() { + emitTs.set(0); + sendData(); + } + + @Override + public void onEvent(Object event, long sequence, boolean endOfBatch) + throws Exception { + if (event == null) { + return; + } + + TaskMessage message = (TaskMessage) event; + + MessageBatch messageBatch = messageBatchRef.getAndSet(null); + if (null == messageBatch) { + messageBatch = new MessageBatch(messageBatchSize); + } + + messageBatch.add(message); + + if (messageBatch.isFull()) { + batchQueue.offer(messageBatch); + } else if (endOfBatch == true) { + batchQueue.offer(messageBatch); + } else { + messageBatchRef.set(messageBatch); + } + } + + /** + * Handle lost message case + */ + void trigger() { + if (isClosed() == true) { + return; + } + + // if long time no receive NettyServer response + // it is likely lost message + long emitTime = emitTs.get(); + if (emitTime == 0) { + return; + } + + long now = System.currentTimeMillis(); + + long delt = now - emitTime; + if (delt < timeoutMs) { + return; + } + + Channel channel = channelRef.get(); + if (channel != null) { + LOG.info("Long time no response of {}, {}s", name, delt / 1000); + channel.write(ControlMessage.EOB_MESSAGE); + } + + } + + protected void shutdownPool() { + bossExecutor.shutdownNow(); + workerExecutor.shutdownNow(); + + try { + bossExecutor.awaitTermination(1, TimeUnit.SECONDS); + workerExecutor.awaitTermination(1, TimeUnit.SECONDS); + } catch (InterruptedException e) { + LOG.error("Error when shutting down client scheduler", e); + } + + clientChannelFactory.releaseExternalResources(); + } + + public void unregisterSyncMetrics() { + JStormMetrics.unregisterWorkerMetric( + MetricDef.NETTY_CLI_SYNC_BATCH_QUEUE, + nettyConnection.toString()); + JStormMetrics + .unregisterWorkerMetric(MetricDef.NETTY_CLI_SYNC_DISR_QUEUE, + nettyConnection.toString()); + JStormHealthCheck + .unregisterWorkerHealthCheck(MetricDef.NETTY_CLI_SYNC_DISR_QUEUE + + ":" + nettyConnection.toString()); + } + + @Override + public void close() { + LOG.info( + "Begin to close connection to {} and flush all data, batchQueue {}, disruptor {}", + name, batchQueue.size(), disruptorQueue.population()); + sendAllData(); + disruptorQueue.haltWithInterrupt(); + if (connectMyself == false) { + unregisterSyncMetrics(); + } + + super.close(); + + shutdownPool(); + + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyConnection.java b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyConnection.java new file mode 100755 index 000000000..cd8c0fade --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyConnection.java @@ -0,0 +1,91 @@ +/** + * 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.message.netty; + +import java.io.Serializable; + +import com.alibaba.jstorm.utils.NetWorkUtils; + +public class NettyConnection implements Serializable{ + protected String clientPort; + protected String serverPort; + + public String getClientPort() { + return clientPort; + } + + public void setClientPort(String client, int port) { + String ip = NetWorkUtils.host2Ip(client); + clientPort = ip + ":" + port; + } + + public String getServerPort() { + return serverPort; + } + + public void setServerPort(String server, int port) { + String ip = NetWorkUtils.host2Ip(server); + serverPort = ip + ":" + port; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = + prime * result + + ((clientPort == null) ? 0 : clientPort.hashCode()); + result = + prime * result + + ((serverPort == null) ? 0 : serverPort.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + NettyConnection other = (NettyConnection) obj; + if (clientPort == null) { + if (other.clientPort != null) + return false; + } else if (!clientPort.equals(other.clientPort)) + return false; + if (serverPort == null) { + if (other.serverPort != null) + return false; + } else if (!serverPort.equals(other.serverPort)) + return false; + return true; + } + + @Override + public String toString() { + return clientPort + "->" + serverPort; + } + + public static String mkString(String client, int clientPort, + String server, int serverPort) { + return client + ":" + clientPort + "->" + server + ":" + serverPort; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyContext.java b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyContext.java new file mode 100644 index 000000000..a6ddd9a60 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyContext.java @@ -0,0 +1,150 @@ +/** + * 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.message.netty; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; + +import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.messaging.IConnection; +import backtype.storm.messaging.IContext; +import backtype.storm.utils.DisruptorQueue; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.callback.AsyncLoopThread; +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.utils.JStormUtils; + +public class NettyContext implements IContext { + private final static Logger LOG = LoggerFactory + .getLogger(NettyContext.class); + @SuppressWarnings("rawtypes") + private Map storm_conf; + + private NioClientSocketChannelFactory clientChannelFactory; + + private ScheduledExecutorService clientScheduleService; + private final int MAX_CLIENT_SCHEDULER_THREAD_POOL_SIZE = 5; + + private ReconnectRunnable reconnector; + + private boolean isSyncMode = false; + + @SuppressWarnings("unused") + public NettyContext() { + } + + /** + * initialization per Storm configuration + */ + @SuppressWarnings("rawtypes") + public void prepare(Map storm_conf) { + this.storm_conf = storm_conf; + + int maxWorkers = + Utils.getInt(storm_conf + .get(Config.STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS)); + ThreadFactory bossFactory = + new NettyRenameThreadFactory(MetricDef.NETTY_CLI + "boss"); + ThreadFactory workerFactory = + new NettyRenameThreadFactory(MetricDef.NETTY_CLI + "worker"); + + if (maxWorkers > 0) { + clientChannelFactory = + new NioClientSocketChannelFactory( + Executors.newCachedThreadPool(bossFactory), + Executors.newCachedThreadPool(workerFactory), + maxWorkers); + } else { + clientChannelFactory = + new NioClientSocketChannelFactory( + Executors.newCachedThreadPool(bossFactory), + Executors.newCachedThreadPool(workerFactory)); + } + int otherWorkers = + Utils.getInt(storm_conf.get(Config.TOPOLOGY_WORKERS), 1) - 1; + int poolSize = + Math.min(Math.max(1, otherWorkers), + MAX_CLIENT_SCHEDULER_THREAD_POOL_SIZE); + clientScheduleService = + Executors + .newScheduledThreadPool(poolSize, + new NettyRenameThreadFactory( + "client-schedule-service")); + + reconnector = new ReconnectRunnable(); + new AsyncLoopThread(reconnector, true, Thread.MIN_PRIORITY, true); + + isSyncMode = ConfigExtension.isNettySyncMode(storm_conf); + } + + @Override + public IConnection bind(String topology_id, int port, ConcurrentHashMap deserializedueue) { + IConnection retConnection = null; + try { + + retConnection = new NettyServer(storm_conf, port, isSyncMode, deserializedueue); + } catch (Throwable e) { + LOG.error("Failed to instance NettyServer", e.getCause()); + JStormUtils.halt_process(-1, "Failed to bind " + port); + } + + return retConnection; + } + + @Override + public IConnection connect(String topology_id, String host, int port) { + if (isSyncMode == true) { + return new NettyClientSync(storm_conf, clientChannelFactory, + clientScheduleService, host, port, reconnector); + } else { + return new NettyClientAsync(storm_conf, clientChannelFactory, + clientScheduleService, host, port, reconnector); + } + } + + /** + * terminate this context + */ + public void term() { + clientScheduleService.shutdown(); + // for (IConnection conn : connections) { + // conn.close(); + // } + try { + clientScheduleService.awaitTermination(10, TimeUnit.SECONDS); + } catch (InterruptedException e) { + LOG.error("Error when shutting down client scheduler", e); + } + // connections = null; + + clientChannelFactory.releaseExternalResources(); + + reconnector.shutdown(); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyRenameThreadFactory.java b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyRenameThreadFactory.java new file mode 100755 index 000000000..2e060c20e --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyRenameThreadFactory.java @@ -0,0 +1,55 @@ +/** + * 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.message.netty; + +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicInteger; + +import org.jboss.netty.util.ThreadNameDeterminer; +import org.jboss.netty.util.ThreadRenamingRunnable; + +public class NettyRenameThreadFactory implements ThreadFactory { + + static { + // Rename Netty threads + ThreadRenamingRunnable + .setThreadNameDeterminer(ThreadNameDeterminer.CURRENT); + } + + final ThreadGroup group; + final AtomicInteger index = new AtomicInteger(1); + final String name; + + NettyRenameThreadFactory(String name) { + SecurityManager s = System.getSecurityManager(); + group = + (s != null) ? s.getThreadGroup() : Thread.currentThread() + .getThreadGroup(); + this.name = name; + } + + public Thread newThread(Runnable r) { + Thread t = + new Thread(group, r, name + "-" + index.getAndIncrement(), 0); + if (t.isDaemon()) + t.setDaemon(false); + if (t.getPriority() != Thread.NORM_PRIORITY) + t.setPriority(Thread.NORM_PRIORITY); + return t; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyServer.java b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyServer.java new file mode 100644 index 000000000..d00b24f95 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/NettyServer.java @@ -0,0 +1,234 @@ +/** + * 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.message.netty; + +import java.net.InetSocketAddress; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; + +import org.jboss.netty.bootstrap.ServerBootstrap; +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelFactory; +import org.jboss.netty.channel.group.ChannelGroup; +import org.jboss.netty.channel.group.DefaultChannelGroup; +import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.messaging.IConnection; +import backtype.storm.messaging.TaskMessage; +import backtype.storm.utils.DisruptorQueue; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.client.ConfigExtension; + +class NettyServer implements IConnection { + private static final Logger LOG = LoggerFactory + .getLogger(NettyServer.class); + @SuppressWarnings("rawtypes") + Map storm_conf; + int port; + + // private LinkedBlockingQueue message_queue; + volatile ChannelGroup allChannels = + new DefaultChannelGroup("jstorm-server"); + final ChannelFactory factory; + final ServerBootstrap bootstrap; + + // ayncBatch is only one solution, so directly set it as true + private final boolean isSyncMode; + + private ConcurrentHashMap deserializeQueues; + + @SuppressWarnings("rawtypes") + NettyServer(Map storm_conf, int port, boolean isSyncMode, ConcurrentHashMap deserializeQueues) { + this.storm_conf = storm_conf; + this.port = port; + this.isSyncMode = isSyncMode; + this.deserializeQueues = deserializeQueues; + + // Configure the server. + int buffer_size = + Utils.getInt(storm_conf + .get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE)); + int maxWorkers = + Utils.getInt(storm_conf + .get(Config.STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS)); + + // asyncBatch = ConfigExtension.isNettyTransferAsyncBatch(storm_conf); + + ThreadFactory bossFactory = + new NettyRenameThreadFactory("server" + "-boss"); + ThreadFactory workerFactory = + new NettyRenameThreadFactory("server" + "-worker"); + if (maxWorkers > 0) { + factory = + new NioServerSocketChannelFactory( + Executors.newCachedThreadPool(bossFactory), + Executors.newCachedThreadPool(workerFactory), + maxWorkers); + } else { + factory = + new NioServerSocketChannelFactory( + Executors.newCachedThreadPool(bossFactory), + Executors.newCachedThreadPool(workerFactory)); + } + + bootstrap = new ServerBootstrap(factory); + bootstrap.setOption("reuserAddress", true); + bootstrap.setOption("child.tcpNoDelay", true); + bootstrap.setOption("child.receiveBufferSize", buffer_size); + bootstrap.setOption("child.keepAlive", true); + + // Set up the pipeline factory. + bootstrap.setPipelineFactory(new StormServerPipelineFactory(this, storm_conf)); + + // Bind and start to accept incoming connections. + Channel channel = bootstrap.bind(new InetSocketAddress(port)); + allChannels.add(channel); + + LOG.info("Successfull bind {}, buffer_size:{}, maxWorkers:{}", port, + buffer_size, maxWorkers); + } + + @Override + public void registerQueue(Integer taskId, DisruptorQueue recvQueu) { + deserializeQueues.put(taskId, recvQueu); + } + + /** + * enqueue a received message + * + * @param message + * @throws InterruptedException + */ + public void enqueue(TaskMessage message) { + + int task = message.task(); + + DisruptorQueue queue = deserializeQueues.get(task); + if (queue == null) { + LOG.debug("Received invalid message directed at port " + task + + ". Dropping..."); + return; + } + + queue.publish(message.message()); + } + + /** + * fetch a message from message queue synchronously (flags != 1) or + * asynchronously (flags==1) + */ + public Object recv(Integer taskId, int flags) { + try { + DisruptorQueue recvQueue = deserializeQueues.get(taskId); + if ((flags & 0x01) == 0x01) { + return recvQueue.poll(); + // non-blocking + + } else { + return recvQueue.take(); + + } + + } catch (Exception e) { + LOG.warn("Occur unexception ", e); + return null; + } + + } + + /** + * register a newly created channel + * + * @param channel + */ + protected void addChannel(Channel channel) { + allChannels.add(channel); + } + + /** + * close a channel + * + * @param channel + */ + protected void closeChannel(Channel channel) { + MessageDecoder.removeTransmitHistogram(channel); + channel.close().awaitUninterruptibly(); + allChannels.remove(channel); + } + + /** + * close all channels, and release resources + */ + public synchronized void close() { + LOG.info("Begin to shutdown NettyServer"); + if (allChannels != null) { + new Thread(new Runnable() { + + @Override + public void run() { + try { + // await(5, TimeUnit.SECONDS) + // sometimes allChannels.close() will block the exit + // thread + allChannels.close().await(1, TimeUnit.SECONDS); + LOG.info("Successfully close all channel"); + factory.releaseExternalResources(); + } catch (Exception e) { + + } + allChannels = null; + } + }).start(); + + JStormUtils.sleepMs(1 * 1000); + } + LOG.info("Successfully shutdown NettyServer"); + } + + @Override + public void send(List messages) { + throw new UnsupportedOperationException( + "Server connection should not send any messages"); + } + + @Override + public void send(TaskMessage message) { + throw new UnsupportedOperationException( + "Server connection should not send any messages"); + } + + @Override + public boolean isClosed() { + // TODO Auto-generated method stub + return false; + } + + public boolean isSyncMode() { + return isSyncMode; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/ReconnectRunnable.java b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/ReconnectRunnable.java new file mode 100755 index 000000000..dcf2a5d1c --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/ReconnectRunnable.java @@ -0,0 +1,72 @@ +/** + * 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.message.netty; + +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingDeque; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.callback.RunnableCallback; + +public class ReconnectRunnable extends RunnableCallback { + private static final Logger LOG = LoggerFactory + .getLogger(ReconnectRunnable.class); + + private BlockingQueue queue = + new LinkedBlockingDeque(); + + public void pushEvent(NettyClient client) { + queue.offer(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 { + client = queue.take(); + } catch (InterruptedException e) { + continue; + } + if (client != null) { + client.doReconnect(); + } + + } + + LOG.info("Successfully shutdown reconnect thread"); + } + + @Override + public void shutdown() { + closed = true; + thread.interrupt(); + } + + @Override + public Object getResult() { + return -1; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/StormClientHandler.java b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/StormClientHandler.java new file mode 100755 index 000000000..f84c2f0c3 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/StormClientHandler.java @@ -0,0 +1,107 @@ +/** + * 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.message.netty; + +import java.net.ConnectException; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelHandlerContext; +import org.jboss.netty.channel.ChannelStateEvent; +import org.jboss.netty.channel.ExceptionEvent; +import org.jboss.netty.channel.MessageEvent; +import org.jboss.netty.channel.SimpleChannelUpstreamHandler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class StormClientHandler extends SimpleChannelUpstreamHandler { + private static final Logger LOG = LoggerFactory + .getLogger(StormClientHandler.class); + private NettyClient client; + private AtomicBoolean being_closed; + + StormClientHandler(NettyClient client) { + this.client = client; + being_closed = client.getBeing_closed(); + } + + /** + * Sometime when connect one bad channel which isn't writable, it will call + * this function + */ + @Override + public void channelConnected(ChannelHandlerContext ctx, + ChannelStateEvent event) { + // register the newly established channel + Channel channel = event.getChannel(); + LOG.info("connection established to :{}, local port:{}", + client.getRemoteAddr(), channel.getLocalAddress()); + + client.handleResponse(); + } + + @Override + public void messageReceived(ChannelHandlerContext ctx, MessageEvent event) { + client.handleResponse(); + + } + + /** + * + * @see org.jboss.netty.channel.SimpleChannelUpstreamHandler#exceptionCaught(org.jboss.netty.channel.ChannelHandlerContext, + * org.jboss.netty.channel.ExceptionEvent) + */ + @Override + public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent event) { + Throwable cause = event.getCause(); + if (being_closed.get() == false) { + if (!(cause instanceof ConnectException)) { + LOG.info("Connection failed:" + client.getRemoteAddr(), cause); + } + + client.exceptionChannel(event.getChannel()); + client.reconnect(); + } + } + + /** + * Attention please, + * + * @see org.jboss.netty.channel.SimpleChannelUpstreamHandler#channelDisconnected(org.jboss.netty.channel.ChannelHandlerContext, + * org.jboss.netty.channel.ChannelStateEvent) + */ + @Override + public void channelDisconnected(ChannelHandlerContext ctx, + ChannelStateEvent e) throws Exception { + LOG.info("Receive channelDisconnected to {}, channel = {}", + client.getRemoteAddr(), e.getChannel()); + // ctx.sendUpstream(e); + super.channelDisconnected(ctx, e); + + client.disconnectChannel(e.getChannel()); + } + + @Override + public void channelClosed(ChannelHandlerContext ctx, ChannelStateEvent e) + throws Exception { + LOG.info("Connection to {} has been closed, channel = {}", + client.getRemoteAddr(), e.getChannel()); + super.channelClosed(ctx, e); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/StormClientPipelineFactory.java b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/StormClientPipelineFactory.java new file mode 100755 index 000000000..080f91cef --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/StormClientPipelineFactory.java @@ -0,0 +1,51 @@ +/** + * 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.message.netty; + +import java.util.Map; + +import org.jboss.netty.channel.ChannelPipeline; +import org.jboss.netty.channel.ChannelPipelineFactory; +import org.jboss.netty.channel.Channels; + +import com.alibaba.jstorm.client.ConfigExtension; + +class StormClientPipelineFactory implements ChannelPipelineFactory { + private NettyClient client; + private Map conf; + + StormClientPipelineFactory(NettyClient client, Map conf) { + this.client = client; + this.conf = conf; + + } + + public ChannelPipeline getPipeline() throws Exception { + // Create a default pipeline implementation. + ChannelPipeline pipeline = Channels.pipeline(); + + // Decoder + pipeline.addLast("decoder", new MessageDecoder(false, conf)); + // Encoder + pipeline.addLast("encoder", new MessageEncoder()); + // business logic. + pipeline.addLast("handler", new StormClientHandler(client)); + + return pipeline; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/StormServerHandler.java b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/StormServerHandler.java new file mode 100755 index 000000000..916ce93d3 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/StormServerHandler.java @@ -0,0 +1,140 @@ +/** + * 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.message.netty; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelHandlerContext; +import org.jboss.netty.channel.ChannelStateEvent; +import org.jboss.netty.channel.ChildChannelStateEvent; +import org.jboss.netty.channel.ExceptionEvent; +import org.jboss.netty.channel.MessageEvent; +import org.jboss.netty.channel.SimpleChannelUpstreamHandler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.messaging.TaskMessage; + +class StormServerHandler extends SimpleChannelUpstreamHandler { + private static final Logger LOG = LoggerFactory + .getLogger(StormServerHandler.class); + private NettyServer server; + private Map failureCounters; + + StormServerHandler(NettyServer server) { + this.server = server; + failureCounters = new ConcurrentHashMap(); + } + + protected int getFailureCounter(Channel channel) { + Integer num = failureCounters.get(channel); + if (num == null) { + return 0; + } + + return num; + } + + protected void incFailureCounter(Channel channel) { + Integer num = failureCounters.get(channel); + if (num == null) { + num = Integer.valueOf(0); + } + num = num + 1; + + failureCounters.put(channel, num); + } + + protected void removeFailureCounter(Channel channel) { + failureCounters.remove(channel); + } + + @Override + public void channelConnected(ChannelHandlerContext ctx, ChannelStateEvent e) { + LOG.info("Connection established {}", e.getChannel().getRemoteAddress()); + server.addChannel(e.getChannel()); + } + + @Override + public void childChannelClosed( + ChannelHandlerContext ctx, ChildChannelStateEvent e) throws Exception { + super.childChannelClosed(ctx, e); + LOG.info("Connection closed {}", e.getChildChannel().getRemoteAddress()); + + MessageDecoder.removeTransmitHistogram(e.getChildChannel()); + } + + @Override + public void channelDisconnected(ChannelHandlerContext ctx, ChannelStateEvent e) throws Exception { + super.channelDisconnected(ctx, e); + LOG.info("Connection channelDisconnected {}", e.getChannel().getRemoteAddress()); + + MessageDecoder.removeTransmitHistogram(e.getChannel()); + }; + + @Override + public void channelClosed(ChannelHandlerContext ctx, ChannelStateEvent e) throws Exception { + super.channelClosed(ctx, e); + LOG.info("Connection channelClosed {}", e.getChannel().getRemoteAddress()); + + MessageDecoder.removeTransmitHistogram(e.getChannel()); + }; + + @Override + public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) { + + Object msg = e.getMessage(); + if (msg == null) + return; + + // end of batch? + if (msg == ControlMessage.EOB_MESSAGE) { + if (server.isSyncMode() == true) { + Channel channel = ctx.getChannel(); + // simplify the logic, just send OK_RESPONSE + channel.write(ControlMessage.OK_RESPONSE); + } + return; + } else if (msg instanceof ControlMessage) { + // LOG.debug("Receive ...{}", msg); + return; + } + + // enqueue the received message for processing + try { + server.enqueue((TaskMessage) msg); + } catch (Exception e1) { + LOG.warn("Failed to enqueue a request message" + e1.toString(), e); + // Channel channel = ctx.getChannel(); + // incFailureCounter(channel); + } + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) { + // removeFailureCounter(e.getChannel()); + if (e.getChannel() != null) { + LOG.info("Channel occur exception {}", e.getChannel() + .getRemoteAddress()); + } + + server.closeChannel(e.getChannel()); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/StormServerPipelineFactory.java b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/StormServerPipelineFactory.java new file mode 100755 index 000000000..9dead91cf --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/message/netty/StormServerPipelineFactory.java @@ -0,0 +1,48 @@ +/** + * 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.message.netty; + +import java.util.Map; + +import org.jboss.netty.channel.ChannelPipeline; +import org.jboss.netty.channel.ChannelPipelineFactory; +import org.jboss.netty.channel.Channels; + +class StormServerPipelineFactory implements ChannelPipelineFactory { + private NettyServer server; + private Map conf; + + StormServerPipelineFactory(NettyServer server, Map conf) { + this.server = server; + this.conf = conf; + } + + public ChannelPipeline getPipeline() throws Exception { + // Create a default pipeline implementation. + ChannelPipeline pipeline = Channels.pipeline(); + + // Decoder + pipeline.addLast("decoder", new MessageDecoder(true, conf)); + // Encoder + pipeline.addLast("encoder", new MessageEncoder()); + // business logic. + pipeline.addLast("handler", new StormServerHandler(server)); + + return pipeline; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/metric/AlimonitorClient.java b/jstorm-core/src/main/java/com/alibaba/jstorm/metric/AlimonitorClient.java new file mode 100755 index 000000000..760e538b8 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/metric/AlimonitorClient.java @@ -0,0 +1,267 @@ +package com.alibaba.jstorm.metric; + +import java.net.URLEncoder; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.http.HttpEntity; +import org.apache.http.NameValuePair; +import org.apache.http.client.entity.UrlEncodedFormEntity; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClientBuilder; +import org.apache.http.message.BasicNameValuePair; +import org.apache.http.util.EntityUtils; +import org.apache.log4j.Logger; + +import backtype.storm.utils.Utils; + +public class AlimonitorClient extends MetricSendClient { + + public static Logger LOG = Logger.getLogger(AlimonitorClient.class); + + // Send to localhost:15776 by default + public static final String DEFAUT_ADDR = "127.0.0.1"; + public static final String DEFAULT_PORT = "15776"; + public static final int DEFAUTL_FLAG = 0; + public static final String DEFAULT_ERROR_INFO = ""; + + private final String COLLECTION_FLAG = "collection_flag"; + private final String ERROR_INFO = "error_info"; + private final String MSG = "MSG"; + + private String port; + private String requestIP; + private String monitorName; + private int collectionFlag; + private String errorInfo; + + private boolean post; + + public AlimonitorClient() { + } + + public AlimonitorClient(String requestIP, String port, boolean post) { + this.requestIP = requestIP; + this.port = port; + this.post = post; + this.monitorName = null; + this.collectionFlag = 0; + this.errorInfo = null; + } + + public void setIpAddr(String ipAddr) { + this.requestIP = ipAddr; + } + + public void setPort(String port) { + this.port = port; + } + + public void setMonitorName(String monitorName) { + this.monitorName = monitorName; + } + + public void setCollectionFlag(int flag) { + this.collectionFlag = flag; + } + + public void setErrorInfo(String msg) { + this.errorInfo = msg; + } + + public void setPostFlag(boolean post) { + this.post = post; + } + + public String buildURL() { + return "http://" + requestIP + ":" + port + "/passive"; + } + + public String buildRqstAddr() { + return "http://" + requestIP + ":" + port + "/passive?name=" + + monitorName + "&msg="; + } + + @Override + public boolean send(Map msg) { + try { + if (monitorName == null) { + LOG.warn("monitor name is null"); + return false; + } + return sendRequest(collectionFlag, errorInfo, msg); + } catch (Exception e) { + LOG.error("Failed to sendRequest", e); + return false; + } + } + + @Override + public boolean send(List> msg) { + try { + if (monitorName == null) { + LOG.warn("monitor name is null"); + return false; + } + return sendRequest(collectionFlag, errorInfo, msg); + } catch (Exception e) { + LOG.error("Failed to sendRequest", e); + return false; + } + } + + public Map buildAliMonitorMsg(int collection_flag, String error_message) { + // Json format of the message sent to Alimonitor + // { + // "collection_flag":int, + // "error_info":string, + // "MSG": ojbect | array + // } + Map ret = new HashMap(); + ret.put(COLLECTION_FLAG, collection_flag); + ret.put(ERROR_INFO, error_message); + ret.put(MSG, null); + + return ret; + } + + private void addMsgData(Map jsonObj, Map map) { + jsonObj.put(MSG, map); + } + + private void addMsgData(Map jsonObj, List> mapList) { + // JSONArray jsonArray = new JSONArray(); + // for(Map map : mapList) { + // jsonArray.add(map); + // } + + jsonObj.put(MSG, mapList); + } + + private boolean sendRequest(int collection_flag, String error_message, + Map msg) throws Exception { + boolean ret = false; + + if (msg.size() == 0) + return ret; + + Map jsonObj = buildAliMonitorMsg(collection_flag, error_message); + addMsgData(jsonObj, msg); + String jsonMsg = jsonObj.toString(); + LOG.info(jsonMsg); + + if (post == true) { + String url = buildURL(); + ret = httpPost(url, jsonMsg); + } else { + String request = buildRqstAddr(); + StringBuilder postAddr = new StringBuilder(); + postAddr.append(request); + postAddr.append(URLEncoder.encode(jsonMsg)); + + ret = httpGet(postAddr); + } + + return ret; + } + + private boolean sendRequest(int collection_flag, String error_message, + List> msgList) throws Exception { + boolean ret = false; + + if (msgList.size() == 0) + return ret; + + Map jsonObj = buildAliMonitorMsg(collection_flag, error_message); + addMsgData(jsonObj, msgList); + + String jsonMsg = Utils.to_json(jsonObj); + LOG.info(jsonMsg); + + if (post == true) { + String url = buildURL(); + ret = httpPost(url, jsonMsg); + } else { + String request = buildRqstAddr(); + StringBuilder postAddr = new StringBuilder(); + postAddr.append(request); + postAddr.append(URLEncoder.encode(jsonMsg)); + + ret = httpGet(postAddr); + } + + return ret; + } + + private boolean httpGet(StringBuilder postAddr) { + boolean ret = false; + + CloseableHttpClient httpClient = HttpClientBuilder.create().build(); + CloseableHttpResponse response = null; + + try { + HttpGet request = new HttpGet(postAddr.toString()); + response = httpClient.execute(request); + HttpEntity entity = response.getEntity(); + if (entity != null) { + LOG.info(EntityUtils.toString(entity)); + } + EntityUtils.consume(entity); + ret = true; + } catch (Exception e) { + LOG.error("Exception when sending http request to alimonitor", e); + } finally { + try { + if (response != null) + response.close(); + httpClient.close(); + } catch (Exception e) { + LOG.error("Exception when closing httpclient", e); + } + } + + return ret; + } + + private boolean httpPost(String url, String msg) { + boolean ret = false; + + CloseableHttpClient httpClient = HttpClientBuilder.create().build(); + CloseableHttpResponse response = null; + + try { + HttpPost request = new HttpPost(url); + List nvps = new ArrayList(); + nvps.add(new BasicNameValuePair("name", monitorName)); + nvps.add(new BasicNameValuePair("msg", msg)); + request.setEntity(new UrlEncodedFormEntity(nvps)); + response = httpClient.execute(request); + HttpEntity entity = response.getEntity(); + if (entity != null) { + LOG.info(EntityUtils.toString(entity)); + } + EntityUtils.consume(entity); + ret = true; + } catch (Exception e) { + LOG.error("Exception when sending http request to alimonitor", e); + } finally { + try { + if (response != null) + response.close(); + httpClient.close(); + } catch (Exception e) { + LOG.error("Exception when closing httpclient", e); + } + } + + return ret; + } + + public void close() { + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/metric/JStormHealthCheck.java b/jstorm-core/src/main/java/com/alibaba/jstorm/metric/JStormHealthCheck.java new file mode 100755 index 000000000..631c38b6e --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/metric/JStormHealthCheck.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.metric; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.codahale.metrics.health.HealthCheck; +import com.codahale.metrics.health.HealthCheckRegistry; + +public class JStormHealthCheck { + private static final Logger LOG = LoggerFactory + .getLogger(JStormHealthCheck.class); + + private final static Map taskHealthCheckMap = + new ConcurrentHashMap(); + + private final static HealthCheckRegistry workerHealthCheck = + new HealthCheckRegistry(); + + public static void registerTaskHealthCheck(int taskId, String name, + HealthCheck healthCheck) { + HealthCheckRegistry healthCheckRegister = + taskHealthCheckMap.get(taskId); + + if (healthCheckRegister == null) { + healthCheckRegister = new HealthCheckRegistry(); + taskHealthCheckMap.put(taskId, healthCheckRegister); + } + + healthCheckRegister.register(name, healthCheck); + } + + public static void registerWorkerHealthCheck(String name, + HealthCheck healthCheck) { + workerHealthCheck.register(name, healthCheck); + } + + public static void unregisterTaskHealthCheck(int taskId, String name) { + HealthCheckRegistry healthCheckRegister = + taskHealthCheckMap.get(taskId); + + if (healthCheckRegister != null) { + healthCheckRegister.unregister(name); + } + + } + + public static void unregisterWorkerHealthCheck(String name) { + workerHealthCheck.unregister(name); + } + + public static Map getTaskhealthcheckmap() { + return taskHealthCheckMap; + } + + public static HealthCheckRegistry getWorkerhealthcheck() { + return workerHealthCheck; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/metric/JStormMetrics.java b/jstorm-core/src/main/java/com/alibaba/jstorm/metric/JStormMetrics.java new file mode 100755 index 000000000..8221cd88b --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/metric/JStormMetrics.java @@ -0,0 +1,281 @@ +/** + * 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.metric; + +import java.io.Serializable; +import java.lang.management.ManagementFactory; +import java.lang.management.MemoryMXBean; +import java.lang.management.MemoryUsage; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.generated.MetricInfo; + +import com.alibaba.jstorm.common.metric.Counter; +import com.alibaba.jstorm.common.metric.Gauge; +import com.alibaba.jstorm.common.metric.Histogram; +import com.alibaba.jstorm.common.metric.Meter; +import com.alibaba.jstorm.common.metric.MetricRegistry; +import com.alibaba.jstorm.common.metric.Timer; +import com.alibaba.jstorm.common.metric.window.Metric; +import com.alibaba.jstorm.utils.JStormUtils; + +public class JStormMetrics implements Serializable { + private static final Logger LOG = LoggerFactory + .getLogger(JStormMetrics.class); + private static final long serialVersionUID = 2046603514943797241L; + + /** + * Metrics in this object will be uploaded to nimbus + */ + static MetricRegistry workerMetrics = new MetricRegistry(); + static Map taskMetrics = + new ConcurrentHashMap(); + /** + * Metrics in this object will be just be output to log, won't be uploaded + * to nimbus + */ + static MetricRegistry skipMetrics = new MetricRegistry(); + + protected static MetricInfo exposeWorkerMetrics; + protected static Map exposeNettyMetrics; + protected static Map exposeTaskMetrics; + + static { + registerWorkerGauge(new com.codahale.metrics.Gauge() { + + @Override + public Double getValue() { + // TODO Auto-generated method stub + return JStormUtils.getCpuUsage(); + } + + }, MetricDef.CPU_USED_RATIO); + + registerWorkerGauge(new com.codahale.metrics.Gauge() { + + @Override + public Double getValue() { + // TODO Auto-generated method stub + return JStormUtils.getMemUsage(); + } + + }, MetricDef.MEMORY_USED); + } + + public static MetricRegistry registerTask(int taskId) { + MetricRegistry ret = taskMetrics.get(taskId); + if (ret == null) { + ret = new MetricRegistry(); + taskMetrics.put(taskId, ret); + LOG.info("Register task MetricRegistry " + taskId); + } + + return ret; + } + + public static void unregisterTask(int taskId) { + taskMetrics.remove(taskId); + } + + // the Metric should be one of metrics of task + // if register this metric through this function, + // the web UI would do sum operation for the metric + // the metric will display in component/topology level in web UI + public static void registerSumMetric(String name) { + MetricDef.MERGE_SUM_TAG.add(name); + } + + public static void unregisterSumMetric(String name) { + MetricDef.MERGE_SUM_TAG.remove(name); + } + + // the Metric should be one of metrics of task + // if register this metric through this function, + // the web UI would do sum operation for the metric + // the metric will display in component/topology level in web UI + public static void registerAvgMetric(String name) { + MetricDef.MERGE_AVG_TAG.add(name); + } + + public static void unregisterAvgMetric(String name) { + MetricDef.MERGE_AVG_TAG.remove(name); + } + + public static T registerWorkerMetric(T metric, + String name, String... args) throws IllegalArgumentException { + String registerName = MetricRegistry.name(name, args); + + return workerMetrics.register(registerName, metric); + } + + public static void unregisterWorkerMetric(String name, String... args) { + String registerName = MetricRegistry.name(name, args); + + workerMetrics.remove(registerName); + } + + public static T registerTaskMetric(T metric, int taskId, + String name, String... args) throws IllegalArgumentException { + MetricRegistry metrics = taskMetrics.get(taskId); + if (metrics == null) { + throw new IllegalArgumentException("Invalid taskId " + taskId); + } + + String registerName = MetricRegistry.name(name, args); + + return metrics.register(registerName, metric); + } + + public static void unregisterTaskMetric(int taskId, String name, + String... args) throws IllegalArgumentException { + String registerName = MetricRegistry.name(name, args); + MetricRegistry metrics = taskMetrics.get(taskId); + if (metrics == null) { + throw new IllegalArgumentException("Invalid taskId"); + } + metrics.remove(registerName); + } + + public static Gauge registerWorkerGauge( + com.codahale.metrics.Gauge rawGauge, String name, + String... args) { + Gauge ret = new Gauge(rawGauge); + registerWorkerMetric(ret, name, args); + return ret; + } + + public static Gauge registerTaskGauge( + com.codahale.metrics.Gauge rawGauge, int taskId, + String name, String... args) { + Gauge ret = new Gauge(rawGauge); + registerTaskMetric(ret, taskId, name, args); + return ret; + } + + public static Counter registerWorkerCounter(String name, + String... args) throws IllegalArgumentException { + Counter ret = + (Counter) Builder.mkInstance(Builder.COUNTER); + registerWorkerMetric(ret, name, args); + return ret; + } + + public static Counter registerTaskCounter(int taskId, String name, + String... args) { + Counter ret = + (Counter) Builder.mkInstance(Builder.COUNTER); + registerTaskMetric(ret, taskId, name, args); + return ret; + } + + public static Meter registerWorkerMeter(String name, String... args) + throws IllegalArgumentException { + Meter ret = (Meter) Builder.mkInstance(Builder.METER); + registerWorkerMetric(ret, name, args); + return ret; + } + + public static Meter registerTaskMeter(int taskId, String name, + String... args) { + Meter ret = (Meter) Builder.mkInstance(Builder.METER); + registerTaskMetric(ret, taskId, name, args); + return ret; + } + + public static Histogram registerWorkerHistogram(String name, String... args) + throws IllegalArgumentException { + Histogram ret = (Histogram) Builder.mkInstance(Builder.HISTOGRAM); + registerWorkerMetric(ret, name, args); + return ret; + } + + public static Histogram registerTaskHistogram(int taskId, String name, + String... args) { + Histogram ret = (Histogram) Builder.mkInstance(Builder.HISTOGRAM); + registerTaskMetric(ret, taskId, name, args); + return ret; + } + + public static Timer registerWorkerTimer(String name, String... args) + throws IllegalArgumentException { + Timer ret = (Timer) Builder.mkInstance(Builder.TIMER); + registerWorkerMetric(ret, name, args); + return ret; + } + + public static Timer registerTaskTimer(int taskId, String name, + String... args) { + Timer ret = (Timer) Builder.mkInstance(Builder.TIMER); + registerTaskMetric(ret, taskId, name, args); + return ret; + } + + public static class Builder { + public static final int COUNTER = 1; + public static final int METER = 2; + public static final int HISTOGRAM = 3; + public static final int TIMER = 4; + + public static Metric mkInstance(int type) { + if (type == COUNTER) { + return new Counter(Double.valueOf(0)); + } else if (type == METER) { + return new Meter(); + } else if (type == HISTOGRAM) { + return new Histogram(); + } else if (type == TIMER) { + return new Timer(); + } else { + throw new IllegalArgumentException(); + } + } + } + + public static MetricInfo getExposeWorkerMetrics() { + return exposeWorkerMetrics; + } + + public static void setExposeWorkerMetrics(MetricInfo exposeWorkerMetrics) { + JStormMetrics.exposeWorkerMetrics = exposeWorkerMetrics; + } + + public static Map getExposeTaskMetrics() { + return exposeTaskMetrics; + } + + public static void setExposeTaskMetrics( + Map exposeTaskMetrics) { + JStormMetrics.exposeTaskMetrics = exposeTaskMetrics; + } + + public static Map getExposeNettyMetrics() { + return exposeNettyMetrics; + } + + public static void setExposeNettyMetrics(Map exposeNettyMetrics) { + JStormMetrics.exposeNettyMetrics = exposeNettyMetrics; + } + + + + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/metric/JStormMetricsReporter.java b/jstorm-core/src/main/java/com/alibaba/jstorm/metric/JStormMetricsReporter.java new file mode 100755 index 000000000..489bec8b6 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/metric/JStormMetricsReporter.java @@ -0,0 +1,406 @@ +/** + * 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.metric; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.LocalCluster; +import backtype.storm.generated.MetricInfo; +import backtype.storm.generated.MetricWindow; +import backtype.storm.generated.NettyMetric; +import backtype.storm.generated.WorkerUploadMetrics; +import backtype.storm.utils.NimbusClient; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.common.metric.Gauge; +import com.alibaba.jstorm.common.metric.MetricFilter; +import com.alibaba.jstorm.common.metric.MetricRegistry; +import com.alibaba.jstorm.common.metric.window.Metric; +import com.alibaba.jstorm.daemon.worker.WorkerData; +import com.alibaba.jstorm.utils.JStormUtils; +import com.codahale.metrics.health.HealthCheck; +import com.codahale.metrics.health.HealthCheckRegistry; + +public class JStormMetricsReporter extends RunnableCallback { + private static final Logger LOG = LoggerFactory + .getLogger(JStormMetricsReporter.class); + + private MetricRegistry workerMetrics = JStormMetrics.workerMetrics; + private Map taskMetrics = + JStormMetrics.taskMetrics; + private MetricRegistry skipMetrics = JStormMetrics.skipMetrics; + + private JStormMetricFilter inputFilter; + + private JStormMetricFilter outputFilter; + + private Map conf; + private String topologyId; + private String supervisorId; + private int port; + private int frequence; + + private StormClusterState clusterState; + private boolean localMode = false; + private NimbusClient client; + + public JStormMetricsReporter(WorkerData workerData) { + this.conf = workerData.getStormConf(); + this.topologyId = (String) conf.get(Config.TOPOLOGY_ID); + this.supervisorId = workerData.getSupervisorId(); + this.port = workerData.getPort(); + this.frequence = ConfigExtension.getWorkerMetricReportFrequency(conf); + this.clusterState = workerData.getZkCluster(); + + outputFilter = new JStormMetricFilter(MetricDef.OUTPUT_TAG); + inputFilter = new JStormMetricFilter(MetricDef.INPUT_TAG); + localMode = StormConfig.local_mode(conf); + LOG.info("Successfully start "); + } + + protected boolean getMoreMetric( + Map> extraMap, + JStormMetricFilter metricFilter, String metricFullName, + Map metricWindow) { + if (metricFilter.matches(metricFullName, null) == false) { + return false; + } + + int pos = metricFullName.indexOf(MetricRegistry.NAME_SEPERATOR); + if (pos <= 0 || pos >= metricFullName.length() - 1) { + return false; + } + + String metricName = metricFullName.substring(0, pos); + String extraName = metricFullName.substring(pos + 1); + + Map item = extraMap.get(metricName); + if (item == null) { + item = new HashMap(); + extraMap.put(metricName, item); + } + + MetricWindow metricWindowThrift = new MetricWindow(); + metricWindowThrift.set_metricWindow(metricWindow); + + item.put(extraName, metricWindowThrift); + + return true; + } + + protected void insertNettyMetrics(Map nettyMetricInfo, + Map snapshot, + String metricFullName) { + int pos = metricFullName.indexOf(MetricRegistry.NAME_SEPERATOR); + if (pos < 0 || pos >= metricFullName.length() - 1) { + return ; + } + + String realHeader = metricFullName.substring(0, pos); + String nettyConnection = metricFullName.substring(pos + 1); + + MetricInfo metricInfo = nettyMetricInfo.get(nettyConnection); + if (metricInfo == null) { + metricInfo = MetricThrift.mkMetricInfo(); + + nettyMetricInfo.put(nettyConnection, metricInfo); + } + + MetricThrift.insert(metricInfo, realHeader, snapshot); + } + + protected void insertMergeList(Map > > mergeMap, + List mergeList, + Map snapshot, + String name) { + for (String tag : mergeList) { + if (name.startsWith(tag) == false) { + continue; + } + List > list = mergeMap.get(tag); + if (list == null) { + list = new ArrayList>(); + mergeMap.put(tag, list); + } + + list.add(snapshot); + + } + } + + protected void doMergeList(MetricInfo workerMetricInfo, + Map > > mergeMap) { + for (Entry > > entry : mergeMap.entrySet()) { + String name = entry.getKey(); + List> list = entry.getValue(); + + Map merged = JStormUtils.mergeMapList(list); + + MetricThrift.insert(workerMetricInfo, name, merged); + } + } + + public MetricInfo computWorkerMetrics() { + MetricInfo workerMetricInfo = MetricThrift.mkMetricInfo(); + Map nettyMetricInfo = new HashMap(); + + Map > > mergeMap = + new HashMap > >(); + List mergeList = new ArrayList(); + mergeList.add(MetricDef.NETTY_CLI_SEND_SPEED); + + Map workerMetricMap = workerMetrics.getMetrics(); + for (Entry entry : workerMetricMap.entrySet()) { + String name = entry.getKey(); + Map snapshot = entry.getValue().getSnapshot(); + + if (MetricDef.isNettyDetails(name) == false) { + MetricThrift.insert(workerMetricInfo, name, snapshot); + continue; + } + + insertNettyMetrics(nettyMetricInfo, snapshot, name); + + insertMergeList(mergeMap, mergeList, snapshot, name); + + } + + doMergeList(workerMetricInfo, mergeMap); + + JStormMetrics.setExposeWorkerMetrics(workerMetricInfo); + JStormMetrics.setExposeNettyMetrics(nettyMetricInfo); + return workerMetricInfo; + } + + public boolean isTaskQueueFull(Metric metric, + Map snapshot, String name) { + if (metric instanceof Gauge) { + if (MetricDef.TASK_QUEUE_SET.contains(name)) { + for (Entry entry : snapshot.entrySet()) { + if (entry.getValue() == MetricDef.FULL_RATIO) { + return true; + } + } + } + } + + return false; + } + + public Map computeTaskMetrics() { + Map ret = new HashMap(); + + for (Entry entry : taskMetrics.entrySet()) { + Integer taskId = entry.getKey(); + MetricRegistry taskMetrics = entry.getValue(); + + Map> inputMap = + new HashMap>(); + Map> outputMap = + new HashMap>(); + + MetricInfo taskMetricInfo = MetricThrift.mkMetricInfo(); + taskMetricInfo.set_inputMetric(inputMap); + taskMetricInfo.set_outputMetric(outputMap); + ret.put(taskId, taskMetricInfo); + + for (Entry metricEntry : taskMetrics.getMetrics() + .entrySet()) { + String name = metricEntry.getKey(); + Metric metric = metricEntry.getValue(); + Map snapshot = metric.getSnapshot(); + + boolean isInput = + getMoreMetric(inputMap, inputFilter, name, snapshot); + boolean isOutput = + getMoreMetric(outputMap, outputFilter, name, snapshot); + + if (isInput == false && isOutput == false) { + MetricThrift.insert(taskMetricInfo, name, snapshot); + } + } + + MetricThrift.merge(taskMetricInfo, inputMap); + MetricThrift.merge(taskMetricInfo, outputMap); + + } + + JStormMetrics.setExposeTaskMetrics(ret); + return ret; + } + + public void healthCheck(Integer taskId, HealthCheckRegistry healthCheck) { + if (taskId == null) { + return ; + } + + final Map results = + healthCheck.runHealthChecks(); + for (Entry resultEntry : results + .entrySet()) { + HealthCheck.Result result = resultEntry.getValue(); + if (result.isHealthy() == false) { + LOG.warn("{}:{}", taskId, result.getMessage()); + try { + clusterState.report_task_error(topologyId, taskId, + result.getMessage()); + } catch (Exception e) { + // TODO Auto-generated catch block + LOG.error(e.getMessage(), e); + } + + } + } + } + + public void healthCheck() { + Integer firstTask = null; + + Map taskHealthCheckMap = + JStormHealthCheck.getTaskhealthcheckmap(); + + for (Entry entry : taskHealthCheckMap + .entrySet()) { + Integer taskId = entry.getKey(); + HealthCheckRegistry taskHealthCheck = entry.getValue(); + + healthCheck(taskId, taskHealthCheck); + + if (firstTask != null) { + firstTask = taskId; + } + } + + HealthCheckRegistry workerHealthCheck = + JStormHealthCheck.getWorkerhealthcheck(); + healthCheck(firstTask, workerHealthCheck); + + + } + + @Override + public void run() { + + try { + // TODO Auto-generated method stub + MetricInfo workerMetricInfo = computWorkerMetrics(); + + Map taskMetricMap = computeTaskMetrics(); + + WorkerUploadMetrics upload = new WorkerUploadMetrics(); + upload.set_topology_id(topologyId); + upload.set_supervisor_id(supervisorId); + upload.set_port(port); + upload.set_workerMetric(workerMetricInfo); + upload.set_nettyMetric( + new NettyMetric( + JStormMetrics.getExposeNettyMetrics(), + JStormMetrics.getExposeNettyMetrics().size())); + upload.set_taskMetric(taskMetricMap); + + uploadMetric(upload); + + healthCheck(); + + LOG.info("Successfully upload worker's metrics"); + LOG.info(Utils.toPrettyJsonString(workerMetricInfo)); + LOG.info(Utils.toPrettyJsonString(JStormMetrics.getExposeNettyMetrics())); + LOG.info(Utils.toPrettyJsonString(taskMetricMap)); + } catch (Exception e) { + LOG.error("Failed to upload worker metrics", e); + } + + } + + public void uploadMetric(WorkerUploadMetrics upload) { + if (StormConfig.local_mode(conf)) { + try { + byte[] temp = Utils.serialize(upload); + + LocalCluster.getInstance().getLocalClusterMap().getNimbus() + .workerUploadMetric(upload); + } catch (TException e) { + // TODO Auto-generated catch block + LOG.error("Failed to upload worker metrics", e); + } + } else { + try { + if (client == null) { + client = NimbusClient.getConfiguredClient(conf); + } + client.getClient().workerUploadMetric(upload); + } catch (Exception e) { + LOG.error("Failed to upload worker metrics", e); + if (client != null) { + client.close(); + client = null; + } + } finally { + + } + } + } + + @Override + public Object getResult() { + return frequence; + } + + @Override + public void shutdown() { + if (client != null) { + client.close(); + client = null; + } + } + + public static class JStormMetricFilter implements MetricFilter { + private static final long serialVersionUID = -8886536175626248855L; + private String[] tags; + + public JStormMetricFilter(String[] tags) { + this.tags = tags; + } + + @Override + public boolean matches(String name, Metric metric) { + // TODO Auto-generated method stub + for (String tag : tags) { + if (name.startsWith(tag)) { + return true; + } + } + return false; + } + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/metric/MetricDef.java b/jstorm-core/src/main/java/com/alibaba/jstorm/metric/MetricDef.java new file mode 100755 index 000000000..58413bb16 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/metric/MetricDef.java @@ -0,0 +1,163 @@ +/** + * 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.metric; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +public class MetricDef { + // metric name for task + public static final String QUEUE_TYPE = "Queue"; + public static final String TIME_TYPE = "Time"; + + public static final String DESERIALIZE_THREAD = "Deserialize"; + public static final String DESERIALIZE_QUEUE = DESERIALIZE_THREAD + + QUEUE_TYPE; + public static final String DESERIALIZE_TIME = DESERIALIZE_THREAD + + TIME_TYPE; + + public static final String SERIALIZE_THREAD = "Serialize"; + public static final String SERIALIZE_QUEUE = SERIALIZE_THREAD + QUEUE_TYPE; + public static final String SERIALIZE_TIME = SERIALIZE_THREAD + TIME_TYPE; + + public static final String EXECUTE_THREAD = "Executor"; + public static final String EXECUTE_QUEUE = EXECUTE_THREAD + QUEUE_TYPE; + public static final String EXECUTE_TIME = EXECUTE_THREAD + TIME_TYPE; + + public static final String ACKER_TIME = "AckerTime"; + public static final String EMPTY_CPU_RATIO = "EmptyCpuRatio"; + public static final String PENDING_MAP = "PendingNum"; + public static final String COLLECTOR_EMIT_TIME = "EmitTime"; + + + + public static final String DISPATCH_THREAD = "VirtualPortDispatch"; + public static final String DISPATCH_QUEUE = DISPATCH_THREAD + QUEUE_TYPE; + public static final String DISPATCH_TIME = DISPATCH_THREAD + TIME_TYPE; + + public static final String BATCH_DRAINER_THREAD = "BatchDrainer"; + public static final String BATCH_DRAINER_QUEUE = BATCH_DRAINER_THREAD + + QUEUE_TYPE; + public static final String BATCH_DRAINER_TIME = BATCH_DRAINER_THREAD + + TIME_TYPE; + + public static final String DRAINER_THREAD = "Drainer"; + public static final String DRAINER_QUEUE = DRAINER_THREAD + QUEUE_TYPE; + public static final String DRAINER_TIME = DRAINER_THREAD + TIME_TYPE; + + + public static final String NETWORK_MSG_DECODE_TIME = "NetworkMsgDecodeTime"; + + // all tag start with "Netty" will be specially display in Web UI + public static final String NETTY = "Netty"; + public static final String NETTY_CLI = NETTY + "Client"; + public static final String NETTY_SRV = NETTY + "Server"; + public static final String NETTY_CLI_SEND_SPEED = NETTY_CLI + "SendSpeed"; + public static final String NETTY_SRV_RECV_SPEED = NETTY_SRV + "RecvSpeed"; + + public static final String NETTY_CLI_SEND_TIME = NETTY_CLI + "SendTime"; + public static final String NETTY_CLI_BATCH_SIZE = + NETTY_CLI + "SendBatchSize"; + public static final String NETTY_CLI_SEND_PENDING = + NETTY_CLI + "SendPendings"; + public static final String NETTY_CLI_SYNC_BATCH_QUEUE = + NETTY_CLI + "SyncBatchQueue"; + public static final String NETTY_CLI_SYNC_DISR_QUEUE = + NETTY_CLI + "SyncDisrQueue"; + public static final String NETTY_CLI_CACHE_SIZE = NETTY_CLI + "CacheSize"; + public static final String NETTY_CLI_CONNECTION = NETTY_CLI + "ConnectionCheck"; + + // metric name for worker + public static final String NETTY_SRV_MSG_TRANS_TIME = NETTY_SRV + "TransmitTime"; + + + public static final String ZMQ_SEND_TIME = "ZMQSendTime"; + public static final String ZMQ_SEND_MSG_SIZE = "ZMQSendMSGSize"; + + public static final String CPU_USED_RATIO = "CpuUsedRatio"; + public static final String MEMORY_USED = "MemoryUsed"; + + public static final String REMOTE_CLI_ADDR = "RemoteClientAddress"; + public static final String REMOTE_SERV_ADDR = "RemoteServerAddress"; + + public static final String EMMITTED_NUM = "Emitted"; + public static final String ACKED_NUM = "Acked"; + public static final String FAILED_NUM = "Failed"; + public static final String SEND_TPS = "SendTps"; + public static final String RECV_TPS = "RecvTps"; + public static final String PROCESS_LATENCY = "ProcessLatency"; + + public static final String[] OUTPUT_TAG = { EMMITTED_NUM, SEND_TPS }; + public static final String[] INPUT_TAG = { RECV_TPS, ACKED_NUM, FAILED_NUM, + PROCESS_LATENCY }; + + public static final Set MERGE_SUM_TAG = new HashSet(); + static { + MERGE_SUM_TAG.add(MetricDef.EMMITTED_NUM); + MERGE_SUM_TAG.add(MetricDef.SEND_TPS); + MERGE_SUM_TAG.add(MetricDef.RECV_TPS); + MERGE_SUM_TAG.add(MetricDef.ACKED_NUM); + MERGE_SUM_TAG.add(MetricDef.FAILED_NUM); + + } + + public static final Set MERGE_AVG_TAG = new HashSet(); + static { + MERGE_AVG_TAG.add(PROCESS_LATENCY); + } + + public static final double FULL_RATIO = 100.0; + + public static final String QEUEU_IS_FULL = "queue is full"; + + public static final Set TASK_QUEUE_SET = new HashSet(); + static { + TASK_QUEUE_SET.add(DESERIALIZE_QUEUE); + TASK_QUEUE_SET.add(SERIALIZE_QUEUE); + TASK_QUEUE_SET.add(EXECUTE_QUEUE); + + } + + public static final Set WORKER_QUEUE_SET = new HashSet(); + static { + WORKER_QUEUE_SET.add(DISPATCH_QUEUE); + WORKER_QUEUE_SET.add(BATCH_DRAINER_QUEUE); + WORKER_QUEUE_SET.add(DRAINER_QUEUE); + } + + + public static final int NETTY_METRICS_PACKAGE_SIZE = 200; + public static boolean isNettyDetails(String metricName) { + + Set specialNettySet = new HashSet(); + specialNettySet.add(MetricDef.NETTY_CLI_SEND_SPEED); + specialNettySet.add(MetricDef.NETTY_SRV_RECV_SPEED); + + if (specialNettySet.contains(metricName)) { + return false; + } + if (metricName.startsWith(MetricDef.NETTY)) { + return true; + } + + return false; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/metric/MetricJstack.java b/jstorm-core/src/main/java/com/alibaba/jstorm/metric/MetricJstack.java new file mode 100755 index 000000000..2989196bb --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/metric/MetricJstack.java @@ -0,0 +1,140 @@ +/** + * 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.metric; + +import java.lang.management.ManagementFactory; +import java.lang.management.ThreadInfo; +import java.lang.management.ThreadMXBean; + +import com.codahale.metrics.Gauge; + +public class MetricJstack implements Gauge { + + private String getTaskName(long id, String name) { + if (name == null) { + return Long.toString(id); + } + return id + " (" + name + ")"; + } + + public String dumpThread() throws Exception { + StringBuilder writer = new StringBuilder(); + + ThreadMXBean threadMXBean = ManagementFactory.getThreadMXBean(); + + boolean contention = threadMXBean.isThreadContentionMonitoringEnabled(); + + long[] threadIds = threadMXBean.getAllThreadIds(); + writer.append(threadIds.length + " active threads:"); + for (long tid : threadIds) { + writer.append(tid).append(" "); + } + writer.append("\n"); + + long[] deadLockTids = threadMXBean.findDeadlockedThreads(); + if (deadLockTids != null) { + writer.append(threadIds.length + " deadlocked threads:"); + for (long tid : deadLockTids) { + writer.append(tid).append(" "); + } + writer.append("\n"); + } + + long[] deadLockMonitorTids = + threadMXBean.findMonitorDeadlockedThreads(); + if (deadLockMonitorTids != null) { + writer.append(threadIds.length + " deadlocked monitor threads:"); + for (long tid : deadLockMonitorTids) { + writer.append(tid).append(" "); + } + writer.append("\n"); + } + + for (long tid : threadIds) { + ThreadInfo info = + threadMXBean.getThreadInfo(tid, Integer.MAX_VALUE); + if (info == null) { + writer.append(" Inactive").append("\n"); + continue; + } + writer.append( + "Thread " + + getTaskName(info.getThreadId(), + info.getThreadName()) + ":").append("\n"); + Thread.State state = info.getThreadState(); + writer.append(" State: " + state).append("\n"); + writer.append(" Blocked count: " + info.getBlockedCount()).append( + "\n"); + writer.append(" Waited count: " + info.getWaitedCount()).append( + "\n"); + writer.append(" Cpu time:") + .append(threadMXBean.getThreadCpuTime(tid) / 1000000) + .append("ms").append("\n"); + writer.append(" User time:") + .append(threadMXBean.getThreadUserTime(tid) / 1000000) + .append("ms").append("\n"); + if (contention) { + writer.append(" Blocked time: " + info.getBlockedTime()) + .append("\n"); + writer.append(" Waited time: " + info.getWaitedTime()).append( + "\n"); + } + if (state == Thread.State.WAITING) { + writer.append(" Waiting on " + info.getLockName()) + .append("\n"); + } else if (state == Thread.State.BLOCKED) { + writer.append(" Blocked on " + info.getLockName()) + .append("\n"); + writer.append( + " Blocked by " + + getTaskName(info.getLockOwnerId(), + info.getLockOwnerName())).append("\n"); + } + + } + for (long tid : threadIds) { + ThreadInfo info = + threadMXBean.getThreadInfo(tid, Integer.MAX_VALUE); + if (info == null) { + writer.append(" Inactive").append("\n"); + continue; + } + + writer.append( + "Thread " + + getTaskName(info.getThreadId(), + info.getThreadName()) + ": Stack").append( + "\n"); + for (StackTraceElement frame : info.getStackTrace()) { + writer.append(" " + frame.toString()).append("\n"); + } + } + + return writer.toString(); + } + + @Override + public String getValue() { + try { + return dumpThread(); + } catch (Exception e) { + return "Failed to get jstack thread info"; + } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/metric/MetricSendClient.java b/jstorm-core/src/main/java/com/alibaba/jstorm/metric/MetricSendClient.java new file mode 100755 index 000000000..e1313f620 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/metric/MetricSendClient.java @@ -0,0 +1,18 @@ +package com.alibaba.jstorm.metric; + +import java.util.List; +import java.util.Map; + +public class MetricSendClient { + + public MetricSendClient() { + } + + public boolean send(Map msg) { + return true; + } + + public boolean send(List> msgList) { + return true; + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/metric/MetricThrift.java b/jstorm-core/src/main/java/com/alibaba/jstorm/metric/MetricThrift.java new file mode 100755 index 000000000..5286a1c91 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/metric/MetricThrift.java @@ -0,0 +1,129 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.metric; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.generated.MetricInfo; +import backtype.storm.generated.MetricWindow; + +import com.alibaba.jstorm.utils.JStormUtils; + +public class MetricThrift { + private static final Logger LOG = LoggerFactory + .getLogger(MetricThrift.class); + + public static MetricInfo mkMetricInfo() { + MetricInfo metricInfo = + new MetricInfo(new HashMap()); + + return metricInfo; + } + + public static void insert(MetricInfo metricInfo, String key, + Map windowSet) { + MetricWindow metricWindow = new MetricWindow(); + metricWindow.set_metricWindow(windowSet); + + metricInfo.put_to_baseMetric(key, metricWindow); + + } + + public static MetricWindow merge(Map details) { + Map merge = new HashMap(); + + for (Entry entry : details.entrySet()) { + MetricWindow metricWindow = entry.getValue(); + Map metric = metricWindow.get_metricWindow(); + + for (Entry metricEntry : metric.entrySet()) { + Integer key = metricEntry.getKey(); + try { + Double value = + ((Number) JStormUtils.add(metricEntry.getValue(), + merge.get(key))).doubleValue(); + + merge.put(key, value); + } catch (Exception e) { + LOG.error("Invalid type of " + entry.getKey() + ":" + key, + e); + continue; + } + } + } + + MetricWindow ret = new MetricWindow(); + + ret.set_metricWindow(merge); + return ret; + } + + public static void merge(MetricInfo metricInfo, + Map> extraMap) { + for (Entry> entry : extraMap + .entrySet()) { + String metricName = entry.getKey(); + + metricInfo.put_to_baseMetric(metricName, merge(entry.getValue())); + + } + } + + public static MetricWindow mergeMetricWindow(MetricWindow fromMetric, + MetricWindow toMetric) { + if (toMetric == null) { + toMetric = new MetricWindow(new HashMap()); + } + + if (fromMetric == null) { + return toMetric; + } + + List> list = new ArrayList>(); + list.add(fromMetric.get_metricWindow()); + list.add(toMetric.get_metricWindow()); + Map merged = JStormUtils.mergeMapList(list); + + toMetric.set_metricWindow(merged); + + return toMetric; + } + + public static MetricInfo mergeMetricInfo(MetricInfo from, MetricInfo to) { + if (to == null) { + to = mkMetricInfo(); + } + + if (from == null) { + return to; + } + + to.get_baseMetric().putAll(from.get_baseMetric()); + + return to; + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/metric/SimpleJStormMetric.java b/jstorm-core/src/main/java/com/alibaba/jstorm/metric/SimpleJStormMetric.java new file mode 100755 index 000000000..1d770897e --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/metric/SimpleJStormMetric.java @@ -0,0 +1,96 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.metric; + +import java.util.Map; +import java.util.Map.Entry; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.common.metric.Histogram; +import com.alibaba.jstorm.common.metric.MetricRegistry; +import com.alibaba.jstorm.common.metric.window.Metric; + +public class SimpleJStormMetric extends JStormMetrics implements Runnable{ + private static final Logger LOG = LoggerFactory.getLogger(SimpleJStormMetric.class); + + protected static MetricRegistry metrics = JStormMetrics.workerMetrics; + static { + Metric.setEnable(true); + } + + protected static SimpleJStormMetric instance = null; + + + public static SimpleJStormMetric mkInstance() { + synchronized (SimpleJStormMetric.class) { + if (instance == null) { + instance = new SimpleJStormMetric(); + } + + return instance; + } + } + + protected SimpleJStormMetric() { + + } + + public static Histogram registerHistorgram(String name) { + return JStormMetrics.registerWorkerHistogram(name); + } + + public static void updateHistorgram(String name, Number obj) { + LOG.debug(name + ":" + obj.doubleValue()); + Histogram histogram = (Histogram)metrics.getMetric(name); + if (histogram == null) { + try { + histogram = registerHistorgram(name); + }catch(Exception e) { + LOG.info("{} has been register", name); + return; + } + } + + histogram.update(obj); + + } + + @Override + public void run() { + // TODO Auto-generated method stub + Map map = metrics.getMetrics(); + + for (Entry entry : map.entrySet()) { + String key = entry.getKey(); + Metric metric = entry.getValue(); + + LOG.info(key + ":" + metric.getSnapshot()); + } + } + + + public static void main(String[] args) { + updateHistorgram("test", 11100.0); + + SimpleJStormMetric instance = new SimpleJStormMetric(); + + instance.run(); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/queue/disruptor/JstormEvent.java b/jstorm-core/src/main/java/com/alibaba/jstorm/queue/disruptor/JstormEvent.java new file mode 100755 index 000000000..eabcd448c --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/queue/disruptor/JstormEvent.java @@ -0,0 +1,41 @@ +/** + * 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.queue.disruptor; + +import com.lmax.disruptor.EventFactory; + +public class JstormEvent { + + private String msgId; + + public String getMsgId() { + return msgId; + } + + public void setMsgId(String msgId) { + this.msgId = msgId; + } + + public final static EventFactory EVENT_FACTORY = + new EventFactory() { + public JstormEvent newInstance() { + return new JstormEvent(); + } + }; + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/queue/disruptor/JstormEventHandler.java b/jstorm-core/src/main/java/com/alibaba/jstorm/queue/disruptor/JstormEventHandler.java new file mode 100755 index 000000000..dfc43d1de --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/queue/disruptor/JstormEventHandler.java @@ -0,0 +1,47 @@ +/** + * 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.queue.disruptor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.lmax.disruptor.EventHandler; + +public class JstormEventHandler implements EventHandler { + + Logger logger = LoggerFactory.getLogger(JstormEventHandler.class); + + private int count; + + public JstormEventHandler(int count) { + this.count = count; + } + + @Override + public void onEvent(Object event, long sequence, boolean endOfBatch) + throws Exception { + long msgId = Long.parseLong(((JstormEvent) event).getMsgId()); + // if (msgId % size ==0) { + // logger.warn("consumer msgId=" + msgId + ", seq=" + sequence); + // } + if (msgId == count - 1) { + logger.warn("end..." + System.currentTimeMillis()); + } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/queue/disruptor/JstormProducer.java b/jstorm-core/src/main/java/com/alibaba/jstorm/queue/disruptor/JstormProducer.java new file mode 100755 index 000000000..999367d79 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/queue/disruptor/JstormProducer.java @@ -0,0 +1,63 @@ +/** + * 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.queue.disruptor; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.lmax.disruptor.RingBuffer; + +public class JstormProducer implements Runnable { + + Logger logger = LoggerFactory.getLogger(JstormProducer.class); + + private RingBuffer ringBuffer; + private int size; + + public JstormProducer(RingBuffer ringBuffer, int size) { + this.ringBuffer = ringBuffer; + this.size = size; + } + + @Override + public void run() { + logger.warn("producer start..." + System.currentTimeMillis()); + + // while (true) { + // long seqId = ringBuffer.next(); + // + // ringBuffer.get(seqId).setMsgId(String.valueOf(seqId)); + // ringBuffer.publish(seqId); + // + // try { + // double random = Math.random(); + // Thread.sleep((long)(random * 1000)); + // } catch (InterruptedException e) { + // e.printStackTrace(); + // } + // } + + for (int i = 0; i < size; i++) { + long seqId = ringBuffer.next(); + + ringBuffer.get(seqId).setMsgId(String.valueOf(seqId)); + ringBuffer.publish(seqId); + } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/Assignment.java b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/Assignment.java new file mode 100644 index 000000000..69620bd22 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/Assignment.java @@ -0,0 +1,232 @@ +/** + * 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.schedule; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; + +import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; + +/** + * Assignment of one Toplogy, stored in /ZK-DIR/assignments/{topologyid} + * nodeHost {supervisorid: hostname} -- assigned supervisor Map + * taskStartTimeSecs: {taskid, taskStartSeconds} masterCodeDir: topology source + * code's dir in Nimbus taskToResource: {taskid, ResourceAssignment} + * + * @author Lixin/Longda + */ +public class Assignment implements Serializable { + public enum AssignmentType { + Assign, Config + } + + private static final long serialVersionUID = 6087667851333314069L; + + private final String masterCodeDir; + /** + * @@@ nodeHost store , this will waste some zk + * storage + */ + private final Map nodeHost; + private final Map taskStartTimeSecs; + private final Set workers; + + private long timeStamp; + + private AssignmentType type; + + public Assignment() { + masterCodeDir = null; + this.nodeHost = new HashMap(); + this.taskStartTimeSecs = new HashMap(); + this.workers = new HashSet(); + this.timeStamp = System.currentTimeMillis(); + this.type = AssignmentType.Assign; + } + + public Assignment(String masterCodeDir, Set workers, + Map nodeHost, + Map taskStartTimeSecs) { + this.workers = workers; + this.nodeHost = nodeHost; + this.taskStartTimeSecs = taskStartTimeSecs; + this.masterCodeDir = masterCodeDir; + this.timeStamp = System.currentTimeMillis(); + this.type = AssignmentType.Assign; + } + + public void setAssignmentType(AssignmentType type) { + this.type = type; + } + + public AssignmentType getAssignmentType() { + return type; + } + + public Map getNodeHost() { + return nodeHost; + } + + public Map getTaskStartTimeSecs() { + return taskStartTimeSecs; + } + + public String getMasterCodeDir() { + return masterCodeDir; + } + + public Set getWorkers() { + return workers; + } + + /** + * find workers for every supervisorId (node) + * + * @param supervisorId + * @return Map + */ + public Map getTaskToNodePortbyNode( + String supervisorId) { + + Map result = + new HashMap(); + for (ResourceWorkerSlot worker : workers) { + if (worker.getNodeId().equals(supervisorId)) { + result.put(worker.getPort(), worker); + } + } + return result; + } + + public Set getCurrentSuperviosrTasks(String supervisorId) { + Set Tasks = new HashSet(); + + for (ResourceWorkerSlot worker : workers) { + if (worker.getNodeId().equals(supervisorId)) + Tasks.addAll(worker.getTasks()); + } + + return Tasks; + } + + public Set getCurrentSuperviosrWorkers(String supervisorId) { + Set workerSet = new HashSet(); + + for (ResourceWorkerSlot worker : workers) { + if (worker.getNodeId().equals(supervisorId)) + workerSet.add(worker.getPort()); + } + + return workerSet; + } + + public Set getCurrentWorkerTasks(String supervisorId, int port) { + + for (ResourceWorkerSlot worker : workers) { + if (worker.getNodeId().equals(supervisorId) + && worker.getPort() == port) + return worker.getTasks(); + } + + return new HashSet(); + } + + public ResourceWorkerSlot getWorkerByTaskId(Integer taskId) { + for (ResourceWorkerSlot worker : workers) { + if (worker.getTasks().contains(taskId)) + return worker; + } + return null; + } + + public long getTimeStamp() { + return this.timeStamp; + } + + public void updateTimeStamp() { + timeStamp = System.currentTimeMillis(); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = + prime + * result + + ((masterCodeDir == null) ? 0 : masterCodeDir + .hashCode()); + result = + prime * result + ((nodeHost == null) ? 0 : nodeHost.hashCode()); + result = + prime + * result + + ((taskStartTimeSecs == null) ? 0 : taskStartTimeSecs + .hashCode()); + result = prime * result + ((workers == null) ? 0 : workers.hashCode()); + result = prime * result + (int) (timeStamp & 0xFFFFFFFF); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + Assignment other = (Assignment) obj; + if (masterCodeDir == null) { + if (other.masterCodeDir != null) + return false; + } else if (!masterCodeDir.equals(other.masterCodeDir)) + return false; + if (nodeHost == null) { + if (other.nodeHost != null) + return false; + } else if (!nodeHost.equals(other.nodeHost)) + return false; + if (taskStartTimeSecs == null) { + if (other.taskStartTimeSecs != null) + return false; + } else if (!taskStartTimeSecs.equals(other.taskStartTimeSecs)) + return false; + if (workers == null) { + if (other.workers != null) + return false; + } else if (!workers.equals(other.workers)) + return false; + if (timeStamp != other.timeStamp) + return false; + return true; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/AssignmentBak.java b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/AssignmentBak.java new file mode 100755 index 000000000..3e7a770b7 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/AssignmentBak.java @@ -0,0 +1,55 @@ +/** + * 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.schedule; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; + +public class AssignmentBak implements Serializable { + + /** */ + private static final long serialVersionUID = 7633746649144483965L; + + private final Map> componentTasks; + private final Assignment assignment; + + public AssignmentBak(Map> componentTasks, + Assignment assignment) { + super(); + this.componentTasks = componentTasks; + this.assignment = assignment; + } + + public Map> getComponentTasks() { + return componentTasks; + } + + public Assignment getAssignment() { + return assignment; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/CleanRunnable.java b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/CleanRunnable.java new file mode 100755 index 000000000..d73adfd27 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/CleanRunnable.java @@ -0,0 +1,83 @@ +/** + * 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.schedule; + +import java.io.File; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.utils.OlderFileFilter; + +/** + * clean /nimbus/inbox jar every 600 seconds + * + * Default expire time is 3600 seconds + * + * @author lixin + * + */ +public class CleanRunnable implements Runnable { + + private static Logger log = LoggerFactory.getLogger(CleanRunnable.class); + + private String dir_location; + + private int seconds; + + public CleanRunnable(String dir_location, int inbox_jar_expiration_secs) { + this.dir_location = dir_location; + this.seconds = inbox_jar_expiration_secs; + } + + @Override + public void run() { + File inboxdir = new File(dir_location); + clean(inboxdir); + } + + private void clean(File file) { + // filter + OlderFileFilter filter = new OlderFileFilter(seconds); + + File[] files = file.listFiles(filter); + for (File f : files) { + if (f.isFile()) { + log.info("Cleaning inbox ... deleted: " + f.getName()); + try { + f.delete(); + } catch (Exception e) { + log.error("Cleaning inbox ... error deleting:" + + f.getName() + "," + e); + } + } else { + clean(f); + if (f.listFiles().length == 0) { + log.info("Cleaning inbox ... deleted: " + f.getName()); + try { + f.delete(); + } catch (Exception e) { + log.error("Cleaning inbox ... error deleting:" + + f.getName() + "," + e); + } + } + } + } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/DelayEventRunnable.java b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/DelayEventRunnable.java new file mode 100755 index 000000000..14d38d830 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/DelayEventRunnable.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.schedule; + +import com.alibaba.jstorm.daemon.nimbus.NimbusData; +import com.alibaba.jstorm.daemon.nimbus.NimbusUtils; +import com.alibaba.jstorm.daemon.nimbus.StatusType; + +public class DelayEventRunnable implements Runnable { + + private NimbusData data; + private String topologyid; + private StatusType status; + private Object[] args; + + public DelayEventRunnable(NimbusData data, String topologyid, + StatusType status, Object[] args) { + this.data = data; + this.topologyid = topologyid; + this.status = status; + this.args = args; + } + + @Override + public void run() { + NimbusUtils.transition(data, topologyid, false, status, args); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/FollowerRunnable.java b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/FollowerRunnable.java new file mode 100644 index 000000000..e62c61b68 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/FollowerRunnable.java @@ -0,0 +1,306 @@ +/** + * 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.schedule; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang.StringUtils; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.Cluster; +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.cluster.StormConfig; +import com.alibaba.jstorm.daemon.nimbus.NimbusData; +import com.alibaba.jstorm.utils.JStormServerUtils; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.NetWorkUtils; +import com.alibaba.jstorm.utils.PathUtils; + +public class FollowerRunnable implements Runnable { + + private static final Logger LOG = LoggerFactory + .getLogger(FollowerRunnable.class); + + private NimbusData data; + + private int sleepTime; + + private volatile boolean state = true; + + private RunnableCallback callback; + + private final String hostPort; + + @SuppressWarnings("unchecked") + public FollowerRunnable(final NimbusData data, int sleepTime) { + this.data = data; + this.sleepTime = sleepTime; + if (!ConfigExtension.isNimbusUseIp(data.getConf())) { + this.hostPort = + NetWorkUtils.hostname() + + ":" + + String.valueOf(Utils.getInt(data.getConf().get( + Config.NIMBUS_THRIFT_PORT))); + } else { + this.hostPort = + NetWorkUtils.ip() + + ":" + + String.valueOf(Utils.getInt(data.getConf().get( + Config.NIMBUS_THRIFT_PORT))); + } + try { + String[] hostfigs = this.hostPort.split(":"); + boolean isLocaliP = false; + if(hostfigs.length > 0){ + isLocaliP = hostfigs[0].equals("127.0.0.1"); + } + if(isLocaliP){ + throw new Exception("the hostname which Nimbus get is localhost"); + } + }catch(Exception e1){ + LOG.error("get nimbus host error!", e1); + throw new RuntimeException(e1); + } + try { + this.tryToBeLeader(data.getConf()); + } catch (Exception e1) { + // TODO Auto-generated catch block + LOG.error("try to be leader error.", e1); + throw new RuntimeException(e1); + } + try { + data.getStormClusterState().update_nimbus_slave(hostPort, + data.uptime()); + } catch (Exception e) { + LOG.error("register nimbus host fail!", e); + throw new RuntimeException(); + } + callback = new RunnableCallback() { + @Override + public void run() { + if (!data.isLeader()) + check(); + } + }; + } + + 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 (!zkClusterState.leader_existed()) { + this.tryToBeLeader(data.getConf()); + continue; + } + + 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(); + zkClusterState.update_nimbus_slave(hostPort, data.uptime()); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + continue; + } catch (Exception e) { + if (state) { + LOG.error("Unknow exception ", e); + } + } + } + LOG.info("Follower Thread has closed!"); + } + + public void clean() { + state = false; + } + + private synchronized void check() { + + StormClusterState clusterState = data.getStormClusterState(); + + try { + String master_stormdist_root = + StormConfig.masterStormdistRoot(data.getConf()); + + List code_ids = + PathUtils.read_dir_contents(master_stormdist_root); + + List assignments_ids = clusterState.assignments(callback); + + List done_ids = new ArrayList(); + + for (String id : code_ids) { + if (assignments_ids.contains(id)) { + done_ids.add(id); + } + } + + for (String id : done_ids) { + assignments_ids.remove(id); + code_ids.remove(id); + } + + for (String topologyId : code_ids) { + deleteLocalTopology(topologyId); + } + + for (String id : assignments_ids) { + Assignment assignment = clusterState.assignment_info(id, null); + downloadCodeFromMaster(assignment, id); + } + } catch (IOException e) { + // TODO Auto-generated catch block + LOG.error("Get stormdist dir error!", e); + return; + } catch (Exception e) { + // TODO Auto-generated catch block + LOG.error("Check error!", e); + return; + } + } + + private void deleteLocalTopology(String topologyId) throws IOException { + String dir_to_delete = + StormConfig.masterStormdistRoot(data.getConf(), topologyId); + try { + PathUtils.rmr(dir_to_delete); + LOG.info("delete:" + dir_to_delete + "successfully!"); + } catch (IOException e) { + // TODO Auto-generated catch block + LOG.error("delete:" + dir_to_delete + "fail!", e); + } + } + + private void downloadCodeFromMaster(Assignment assignment, String topologyId) + throws IOException, TException { + try { + String localRoot = + StormConfig.masterStormdistRoot(data.getConf(), topologyId); + String tmpDir = + StormConfig.masterInbox(data.getConf()) + "/" + + UUID.randomUUID().toString(); + String masterCodeDir = assignment.getMasterCodeDir(); + JStormServerUtils.downloadCodeFromMaster(data.getConf(), tmpDir, + masterCodeDir, topologyId, false); + + FileUtils.moveDirectory(new File(tmpDir), new File(localRoot)); + } catch (TException e) { + // TODO Auto-generated catch block + LOG.error(e + " downloadStormCode failed " + "topologyId:" + + topologyId + "masterCodeDir:" + + assignment.getMasterCodeDir()); + throw e; + } + LOG.info("Finished downloading code for topology id " + topologyId + + " from " + assignment.getMasterCodeDir()); + } + + private void tryToBeLeader(final Map conf) throws Exception { + RunnableCallback masterCallback = new RunnableCallback() { + @Override + public void run() { + try { + tryToBeLeader(conf); + } catch (Exception e) { + LOG.error("To be master error", e); + JStormUtils.halt_process(30, + "Cant't to be master" + e.getMessage()); + } + } + }; + data.getStormClusterState().try_to_be_leader(Cluster.MASTER_SUBTREE, + hostPort, masterCallback); + } + + /** + * Check whether current node is master or not + * + * @throws Exception + */ + private void checkOwnMaster() throws Exception { + int retry_times = 10; + + StormClusterState zkClient = data.getStormClusterState(); + for (int i = 0; i < retry_times; i++, JStormUtils.sleepMs(sleepTime)) { + + if (zkClient.leader_existed() == false) { + continue; + } + + String zkHost = zkClient.get_leader_host(); + if (hostPort.equals(zkHost) == true) { + // current process own master + return; + } + LOG.warn("Current Nimbus has start thrift, but fail to own zk master :" + + zkHost); + } + + // current process doesn't own master + String err = + "Current Nimubs fail to own nimbus_master, should halt process"; + LOG.error(err); + JStormUtils.halt_process(0, err); + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/IToplogyScheduler.java b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/IToplogyScheduler.java new file mode 100755 index 000000000..a9d9b921c --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/IToplogyScheduler.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.schedule; + +import java.util.Map; +import java.util.Set; + +import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; +import com.alibaba.jstorm.utils.FailedAssignTopologyException; + +public interface IToplogyScheduler { + void prepare(Map conf); + + Set assignTasks(TopologyAssignContext contex) + throws FailedAssignTopologyException; +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/MonitorRunnable.java b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/MonitorRunnable.java new file mode 100644 index 000000000..8342b7917 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/MonitorRunnable.java @@ -0,0 +1,125 @@ +/** + * 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.schedule; + +import java.util.Date; +import java.util.List; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.daemon.nimbus.NimbusData; +import com.alibaba.jstorm.daemon.nimbus.NimbusUtils; +import com.alibaba.jstorm.daemon.nimbus.StatusType; +import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; +import com.alibaba.jstorm.utils.TimeFormat; + +/** + * + * Scan all task's heartbeat, if task isn't alive, DO + * NimbusUtils.transition(monitor) + * + * @author Longda + * + */ +public class MonitorRunnable implements Runnable { + private static Logger LOG = LoggerFactory.getLogger(MonitorRunnable.class); + + private NimbusData data; + + public MonitorRunnable(NimbusData data) { + this.data = data; + } + + /** + * @@@ Todo when one topology is being reassigned, the topology should be + * skip check + * @param data + */ + @Override + public void run() { + StormClusterState clusterState = data.getStormClusterState(); + + try { + // Attetion, need first check Assignments + List active_topologys = clusterState.assignments(null); + + if (active_topologys == null) { + LOG.info("Failed to get active topologies"); + return; + } + + for (String topologyid : active_topologys) { + if (clusterState.storm_base(topologyid, null) == null) { + continue; + } + + LOG.debug("Check tasks " + topologyid); + + // Attention, here don't check /ZK-dir/taskbeats/topologyid to + // get task ids + Set taskIds = clusterState.task_ids(topologyid); + if (taskIds == null) { + LOG.info("Failed to get task ids of " + topologyid); + continue; + } + + boolean needReassign = false; + for (Integer task : taskIds) { + boolean isTaskDead = + NimbusUtils.isTaskDead(data, topologyid, task); + if (isTaskDead == true) { + LOG.info("Found " + topologyid + ",taskid:" + task + + " is dead"); + + ResourceWorkerSlot resource = null; + Assignment assignment = + clusterState.assignment_info(topologyid, null); + if (assignment != null) + resource = assignment.getWorkerByTaskId(task); + if (resource != null) { + Date now = new Date(); + String nowStr = TimeFormat.getSecond(now); + String errorInfo = + "Task-" + task + " is dead on " + + resource.getHostname() + ":" + + resource.getPort() + ", " + + nowStr; + LOG.info(errorInfo); + clusterState.report_task_error(topologyid, task, + errorInfo); + } + needReassign = true; + break; + } + } + if (needReassign == true) { + NimbusUtils.transition(data, topologyid, false, + StatusType.monitor); + } + } + + } catch (Exception e) { + // TODO Auto-generated catch block + LOG.error(e.getMessage(), e); + } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/TopologyAssignContext.java b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/TopologyAssignContext.java new file mode 100755 index 000000000..12cdad033 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/TopologyAssignContext.java @@ -0,0 +1,179 @@ +/** + * 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.schedule; + +import java.util.Map; +import java.util.Set; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; + +import backtype.storm.generated.StormTopology; + +import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; +import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; +import com.alibaba.jstorm.schedule.Assignment; + +public class TopologyAssignContext { + + public static final int ASSIGN_TYPE_NEW = 0; // assign a new topology + + public static final int ASSIGN_TYPE_REBALANCE = 1; // rebalance a topology + + public static final int ASSIGN_TYPE_MONITOR = 2; // monitor a topology, some + // tasks are dead + + protected int assignType; + + protected StormTopology rawTopology; + + protected Map stormConf; + + // if assignType is ASSIGN_TYPE_NEW, oldAssignment is the Assignment last + // time + // otherwise it is the old assignment before assignment. + protected Assignment oldAssignment; + + protected Map cluster; + + protected Map taskToComponent; + + protected Set allTaskIds; // all tasks + protected Set deadTaskIds; // dead tasks + protected Set unstoppedTaskIds; // the task is alive, but his + // supervisor is dead + protected Set unstoppedWorkers; + + protected boolean isReassign; + + public TopologyAssignContext() { + + } + + public TopologyAssignContext(TopologyAssignContext copy) { + this.assignType = copy.getAssignType(); + this.rawTopology = copy.getRawTopology(); + this.stormConf = copy.getStormConf(); + this.oldAssignment = copy.getOldAssignment(); + this.cluster = copy.getCluster(); + this.taskToComponent = copy.getTaskToComponent(); + this.allTaskIds = copy.getAllTaskIds(); + this.deadTaskIds = copy.getDeadTaskIds(); + this.unstoppedTaskIds = copy.getUnstoppedTaskIds(); + this.isReassign = copy.isReassign(); + } + + public int getAssignType() { + return assignType; + } + + public void setAssignType(int assignType) { + this.assignType = assignType; + } + + public StormTopology getRawTopology() { + return rawTopology; + } + + public void setRawTopology(StormTopology rawTopology) { + this.rawTopology = rawTopology; + } + + public Map getStormConf() { + return stormConf; + } + + public void setStormConf(Map stormConf) { + this.stormConf = stormConf; + } + + public Assignment getOldAssignment() { + return oldAssignment; + } + + public void setOldAssignment(Assignment oldAssignment) { + this.oldAssignment = oldAssignment; + } + + public Map getCluster() { + return cluster; + } + + public void setCluster(Map cluster) { + this.cluster = cluster; + } + + public Set getAllTaskIds() { + return allTaskIds; + } + + public void setAllTaskIds(Set allTaskIds) { + this.allTaskIds = allTaskIds; + } + + public Set getDeadTaskIds() { + return deadTaskIds; + } + + public void setDeadTaskIds(Set deadTaskIds) { + this.deadTaskIds = deadTaskIds; + } + + public Set getUnstoppedTaskIds() { + return unstoppedTaskIds; + } + + public void setUnstoppedTaskIds(Set unstoppedTaskIds) { + this.unstoppedTaskIds = unstoppedTaskIds; + } + + public Map getTaskToComponent() { + return taskToComponent; + } + + public void setTaskToComponent(Map taskToComponent) { + this.taskToComponent = taskToComponent; + } + + public static boolean isAssignTypeValid(int type) { + return (type == ASSIGN_TYPE_NEW) || (type == ASSIGN_TYPE_REBALANCE) + || (type == ASSIGN_TYPE_MONITOR); + } + + public Set getUnstoppedWorkers() { + return unstoppedWorkers; + } + + public void setUnstoppedWorkers(Set unstoppedWorkers) { + this.unstoppedWorkers = unstoppedWorkers; + } + + public boolean isReassign() { + return isReassign; + } + + public void setIsReassign(boolean isReassign) { + this.isReassign = isReassign; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/DefaultTopologyAssignContext.java b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/DefaultTopologyAssignContext.java new file mode 100755 index 000000000..9eb27751a --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/DefaultTopologyAssignContext.java @@ -0,0 +1,221 @@ +/** + * 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.schedule.default_assign; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; + +import backtype.storm.Config; +import backtype.storm.generated.Bolt; +import backtype.storm.generated.ComponentCommon; +import backtype.storm.generated.SpoutSpec; +import backtype.storm.generated.StateSpoutSpec; +import backtype.storm.generated.StormTopology; +import backtype.storm.utils.ThriftTopologyUtils; + +import com.alibaba.jstorm.cluster.Common; +import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; +import com.alibaba.jstorm.schedule.TopologyAssignContext; +import com.alibaba.jstorm.utils.FailedAssignTopologyException; +import com.alibaba.jstorm.utils.JStormUtils; + +public class DefaultTopologyAssignContext extends TopologyAssignContext { + + private final StormTopology sysTopology; + private final Map sidToHostname; + private final Map> hostToSid; + private final Set oldWorkers; + private final Map> componentTasks; + private final Set unstoppedWorkers = + new HashSet(); + private final int totalWorkerNum; + private final int unstoppedWorkerNum; + + private int computeWorkerNum() { + Integer settingNum = + JStormUtils.parseInt(stormConf.get(Config.TOPOLOGY_WORKERS)); + + int hintSum = 0; + + Map components = + ThriftTopologyUtils.getComponents(sysTopology); + for (Entry entry : components.entrySet()) { + String componentName = entry.getKey(); + Object component = entry.getValue(); + + ComponentCommon common = null; + if (component instanceof Bolt) { + common = ((Bolt) component).get_common(); + } + if (component instanceof SpoutSpec) { + common = ((SpoutSpec) component).get_common(); + } + if (component instanceof StateSpoutSpec) { + common = ((StateSpoutSpec) component).get_common(); + } + + int hint = common.get_parallelism_hint(); + hintSum += hint; + } + + if (settingNum == null) { + return hintSum; + } else { + return Math.min(settingNum, hintSum); + } + } + + public int computeUnstoppedAssignments() { + for (Integer task : unstoppedTaskIds) { + // if unstoppedTasksIds isn't empty, it should be REASSIGN/Monitor + ResourceWorkerSlot worker = oldAssignment.getWorkerByTaskId(task); + unstoppedWorkers.add(worker); + } + + return unstoppedWorkers.size(); + } + + private void refineDeadTasks() { + Set rawDeadTasks = getDeadTaskIds(); + Set refineDeadTasks = new HashSet(); + refineDeadTasks.addAll(rawDeadTasks); + + Set unstoppedTasks = getUnstoppedTaskIds(); + + // if type is ASSIGN_NEW, rawDeadTasks is empty + // then the oldWorkerTasks should be existingAssignment + for (Integer task : rawDeadTasks) { + if (unstoppedTasks.contains(task)) { + continue; + } + for (ResourceWorkerSlot worker : oldWorkers) { + if (worker.getTasks().contains(task)) { + refineDeadTasks.addAll(worker.getTasks()); + + } + } + } + setDeadTaskIds(refineDeadTasks); + } + + /** + * @@@ Do we need just handle the case whose type is ASSIGN_TYPE_NEW? + * + * @return + */ + private Map generateSidToHost() { + Map sidToHostname = new HashMap(); + if (oldAssignment != null) { + sidToHostname.putAll(oldAssignment.getNodeHost()); + } + + for (Entry entry : cluster.entrySet()) { + String supervisorId = entry.getKey(); + SupervisorInfo supervisorInfo = entry.getValue(); + + sidToHostname.put(supervisorId, supervisorInfo.getHostName()); + + } + + return sidToHostname; + } + + public DefaultTopologyAssignContext(TopologyAssignContext context) { + super(context); + + try { + sysTopology = Common.system_topology(stormConf, rawTopology); + } catch (Exception e) { + throw new FailedAssignTopologyException( + "Failed to generate system topology"); + } + + sidToHostname = generateSidToHost(); + hostToSid = JStormUtils.reverse_map(sidToHostname); + + if (oldAssignment != null && oldAssignment.getWorkers() != null) { + oldWorkers = oldAssignment.getWorkers(); + } else { + oldWorkers = new HashSet(); + } + + refineDeadTasks(); + + componentTasks = JStormUtils.reverse_map(context.getTaskToComponent()); + + for (Entry> entry : componentTasks.entrySet()) { + List componentTaskList = entry.getValue(); + + Collections.sort(componentTaskList); + } + + totalWorkerNum = computeWorkerNum(); + + unstoppedWorkerNum = computeUnstoppedAssignments(); + } + + public StormTopology getSysTopology() { + return sysTopology; + } + + public Map getSidToHostname() { + return sidToHostname; + } + + public Map> getHostToSid() { + return hostToSid; + } + + public Map> getComponentTasks() { + return componentTasks; + } + + public int getTotalWorkerNum() { + return totalWorkerNum; + } + + public int getUnstoppedWorkerNum() { + return unstoppedWorkerNum; + } + + public Set getOldWorkers() { + return oldWorkers; + } + + public Set getUnstoppedWorkers() { + return unstoppedWorkers; + } + + @Override + public String toString() { + return (String) stormConf.get(Config.TOPOLOGY_NAME); + } + + public String toDetailString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/DefaultTopologyScheduler.java b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/DefaultTopologyScheduler.java new file mode 100644 index 000000000..5df7de402 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/DefaultTopologyScheduler.java @@ -0,0 +1,190 @@ +/** + * 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.schedule.default_assign; + +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; +import com.alibaba.jstorm.schedule.Assignment; +import com.alibaba.jstorm.schedule.IToplogyScheduler; +import com.alibaba.jstorm.schedule.TopologyAssignContext; +import com.alibaba.jstorm.utils.FailedAssignTopologyException; + +public class DefaultTopologyScheduler implements IToplogyScheduler { + private static final Logger LOG = LoggerFactory + .getLogger(DefaultTopologyScheduler.class); + + private Map nimbusConf; + + @Override + public void prepare(Map conf) { + nimbusConf = conf; + } + + /** + * @@@ Here maybe exist one problem, some dead slots have been free + * + * @param context + */ + protected void freeUsed(TopologyAssignContext context) { + Set canFree = new HashSet(); + canFree.addAll(context.getAllTaskIds()); + canFree.removeAll(context.getUnstoppedTaskIds()); + + Map cluster = context.getCluster(); + Assignment oldAssigns = context.getOldAssignment(); + for (Integer task : canFree) { + ResourceWorkerSlot worker = oldAssigns.getWorkerByTaskId(task); + if (worker == null) { + LOG.warn("When free rebalance resource, no ResourceAssignment of task " + + task); + continue; + } + + SupervisorInfo supervisorInfo = cluster.get(worker.getNodeId()); + if (supervisorInfo == null) { + continue; + } + supervisorInfo.getAvailableWorkerPorts().add(worker.getPort()); + } + } + + private Set getNeedAssignTasks(DefaultTopologyAssignContext context) { + Set needAssign = new HashSet(); + + int assignType = context.getAssignType(); + if (assignType == TopologyAssignContext.ASSIGN_TYPE_NEW) { + needAssign.addAll(context.getAllTaskIds()); + } else if (assignType == TopologyAssignContext.ASSIGN_TYPE_REBALANCE) { + needAssign.addAll(context.getAllTaskIds()); + needAssign.removeAll(context.getUnstoppedTaskIds()); + } else { + // monitor + needAssign.addAll(context.getDeadTaskIds()); + } + + return needAssign; + } + + /** + * Get the task Map which the task is alive and will be kept Only when type + * is ASSIGN_TYPE_MONITOR, it is valid + * + * @param defaultContext + * @param needAssigns + * @return + */ + public Set getKeepAssign( + DefaultTopologyAssignContext defaultContext, + Set needAssigns) { + + Set keepAssignIds = new HashSet(); + keepAssignIds.addAll(defaultContext.getAllTaskIds()); + keepAssignIds.removeAll(defaultContext.getUnstoppedTaskIds()); + keepAssignIds.removeAll(needAssigns); + Set keeps = new HashSet(); + if (keepAssignIds.isEmpty()) { + return keeps; + } + + Assignment oldAssignment = defaultContext.getOldAssignment(); + if (oldAssignment == null) { + return keeps; + } + keeps.addAll(defaultContext.getOldWorkers()); + for (ResourceWorkerSlot worker : defaultContext.getOldWorkers()) { + for (Integer task : worker.getTasks()) { + if (!keepAssignIds.contains(task)) { + keeps.remove(worker); + break; + } + } + } + return keeps; + } + + @Override + public Set assignTasks(TopologyAssignContext context) + throws FailedAssignTopologyException { + + int assignType = context.getAssignType(); + if (TopologyAssignContext.isAssignTypeValid(assignType) == false) { + throw new FailedAssignTopologyException("Invalide Assign Type " + + assignType); + } + + DefaultTopologyAssignContext defaultContext = + new DefaultTopologyAssignContext(context); + if (assignType == TopologyAssignContext.ASSIGN_TYPE_REBALANCE) { + /** + * Mark all current assigned worker as available. Current assignment + * will be restored in task scheduler. + */ + freeUsed(defaultContext); + } + LOG.info("Dead tasks:" + defaultContext.getDeadTaskIds()); + LOG.info("Unstopped tasks:" + defaultContext.getUnstoppedTaskIds()); + + Set needAssignTasks = getNeedAssignTasks(defaultContext); + + Set keepAssigns = + getKeepAssign(defaultContext, needAssignTasks); + + // please use tree map to make task sequence + Set ret = new HashSet(); + ret.addAll(keepAssigns); + ret.addAll(defaultContext.getUnstoppedWorkers()); + + int allocWorkerNum = + defaultContext.getTotalWorkerNum() + - defaultContext.getUnstoppedWorkerNum() + - keepAssigns.size(); + LOG.info("allocWorkerNum=" + allocWorkerNum + ", totalWorkerNum=" + + defaultContext.getTotalWorkerNum()); + + if (allocWorkerNum <= 0) { + LOG.warn("Don't need assign workers, all workers are fine " + + defaultContext.toDetailString()); + throw new FailedAssignTopologyException( + "Don't need assign worker, all workers are fine "); + } + + List availableWorkers = + WorkerScheduler.getInstance().getAvailableWorkers( + defaultContext, needAssignTasks, allocWorkerNum); + TaskScheduler taskScheduler = + new TaskScheduler(defaultContext, needAssignTasks, + availableWorkers); + Set assignment = + new HashSet(taskScheduler.assign()); + ret.addAll(assignment); + + LOG.info("Keep Alive slots:" + keepAssigns); + LOG.info("Unstopped slots:" + defaultContext.getUnstoppedWorkers()); + LOG.info("New assign slots:" + assignment); + + return ret; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/ResourceWorkerSlot.java b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/ResourceWorkerSlot.java new file mode 100644 index 000000000..c218f521b --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/ResourceWorkerSlot.java @@ -0,0 +1,162 @@ +/** + * 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.schedule.default_assign; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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 { + + public static Logger LOG = LoggerFactory + .getLogger(ResourceWorkerSlot.class); + private static final long serialVersionUID = 9138386287559932411L; + + private String hostname; + private long memSize; + private int cpu; + private Set tasks; + + private String jvm; + + public ResourceWorkerSlot() { + + } + + public ResourceWorkerSlot(String supervisorId, Integer port) { + super(supervisorId, port); + } + + public ResourceWorkerSlot(WorkerAssignment worker, + Map> componentToTask) { + super(worker.getNodeId(), worker.getPort()); + this.hostname = worker.getHostName(); + this.tasks = new HashSet(); + this.cpu = worker.getCpu(); + this.memSize = worker.getMem(); + this.jvm = worker.getJvm(); + for (Entry entry : worker.getComponentToNum() + .entrySet()) { + List tasks = componentToTask.get(entry.getKey()); + if (tasks == null || tasks.size() == 0) + continue; + int num = Math.min(tasks.size(), entry.getValue().intValue()); + List cTasks = new ArrayList(); + cTasks.addAll(tasks.subList(0, num)); + this.tasks.addAll(cTasks); + tasks.removeAll(cTasks); + } + } + + public String getHostname() { + return hostname; + } + + public void setHostname(String hostname) { + this.hostname = hostname; + } + + public Set getTasks() { + return tasks; + } + + public void setTasks(Set tasks) { + this.tasks = tasks; + } + + public String getJvm() { + return jvm; + } + + public void setJvm(String jvm) { + this.jvm = jvm; + } + + public long getMemSize() { + return memSize; + } + + public void setMemSize(long memSize) { + this.memSize = memSize; + } + + public int getCpu() { + return cpu; + } + + public void setCpu(int cpu) { + this.cpu = cpu; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } + + public boolean compareToUserDefineWorker(WorkerAssignment worker, + Map taskToComponent) { + int cpu = worker.getCpu(); + if (cpu != 0 && this.cpu != cpu) + return false; + long mem = worker.getMem(); + if (mem != 0 && this.memSize != mem) + return false; + String jvm = worker.getJvm(); + if (jvm != null && !jvm.equals(this.jvm)) + return false; + String hostName = worker.getHostName(); + if (NetWorkUtils.equals(hostname, hostName) == false) + return false; + int port = worker.getPort(); + if (port != 0 && port != this.getPort()) + return false; + Map componentToNum = worker.getComponentToNum(); + Map myComponentToNum = new HashMap(); + for (Integer task : tasks) { + String component = taskToComponent.get(task); + Integer i = myComponentToNum.get(component); + if (i == null) { + i = 0; + } + myComponentToNum.put(component, ++i); + } + + return myComponentToNum.equals(componentToNum); + } + + public String getHostPort() { + return hostname + ":" + getPort(); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/AbstractSelector.java b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/AbstractSelector.java new file mode 100755 index 000000000..eed0e3978 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/AbstractSelector.java @@ -0,0 +1,71 @@ +/** + * 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.schedule.default_assign.Selector; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; + +import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; +import com.alibaba.jstorm.schedule.default_assign.TaskAssignContext; + +public abstract class AbstractSelector implements Selector { + + protected final TaskAssignContext context; + + protected WorkerComparator workerComparator; + + protected WorkerComparator supervisorComparator; + + public AbstractSelector(TaskAssignContext context) { + this.context = context; + } + + protected List selectWorker( + List list, Comparator c) { + List result = new ArrayList(); + ResourceWorkerSlot best = null; + for (ResourceWorkerSlot worker : list) { + if (best == null) { + best = worker; + result.add(worker); + continue; + } + if (c.compare(best, worker) == 0) { + result.add(worker); + } else if (c.compare(best, worker) > 0) { + best = worker; + result.clear(); + result.add(best); + } + } + return result; + } + + @Override + public List select(List result, + String name) { + if (result.size() == 1) + return result; + result = this.selectWorker(result, workerComparator.get(name)); + if (result.size() == 1) + return result; + return this.selectWorker(result, supervisorComparator.get(name)); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/ComponentNumSelector.java b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/ComponentNumSelector.java new file mode 100755 index 000000000..8170ae268 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/ComponentNumSelector.java @@ -0,0 +1,56 @@ +/** + * 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.schedule.default_assign.Selector; + +import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; +import com.alibaba.jstorm.schedule.default_assign.TaskAssignContext; + +public class ComponentNumSelector extends AbstractSelector { + + public ComponentNumSelector(final TaskAssignContext context) { + super(context); + // TODO Auto-generated constructor stub + this.workerComparator = new WorkerComparator() { + @Override + public int compare(ResourceWorkerSlot o1, ResourceWorkerSlot o2) { + // TODO Auto-generated method stub + int o1Num = context.getComponentNumOnWorker(o1, name); + int o2Num = context.getComponentNumOnWorker(o2, name); + if (o1Num == o2Num) + return 0; + return o1Num > o2Num ? 1 : -1; + } + }; + this.supervisorComparator = new WorkerComparator() { + @Override + public int compare(ResourceWorkerSlot o1, ResourceWorkerSlot o2) { + // TODO Auto-generated method stub + int o1Num = + context.getComponentNumOnSupervisor(o1.getNodeId(), + name); + int o2Num = + context.getComponentNumOnSupervisor(o2.getNodeId(), + name); + if (o1Num == o2Num) + return 0; + return o1Num > o2Num ? 1 : -1; + } + }; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/InputComponentNumSelector.java b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/InputComponentNumSelector.java new file mode 100755 index 000000000..49eb447a3 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/InputComponentNumSelector.java @@ -0,0 +1,55 @@ +/** + * 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.schedule.default_assign.Selector; + +import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; +import com.alibaba.jstorm.schedule.default_assign.TaskAssignContext; + +public class InputComponentNumSelector extends AbstractSelector { + + public InputComponentNumSelector(final TaskAssignContext context) { + super(context); + // TODO Auto-generated constructor stub + this.workerComparator = new WorkerComparator() { + @Override + public int compare(ResourceWorkerSlot o1, ResourceWorkerSlot o2) { + // TODO Auto-generated method stub + int o1Num = context.getInputComponentNumOnWorker(o1, name); + int o2Num = context.getInputComponentNumOnWorker(o2, name); + if (o1Num == o2Num) + return 0; + return o1Num > o2Num ? -1 : 1; + } + }; + this.supervisorComparator = new WorkerComparator() { + @Override + public int compare(ResourceWorkerSlot o1, ResourceWorkerSlot o2) { + // TODO Auto-generated method stub + int o1Num = + context.getInputComponentNumOnSupervisor( + o1.getNodeId(), name); + int o2Num = + context.getInputComponentNumOnSupervisor( + o2.getNodeId(), name); + if (o1Num == o2Num) + return 0; + return o1Num > o2Num ? -1 : 1; + } + }; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/Selector.java b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/Selector.java new file mode 100755 index 000000000..adc8b290c --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/Selector.java @@ -0,0 +1,27 @@ +/** + * 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.schedule.default_assign.Selector; + +import java.util.List; + +import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; + +public interface Selector { + public List select(List result, + String name); +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/TotalTaskNumSelector.java b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/TotalTaskNumSelector.java new file mode 100755 index 000000000..94dfdc354 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/TotalTaskNumSelector.java @@ -0,0 +1,52 @@ +/** + * 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.schedule.default_assign.Selector; + +import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; +import com.alibaba.jstorm.schedule.default_assign.TaskAssignContext; + +public class TotalTaskNumSelector extends AbstractSelector { + + public TotalTaskNumSelector(final TaskAssignContext context) { + super(context); + // TODO Auto-generated constructor stub + this.workerComparator = new WorkerComparator() { + @Override + public int compare(ResourceWorkerSlot o1, ResourceWorkerSlot o2) { + // TODO Auto-generated method stub + int o1Num = context.getTaskNumOnWorker(o1); + int o2Num = context.getTaskNumOnWorker(o2); + if (o1Num == o2Num) + return 0; + return o1Num > o2Num ? 1 : -1; + } + }; + this.supervisorComparator = new WorkerComparator() { + @Override + public int compare(ResourceWorkerSlot o1, ResourceWorkerSlot o2) { + // TODO Auto-generated method stub + int o1Num = context.getTaskNumOnSupervisor(o1.getNodeId()); + int o2Num = context.getTaskNumOnSupervisor(o2.getNodeId()); + if (o1Num == o2Num) + return 0; + return o1Num > o2Num ? 1 : -1; + } + }; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/WorkerComparator.java b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/WorkerComparator.java new file mode 100755 index 000000000..f01ee9ae5 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/WorkerComparator.java @@ -0,0 +1,33 @@ +/** + * 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.schedule.default_assign.Selector; + +import java.util.Comparator; + +import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; + +public abstract class WorkerComparator implements + Comparator { + + protected String name; + + public WorkerComparator get(String name) { + this.name = name; + return this; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/TaskAssignContext.java b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/TaskAssignContext.java new file mode 100644 index 000000000..f81d07265 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/TaskAssignContext.java @@ -0,0 +1,133 @@ +/** + * 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.schedule.default_assign; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +public class TaskAssignContext { + + private final Map> supervisorToWorker; + + private final Map> relationship; + + // Map + private final Map> workerToComponentNum = + new HashMap>(); + + // Map + private final Map workerToTaskNum = + new HashMap(); + + private final Map HostPortToWorkerMap = + new HashMap(); + + public TaskAssignContext( + Map> supervisorToWorker, + Map> relationship) { + this.supervisorToWorker = supervisorToWorker; + this.relationship = relationship; + + for (Entry> entry : supervisorToWorker + .entrySet()) { + for (ResourceWorkerSlot worker : entry.getValue()) { + workerToTaskNum.put(worker, 0); + HostPortToWorkerMap.put(worker.getHostPort(), worker); + } + } + } + + public Map getWorkerToTaskNum() { + return workerToTaskNum; + } + + public Map> getSupervisorToWorker() { + return supervisorToWorker; + } + + public Map> getWorkerToComponentNum() { + return workerToComponentNum; + } + + public Map> getRelationship() { + return relationship; + } + + public int getComponentNumOnSupervisor(String supervisor, String name) { + List workers = supervisorToWorker.get(supervisor); + if (workers == null) + return 0; + int result = 0; + for (ResourceWorkerSlot worker : workers) { + result = result + this.getComponentNumOnWorker(worker, name); + } + return result; + } + + public int getComponentNumOnWorker(ResourceWorkerSlot worker, String name) { + int result = 0; + Map componentNum = workerToComponentNum.get(worker); + if (componentNum != null && componentNum.get(name) != null) + result = componentNum.get(name); + return result; + } + + public int getTaskNumOnSupervisor(String supervisor) { + List workers = supervisorToWorker.get(supervisor); + if (workers == null) + return 0; + int result = 0; + for (ResourceWorkerSlot worker : workers) { + result = result + this.getTaskNumOnWorker(worker); + } + return result; + } + + public int getTaskNumOnWorker(ResourceWorkerSlot worker) { + return worker.getTasks() == null ? 0 : worker.getTasks().size(); + } + + public int getInputComponentNumOnSupervisor(String supervisor, String name) { + int result = 0; + List workers = supervisorToWorker.get(supervisor); + if (workers == null) + return 0; + for (ResourceWorkerSlot worker : workers) + result = result + this.getInputComponentNumOnWorker(worker, name); + return result; + } + + public int getInputComponentNumOnWorker(ResourceWorkerSlot worker, + String name) { + int result = 0; + for (String component : relationship.get(name)) + result = result + this.getComponentNumOnWorker(worker, component); + return result; + } + + public Map getHostPortToWorkerMap() { + return HostPortToWorkerMap; + } + + public ResourceWorkerSlot getWorker(ResourceWorkerSlot worker) { + return HostPortToWorkerMap.get(worker.getHostPort()); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/TaskScheduler.java b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/TaskScheduler.java new file mode 100644 index 000000000..713146379 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/TaskScheduler.java @@ -0,0 +1,334 @@ +/** + * 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.schedule.default_assign; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.Common; +import com.alibaba.jstorm.schedule.TopologyAssignContext; +import com.alibaba.jstorm.schedule.default_assign.Selector.ComponentNumSelector; +import com.alibaba.jstorm.schedule.default_assign.Selector.InputComponentNumSelector; +import com.alibaba.jstorm.schedule.default_assign.Selector.Selector; +import com.alibaba.jstorm.schedule.default_assign.Selector.TotalTaskNumSelector; +import com.alibaba.jstorm.utils.FailedAssignTopologyException; + +public class TaskScheduler { + + public static Logger LOG = LoggerFactory.getLogger(TaskScheduler.class); + + public static final String ACKER_NAME = "__acker"; + + private final TaskAssignContext taskContext; + + private List assignments = + new ArrayList(); + + private int workerNum; + + /** + * For balance purpose, default scheduler is trying to assign the same + * number of tasks to a worker. e.g. There are 4 tasks and 3 available + * workers. Each worker will be assigned one task first. And then one worker + * is chosen for the last one. + */ + private int avgTaskNum; + private int leftTaskNum; + + private Set tasks; + + private DefaultTopologyAssignContext context; + + private Selector componentSelector; + + private Selector inputComponentSelector; + + private Selector totalTaskNumSelector; + + public TaskScheduler(DefaultTopologyAssignContext context, + Set tasks, List workers) { + this.tasks = tasks; + LOG.info("Tasks " + tasks + " is going to be assigned in workers " + + workers); + this.context = context; + this.taskContext = + new TaskAssignContext(this.buildSupervisorToWorker(workers), + Common.buildSpoutOutoputAndBoltInputMap(context)); + this.componentSelector = new ComponentNumSelector(taskContext); + this.inputComponentSelector = + new InputComponentNumSelector(taskContext); + this.totalTaskNumSelector = new TotalTaskNumSelector(taskContext); + if (tasks.size() == 0) + return; + setTaskNum(tasks.size(), workerNum); + + // For Scale-out case, the old assignment should be kept. + if (context.getAssignType() == TopologyAssignContext.ASSIGN_TYPE_REBALANCE + && context.isReassign() == false) { + keepAssignment(context.getOldAssignment().getWorkers()); + } + } + + private void keepAssignment(Set keepAssignments) { + Set keepTasks = new HashSet(); + for (ResourceWorkerSlot worker : keepAssignments) { + for (Integer taskId : worker.getTasks()) { + if (tasks.contains(taskId)) { + ResourceWorkerSlot contextWorker = + taskContext.getWorker(worker); + if (contextWorker != null) { + String componentName = + context.getTaskToComponent().get(taskId); + updateAssignedTasksOfWorker(taskId, contextWorker); + updateComponentsNumOfWorker(componentName, + contextWorker); + keepTasks.add(taskId); + } + } + } + } + + // Try to find the workers which have been assigned too much tasks + // If found, remove the workers from worker resource pool and update + // the avgNum and leftNum + int doneAssignedTaskNum = 0; + while (true) { + boolean found = false; + Set doneAssignedWorkers = + new HashSet(); + for (ResourceWorkerSlot worker : keepAssignments) { + ResourceWorkerSlot contextWorker = + taskContext.getWorker(worker); + if (contextWorker != null && isTaskFullForWorker(contextWorker)) { + found = true; + workerNum--; + taskContext.getWorkerToTaskNum().remove(contextWorker); + assignments.add(contextWorker); + + doneAssignedWorkers.add(worker); + doneAssignedTaskNum += contextWorker.getTasks().size(); + } + } + + if (found) { + setTaskNum((tasks.size() - doneAssignedTaskNum), workerNum); + keepAssignments.removeAll(doneAssignedWorkers); + } else { + break; + } + } + tasks.removeAll(keepTasks); + LOG.info("keep following assignment, " + assignments); + } + + private boolean isTaskFullForWorker(ResourceWorkerSlot worker) { + boolean ret = false; + Set tasks = worker.getTasks(); + + if (tasks != null) { + if ((leftTaskNum == 0 && tasks.size() >= avgTaskNum) + || (leftTaskNum > 0 && tasks.size() >= (avgTaskNum + 1))) { + ret = true; + } + } + return ret; + } + + public List assign() { + if (tasks.size() == 0) + return assignments; + + // Firstly, assign workers to the components which are configured + // by "task.on.differ.node" + Set assignedTasks = assignForDifferNodeTask(); + + // Assign for the tasks except acker + tasks.removeAll(assignedTasks); + Set ackers = new HashSet(); + for (Integer task : tasks) { + String name = context.getTaskToComponent().get(task); + if (name.equals(TaskScheduler.ACKER_NAME)) { + ackers.add(task); + continue; + } + assignForTask(name, task); + } + + // At last, make the assignment for acker + for (Integer task : ackers) { + assignForTask(TaskScheduler.ACKER_NAME, task); + } + return assignments; + } + + private void assignForTask(String name, Integer task) { + ResourceWorkerSlot worker = + chooseWorker(name, new ArrayList( + taskContext.getWorkerToTaskNum().keySet())); + pushTaskToWorker(task, name, worker); + } + + private Set assignForDifferNodeTask() { + Set ret = new HashSet(); + for (Integer task : tasks) { + Map conf = Common.getComponentMap(context, task); + if (ConfigExtension.isTaskOnDifferentNode(conf)) + ret.add(task); + } + for (Integer task : ret) { + String name = context.getTaskToComponent().get(task); + ResourceWorkerSlot worker = + chooseWorker(name, getDifferNodeTaskWokers(name)); + pushTaskToWorker(task, name, worker); + } + return ret; + } + + private Map> buildSupervisorToWorker( + List workers) { + Map> supervisorToWorker = + new HashMap>(); + for (ResourceWorkerSlot worker : workers) { + if (worker.getTasks() == null || worker.getTasks().size() == 0) { + List supervisor = + supervisorToWorker.get(worker.getNodeId()); + if (supervisor == null) { + supervisor = new ArrayList(); + supervisorToWorker.put(worker.getNodeId(), supervisor); + } + supervisor.add(worker); + } else { + assignments.add(worker); + } + } + this.workerNum = workers.size() - assignments.size(); + return supervisorToWorker; + } + + private ResourceWorkerSlot chooseWorker(String name, + List workers) { + List result = + componentSelector.select(workers, name); + result = totalTaskNumSelector.select(result, name); + if (name.equals(TaskScheduler.ACKER_NAME)) + return result.iterator().next(); + result = inputComponentSelector.select(result, name); + return result.iterator().next(); + } + + private void pushTaskToWorker(Integer task, String name, + ResourceWorkerSlot worker) { + LOG.debug("Push task-" + task + " to worker-" + worker.getPort()); + int taskNum = updateAssignedTasksOfWorker(task, worker); + + if (leftTaskNum <= 0) { + if (taskNum == avgTaskNum) { + taskContext.getWorkerToTaskNum().remove(worker); + assignments.add(worker); + } + } else { + if (taskNum == (avgTaskNum + 1)) { + taskContext.getWorkerToTaskNum().remove(worker); + leftTaskNum--; + assignments.add(worker); + } + if (leftTaskNum <= 0) { + List needDelete = + new ArrayList(); + for (Entry entry : taskContext + .getWorkerToTaskNum().entrySet()) { + if (entry.getValue() == avgTaskNum) + needDelete.add(entry.getKey()); + } + for (ResourceWorkerSlot workerToDelete : needDelete) { + taskContext.getWorkerToTaskNum().remove(workerToDelete); + assignments.add(workerToDelete); + } + } + } + + updateComponentsNumOfWorker(name, worker); + } + + private int updateAssignedTasksOfWorker(Integer task, + ResourceWorkerSlot worker) { + int ret = 0; + Set tasks = worker.getTasks(); + if (tasks == null) { + tasks = new HashSet(); + worker.setTasks(tasks); + } + tasks.add(task); + + ret = taskContext.getWorkerToTaskNum().get(worker); + taskContext.getWorkerToTaskNum().put(worker, ++ret); + return ret; + } + + private void updateComponentsNumOfWorker(String name, + ResourceWorkerSlot worker) { + Map components = + taskContext.getWorkerToComponentNum().get(worker); + if (components == null) { + components = new HashMap(); + taskContext.getWorkerToComponentNum().put(worker, components); + } + Integer componentNum = components.get(name); + if (componentNum == null) { + componentNum = 0; + } + components.put(name, ++componentNum); + } + + private void setTaskNum(int taskNum, int workerNum) { + if (taskNum >= 0 && workerNum > 0) { + this.avgTaskNum = taskNum / workerNum; + this.leftTaskNum = taskNum % workerNum; + LOG.debug("avgTaskNum=" + avgTaskNum + ", leftTaskNum=" + + leftTaskNum); + } else { + LOG.debug("Illegal parameters, taskNum=" + taskNum + ", workerNum=" + + workerNum); + } + } + + private List getDifferNodeTaskWokers(String name) { + List workers = new ArrayList(); + workers.addAll(taskContext.getWorkerToTaskNum().keySet()); + + for (Entry> entry : taskContext + .getSupervisorToWorker().entrySet()) { + if (taskContext.getComponentNumOnSupervisor(entry.getKey(), name) != 0) + workers.removeAll(entry.getValue()); + } + if (workers.size() == 0) + throw new FailedAssignTopologyException( + "there's no enough supervisor for making component: " + + name + " 's tasks on different node"); + return workers; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/WorkerScheduler.java b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/WorkerScheduler.java new file mode 100644 index 000000000..c85d72317 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/schedule/default_assign/WorkerScheduler.java @@ -0,0 +1,387 @@ +/** + * 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.schedule.default_assign; + +import java.util.*; +import java.util.Map.Entry; + +import org.apache.commons.lang.math.NumberUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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.schedule.TopologyAssignContext; +import com.alibaba.jstorm.utils.FailedAssignTopologyException; +import com.alibaba.jstorm.utils.NetWorkUtils; + +public class WorkerScheduler { + + public static Logger LOG = LoggerFactory.getLogger(WorkerScheduler.class); + + private static WorkerScheduler instance; + + private WorkerScheduler() { + + } + + public static WorkerScheduler getInstance() { + if (instance == null) { + instance = new WorkerScheduler(); + } + return instance; + } + + public List getAvailableWorkers( + DefaultTopologyAssignContext context, Set needAssign, + int num) { + int workersNum = getWorkersNum(context, num); + if (workersNum == 0) { + throw new FailedAssignTopologyException("there's no enough worker"); + } + List assignedWorkers = + new ArrayList(); + // userdefine assignments + getRightWorkers( + context, + needAssign, + assignedWorkers, + workersNum, + getUserDefineWorkers(context, ConfigExtension + .getUserDefineAssignment(context.getStormConf()))); + // old assignments + if (ConfigExtension.isUseOldAssignment(context.getStormConf())) { + getRightWorkers(context, needAssign, assignedWorkers, workersNum, + context.getOldWorkers()); + } else if (context.getAssignType() == TopologyAssignContext.ASSIGN_TYPE_REBALANCE + && context.isReassign() == false) { + int cnt = 0; + for (ResourceWorkerSlot worker : context.getOldWorkers()) { + if (cnt < workersNum) { + ResourceWorkerSlot resFreeWorker = new ResourceWorkerSlot(); + resFreeWorker.setPort(worker.getPort()); + resFreeWorker.setHostname(worker.getHostname()); + resFreeWorker.setNodeId(worker.getNodeId()); + assignedWorkers.add(resFreeWorker); + cnt++; + } else { + break; + } + } + } + int defaultWorkerNum = + Math.min(workersNum - assignedWorkers.size(), needAssign.size()); + LOG.info("Get workers from user define and old assignments: " + + assignedWorkers); + for (int i = 0; i < defaultWorkerNum; i++) { + assignedWorkers.add(new ResourceWorkerSlot()); + } + List isolationSupervisors = + this.getIsolationSupervisors(context); + if (isolationSupervisors.size() != 0) { + putAllWorkerToSupervisor(assignedWorkers, + getResAvailSupervisors(isolationSupervisors)); + } else { + putAllWorkerToSupervisor(assignedWorkers, + getResAvailSupervisors(context.getCluster())); + } + this.setAllWorkerMemAndCpu(context.getStormConf(), assignedWorkers); + LOG.info("Assigned workers=" + assignedWorkers); + return assignedWorkers; + } + + private void setAllWorkerMemAndCpu(Map conf, + List assignedWorkers) { + long defaultSize = ConfigExtension.getMemSizePerWorker(conf); + int defaultCpu = ConfigExtension.getCpuSlotPerWorker(conf); + for (ResourceWorkerSlot worker : assignedWorkers) { + if (worker.getMemSize() <= 0) + worker.setMemSize(defaultSize); + if (worker.getCpu() <= 0) + worker.setCpu(defaultCpu); + } + } + + private void putAllWorkerToSupervisor( + List assignedWorkers, + List supervisors) { + for (ResourceWorkerSlot worker : assignedWorkers) { + if (worker.getHostname() != null) { + for (SupervisorInfo supervisor : supervisors) { + if (NetWorkUtils.equals(supervisor.getHostName(), + worker.getHostname()) + && supervisor.getAvailableWorkerPorts().size() > 0) { + putWorkerToSupervisor(supervisor, worker); + break; + } + } + } + } + supervisors = getResAvailSupervisors(supervisors); + Collections.sort(supervisors, new Comparator() { + + @Override + public int compare(SupervisorInfo o1, SupervisorInfo o2) { + // TODO Auto-generated method stub + return -NumberUtils.compare(o1.getAvailableWorkerPorts().size(), o2 + .getAvailableWorkerPorts().size()); + } + + }); + putWorkerToSupervisor(assignedWorkers, supervisors); + } + + private void putWorkerToSupervisor(SupervisorInfo supervisor, + ResourceWorkerSlot worker) { + int port = worker.getPort(); + if (!supervisor.getAvailableWorkerPorts().contains(worker.getPort())) { + port = supervisor.getAvailableWorkerPorts().iterator().next(); + } + worker.setPort(port); + supervisor.getAvailableWorkerPorts().remove(port); + worker.setNodeId(supervisor.getSupervisorId()); + } + + private void putWorkerToSupervisor( + List assignedWorkers, + List supervisors) { + int allUsedPorts = 0; + for (SupervisorInfo supervisor : supervisors) { + int supervisorUsedPorts = supervisor.getWorkerPorts().size() + - supervisor.getAvailableWorkerPorts().size(); + allUsedPorts = allUsedPorts + supervisorUsedPorts; + } + // per supervisor should be allocated ports in theory + int theoryAveragePorts = + (allUsedPorts + assignedWorkers.size()) / supervisors.size() + + 1; + // supervisor which use more than theoryAveragePorts ports will be + // pushed overLoadSupervisors + List overLoadSupervisors = + new ArrayList(); + int key = 0; + Iterator iterator = assignedWorkers.iterator(); + while (iterator.hasNext()) { + if (supervisors.size() == 0) + break; + if (key >= supervisors.size()) + key = 0; + SupervisorInfo supervisor = supervisors.get(key); + int supervisorUsedPorts = supervisor.getWorkerPorts().size() + - supervisor.getAvailableWorkerPorts().size(); + if (supervisorUsedPorts < theoryAveragePorts) { + ResourceWorkerSlot worker = iterator.next(); + if (worker.getNodeId() != null) + continue; + worker.setHostname(supervisor.getHostName()); + worker.setNodeId(supervisor.getSupervisorId()); + worker.setPort( + supervisor.getAvailableWorkerPorts().iterator().next()); + supervisor.getAvailableWorkerPorts().remove(worker.getPort()); + if (supervisor.getAvailableWorkerPorts().size() == 0) + supervisors.remove(supervisor); + key++; + } else { + overLoadSupervisors.add(supervisor); + supervisors.remove(supervisor); + } + } + // rest assignedWorkers will be allocate supervisor by deal + Collections.sort(overLoadSupervisors, new Comparator() { + + @Override + public int compare(SupervisorInfo o1, SupervisorInfo o2) { + // TODO Auto-generated method stub + return -NumberUtils.compare(o1.getAvailableWorkerPorts().size(), + o2.getAvailableWorkerPorts().size()); + } + + }); + key = 0; + while (iterator.hasNext()) { + if (overLoadSupervisors.size() == 0) + break; + if (key >= overLoadSupervisors.size()) + key = 0; + ResourceWorkerSlot worker = iterator.next(); + if (worker.getNodeId() != null) + continue; + SupervisorInfo supervisor = overLoadSupervisors.get(key); + worker.setHostname(supervisor.getHostName()); + worker.setNodeId(supervisor.getSupervisorId()); + worker.setPort( + supervisor.getAvailableWorkerPorts().iterator().next()); + supervisor.getAvailableWorkerPorts().remove(worker.getPort()); + if (supervisor.getAvailableWorkerPorts().size() == 0) + overLoadSupervisors.remove(supervisor); + key++; + } + } + + private void getRightWorkers(DefaultTopologyAssignContext context, + Set needAssign, List assignedWorkers, + int workersNum, Collection workers) { + Set assigned = new HashSet(); + List users = new ArrayList(); + if (workers == null) + return; + for (ResourceWorkerSlot worker : workers) { + boolean right = true; + Set tasks = worker.getTasks(); + if (tasks == null) + continue; + for (Integer task : tasks) { + if (!needAssign.contains(task) || assigned.contains(task)) { + right = false; + break; + } + } + if (right) { + assigned.addAll(tasks); + users.add(worker); + } + } + if (users.size() + assignedWorkers.size() > workersNum) { + return; + } + + if (users.size() + assignedWorkers.size() == workersNum + && assigned.size() != needAssign.size()) { + return; + } + assignedWorkers.addAll(users); + needAssign.removeAll(assigned); + } + + 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.getAvailableWorkerPorts().size(); + } + return Math.min(slotNum, workersNum); + } + for (Entry entry : supervisors.entrySet()) { + slotNum = slotNum + entry.getValue().getAvailableWorkerPorts().size(); + } + return Math.min(slotNum, workersNum); + } + + /** + * @param context + * @param workers + * @return + */ + private List getUserDefineWorkers( + DefaultTopologyAssignContext context, List workers) { + List ret = new ArrayList(); + if (workers == null) + return ret; + Map> componentToTask = + (HashMap>) ((HashMap>) context + .getComponentTasks()).clone(); + if (context.getAssignType() != context.ASSIGN_TYPE_NEW) { + checkUserDefineWorkers(context, workers, + context.getTaskToComponent()); + } + for (WorkerAssignment worker : workers) { + ResourceWorkerSlot workerSlot = + new ResourceWorkerSlot(worker, componentToTask); + if (workerSlot.getTasks().size() != 0) { + ret.add(workerSlot); + } + } + return ret; + } + + private void checkUserDefineWorkers(DefaultTopologyAssignContext context, + List workers, Map taskToComponent) { + Set unstoppedWorkers = + context.getUnstoppedWorkers(); + List re = new ArrayList(); + for (WorkerAssignment worker : workers) { + for (ResourceWorkerSlot unstopped : unstoppedWorkers) { + if (unstopped + .compareToUserDefineWorker(worker, taskToComponent)) + re.add(worker); + } + } + workers.removeAll(re); + + } + + private List getResAvailSupervisors( + Map supervisors) { + List availableSupervisors = + new ArrayList(); + for (Entry entry : supervisors.entrySet()) { + SupervisorInfo supervisor = entry.getValue(); + if (supervisor.getAvailableWorkerPorts().size() > 0) + availableSupervisors.add(entry.getValue()); + } + return availableSupervisors; + } + + private List getResAvailSupervisors( + List supervisors) { + List availableSupervisors = + new ArrayList(); + for (SupervisorInfo supervisor : supervisors) { + if (supervisor.getAvailableWorkerPorts().size() > 0) + availableSupervisors.add(supervisor); + } + return availableSupervisors; + } + + private List getIsolationSupervisors( + DefaultTopologyAssignContext context) { + List isolationHosts = + (List) context.getStormConf().get( + Config.ISOLATION_SCHEDULER_MACHINES); + LOG.info("Isolation machines: " + isolationHosts); + if (isolationHosts == null) + return new ArrayList(); + List isolationSupervisors = + new ArrayList(); + 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-core/src/main/java/com/alibaba/jstorm/task/Task.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/Task.java new file mode 100644 index 000000000..6481c5e12 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/Task.java @@ -0,0 +1,370 @@ +/** + * 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.task; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.messaging.IContext; +import backtype.storm.serialization.KryoTupleSerializer; +import backtype.storm.spout.ISpout; +import backtype.storm.task.IBolt; +import backtype.storm.task.TopologyContext; +import backtype.storm.utils.DisruptorQueue; +import backtype.storm.utils.Utils; +import backtype.storm.utils.WorkerClassLoader; +import clojure.lang.Atom; + +import com.alibaba.jstorm.callback.AsyncLoopDefaultKill; +import com.alibaba.jstorm.callback.AsyncLoopThread; +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.Common; +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.cluster.StormZkClusterState; +import com.alibaba.jstorm.daemon.worker.WorkerData; +import com.alibaba.jstorm.schedule.Assignment.AssignmentType; +import com.alibaba.jstorm.task.comm.TaskSendTargets; +import com.alibaba.jstorm.task.comm.UnanchoredSend; +import com.alibaba.jstorm.task.error.ITaskReportErr; +import com.alibaba.jstorm.task.error.TaskReportError; +import com.alibaba.jstorm.task.error.TaskReportErrorAndDie; +import com.alibaba.jstorm.task.execute.BaseExecutors; +import com.alibaba.jstorm.task.execute.BoltExecutors; +import com.alibaba.jstorm.task.execute.spout.MultipleThreadSpoutExecutors; +import com.alibaba.jstorm.task.execute.spout.SingleThreadSpoutExecutors; +import com.alibaba.jstorm.task.execute.spout.SpoutExecutors; +import com.alibaba.jstorm.task.group.MkGrouper; +import com.alibaba.jstorm.task.heartbeat.TaskHeartbeatRunable; +import com.alibaba.jstorm.task.heartbeat.TaskStats; +import com.alibaba.jstorm.utils.JStormServerUtils; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.NetWorkUtils; +import com.lmax.disruptor.WaitStrategy; +import com.lmax.disruptor.dsl.ProducerType; + +/** + * Task instance + * + * @author yannian/Longda + * + */ +public class Task { + + private final static Logger LOG = LoggerFactory.getLogger(Task.class); + + private Map stormConf; + + private TopologyContext topologyContext; + private TopologyContext userContext; + private String topologyid; + private IContext context; + + private TaskTransfer taskTransfer; + private TaskReceiver taskReceiver; + private Map innerTaskTransfer; + private Map deserializeQueues; + private AsyncLoopDefaultKill workHalt; + + private Integer taskid; + private String componentid; + private volatile TaskStatus taskStatus; + private Atom openOrPrepareWasCalled; + // running time counter + private UptimeComputer uptime = new UptimeComputer(); + + private StormClusterState zkCluster; + private Object taskObj; + private TaskBaseMetric taskStats; + private WorkerData workerData; + private String componentType; // "spout" or "bolt" + + private TaskSendTargets taskSendTargets; + + private boolean isTaskBatchTuple; + + @SuppressWarnings("rawtypes") + public Task(WorkerData workerData, int taskId) throws Exception { + openOrPrepareWasCalled = new Atom(Boolean.valueOf(false)); + + this.workerData = workerData; + this.topologyContext = + workerData.getContextMaker().makeTopologyContext( + workerData.getSysTopology(), taskId, + openOrPrepareWasCalled); + this.userContext = + workerData.getContextMaker().makeTopologyContext( + workerData.getRawTopology(), taskId, + openOrPrepareWasCalled); + this.taskid = taskId; + this.componentid = topologyContext.getThisComponentId(); + this.stormConf = + Common.component_conf(workerData.getStormConf(), + topologyContext, componentid); + + this.taskStatus = new TaskStatus(); + this.taskStats = new TaskBaseMetric(taskId); + + this.innerTaskTransfer = workerData.getInnerTaskTransfer(); + this.deserializeQueues = workerData.getDeserializeQueues(); + this.topologyid = workerData.getTopologyId(); + this.context = workerData.getContext(); + this.workHalt = workerData.getWorkHalt(); + this.zkCluster = + new StormZkClusterState(workerData.getZkClusterstate()); + + LOG.info("Begin to deserialize taskObj " + componentid + ":" + taskid); + + WorkerClassLoader.switchThreadContext(); + // get real task object -- spout/bolt/spoutspec + this.taskObj = + Common.get_task_object(topologyContext.getRawTopology(), + componentid, WorkerClassLoader.getInstance()); + WorkerClassLoader.restoreThreadContext(); + + isTaskBatchTuple = ConfigExtension.isTaskBatchTuple(stormConf); + LOG.info("Transfer/receive in batch mode :" + isTaskBatchTuple); + + LOG.info("Loading task " + componentid + ":" + taskid); + } + + private void setComponentType() { + if (taskObj instanceof IBolt) { + componentType = "bolt"; + } else if (taskObj instanceof ISpout) { + componentType = "spout"; + } + } + + private TaskSendTargets makeSendTargets() { + String component = topologyContext.getThisComponentId(); + + // get current task's output + // > + Map> streamComponentGrouper = + Common.outbound_components(topologyContext, workerData); + + return new TaskSendTargets(stormConf, component, + streamComponentGrouper, topologyContext, taskStats); + } + + private void updateSendTargets() { + if (taskSendTargets != null) { + Map> streamComponentGrouper = + Common.outbound_components(topologyContext, workerData); + taskSendTargets.updateStreamCompGrouper(streamComponentGrouper); + } else { + LOG.error("taskSendTargets is null when trying to update it."); + } + } + + private TaskTransfer mkTaskSending(WorkerData workerData) { + + // sending tuple's serializer + KryoTupleSerializer serializer = + new KryoTupleSerializer(workerData.getStormConf(), + topologyContext); + + String taskName = JStormServerUtils.getName(componentid, taskid); + // Task sending all tuples through this Object + TaskTransfer taskTransfer; + if (isTaskBatchTuple) + taskTransfer = + new TaskBatchTransfer(this, taskName, serializer, + taskStatus, workerData); + else + taskTransfer = + new TaskTransfer(this, taskName, serializer, taskStatus, + workerData); + return taskTransfer; + } + + public TaskSendTargets echoToSystemBolt() { + // send "startup" tuple to system bolt + List msg = new ArrayList(); + msg.add("startup"); + + // create task receive object + TaskSendTargets sendTargets = makeSendTargets(); + UnanchoredSend.send(topologyContext, sendTargets, taskTransfer, + Common.SYSTEM_STREAM_ID, msg); + + return sendTargets; + } + + public boolean isSingleThread(Map conf) { + boolean isOnePending = JStormServerUtils.isOnePending(conf); + if (isOnePending == true) { + return true; + } + + return ConfigExtension.isSpoutSingleThread(conf); + } + + public RunnableCallback mk_executors(TaskSendTargets sendTargets, + ITaskReportErr report_error) { + + if (taskObj instanceof IBolt) { + return new BoltExecutors(this, (IBolt) taskObj, taskTransfer, + innerTaskTransfer, stormConf, sendTargets, taskStatus, + topologyContext, userContext, taskStats, report_error); + } else if (taskObj instanceof ISpout) { + if (isSingleThread(stormConf) == true) { + return new SingleThreadSpoutExecutors(this, (ISpout) taskObj, + taskTransfer, innerTaskTransfer, stormConf, + sendTargets, taskStatus, topologyContext, userContext, + taskStats, report_error); + } else { + return new MultipleThreadSpoutExecutors(this, (ISpout) taskObj, + taskTransfer, innerTaskTransfer, stormConf, + sendTargets, taskStatus, topologyContext, userContext, + taskStats, report_error); + } + } + + return null; + } + + /** + * create executor to receive tuples and run bolt/spout execute function + * + * @param puller + * @param sendTargets + * @return + */ + private RunnableCallback mkExecutor(TaskSendTargets sendTargets) { + // create report error callback, + // in fact it is storm_cluster.report-task-error + ITaskReportErr reportError = + new TaskReportError(zkCluster, topologyid, taskid); + + // report error and halt worker + TaskReportErrorAndDie reportErrorDie = + new TaskReportErrorAndDie(reportError, workHalt); + + return mk_executors(sendTargets, reportErrorDie); + } + + public TaskReceiver mkTaskReceiver() { + String taskName = JStormServerUtils.getName(componentid, taskid); + TaskReceiver taskReceiver; + if (isTaskBatchTuple) + taskReceiver = + new TaskBatchReceiver(this, taskid, stormConf, + topologyContext, innerTaskTransfer, taskStatus, + taskName); + else + taskReceiver = + new TaskReceiver(this, taskid, stormConf, topologyContext, + innerTaskTransfer, taskStatus, taskName); + deserializeQueues.put(taskid, taskReceiver.getDeserializeQueue()); + return taskReceiver; + } + + public TaskShutdownDameon execute() throws Exception { + setComponentType(); + + taskSendTargets = echoToSystemBolt(); + + // create thread to get tuple from zeroMQ, + // and pass the tuple to bolt/spout + taskTransfer = mkTaskSending(workerData); + RunnableCallback baseExecutor = mkExecutor(taskSendTargets); + AsyncLoopThread executor_threads = + new AsyncLoopThread(baseExecutor, false, Thread.MAX_PRIORITY, + true); + taskReceiver = mkTaskReceiver(); + + List allThreads = new ArrayList(); + allThreads.add(executor_threads); + + TaskHeartbeatRunable.registerTaskStats(taskid, new TaskStats( + componentType, taskStats)); + LOG.info("Finished loading task " + componentid + ":" + taskid); + + return getShutdown(allThreads, taskReceiver.getDeserializeQueue(), + baseExecutor); + } + + public TaskShutdownDameon getShutdown(List allThreads, + DisruptorQueue deserializeQueue, RunnableCallback baseExecutor) { + + AsyncLoopThread ackerThread = null; + if (baseExecutor instanceof SpoutExecutors) { + ackerThread = + ((SpoutExecutors) baseExecutor).getAckerRunnableThread(); + + if (ackerThread != null) { + allThreads.add(ackerThread); + } + } + AsyncLoopThread recvThread = taskReceiver.getDeserializeThread(); + allThreads.add(recvThread); + + AsyncLoopThread serializeThread = taskTransfer.getSerializeThread(); + allThreads.add(serializeThread); + + TaskShutdownDameon shutdown = + new TaskShutdownDameon(taskStatus, topologyid, taskid, + allThreads, zkCluster, taskObj); + + return shutdown; + } + + public static TaskShutdownDameon mk_task(WorkerData workerData, int taskId) + throws Exception { + + Task t = new Task(workerData, taskId); + + return t.execute(); + } + + /** + * Update the data which can be changed dynamically e.g. when scale-out of a + * task parallelism + */ + public void updateTaskData() { + // Only update the local task list of topologyContext here. Because + // other + // relative parts in context shall be updated while the updating of + // WorkerData (Task2Component and Component2Task map) + List localTasks = JStormUtils.mk_list(workerData.getTaskids()); + topologyContext.setThisWorkerTasks(localTasks); + userContext.setThisWorkerTasks(localTasks); + + // Update the TaskSendTargets + updateSendTargets(); + } + + public long getWorkerAssignmentTs() { + return workerData.getAssignmentTs(); + } + + public AssignmentType getWorkerAssignmentType() { + return workerData.getAssignmentType(); + } + + public void unregisterDeserializeQueue() { + deserializeQueues.remove(taskid); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskBaseMetric.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskBaseMetric.java new file mode 100755 index 000000000..4c9eb0b6e --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskBaseMetric.java @@ -0,0 +1,132 @@ +/** + * 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.task; + +import java.io.Serializable; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.cluster.Common; +import com.alibaba.jstorm.common.metric.MetricRegistry; +import com.alibaba.jstorm.common.metric.window.Metric; +import com.alibaba.jstorm.metric.JStormMetrics; +import com.alibaba.jstorm.metric.MetricDef; + +public class TaskBaseMetric implements Serializable { + private static final Logger LOG = LoggerFactory.getLogger(TaskBaseMetric.class); + private static final long serialVersionUID = -7157987126460293444L; + protected MetricRegistry metrics; + private int taskId; + + public TaskBaseMetric(int taskId) { + metrics = JStormMetrics.registerTask(taskId); + this.taskId = taskId; + } + + public void update(String name, Number value, int type) { + Metric metric = metrics.getMetric(name); + if (metric == null) { + metric = JStormMetrics.Builder.mkInstance(type); + try { + /** + * Here use one hack method to handle competition register metric + * if duplicated metric, just skip it. + * + * this will improve performance + */ + JStormMetrics.registerTaskMetric(metric, taskId, name); + }catch(Exception e) { + LOG.warn("Duplicated metrics of {}, taskId:{}", name, taskId); + return ; + } + + } + metric.update(value); + } + + public void send_tuple(String stream, int num_out_tasks) { + if (num_out_tasks <= 0) { + return; + } + + String emmitedName = + MetricRegistry.name(MetricDef.EMMITTED_NUM, stream); + update(emmitedName, Double.valueOf(num_out_tasks), + JStormMetrics.Builder.COUNTER); + + String sendTpsName = MetricRegistry.name(MetricDef.SEND_TPS, stream); + update(sendTpsName, Double.valueOf(num_out_tasks), + JStormMetrics.Builder.METER); + } + + public void recv_tuple(String component, String stream) { + + String name = + MetricRegistry.name(MetricDef.RECV_TPS, component, stream); + update(name, Double.valueOf(1), JStormMetrics.Builder.METER); + + } + + public void bolt_acked_tuple(String component, String stream, + Double latency_ms) { + + if (latency_ms == null) { + return; + } + + String ackNumName = + MetricRegistry.name(MetricDef.ACKED_NUM, component, stream); + update(ackNumName, Double.valueOf(1), JStormMetrics.Builder.COUNTER); + + String processName = + MetricRegistry.name(MetricDef.PROCESS_LATENCY, component, + stream); + update(processName, latency_ms, + JStormMetrics.Builder.HISTOGRAM); + } + + public void bolt_failed_tuple(String component, String stream) { + + String failNumName = + MetricRegistry.name(MetricDef.FAILED_NUM, component, stream); + update(failNumName, Double.valueOf(1), JStormMetrics.Builder.COUNTER); + } + + public void spout_acked_tuple(String stream, long st) { + + String ackNumName = + MetricRegistry.name(MetricDef.ACKED_NUM, + Common.ACKER_COMPONENT_ID, stream); + update(ackNumName, Double.valueOf(1), JStormMetrics.Builder.COUNTER); + + String processName = + MetricRegistry.name(MetricDef.PROCESS_LATENCY, + Common.ACKER_COMPONENT_ID, stream); + update(processName, Double.valueOf(st), JStormMetrics.Builder.HISTOGRAM); + + } + + public void spout_failed_tuple(String stream) { + String failNumName = + MetricRegistry.name(MetricDef.FAILED_NUM, + Common.ACKER_COMPONENT_ID, stream); + update(failNumName, Double.valueOf(1), JStormMetrics.Builder.COUNTER); + + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskBatchReceiver.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskBatchReceiver.java new file mode 100644 index 000000000..db2e990b4 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskBatchReceiver.java @@ -0,0 +1,99 @@ +/** + * 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.task; + +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.callback.AsyncLoopThread; +import com.alibaba.jstorm.task.TaskReceiver.DeserializeRunnable; +import com.alibaba.jstorm.utils.JStormUtils; + +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.BatchTuple; +import backtype.storm.tuple.Tuple; +import backtype.storm.utils.DisruptorQueue; + +public class TaskBatchReceiver extends TaskReceiver { + private static Logger LOG = LoggerFactory + .getLogger(TaskBatchReceiver.class); + + public TaskBatchReceiver(Task task, int taskId, Map stormConf, + TopologyContext topologyContext, + Map innerTaskTransfer, + TaskStatus taskStatus, String taskName) { + super(task, taskId, stormConf, topologyContext, innerTaskTransfer, + taskStatus, taskName); + } + + @Override + protected void setDeserializeThread() { + this.deserializeThread = + new AsyncLoopThread(new DeserializeBatchRunnable( + deserializeQueue, innerTaskTransfer.get(taskId))); + } + + public class DeserializeBatchRunnable extends DeserializeRunnable { + public DeserializeBatchRunnable(DisruptorQueue deserializeQueue, + DisruptorQueue exeQueue) { + super(deserializeQueue, exeQueue); + } + + @Override + protected Object deserialize(byte[] ser_msg) { + long start = System.nanoTime(); + try { + if (ser_msg == null) { + return null; + } + + if (ser_msg.length == 0) { + return null; + } else if (ser_msg.length == 1) { + byte newStatus = ser_msg[0]; + LOG.info("Change task status as " + newStatus); + taskStatus.setStatus(newStatus); + + return null; + } + + // ser_msg.length > 1 + BatchTuple tuple = deserializer.deserializeBatch(ser_msg); + if (isDebugRecv) { + LOG.info(idStr + " receive " + tuple.toString()); + } + + return tuple; + } catch (Throwable e) { + if (taskStatus.isShutdown() == false) { + LOG.error( + idStr + " recv thread error " + + JStormUtils.toPrintableString(ser_msg) + + "\n", e); + } + } finally { + long end = System.nanoTime(); + deserializeTimer.update((end - start)/1000000.0d); + } + + return null; + } + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskBatchTransfer.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskBatchTransfer.java new file mode 100644 index 000000000..e10fe968f --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskBatchTransfer.java @@ -0,0 +1,141 @@ +/** + * 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.task; + +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.callback.AsyncLoopThread; +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.daemon.worker.WorkerData; + +import backtype.storm.messaging.IConnection; +import backtype.storm.messaging.TaskMessage; +import backtype.storm.serialization.KryoTupleSerializer; +import backtype.storm.tuple.BatchTuple; +import backtype.storm.tuple.TupleExt; +import backtype.storm.utils.DisruptorQueue; + +/** + * Batch Tuples, then send out + * + * @author basti.lj + * + */ +public class TaskBatchTransfer extends TaskTransfer { + + private static Logger LOG = LoggerFactory + .getLogger(TaskBatchTransfer.class); + + private Map batchMap; + private int batchSize; + private Object lock = new Object(); + + public TaskBatchTransfer(Task task, String taskName, + KryoTupleSerializer serializer, TaskStatus taskStatus, + WorkerData workerData) { + super(task, taskName, serializer, taskStatus, workerData); + + batchMap = new HashMap(); + batchSize = + ConfigExtension.getTaskMsgBatchSize(workerData.getStormConf()); + } + + @Override + protected AsyncLoopThread setupSerializeThread() { + return new AsyncLoopThread(new TransferBatchRunnable()); + } + + @Override + public void transfer(TupleExt tuple) { + int targetTaskid = tuple.getTargetTaskId(); + synchronized (lock) { + BatchTuple batch = getBatchTuple(targetTaskid); + + batch.addToBatch(tuple); + if (batch.isBatchFull()) { + pushToQueue(targetTaskid, batch); + } + } + } + + public void flush() { + synchronized (lock) { + for (Entry entry : batchMap.entrySet()) { + int taskId = entry.getKey(); + BatchTuple batch = entry.getValue(); + if (batch != null && batch.currBatchSize() > 0) { + pushToQueue(taskId, batch); + } + } + } + } + + private void pushToQueue(int targetTaskid, BatchTuple batch) { + DisruptorQueue exeQueue = innerTaskTransfer.get(targetTaskid); + if (exeQueue != null) { + exeQueue.publish(batch); + } else { + serializeQueue.publish(batch); + } + resetBatchTuple(targetTaskid); + } + + private BatchTuple getBatchTuple(int targetTaskId) { + BatchTuple ret = batchMap.get(targetTaskId); + if (ret == null) { + ret = new BatchTuple(targetTaskId, batchSize); + batchMap.put(targetTaskId, ret); + } + return ret; + } + + private void resetBatchTuple(int targetTaskId) { + batchMap.put(targetTaskId, null); + } + + protected class TransferBatchRunnable extends TransferRunnable { + @Override + public void onEvent(Object event, long sequence, boolean endOfBatch) + throws Exception { + + if (event == null) { + return; + } + + long start = System.currentTimeMillis(); + try { + BatchTuple tuple = (BatchTuple) event; + int taskid = tuple.getTargetTaskId(); + byte[] tupleMessage = serializer.serializeBatch(tuple); + TaskMessage taskMessage = new TaskMessage(taskid, tupleMessage); + IConnection conn = getConnection(taskid); + if (conn != null) + conn.send(taskMessage); + } finally { + long end = System.currentTimeMillis(); + timer.update(end - start); + } + + } + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskInfo.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskInfo.java new file mode 100755 index 000000000..0eb1c4b6f --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskInfo.java @@ -0,0 +1,78 @@ +/** + * 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.task; + +import java.io.Serializable; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; + +/** + * /storm-zk-root/tasks/{topologyid}/{taskid} data + */ +public class TaskInfo implements Serializable { + + private static final long serialVersionUID = 5625165079055837777L; + private String componentId; + private String componentType; + + public TaskInfo(String componentId, String componentType) { + this.componentId = componentId; + this.componentType = componentType; + } + + public String getComponentId() { + return componentId; + } + + public void setComponentId(String componentId) { + this.componentId = componentId; + } + + public String getComponentType() { + return componentType; + } + + public void setComponentType(String componentType) { + this.componentType = componentType; + } + + @Override + public boolean equals(Object assignment) { + if (assignment instanceof TaskInfo + && ((TaskInfo) assignment).getComponentId().equals( + getComponentId()) + && ((TaskInfo) assignment).getComponentType().equals( + componentType)) { + return true; + } + return false; + } + + @Override + public int hashCode() { + return this.getComponentId().hashCode() + + this.getComponentType().hashCode(); + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskReceiver.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskReceiver.java new file mode 100644 index 000000000..ad32ceb22 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskReceiver.java @@ -0,0 +1,228 @@ +/** + * 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.task; + +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.serialization.KryoTupleDeserializer; +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Tuple; +import backtype.storm.utils.DisruptorQueue; +import backtype.storm.utils.Utils; +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.common.metric.Histogram; +import com.alibaba.jstorm.common.metric.QueueGauge; +import com.alibaba.jstorm.metric.JStormHealthCheck; +import com.alibaba.jstorm.metric.JStormMetrics; +import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.utils.JStormUtils; +import com.lmax.disruptor.EventHandler; +import com.lmax.disruptor.WaitStrategy; +import com.lmax.disruptor.dsl.ProducerType; + +public class TaskReceiver { + private static Logger LOG = LoggerFactory.getLogger(TaskReceiver.class); + + protected Task task; + protected final int taskId; + protected final String idStr; + + protected TopologyContext topologyContext; + protected Map innerTaskTransfer; + + protected final boolean isDebugRecv; + + protected DisruptorQueue deserializeQueue; + protected KryoTupleDeserializer deserializer; + protected AsyncLoopThread deserializeThread; + protected Histogram deserializeTimer; + + protected TaskStatus taskStatus; + + public TaskReceiver(Task task, int taskId, Map stormConf, + TopologyContext topologyContext, + Map innerTaskTransfer, + TaskStatus taskStatus, String taskName) { + this.task = task; + this.taskId = taskId; + this.idStr = taskName; + + this.topologyContext = topologyContext; + this.innerTaskTransfer = innerTaskTransfer; + + this.taskStatus = taskStatus; + + this.isDebugRecv = ConfigExtension.isTopologyDebugRecvTuple(stormConf); + + int queueSize = + JStormUtils + .parseInt( + stormConf + .get(Config.TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE), + 256); + + WaitStrategy waitStrategy = + (WaitStrategy) JStormUtils + .createDisruptorWaitStrategy(stormConf); + this.deserializeQueue = + DisruptorQueue.mkInstance("TaskDeserialize", + ProducerType.MULTI, queueSize, waitStrategy); + setDeserializeThread(); + this.deserializer = + new KryoTupleDeserializer(stormConf, topologyContext); + + deserializeTimer = + JStormMetrics.registerTaskHistogram(taskId, + MetricDef.DESERIALIZE_TIME); + + QueueGauge deserializeQueueGauge = + new QueueGauge(idStr + MetricDef.DESERIALIZE_QUEUE, + deserializeQueue); + JStormMetrics.registerTaskGauge(deserializeQueueGauge, taskId, + MetricDef.DESERIALIZE_QUEUE); + JStormHealthCheck.registerTaskHealthCheck(taskId, + MetricDef.DESERIALIZE_QUEUE, deserializeQueueGauge); + } + + public AsyncLoopThread getDeserializeThread() { + return deserializeThread; + } + + protected void setDeserializeThread() { + this.deserializeThread = + new AsyncLoopThread(new DeserializeRunnable(deserializeQueue, + innerTaskTransfer.get(taskId))); + } + + public DisruptorQueue getDeserializeQueue() { + return deserializeQueue; + } + + class DeserializeRunnable extends RunnableCallback implements EventHandler { + + DisruptorQueue deserializeQueue; + DisruptorQueue exeQueue; + + DeserializeRunnable(DisruptorQueue deserializeQueue, + DisruptorQueue exeQueue) { + this.deserializeQueue = deserializeQueue; + this.exeQueue = exeQueue; + } + + @Override + public String getThreadName() { + return idStr + "-deserializer"; + } + + protected Object deserialize(byte[] ser_msg) { + long start = System.nanoTime(); + try { + if (ser_msg == null) { + return null; + } + + if (ser_msg.length == 0) { + return null; + } else if (ser_msg.length == 1) { + byte newStatus = ser_msg[0]; + LOG.info("Change task status as " + newStatus); + taskStatus.setStatus(newStatus); + + return null; + } + + // ser_msg.length > 1 + Tuple tuple = deserializer.deserialize(ser_msg); + + if (isDebugRecv) { + LOG.info(idStr + " receive " + tuple.toString()); + } + + return tuple; + } catch (Throwable e) { + if (taskStatus.isShutdown() == false) { + LOG.error( + idStr + " recv thread error " + + JStormUtils.toPrintableString(ser_msg) + + "\n", e); + } + } finally { + long end = System.nanoTime(); + deserializeTimer.update((end - start)/1000000.0d); + } + + return null; + } + + @Override + public void onEvent(Object event, long sequence, boolean endOfBatch) + throws Exception { + Object tuple = deserialize((byte[]) event); + + if (tuple != null) { + exeQueue.publish(tuple); + } + } + + @Override + public void preRun() { + WorkerClassLoader.switchThreadContext(); + } + + @Override + public void postRun() { + WorkerClassLoader.restoreThreadContext(); + } + + @Override + public void run() { + deserializeQueue.consumerStarted(); + LOG.info("Successfully start recvThread of " + idStr); + + while (taskStatus.isShutdown() == false) { + try { + + deserializeQueue.consumeBatchWhenAvailable(this); + } catch (Throwable e) { + if (taskStatus.isShutdown() == false) { + LOG.error("Unknow exception ", e); + } + } + + } + + task.unregisterDeserializeQueue(); + + LOG.info("Successfully shutdown recvThread of " + idStr); + } + + public Object getResult() { + LOG.info("Begin to shutdown recvThread of " + idStr); + return -1; + } + + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskShutdownDameon.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskShutdownDameon.java new file mode 100644 index 000000000..c49e9fcfa --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskShutdownDameon.java @@ -0,0 +1,188 @@ +/** + * 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.task; + +import java.util.List; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.spout.ISpout; +import backtype.storm.task.IBolt; +import backtype.storm.topology.IConfig; +import backtype.storm.utils.WorkerClassLoader; + +import com.alibaba.jstorm.callback.AsyncLoopThread; +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.daemon.worker.ShutdownableDameon; +import com.alibaba.jstorm.metric.JStormMetrics; +import com.alibaba.jstorm.task.heartbeat.TaskHeartbeatRunable; +import com.alibaba.jstorm.utils.JStormUtils; + +/** + * shutdown one task + * + * @author yannian/Longda + * + */ +public class TaskShutdownDameon implements ShutdownableDameon { + private static Logger LOG = LoggerFactory + .getLogger(TaskShutdownDameon.class); + + public static final byte QUIT_MSG = (byte) 0xff; + + private TaskStatus taskStatus; + private String topology_id; + private Integer task_id; + private List all_threads; + private StormClusterState zkCluster; + private Object task_obj; + private boolean isClosed = false; + + public TaskShutdownDameon(TaskStatus taskStatus, String topology_id, + Integer task_id, List all_threads, + StormClusterState zkCluster, Object task_obj) { + this.taskStatus = taskStatus; + this.topology_id = topology_id; + this.task_id = task_id; + this.all_threads = all_threads; + this.zkCluster = zkCluster; + this.task_obj = task_obj; + + } + + @Override + public void shutdown() { + synchronized (this) { + if (isClosed == true) { + return; + } + isClosed = true; + } + + LOG.info("Begin to shut down task " + topology_id + ":" + task_id); + + // all thread will check the taskStatus + // once it has been set SHUTDOWN, it will quit + taskStatus.setStatus(TaskStatus.SHUTDOWN); + + // waiting 100ms for executor thread shutting it's own + try { + Thread.sleep(100); + } catch (InterruptedException e) { + } + + for (AsyncLoopThread thr : all_threads) { + LOG.info("Begin to shutdown " + thr.getThread().getName()); + thr.cleanup(); + JStormUtils.sleepMs(10); + thr.interrupt(); + // try { + // //thr.join(); + // thr.getThread().stop(new RuntimeException()); + // } catch (Throwable e) { + // } + LOG.info("Successfully shutdown " + thr.getThread().getName()); + } + + closeComponent(task_obj); + + try { + JStormMetrics.unregisterTask(task_id); + TaskHeartbeatRunable.unregisterTaskStats(task_id); + zkCluster.remove_task_heartbeat(topology_id, task_id); + } catch (Exception e) { + // TODO Auto-generated catch block + LOG.info("Failed to cleanup"); + } finally { + try { + zkCluster.disconnect(); + } catch (Exception e) { + LOG.info("Failed to disconnect", e); + } + } + + LOG.info("Successfully shutdown task " + topology_id + ":" + task_id); + + } + + public void join() throws InterruptedException { + for (AsyncLoopThread t : all_threads) { + t.join(); + } + } + + private void closeComponent(Object _task_obj) { + if (_task_obj instanceof IBolt) { + ((IBolt) _task_obj).cleanup(); + } + + if (_task_obj instanceof ISpout) { + ((ISpout) _task_obj).close(); + } + } + + @Override + public boolean waiting() { + return taskStatus.isRun(); + } + + public void deactive() { + + if (task_obj instanceof ISpout) { + taskStatus.setStatus(TaskStatus.PAUSE); + WorkerClassLoader.switchThreadContext(); + + try { + ((ISpout) task_obj).deactivate(); + } finally { + WorkerClassLoader.restoreThreadContext(); + } + } + + } + + public void active() { + if (task_obj instanceof ISpout) { + taskStatus.setStatus(TaskStatus.RUN); + WorkerClassLoader.switchThreadContext(); + try { + ((ISpout) task_obj).activate(); + } finally { + WorkerClassLoader.restoreThreadContext(); + } + } + } + + public void updateConf(Map conf) { + if (task_obj instanceof IConfig) { + ((IConfig) task_obj).updateConf(conf); + } + } + + @Override + public void run() { + // TODO Auto-generated method stub + shutdown(); + } + + public int getTaskId() { + return this.task_id; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskStatus.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskStatus.java new file mode 100755 index 000000000..10a7e7c23 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskStatus.java @@ -0,0 +1,58 @@ +/** + * 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.task; + +public class TaskStatus { + // task is alive, and it will run BaseExecutor's run + public static final byte RUN = 0; + // task is alive, but it won't run BaseExecutor's run + public static final byte PAUSE = 1; + // task is shutdown + public static final byte SHUTDOWN = 2; + + private volatile byte status = TaskStatus.RUN; + + public byte getStatus() { + return status; + } + + public void setStatus(byte status) { + this.status = status; + } + + public boolean isRun() { + return status == TaskStatus.RUN; + } + + public boolean isShutdown() { + return status == TaskStatus.SHUTDOWN; + } + + public boolean isPause() { + return status == TaskStatus.PAUSE; + } + + /** + * @param args + */ + public static void main(String[] args) { + // TODO Auto-generated method stub + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskTransfer.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskTransfer.java new file mode 100644 index 000000000..efe6deea8 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/TaskTransfer.java @@ -0,0 +1,216 @@ +/** + * 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.task; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.messaging.IConnection; +import backtype.storm.messaging.TaskMessage; +import backtype.storm.scheduler.WorkerSlot; +import backtype.storm.serialization.KryoTupleSerializer; +import backtype.storm.tuple.TupleExt; +import backtype.storm.utils.DisruptorQueue; +import backtype.storm.utils.Utils; +import backtype.storm.utils.WorkerClassLoader; + +import com.alibaba.jstorm.callback.AsyncLoopRunnable; +import com.alibaba.jstorm.callback.AsyncLoopThread; +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.common.metric.MetricRegistry; +import com.alibaba.jstorm.common.metric.QueueGauge; +import com.alibaba.jstorm.common.metric.Timer; +import com.alibaba.jstorm.daemon.worker.WorkerData; +import com.alibaba.jstorm.metric.JStormHealthCheck; +import com.alibaba.jstorm.metric.JStormMetrics; +import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.utils.JStormUtils; +import com.lmax.disruptor.EventHandler; +import com.lmax.disruptor.WaitStrategy; +import com.lmax.disruptor.dsl.ProducerType; + +/** + * Sending entrance + * + * Task sending all tuples through this Object + * + * Serialize the Tuple and put the serialized data to the sending queue + * + * @author yannian + * + */ +public class TaskTransfer { + + private static Logger LOG = LoggerFactory.getLogger(TaskTransfer.class); + + protected Map storm_conf; + protected DisruptorQueue transferQueue; + protected KryoTupleSerializer serializer; + protected Map innerTaskTransfer; + protected DisruptorQueue serializeQueue; + protected final AsyncLoopThread serializeThread; + protected volatile TaskStatus taskStatus; + protected String taskName; + protected Timer timer; + protected Task task; + + protected ConcurrentHashMap nodeportSocket; + protected ConcurrentHashMap taskNodeport; + + public TaskTransfer(Task task, String taskName, + KryoTupleSerializer serializer, TaskStatus taskStatus, + WorkerData workerData) { + this.task = task; + this.taskName = taskName; + this.serializer = serializer; + this.taskStatus = taskStatus; + this.storm_conf = workerData.getStormConf(); + this.transferQueue = workerData.getTransferQueue(); + this.innerTaskTransfer = workerData.getInnerTaskTransfer(); + + this.nodeportSocket = workerData.getNodeportSocket(); + this.taskNodeport = workerData.getTaskNodeport(); + + int queue_size = + Utils.getInt(storm_conf + .get(Config.TOPOLOGY_EXECUTOR_SEND_BUFFER_SIZE)); + WaitStrategy waitStrategy = + (WaitStrategy) JStormUtils.createDisruptorWaitStrategy(storm_conf); + this.serializeQueue = + DisruptorQueue.mkInstance(taskName, ProducerType.MULTI, + queue_size, waitStrategy); + this.serializeQueue.consumerStarted(); + + String taskId = taskName.substring(taskName.indexOf(":") + 1); + String metricName = + MetricRegistry.name(MetricDef.SERIALIZE_QUEUE, taskName); + QueueGauge serializeQueueGauge = + new QueueGauge(metricName, serializeQueue); + JStormMetrics.registerTaskGauge(serializeQueueGauge, + Integer.valueOf(taskId), MetricDef.SERIALIZE_QUEUE); + JStormHealthCheck.registerTaskHealthCheck(Integer.valueOf(taskId), + MetricDef.SERIALIZE_QUEUE, serializeQueueGauge); + timer = + JStormMetrics.registerTaskTimer(Integer.valueOf(taskId), + MetricDef.SERIALIZE_TIME); + + serializeThread = setupSerializeThread(); + LOG.info("Successfully start TaskTransfer thread"); + + } + + public void transfer(TupleExt tuple) { + + int taskid = tuple.getTargetTaskId(); + + DisruptorQueue exeQueue = innerTaskTransfer.get(taskid); + if (exeQueue != null) { + exeQueue.publish(tuple); + } else { + serializeQueue.publish(tuple); + } + + } + + protected AsyncLoopThread setupSerializeThread() { + return new AsyncLoopThread(new TransferRunnable()); + } + + public AsyncLoopThread getSerializeThread() { + return serializeThread; + } + + protected class TransferRunnable extends RunnableCallback implements EventHandler { + + private AtomicBoolean shutdown = AsyncLoopRunnable.getShutdown(); + + @Override + public String getThreadName() { + return taskName + "-" + TransferRunnable.class.getSimpleName(); + } + + @Override + public void preRun() { + WorkerClassLoader.switchThreadContext(); + } + + @Override + public void run() { + + while (shutdown.get() == false) { + serializeQueue.consumeBatchWhenAvailable(this); + + } + + } + + @Override + public void postRun() { + WorkerClassLoader.restoreThreadContext(); + } + + @Override + public void onEvent(Object event, long sequence, boolean endOfBatch) + throws Exception { + + if (event == null) { + return; + } + + long start = System.currentTimeMillis(); + + try { + TupleExt tuple = (TupleExt) event; + int taskid = tuple.getTargetTaskId(); + byte[] tupleMessage = serializer.serialize(tuple); + TaskMessage taskMessage = new TaskMessage(taskid, tupleMessage); + IConnection conn = getConnection(taskid); + if (conn != null) { + conn.send(taskMessage); + } + } finally { + long end = System.currentTimeMillis(); + timer.update(end - start); + } + + } + + protected IConnection getConnection(int taskId) { + IConnection conn = null; + WorkerSlot nodePort = taskNodeport.get(taskId); + if (nodePort == null) { + String errormsg = "can`t not found IConnection to " + taskId; + LOG.warn("Intra transfer warn", new Exception(errormsg)); + } else { + conn = nodeportSocket.get(nodePort); + if (conn == null) { + String errormsg = "can`t not found nodePort " + nodePort; + LOG.warn("Intra transfer warn", new Exception(errormsg)); + } + } + return conn; + } + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/TkHbCacheTime.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/TkHbCacheTime.java new file mode 100755 index 000000000..596fa35f2 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/TkHbCacheTime.java @@ -0,0 +1,65 @@ +/** + * 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.task; + +import com.alibaba.jstorm.task.heartbeat.TaskHeartbeat; +import com.alibaba.jstorm.utils.TimeUtils; + +/** + * TkHbCacheTime is describle taskheartcache (Map>>) + */ + +public class TkHbCacheTime { + private int nimbusTime; + private int taskReportedTime; + private int taskAssignedTime; + + public int getNimbusTime() { + return nimbusTime; + } + + public void setNimbusTime(int nimbusTime) { + this.nimbusTime = nimbusTime; + } + + public int getTaskReportedTime() { + return taskReportedTime; + } + + public void setTaskReportedTime(int taskReportedTime) { + this.taskReportedTime = taskReportedTime; + } + + public int getTaskAssignedTime() { + return taskAssignedTime; + } + + public void setTaskAssignedTime(int taskAssignedTime) { + this.taskAssignedTime = taskAssignedTime; + } + + public void update(TaskHeartbeat zkTaskHeartbeat) { + int nowSecs = TimeUtils.current_time_secs(); + this.nimbusTime = nowSecs; + this.taskReportedTime = zkTaskHeartbeat.getTimeSecs(); + this.taskAssignedTime = + zkTaskHeartbeat.getTimeSecs() - zkTaskHeartbeat.getUptimeSecs(); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/UptimeComputer.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/UptimeComputer.java new file mode 100755 index 000000000..4c9f1504b --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/UptimeComputer.java @@ -0,0 +1,38 @@ +/** + * 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.task; + +import com.alibaba.jstorm.utils.TimeUtils; + +/** + * Get how long task runs + * + * @author yannian + * + */ +public class UptimeComputer { + int start_time = 0; + + public UptimeComputer() { + start_time = TimeUtils.current_time_secs(); + } + + public synchronized int uptime() { + return TimeUtils.time_delta(start_time); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/acker/AckObject.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/acker/AckObject.java new file mode 100755 index 000000000..f44aad719 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/acker/AckObject.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.task.acker; + +import com.alibaba.jstorm.utils.JStormUtils; + +public class AckObject { + public Long val = null; + public Integer spout_task = null; + public boolean failed = false; + + // val xor value + public void update_ack(Object value) { + synchronized (this) { + if (val == null) { + val = Long.valueOf(0); + } + val = JStormUtils.bit_xor(val, value); + } + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/acker/Acker.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/acker/Acker.java new file mode 100755 index 000000000..2be15922f --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/acker/Acker.java @@ -0,0 +1,164 @@ +/** + * 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.task.acker; + +import java.util.List; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.task.IBolt; +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Tuple; + +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.RotatingMap; + +/** + * + * @author yannian/Longda + * + */ +public class Acker implements IBolt { + + private static final Logger LOG = LoggerFactory.getLogger(Acker.class); + + private static final long serialVersionUID = 4430906880683183091L; + + public static final String ACKER_COMPONENT_ID = "__acker"; + public static final String ACKER_INIT_STREAM_ID = "__ack_init"; + public static final String ACKER_ACK_STREAM_ID = "__ack_ack"; + public static final String ACKER_FAIL_STREAM_ID = "__ack_fail"; + + public static final int TIMEOUT_BUCKET_NUM = 3; + + private OutputCollector collector = null; + // private TimeCacheMap pending = null; + private RotatingMap pending = null; + private long lastRotate = System.currentTimeMillis(); + private long rotateTime; + + @Override + public void prepare(Map stormConf, TopologyContext context, + OutputCollector collector) { + this.collector = collector; + // pending = new TimeCacheMap(timeoutSec, + // TIMEOUT_BUCKET_NUM); + this.pending = new RotatingMap(TIMEOUT_BUCKET_NUM); + this.rotateTime = + 1000L + * JStormUtils.parseInt(stormConf + .get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS), 30) + / (TIMEOUT_BUCKET_NUM - 1); + } + + @Override + public void execute(Tuple input) { + Object id = input.getValue(0); + + AckObject curr = pending.get(id); + + String stream_id = input.getSourceStreamId(); + + if (Acker.ACKER_INIT_STREAM_ID.equals(stream_id)) { + if (curr == null) { + curr = new AckObject(); + + curr.val = input.getLong(1); + curr.spout_task = input.getInteger(2); + + pending.put(id, curr); + } else { + // bolt's ack first come + curr.update_ack(input.getValue(1)); + curr.spout_task = input.getInteger(2); + } + + } else if (Acker.ACKER_ACK_STREAM_ID.equals(stream_id)) { + if (curr != null) { + curr.update_ack(input.getValue(1)); + + } else { + // two case + // one is timeout + // the other is bolt's ack first come + curr = new AckObject(); + + curr.val = Long.valueOf(input.getLong(1)); + + pending.put(id, curr); + + } + } else if (Acker.ACKER_FAIL_STREAM_ID.equals(stream_id)) { + if (curr == null) { + // do nothing + // already timeout, should go fail + return; + } + + curr.failed = true; + + } else { + LOG.info("Unknow source stream"); + return; + } + + Integer task = curr.spout_task; + + if (task != null) { + + if (curr.val == 0) { + pending.remove(id); + List values = JStormUtils.mk_list(id); + + collector.emitDirect(task, Acker.ACKER_ACK_STREAM_ID, values); + + } else { + + if (curr.failed) { + pending.remove(id); + List values = JStormUtils.mk_list(id); + collector.emitDirect(task, Acker.ACKER_FAIL_STREAM_ID, + values); + } + } + } else { + + } + + // add this operation to update acker's ACK statics + collector.ack(input); + + long now = System.currentTimeMillis(); + if (now - lastRotate > rotateTime) { + lastRotate = now; + Map tmp = pending.rotate(); + LOG.info("Acker's timeout item size:{}", tmp.size()); + } + + } + + @Override + public void cleanup() { + LOG.info("Successfully cleanup"); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/comm/TaskSendTargets.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/comm/TaskSendTargets.java new file mode 100755 index 000000000..8a55ec5c1 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/comm/TaskSendTargets.java @@ -0,0 +1,151 @@ +/** + * 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.task.comm; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.task.TopologyContext; + +import com.alibaba.jstorm.task.TaskBaseMetric; +import com.alibaba.jstorm.task.group.GrouperType; +import com.alibaba.jstorm.task.group.MkGrouper; +import com.alibaba.jstorm.utils.JStormUtils; + +/** + * + * tuple sending object, which get which task should tuple be send to, and + * update statics + * + * @author yannian/Longda + * + */ +public class TaskSendTargets { + private static Logger LOG = LoggerFactory.getLogger(TaskSendTargets.class); + + private Map stormConf; + // it is system TopologyContext + private TopologyContext topologyContext; + + // > + private volatile Map> streamComponentgrouper; + // SpoutTaskStatsRolling or BoltTaskStatsRolling + private TaskBaseMetric taskStats; + + private String componentId; + private int taskId; + private boolean isDebuging = false; + private String debugIdStr; + + public TaskSendTargets(Map _storm_conf, String _component, + Map> _stream_component_grouper, + TopologyContext _topology_context, TaskBaseMetric _task_stats) { + this.stormConf = _storm_conf; + this.componentId = _component; + this.streamComponentgrouper = _stream_component_grouper; + this.topologyContext = _topology_context; + this.taskStats = _task_stats; + + isDebuging = + JStormUtils.parseBoolean(stormConf.get(Config.TOPOLOGY_DEBUG), + false); + + taskId = topologyContext.getThisTaskId(); + debugIdStr = " Emit from " + componentId + ":" + taskId + " "; + } + + // direct send tuple to special task + public java.util.List get(Integer out_task_id, String stream, + List tuple) { + + // in order to improve acker's speed, skip checking + // String target_component = + // topologyContext.getComponentId(out_task_id); + // Map component_prouping = streamComponentgrouper + // .get(stream); + // MkGrouper grouping = component_prouping.get(target_component); + // if (grouping != null && + // !GrouperType.direct.equals(grouping.gettype())) { + // throw new IllegalArgumentException( + // "Cannot emitDirect to a task expecting a regular grouping"); + // } + + if (isDebuging) { + LOG.info(debugIdStr + stream + " to " + out_task_id + ":" + + tuple.toString()); + } + + taskStats.send_tuple(stream, 1); + + java.util.List out_tasks = new ArrayList(); + out_tasks.add(out_task_id); + return out_tasks; + } + + // send tuple according to grouping + public java.util.List get(String stream, List tuple) { + java.util.List out_tasks = new ArrayList(); + + // get grouper, then get which task should tuple be sent to. + Map componentCrouping = + streamComponentgrouper.get(stream); + if (componentCrouping == null) { + // if the target component's parallelism is 0, don't need send to + // them + LOG.debug("Failed to get Grouper of " + stream + " in " + + debugIdStr); + return out_tasks; + } + + for (Entry ee : componentCrouping.entrySet()) { + String targetComponent = ee.getKey(); + MkGrouper g = ee.getValue(); + + if (GrouperType.direct.equals(g.gettype())) { + throw new IllegalArgumentException( + "Cannot do regular emit to direct stream"); + } + + out_tasks.addAll(g.grouper(tuple)); + + } + + if (isDebuging) { + + LOG.info(debugIdStr + stream + " to " + out_tasks + ":" + + tuple.toString()); + } + + int num_out_tasks = out_tasks.size(); + + taskStats.send_tuple(stream, num_out_tasks); + + return out_tasks; + } + + public void updateStreamCompGrouper( + Map> streamComponentgrouper) { + this.streamComponentgrouper = streamComponentgrouper; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/comm/TupleInfo.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/comm/TupleInfo.java new file mode 100755 index 000000000..a6a340666 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/comm/TupleInfo.java @@ -0,0 +1,73 @@ +/** + * 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.task.comm; + +import java.io.Serializable; +import java.util.List; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; + +public class TupleInfo implements Serializable { + + private static final long serialVersionUID = -3348670497595864118L; + + private Object messageId; + private String stream; + private List values; + private long timestamp; + + public Object getMessageId() { + return messageId; + } + + public void setMessageId(Object messageId) { + this.messageId = messageId; + } + + public long getTimestamp() { + return timestamp; + } + + public void setTimestamp(long timestamp) { + this.timestamp = timestamp; + } + + public String getStream() { + return stream; + } + + public void setStream(String stream) { + this.stream = stream; + } + + public List getValues() { + return values; + } + + public void setValues(List values) { + this.values = values; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/comm/UnanchoredSend.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/comm/UnanchoredSend.java new file mode 100755 index 000000000..72c40611e --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/comm/UnanchoredSend.java @@ -0,0 +1,54 @@ +/** + * 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.task.comm; + +import java.util.List; + +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.TupleImplExt; + +import com.alibaba.jstorm.task.TaskTransfer; + +/** + * Send init/ack/fail tuple to acker + * + * @author yannian + * + */ + +public class UnanchoredSend { + public static void send(TopologyContext topologyContext, + TaskSendTargets taskTargets, TaskTransfer transfer_fn, + String stream, List values) { + + java.util.List tasks = taskTargets.get(stream, values); + if (tasks.size() == 0) { + return; + } + + Integer taskId = topologyContext.getThisTaskId(); + + for (Integer task : tasks) { + TupleImplExt tup = + new TupleImplExt(topologyContext, values, taskId, stream); + tup.setTargetTaskId(task); + + transfer_fn.transfer(tup); + } + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/error/ITaskReportErr.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/error/ITaskReportErr.java new file mode 100755 index 000000000..d0d70bed3 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/error/ITaskReportErr.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.task.error; + +/** + * task report error interface + * + * @author yannian + * + */ +public interface ITaskReportErr { + public void report(Throwable error); +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/error/TaskError.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/error/TaskError.java new file mode 100755 index 000000000..96f3aadcd --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/error/TaskError.java @@ -0,0 +1,55 @@ +/** + * 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.task.error; + +import java.io.Serializable; + +/** + * Task error stored in Zk(/storm-zk-root/taskerrors/{topologyid}/{taskid}) + * + * @author yannian + * + */ +public class TaskError implements Serializable { + + private static final long serialVersionUID = 5028789764629555542L; + private String error; + private int timSecs; + + public TaskError(String error, int timSecs) { + this.error = error; + this.timSecs = timSecs; + } + + public String getError() { + return error; + } + + public void setError(String error) { + this.error = error; + } + + public int getTimSecs() { + return timSecs; + } + + public void setTimSecs(int timSecs) { + this.timSecs = timSecs; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/error/TaskErrorRunable.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/error/TaskErrorRunable.java new file mode 100755 index 000000000..d460e5a33 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/error/TaskErrorRunable.java @@ -0,0 +1,48 @@ +/** + * 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.task.error; + +import com.alibaba.jstorm.callback.RunnableCallback; + +/** + * The callback will be called, when task occur error It just call + * TaskReportErrorAndDie + * + * @author yannian + * + */ +public class TaskErrorRunable extends RunnableCallback { + private TaskReportErrorAndDie report_error_and_die; + + public TaskErrorRunable(TaskReportErrorAndDie _report_error_and_die) { + this.report_error_and_die = _report_error_and_die; + } + + @Override + public Object execute(T... args) { + Exception e = null; + if (args != null && args.length > 0) { + e = (Exception) args[0]; + } + if (e != null) { + report_error_and_die.report(e); + } + return null; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/error/TaskReportError.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/error/TaskReportError.java new file mode 100755 index 000000000..e7506b2bc --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/error/TaskReportError.java @@ -0,0 +1,59 @@ +/** + * 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.task.error; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.cluster.StormClusterState; + +/** + * Task error report callback + * + * @author yannian + * + */ +public class TaskReportError implements ITaskReportErr { + private static Logger LOG = LoggerFactory.getLogger(TaskReportError.class); + private StormClusterState zkCluster; + private String topology_id; + private int task_id; + + public TaskReportError(StormClusterState _storm_cluster_state, + String _topology_id, int _task_id) { + this.zkCluster = _storm_cluster_state; + this.topology_id = _topology_id; + this.task_id = _task_id; + } + + @Override + public void report(Throwable error) { + + LOG.error("Report error to /ZK/taskerrors/" + topology_id + "/" + + task_id + "\n", error); + try { + zkCluster.report_task_error(topology_id, task_id, error); + } catch (Exception e) { + // TODO Auto-generated catch block + LOG.error("Failed update error to /ZK/taskerrors/" + topology_id + + "/" + task_id + "\n", e); + } + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/error/TaskReportErrorAndDie.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/error/TaskReportErrorAndDie.java new file mode 100755 index 000000000..4f4eab36a --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/error/TaskReportErrorAndDie.java @@ -0,0 +1,43 @@ +/** + * 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.task.error; + +import com.alibaba.jstorm.callback.RunnableCallback; + +/** + * Task report error to ZK and halt the process + * + * @author yannian + * + */ +public class TaskReportErrorAndDie implements ITaskReportErr { + private ITaskReportErr reporterror; + private RunnableCallback haltfn; + + public TaskReportErrorAndDie(ITaskReportErr _reporterror, + RunnableCallback _haltfn) { + this.reporterror = _reporterror; + this.haltfn = _haltfn; + } + + @Override + public void report(Throwable error) { + this.reporterror.report(error); + this.haltfn.run(); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/BaseExecutors.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/BaseExecutors.java new file mode 100644 index 000000000..3f4c18fa9 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/BaseExecutors.java @@ -0,0 +1,232 @@ +/** + * 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.task.execute; + +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingDeque; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.serialization.KryoTupleDeserializer; +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Tuple; +import backtype.storm.utils.DisruptorQueue; +import backtype.storm.utils.Utils; +import backtype.storm.utils.WorkerClassLoader; + +import com.alibaba.jstorm.callback.AsyncLoopRunnable; +import com.alibaba.jstorm.callback.AsyncLoopThread; +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.common.metric.Histogram; +import com.alibaba.jstorm.common.metric.QueueGauge; +import com.alibaba.jstorm.daemon.worker.timer.RotatingMapTrigger; +import com.alibaba.jstorm.daemon.worker.timer.TaskHeartbeatTrigger; +import com.alibaba.jstorm.metric.JStormHealthCheck; +import com.alibaba.jstorm.metric.JStormMetrics; +import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.task.Task; +import com.alibaba.jstorm.task.TaskBaseMetric; +import com.alibaba.jstorm.task.TaskStatus; +import com.alibaba.jstorm.task.TaskTransfer; +import com.alibaba.jstorm.task.error.ITaskReportErr; +import com.alibaba.jstorm.utils.JStormServerUtils; +import com.alibaba.jstorm.utils.JStormUtils; +import com.lmax.disruptor.EventHandler; +import com.lmax.disruptor.WaitStrategy; +import com.lmax.disruptor.dsl.ProducerType; + +//import com.alibaba.jstorm.message.zeroMq.IRecvConnection; + +/** + * Base executor share between spout and bolt + * + * + * @author Longda + * + */ +public class BaseExecutors extends RunnableCallback { + private static Logger LOG = LoggerFactory.getLogger(BaseExecutors.class); + + protected final String component_id; + protected final int taskId; + protected final String idStr; + + protected Map storm_conf; + + protected final boolean isDebug; + + protected TopologyContext userTopologyCtx; + protected TaskBaseMetric task_stats; + + protected volatile TaskStatus taskStatus; + + protected int message_timeout_secs = 30; + + protected Throwable error = null; + + protected ITaskReportErr report_error; + + protected DisruptorQueue exeQueue; + protected BlockingQueue controlQueue; + protected Map innerTaskTransfer; + + protected Task task; + protected long assignmentTs; + protected TaskTransfer taskTransfer; + + // protected IntervalCheck intervalCheck = new IntervalCheck(); + + public BaseExecutors(Task task, TaskTransfer _transfer_fn, Map _storm_conf, + Map innerTaskTransfer, + TopologyContext topology_context, TopologyContext _user_context, + TaskBaseMetric _task_stats, TaskStatus taskStatus, + ITaskReportErr _report_error) { + + this.task = task; + this.storm_conf = _storm_conf; + + this.userTopologyCtx = _user_context; + this.task_stats = _task_stats; + this.taskId = topology_context.getThisTaskId(); + this.innerTaskTransfer = innerTaskTransfer; + this.component_id = topology_context.getThisComponentId(); + this.idStr = JStormServerUtils.getName(component_id, taskId); + + this.taskStatus = taskStatus; + this.report_error = _report_error; + + this.isDebug = + JStormUtils.parseBoolean(storm_conf.get(Config.TOPOLOGY_DEBUG), + false); + + message_timeout_secs = + JStormUtils.parseInt( + storm_conf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS), + 30); + + int queue_size = + Utils.getInt(storm_conf + .get(Config.TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE), 256); + WaitStrategy waitStrategy = + (WaitStrategy) JStormUtils.createDisruptorWaitStrategy(storm_conf); + this.exeQueue = + DisruptorQueue.mkInstance(idStr, ProducerType.MULTI, + queue_size, waitStrategy); + this.exeQueue.consumerStarted(); + this.controlQueue = new LinkedBlockingDeque(8); + + this.registerInnerTransfer(exeQueue); + + QueueGauge exeQueueGauge = + new QueueGauge(idStr + MetricDef.EXECUTE_QUEUE, exeQueue); + JStormMetrics.registerTaskGauge(exeQueueGauge, taskId, + MetricDef.EXECUTE_QUEUE); + JStormHealthCheck.registerTaskHealthCheck(taskId, + MetricDef.EXECUTE_QUEUE, exeQueueGauge); + + RotatingMapTrigger rotatingMapTrigger = + new RotatingMapTrigger(storm_conf, idStr + "_rotating", + exeQueue); + rotatingMapTrigger.register(); + TaskHeartbeatTrigger taskHbTrigger = + new TaskHeartbeatTrigger(storm_conf, idStr + "_taskHeartbeat", + exeQueue, controlQueue, taskId); + taskHbTrigger.register(); + + assignmentTs = System.currentTimeMillis(); + + this.taskTransfer = _transfer_fn; + } + + @Override + public void preRun() { + WorkerClassLoader.switchThreadContext(); + } + + @Override + public void postRun() { + WorkerClassLoader.restoreThreadContext(); + } + + @Override + public void run() { + // this function will be override by SpoutExecutor or BoltExecutor + throw new RuntimeException("Should implement this function"); + } + + // @Override + // public Object getResult() { + // if (taskStatus.isRun()) { + // return 0; + // } else if (taskStatus.isPause()) { + // return 0; + // } else if (taskStatus.isShutdown()) { + // this.shutdown(); + // return -1; + // } else { + // LOG.info("Unknow TaskStatus, shutdown executing thread of " + idStr); + // this.shutdown(); + // return -1; + // } + // } + + @Override + public Exception error() { + if (error == null) { + return null; + } + + return new Exception(error); + } + + @Override + public void shutdown() { + LOG.info("Shutdown executing thread of " + idStr); + if (taskStatus.isShutdown() == false) { + LOG.info("Taskstatus isn't shutdown, but enter shutdown method, Occur exception"); + } + this.unregistorInnerTransfer(); + + } + + protected void registerInnerTransfer(DisruptorQueue disruptorQueue) { + LOG.info("Registor inner transfer for executor thread of " + idStr); + DisruptorQueue existInnerTransfer = innerTaskTransfer.get(taskId); + if (existInnerTransfer != null) { + LOG.info("Exist inner task transfer for executing thread of " + + idStr); + if (existInnerTransfer != disruptorQueue) { + throw new RuntimeException( + "Inner task transfer must be only one in executing thread of " + + idStr); + } + } + innerTaskTransfer.put(taskId, disruptorQueue); + } + + protected void unregistorInnerTransfer() { + LOG.info("Unregistor inner transfer for executor thread of " + idStr); + innerTaskTransfer.remove(taskId); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/BoltCollector.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/BoltCollector.java new file mode 100755 index 000000000..a51d09a43 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/BoltCollector.java @@ -0,0 +1,259 @@ +/** + * 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.task.execute; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Random; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.task.IOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.MessageId; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.TupleImplExt; + +import com.alibaba.jstorm.common.metric.Histogram; +import com.alibaba.jstorm.metric.JStormMetrics; +import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.task.TaskBaseMetric; +import com.alibaba.jstorm.task.TaskTransfer; +import com.alibaba.jstorm.task.acker.Acker; +import com.alibaba.jstorm.task.comm.TaskSendTargets; +import com.alibaba.jstorm.task.comm.UnanchoredSend; +import com.alibaba.jstorm.task.error.ITaskReportErr; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.RotatingMap; +import com.alibaba.jstorm.utils.TimeUtils; + +/** + * bolt output interface, do emit/ack/fail + * + * @author yannian/Longda + * + */ +public class BoltCollector implements IOutputCollector { + private static Logger LOG = LoggerFactory.getLogger(BoltCollector.class); + + private ITaskReportErr reportError; + private TaskSendTargets sendTargets; + private TaskTransfer taskTransfer; + private TopologyContext topologyContext; + private Integer task_id; + // private TimeCacheMap tuple_start_times; + private RotatingMap tuple_start_times; + private TaskBaseMetric task_stats; + // private TimeCacheMap pending_acks; + private RotatingMap pending_acks; + private long lastRotate = System.currentTimeMillis(); + private long rotateTime; + + private Map storm_conf; + private Integer ackerNum; + private Histogram timer; + private Random random; + + public BoltCollector(int message_timeout_secs, ITaskReportErr report_error, + TaskSendTargets _send_fn, Map _storm_conf, + TaskTransfer _transfer_fn, TopologyContext _topology_context, + Integer task_id, RotatingMap tuple_start_times, + TaskBaseMetric _task_stats) { + + this.rotateTime = + 1000L * message_timeout_secs / (Acker.TIMEOUT_BUCKET_NUM - 1); + this.reportError = report_error; + this.sendTargets = _send_fn; + this.storm_conf = _storm_conf; + this.taskTransfer = _transfer_fn; + this.topologyContext = _topology_context; + this.task_id = task_id; + this.task_stats = _task_stats; + + this.pending_acks = + new RotatingMap(Acker.TIMEOUT_BUCKET_NUM); + // this.pending_acks = new TimeCacheMap(message_timeout_secs, + // Acker.TIMEOUT_BUCKET_NUM); + this.tuple_start_times = tuple_start_times; + + this.ackerNum = + JStormUtils.parseInt(storm_conf + .get(Config.TOPOLOGY_ACKER_EXECUTORS)); + + String componentId = topologyContext.getThisComponentId(); + timer = + JStormMetrics.registerTaskHistogram(task_id, + MetricDef.COLLECTOR_EMIT_TIME); + + random = new Random(); + random.setSeed(System.currentTimeMillis()); + } + + @Override + public List emit(String streamId, Collection anchors, + List tuple) { + return boltEmit(streamId, anchors, tuple, null); + } + + @Override + public void emitDirect(int taskId, String streamId, + Collection anchors, List tuple) { + boltEmit(streamId, anchors, tuple, taskId); + } + + private List boltEmit(String out_stream_id, + Collection anchors, List values, Integer out_task_id) { + long start = System.nanoTime(); + try { + java.util.List out_tasks = null; + if (out_task_id != null) { + out_tasks = sendTargets.get(out_task_id, out_stream_id, values); + } else { + out_tasks = sendTargets.get(out_stream_id, values); + } + + for (Integer t : out_tasks) { + Map anchors_to_ids = new HashMap(); + if (anchors != null) { + for (Tuple a : anchors) { + // Long edge_id = MessageId.generateId(); + Long edge_id = MessageId.generateId(random); + long now = System.currentTimeMillis(); + if (now - lastRotate > rotateTime) { + pending_acks.rotate(); + lastRotate = now; + } + put_xor(pending_acks, a, edge_id); + for (Long root_id : a.getMessageId().getAnchorsToIds() + .keySet()) { + put_xor(anchors_to_ids, root_id, edge_id); + } + } + } + MessageId msgid = MessageId.makeId(anchors_to_ids); + TupleImplExt tupleExt = + new TupleImplExt(topologyContext, values, task_id, + out_stream_id, msgid); + tupleExt.setTargetTaskId(t); + + taskTransfer.transfer(tupleExt); + + } + return out_tasks; + } catch (Exception e) { + LOG.error("bolt emit", e); + } finally { + long end = System.nanoTime(); + timer.update((end - start)/1000000.0d); + } + return new ArrayList(); + } + + @Override + public void ack(Tuple input) { + + if (ackerNum > 0) { + + Long ack_val = Long.valueOf(0); + Object pend_val = pending_acks.remove(input); + if (pend_val != null) { + ack_val = (Long) (pend_val); + } + + for (Entry e : input.getMessageId().getAnchorsToIds() + .entrySet()) { + + UnanchoredSend.send( + topologyContext, + sendTargets, + taskTransfer, + Acker.ACKER_ACK_STREAM_ID, + JStormUtils.mk_list((Object) e.getKey(), + JStormUtils.bit_xor(e.getValue(), ack_val))); + } + } + + Long delta = tuple_time_delta(tuple_start_times, input); + if (delta != null) { + task_stats.bolt_acked_tuple(input.getSourceComponent(), + input.getSourceStreamId(), Double.valueOf(delta)); + } + } + + @Override + public void fail(Tuple input) { + // if ackerNum == 0, we can just return + if (ackerNum > 0) { + pending_acks.remove(input); + for (Entry e : input.getMessageId().getAnchorsToIds() + .entrySet()) { + UnanchoredSend.send(topologyContext, sendTargets, taskTransfer, + Acker.ACKER_FAIL_STREAM_ID, + JStormUtils.mk_list((Object) e.getKey())); + } + } + + task_stats.bolt_failed_tuple(input.getSourceComponent(), + input.getSourceStreamId()); + + } + + @Override + public void reportError(Throwable error) { + reportError.report(error); + } + + // Utility functions, just used here + public static Long tuple_time_delta(RotatingMap start_times, + Tuple tuple) { + Long start_time = (Long) start_times.remove(tuple); + if (start_time != null) { + return TimeUtils.time_delta_ms(start_time); + } + return null; + } + + public static void put_xor(RotatingMap pending, Tuple key, + Long id) { + // synchronized (pending) { + Long curr = pending.get(key); + if (curr == null) { + curr = Long.valueOf(0); + } + pending.put(key, JStormUtils.bit_xor(curr, id)); + // } + } + + public static void put_xor(Map pending, Long key, Long id) { + // synchronized (pending) { + Long curr = pending.get(key); + if (curr == null) { + curr = Long.valueOf(0); + } + pending.put(key, JStormUtils.bit_xor(curr, id)); + // } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/BoltExecutors.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/BoltExecutors.java new file mode 100644 index 000000000..15adbf291 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/BoltExecutors.java @@ -0,0 +1,290 @@ +/** + * 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.task.execute; + +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.Constants; +import backtype.storm.task.IBolt; +import backtype.storm.task.IOutputCollector; +import backtype.storm.task.OutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.BatchTuple; +import backtype.storm.tuple.Tuple; +import backtype.storm.utils.DisruptorQueue; +import backtype.storm.utils.WorkerClassLoader; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.common.metric.Histogram; +import com.alibaba.jstorm.daemon.worker.timer.TaskBatchFlushTrigger; +import com.alibaba.jstorm.daemon.worker.timer.TickTupleTrigger; +import com.alibaba.jstorm.daemon.worker.timer.TimerConstants; +import com.alibaba.jstorm.daemon.worker.timer.TimerTrigger; +import com.alibaba.jstorm.metric.JStormMetrics; +import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.task.Task; +import com.alibaba.jstorm.task.TaskBaseMetric; +import com.alibaba.jstorm.task.TaskStatus; +import com.alibaba.jstorm.task.TaskTransfer; +import com.alibaba.jstorm.task.TaskBatchTransfer; +import com.alibaba.jstorm.task.acker.Acker; +import com.alibaba.jstorm.task.comm.TaskSendTargets; +import com.alibaba.jstorm.task.error.ITaskReportErr; +import com.alibaba.jstorm.task.heartbeat.TaskHeartbeatRunable; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.RotatingMap; +import com.alibaba.jstorm.utils.TimeUtils; +import com.lmax.disruptor.EventHandler; + +/** + * + * BoltExecutor + * + * @author yannian/Longda + * + */ +public class BoltExecutors extends BaseExecutors implements EventHandler { + private static Logger LOG = LoggerFactory.getLogger(BoltExecutors.class); + + protected IBolt bolt; + + protected RotatingMap tuple_start_times; + + private int ackerNum = 0; + + // internal outputCollector is BoltCollector + private OutputCollector outputCollector; + + private Histogram boltExeTimer; + + public BoltExecutors(Task task, IBolt _bolt, TaskTransfer _transfer_fn, + Map innerTaskTransfer, Map storm_conf, + TaskSendTargets _send_fn, TaskStatus taskStatus, + TopologyContext sysTopologyCxt, TopologyContext userTopologyCxt, + TaskBaseMetric _task_stats, ITaskReportErr _report_error) { + + super(task, _transfer_fn, storm_conf, innerTaskTransfer, + sysTopologyCxt, userTopologyCxt, _task_stats, taskStatus, + _report_error); + + this.bolt = _bolt; + + // create TimeCacheMap + + this.tuple_start_times = + new RotatingMap(Acker.TIMEOUT_BUCKET_NUM); + + this.ackerNum = + JStormUtils.parseInt(storm_conf + .get(Config.TOPOLOGY_ACKER_EXECUTORS)); + + // don't use TimeoutQueue for recv_tuple_queue, + // then other place should check the queue size + // TimeCacheQueue.DefaultExpiredCallback logExpireCb = new + // TimeCacheQueue.DefaultExpiredCallback( + // idStr); + // this.recv_tuple_queue = new + // TimeCacheQueue(message_timeout_secs, + // TimeCacheQueue.DEFAULT_NUM_BUCKETS, logExpireCb); + + // create BoltCollector + IOutputCollector output_collector = + new BoltCollector(message_timeout_secs, _report_error, + _send_fn, storm_conf, _transfer_fn, sysTopologyCxt, + taskId, tuple_start_times, _task_stats); + + outputCollector = new OutputCollector(output_collector); + + boltExeTimer = + JStormMetrics.registerTaskHistogram(taskId, + MetricDef.EXECUTE_TIME); + + 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(); + } + + if (ConfigExtension.isTaskBatchTuple(storm_conf)) { + TaskBatchFlushTrigger batchFlushTrigger = + new TaskBatchFlushTrigger(5, idStr + + Constants.SYSTEM_COMPONENT_ID, + (TaskBatchTransfer) _transfer_fn); + batchFlushTrigger.register(TimeUnit.MILLISECONDS); + } + + try { + // do prepare + WorkerClassLoader.switchThreadContext(); + + // 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; + LOG.error("bolt prepare error ", e); + report_error.report(e); + } finally { + WorkerClassLoader.restoreThreadContext(); + } + + LOG.info("Successfully create BoltExecutors " + idStr); + + } + + @Override + public String getThreadName() { + return idStr + "-" + BoltExecutors.class.getSimpleName(); + } + + @Override + public void run() { + while (taskStatus.isShutdown() == false) { + try { + exeQueue.consumeBatchWhenAvailable(this); + + processControlEvent(); + } catch (Throwable e) { + if (taskStatus.isShutdown() == false) { + LOG.error(idStr + " bolt exeutor error", e); + } + } + } + } + + @Override + public void onEvent(Object event, long sequence, boolean endOfBatch) + throws Exception { + + if (event == null) { + return; + } + + long start = System.nanoTime(); + + try { + if (event instanceof Tuple) { + processTupleEvent((Tuple) event); + } else if (event instanceof BatchTuple) { + for (Tuple tuple : ((BatchTuple) event).getTuples()) { + processTupleEvent((Tuple) tuple); + } + } else if (event instanceof TimerTrigger.TimerEvent) { + processTimerEvent((TimerTrigger.TimerEvent) event); + } else { + LOG.warn("Bolt executor received unknown message"); + } + } finally { + long end = System.nanoTime(); + boltExeTimer.update((end - start) / 1000000.0d); + } + } + + private void processTupleEvent(Tuple tuple) { + task_stats.recv_tuple(tuple.getSourceComponent(), + tuple.getSourceStreamId()); + + tuple_start_times.put(tuple, System.currentTimeMillis()); + + try { + bolt.execute(tuple); + } catch (Throwable e) { + error = e; + LOG.error("bolt execute error ", e); + report_error.report(e); + } + + if (ackerNum == 0) { + // only when acker is disable + // get tuple process latency + Long start_time = (Long) tuple_start_times.remove(tuple); + if (start_time != null) { + Long delta = TimeUtils.time_delta_ms(start_time); + task_stats.bolt_acked_tuple(tuple.getSourceComponent(), + tuple.getSourceStreamId(), Double.valueOf(delta)); + } + } + } + + private void processTimerEvent(TimerTrigger.TimerEvent event) { + switch (event.getOpCode()) { + case TimerConstants.ROTATING_MAP: { + Map timeoutMap = tuple_start_times.rotate(); + + if (ackerNum > 0) { + // only when acker is enable + for (Entry entry : timeoutMap.entrySet()) { + Tuple input = entry.getKey(); + task_stats.bolt_failed_tuple(input.getSourceComponent(), + input.getSourceStreamId()); + } + } + break; + } + case TimerConstants.TICK_TUPLE: { + try { + Tuple tuple = (Tuple) event.getMsg(); + bolt.execute(tuple); + } catch (Throwable e) { + error = e; + LOG.error("bolt execute error ", e); + report_error.report(e); + } + break; + } + case TimerConstants.TASK_HEARTBEAT: { + Integer taskId = (Integer) event.getMsg(); + TaskHeartbeatRunable.updateTaskHbStats(taskId, task); + break; + } + default: { + LOG.warn("Receive unsupported timer event, opcode=" + + event.getOpCode()); + break; + } + } + } + + protected void processControlEvent() { + Object event = controlQueue.poll(); + + if (event != null) { + if (event instanceof TimerTrigger.TimerEvent) { + processTimerEvent((TimerTrigger.TimerEvent) event); + LOG.debug("Received one event from control queue"); + } else { + LOG.warn("Received unknown control event, " + + event.getClass().getName()); + } + } + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/AckSpoutMsg.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/AckSpoutMsg.java new file mode 100755 index 000000000..d554efc07 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/AckSpoutMsg.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.task.execute.spout; + +import java.util.List; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.spout.ISpout; + +import com.alibaba.jstorm.client.spout.IAckValueSpout; +import com.alibaba.jstorm.task.TaskBaseMetric; +import com.alibaba.jstorm.task.comm.TupleInfo; + +/** + * The action after spout receive one ack tuple + * + * @author yannian/Longda + * + */ +public class AckSpoutMsg implements IAckMsg { + private static Logger LOG = LoggerFactory.getLogger(AckSpoutMsg.class); + + private ISpout spout; + private Object msgId; + private String stream; + private long timeStamp; + private List values; + private TaskBaseMetric task_stats; + private boolean isDebug = false; + + public AckSpoutMsg(ISpout _spout, TupleInfo tupleInfo, + TaskBaseMetric _task_stats, boolean _isDebug) { + + this.task_stats = _task_stats; + + this.spout = _spout; + this.isDebug = _isDebug; + + this.msgId = tupleInfo.getMessageId(); + this.stream = tupleInfo.getStream(); + if (tupleInfo.getTimestamp() != 0) { + this.timeStamp = System.currentTimeMillis() - tupleInfo.getTimestamp(); + } + this.values = tupleInfo.getValues(); + } + + public void run() { + if (isDebug) { + LOG.info("Acking message {}", msgId); + } + + if (spout instanceof IAckValueSpout) { + IAckValueSpout ackValueSpout = (IAckValueSpout) spout; + ackValueSpout.ack(msgId, values); + } else { + spout.ack(msgId); + } + + task_stats.spout_acked_tuple(stream, timeStamp); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/FailSpoutMsg.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/FailSpoutMsg.java new file mode 100755 index 000000000..7b5d37bf4 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/FailSpoutMsg.java @@ -0,0 +1,71 @@ +/** + * 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.task.execute.spout; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.spout.ISpout; + +import com.alibaba.jstorm.client.spout.IFailValueSpout; +import com.alibaba.jstorm.task.TaskBaseMetric; +import com.alibaba.jstorm.task.comm.TupleInfo; + +/** + * Do the action after spout receive one failed tuple or sending tuple timeout + * + * @author yannian/Longda + * + */ +public class FailSpoutMsg implements IAckMsg { + private static Logger LOG = LoggerFactory.getLogger(FailSpoutMsg.class); + private Object id; + private ISpout spout; + private TupleInfo tupleInfo; + private TaskBaseMetric task_stats; + private boolean isDebug = false; + + public FailSpoutMsg(Object id, ISpout _spout, TupleInfo _tupleInfo, + TaskBaseMetric _task_stats, boolean _isDebug) { + this.id = id; + this.spout = _spout; + this.tupleInfo = _tupleInfo; + this.task_stats = _task_stats; + this.isDebug = _isDebug; + } + + public void run() { + + Object msg_id = tupleInfo.getMessageId(); + + if (spout instanceof IFailValueSpout) { + IFailValueSpout enhanceSpout = (IFailValueSpout) spout; + enhanceSpout.fail(msg_id, tupleInfo.getValues()); + } else { + spout.fail(msg_id); + } + + task_stats.spout_failed_tuple(tupleInfo.getStream()); + + if (isDebug) { + LOG.info("Failed message rootId: {}, messageId:{} : {}", id, + msg_id, tupleInfo.getValues().toString()); + } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/IAckMsg.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/IAckMsg.java new file mode 100755 index 000000000..c1886630e --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/IAckMsg.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.task.execute.spout; + +public interface IAckMsg extends Runnable { + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/MultipleThreadSpoutExecutors.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/MultipleThreadSpoutExecutors.java new file mode 100644 index 000000000..a74079fd2 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/MultipleThreadSpoutExecutors.java @@ -0,0 +1,131 @@ +/** + * 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.task.execute.spout; + +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.task.TopologyContext; +import backtype.storm.utils.DisruptorQueue; +import backtype.storm.utils.WorkerClassLoader; + +import com.alibaba.jstorm.callback.AsyncLoopRunnable; +import com.alibaba.jstorm.callback.AsyncLoopThread; +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.task.Task; +import com.alibaba.jstorm.task.TaskBaseMetric; +import com.alibaba.jstorm.task.TaskStatus; +import com.alibaba.jstorm.task.TaskTransfer; +import com.alibaba.jstorm.task.acker.Acker; +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.RotatingMap; + +/** + * spout executor + * + * All spout actions will be done here + * + * @author yannian/Longda + * + */ +public class MultipleThreadSpoutExecutors extends SpoutExecutors { + private static Logger LOG = LoggerFactory + .getLogger(MultipleThreadSpoutExecutors.class); + + public MultipleThreadSpoutExecutors(Task task, + backtype.storm.spout.ISpout _spout, TaskTransfer _transfer_fn, + Map innerTaskTransfer, Map _storm_conf, + TaskSendTargets sendTargets, TaskStatus taskStatus, + TopologyContext topology_context, TopologyContext _user_context, + TaskBaseMetric _task_stats, ITaskReportErr _report_error) { + super(task, _spout, _transfer_fn, innerTaskTransfer, _storm_conf, + sendTargets, taskStatus, topology_context, _user_context, + _task_stats, _report_error); + + ackerRunnableThread = new AsyncLoopThread(new AckerRunnable()); + pending = + new RotatingMap(Acker.TIMEOUT_BUCKET_NUM, + null, false); + + super.prepare(sendTargets, _transfer_fn, topology_context); + } + + @Override + public String getThreadName() { + return idStr + "-" + MultipleThreadSpoutExecutors.class.getSimpleName(); + } + + @Override + public void run() { + + super.nextTuple(); + } + + class AckerRunnable extends RunnableCallback { + + private AtomicBoolean shutdown = AsyncLoopRunnable.getShutdown(); + + @Override + public String getThreadName() { + return idStr + "-" + AckerRunnable.class.getSimpleName(); + } + + @Override + public void preRun() { + WorkerClassLoader.switchThreadContext(); + } + + @Override + public void postRun() { + WorkerClassLoader.restoreThreadContext(); + } + + @Override + public void run() { + LOG.info("Successfully start Spout's acker thread " + idStr); + + while (shutdown.get() == false) { + + try { + exeQueue.consumeBatchWhenAvailable(MultipleThreadSpoutExecutors.this); + processControlEvent(); + } catch (Exception e) { + if (shutdown.get() == false) { + LOG.error("Actor occur unknow exception ", e); + report_error.report(e); + } + } + + } + + LOG.info("Successfully shutdown Spout's acker thread " + idStr); + } + + public Object getResult() { + LOG.info("Begin to shutdown Spout's acker thread " + idStr); + return -1; + } + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/SingleThreadSpoutExecutors.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/SingleThreadSpoutExecutors.java new file mode 100644 index 000000000..9e4dd212d --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/SingleThreadSpoutExecutors.java @@ -0,0 +1,96 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.task.execute.spout; + +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.task.TopologyContext; +import backtype.storm.utils.DisruptorQueue; +import backtype.storm.utils.WorkerClassLoader; + +import com.alibaba.jstorm.task.Task; +import com.alibaba.jstorm.task.TaskBaseMetric; +import com.alibaba.jstorm.task.TaskStatus; +import com.alibaba.jstorm.task.TaskTransfer; +import com.alibaba.jstorm.task.acker.Acker; +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.RotatingMap; + +/** + * spout executor + * + * All spout actions will be done here + * + * @author yannian/Longda + * + */ +public class SingleThreadSpoutExecutors extends SpoutExecutors { + private static Logger LOG = LoggerFactory + .getLogger(SingleThreadSpoutExecutors.class); + + public SingleThreadSpoutExecutors(Task task, + backtype.storm.spout.ISpout _spout, TaskTransfer _transfer_fn, + Map innerTaskTransfer, Map _storm_conf, + TaskSendTargets sendTargets, TaskStatus taskStatus, + TopologyContext topology_context, TopologyContext _user_context, + TaskBaseMetric _task_stats, ITaskReportErr _report_error) { + super(task, _spout, _transfer_fn, innerTaskTransfer, _storm_conf, + sendTargets, taskStatus, topology_context, _user_context, + _task_stats, _report_error); + + // sending Tuple's TimeCacheMap + pending = + new RotatingMap(Acker.TIMEOUT_BUCKET_NUM, + null, true); + + super.prepare(sendTargets, _transfer_fn, topology_context); + } + + @Override + public String getThreadName() { + return idStr + "-" + SingleThreadSpoutExecutors.class.getSimpleName(); + } + + @Override + public void run() { + executeEvent(); + + super.nextTuple(); + + processControlEvent(); + + } + + private void executeEvent() { + try { + exeQueue.consumeBatch(this); + + } catch (Exception e) { + if (taskStatus.isShutdown() == false) { + LOG.error("Actor occur unknow exception ", e); + } + } + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/SpoutCollector.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/SpoutCollector.java new file mode 100755 index 000000000..d913a9eb1 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/SpoutCollector.java @@ -0,0 +1,218 @@ +/** + * 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.task.execute.spout; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Random; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.spout.ISpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.MessageId; +import backtype.storm.tuple.TupleImplExt; +import backtype.storm.utils.DisruptorQueue; + +import com.alibaba.jstorm.common.metric.Histogram; +import com.alibaba.jstorm.metric.JStormMetrics; +import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.task.TaskBaseMetric; +import com.alibaba.jstorm.task.TaskTransfer; +import com.alibaba.jstorm.task.acker.Acker; +import com.alibaba.jstorm.task.comm.TaskSendTargets; +import com.alibaba.jstorm.task.comm.TupleInfo; +import com.alibaba.jstorm.task.comm.UnanchoredSend; +import com.alibaba.jstorm.task.error.ITaskReportErr; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.TimeOutMap; + +/** + * spout collector, sending tuple through this Object + * + * @author yannian/Longda + * + */ +public class SpoutCollector implements ISpoutOutputCollector { + private static Logger LOG = LoggerFactory.getLogger(SpoutCollector.class); + + private TaskSendTargets sendTargets; + private Map storm_conf; + private TaskTransfer transfer_fn; + // private TimeCacheMap pending; + private TimeOutMap pending; + // topology_context is system topology context + private TopologyContext topology_context; + + private DisruptorQueue disruptorAckerQueue; + private TaskBaseMetric task_stats; + private backtype.storm.spout.ISpout spout; + private ITaskReportErr report_error; + + private Integer task_id; + private Integer ackerNum; + private boolean isDebug = false; + + private Histogram emitTotalTimer; + Random random; + + public SpoutCollector(Integer task_id, backtype.storm.spout.ISpout spout, + TaskBaseMetric task_stats, TaskSendTargets sendTargets, + Map _storm_conf, TaskTransfer _transfer_fn, + TimeOutMap pending, + TopologyContext topology_context, + DisruptorQueue disruptorAckerQueue, ITaskReportErr _report_error) { + this.sendTargets = sendTargets; + this.storm_conf = _storm_conf; + this.transfer_fn = _transfer_fn; + this.pending = pending; + this.topology_context = topology_context; + + this.disruptorAckerQueue = disruptorAckerQueue; + + this.task_stats = task_stats; + this.spout = spout; + this.task_id = task_id; + this.report_error = _report_error; + + ackerNum = + JStormUtils.parseInt(storm_conf + .get(Config.TOPOLOGY_ACKER_EXECUTORS)); + isDebug = + JStormUtils.parseBoolean(storm_conf.get(Config.TOPOLOGY_DEBUG), + false); + + random = new Random(); + random.setSeed(System.currentTimeMillis()); + + String componentId = topology_context.getThisComponentId(); + emitTotalTimer = + JStormMetrics.registerTaskHistogram(task_id, + MetricDef.COLLECTOR_EMIT_TIME); + + } + + @Override + public List emit(String streamId, List tuple, + Object messageId) { + return sendSpoutMsg(streamId, tuple, messageId, null); + } + + @Override + public void emitDirect(int taskId, String streamId, List tuple, + Object messageId) { + sendSpoutMsg(streamId, tuple, messageId, taskId); + } + + private List sendSpoutMsg(String out_stream_id, + List values, Object message_id, Integer out_task_id) { + + long startTime = System.nanoTime(); + + try { + java.util.List out_tasks = null; + if (out_task_id != null) { + out_tasks = sendTargets.get(out_task_id, out_stream_id, values); + } else { + out_tasks = sendTargets.get(out_stream_id, values); + } + + if (out_tasks.size() == 0) { + // don't need send tuple to other task + return out_tasks; + } + List ackSeq = new ArrayList(); + Boolean needAck = (message_id != null) && (ackerNum > 0); + + // This change storm logic + // Storm can't make sure root_id is unique + // storm's logic is root_id = MessageId.generateId(random); + // when duplicate root_id, it will miss call ack/fail + Long root_id = MessageId.generateId(random); + if (needAck) { + while (pending.containsKey(root_id) == true) { + root_id = MessageId.generateId(random); + } + } + for (Integer t : out_tasks) { + MessageId msgid; + if (needAck) { + // Long as = MessageId.generateId(); + Long as = MessageId.generateId(random); + msgid = MessageId.makeRootId(root_id, as); + ackSeq.add(as); + } else { + msgid = MessageId.makeUnanchored(); + } + + TupleImplExt tp = + new TupleImplExt(topology_context, values, task_id, + out_stream_id, msgid); + tp.setTargetTaskId(t); + transfer_fn.transfer(tp); + + } + + if (needAck) { + + TupleInfo info = new TupleInfo(); + info.setStream(out_stream_id); + info.setValues(values); + info.setMessageId(message_id); + info.setTimestamp(System.currentTimeMillis()); + + pending.putHead(root_id, info); + + List ackerTuple = + JStormUtils.mk_list((Object) root_id, + JStormUtils.bit_xor_vals(ackSeq), task_id); + + UnanchoredSend.send(topology_context, sendTargets, transfer_fn, + Acker.ACKER_INIT_STREAM_ID, ackerTuple); + + } else if (message_id != null) { + TupleInfo info = new TupleInfo(); + info.setStream(out_stream_id); + info.setValues(values); + info.setMessageId(message_id); + info.setTimestamp(0); + + AckSpoutMsg ack = + new AckSpoutMsg(spout, info, task_stats, isDebug); + ack.run(); + + } + + return out_tasks; + } finally { + long endTime = System.nanoTime(); + emitTotalTimer.update((endTime - startTime)/1000000.0d); + } + + } + + @Override + public void reportError(Throwable error) { + // TODO Auto-generated method stub + report_error.report(error); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/SpoutExecutors.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/SpoutExecutors.java new file mode 100644 index 000000000..05f745cff --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/SpoutExecutors.java @@ -0,0 +1,357 @@ +/** + * 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.task.execute.spout; + +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.Constants; +import backtype.storm.spout.ISpoutOutputCollector; +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.BatchTuple; +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.client.ConfigExtension; +import com.alibaba.jstorm.common.metric.Histogram; +import com.alibaba.jstorm.common.metric.TimerRatio; +import com.alibaba.jstorm.daemon.worker.timer.TaskBatchFlushTrigger; +import com.alibaba.jstorm.daemon.worker.timer.TimerConstants; +import com.alibaba.jstorm.daemon.worker.timer.TimerTrigger; +import com.alibaba.jstorm.metric.JStormMetrics; +import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.task.Task; +import com.alibaba.jstorm.task.TaskBaseMetric; +import com.alibaba.jstorm.task.TaskStatus; +import com.alibaba.jstorm.task.TaskTransfer; +import com.alibaba.jstorm.task.TaskBatchTransfer; +import com.alibaba.jstorm.task.acker.Acker; +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.task.execute.BaseExecutors; +import com.alibaba.jstorm.task.heartbeat.TaskHeartbeatRunable; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.RotatingMap; +import com.lmax.disruptor.EventHandler; + +/** + * spout executor + * + * All spout actions will be done here + * + * @author yannian/Longda + * + */ +public class SpoutExecutors extends BaseExecutors implements EventHandler { + private static Logger LOG = LoggerFactory.getLogger(SpoutExecutors.class); + + protected final Integer max_spout_pending; + + protected backtype.storm.spout.ISpout spout; + protected RotatingMap pending; + + protected ISpoutOutputCollector output_collector; + + protected boolean firstTime = true; + + protected Histogram nextTupleTimer; + protected Histogram ackerTimer; + protected TimerRatio emptyCpuCounter; + + protected AsyncLoopThread ackerRunnableThread; + + protected boolean isSpoutFullSleep; + + public SpoutExecutors(Task task, backtype.storm.spout.ISpout _spout, + TaskTransfer _transfer_fn, + Map innerTaskTransfer, Map _storm_conf, + TaskSendTargets sendTargets, TaskStatus taskStatus, + TopologyContext topology_context, TopologyContext _user_context, + TaskBaseMetric _task_stats, ITaskReportErr _report_error) { + super(task, _transfer_fn, _storm_conf, innerTaskTransfer, + topology_context, _user_context, _task_stats, taskStatus, + _report_error); + + this.spout = _spout; + + this.max_spout_pending = + JStormUtils.parseInt(storm_conf + .get(Config.TOPOLOGY_MAX_SPOUT_PENDING)); + + this.nextTupleTimer = + JStormMetrics.registerTaskHistogram(taskId, + MetricDef.EXECUTE_TIME); + + this.ackerTimer = + JStormMetrics.registerTaskHistogram(taskId, + MetricDef.ACKER_TIME); + + this.emptyCpuCounter = new TimerRatio(); + JStormMetrics.registerTaskGauge(emptyCpuCounter, taskId, + MetricDef.EMPTY_CPU_RATIO); + + isSpoutFullSleep = ConfigExtension.isSpoutPendFullSleep(storm_conf); + + if (ConfigExtension.isTaskBatchTuple(storm_conf)) { + TaskBatchFlushTrigger batchFlushTrigger = + new TaskBatchFlushTrigger(5, idStr + + Constants.SYSTEM_COMPONENT_ID, + (TaskBatchTransfer) _transfer_fn); + batchFlushTrigger.register(TimeUnit.MILLISECONDS); + } + + LOG.info("isSpoutFullSleep:" + isSpoutFullSleep); + + } + + public void prepare(TaskSendTargets sendTargets, TaskTransfer transferFn, + TopologyContext topologyContext) { + + JStormMetrics.registerTaskGauge( + new com.codahale.metrics.Gauge() { + + @Override + public Double getValue() { + return (double) pending.size(); + } + + }, taskId, MetricDef.PENDING_MAP); + + // collector, in fact it call send_spout_msg + this.output_collector = + new SpoutCollector(taskId, spout, task_stats, sendTargets, + storm_conf, transferFn, pending, topologyContext, + exeQueue, report_error); + + try { + WorkerClassLoader.switchThreadContext(); + this.spout.open(storm_conf, userTopologyCtx, + new SpoutOutputCollector(output_collector)); + } catch (Throwable e) { + error = e; + LOG.error("spout open error ", e); + report_error.report(e); + } finally { + WorkerClassLoader.restoreThreadContext(); + } + + LOG.info("Successfully create SpoutExecutors " + idStr); + + } + + 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; + LOG.info(idStr + " is ready "); + } + + if (taskStatus.isRun() == false) { + JStormUtils.sleepMs(1); + return; + } + + // if don't need ack, pending map will be always empty + if (max_spout_pending == null || pending.size() < max_spout_pending) { + emptyCpuCounter.stop(); + + long start = System.nanoTime(); + try { + spout.nextTuple(); + } catch (Throwable e) { + error = e; + LOG.error("spout execute error ", e); + report_error.report(e); + } finally { + long end = System.nanoTime(); + nextTupleTimer.update((end - start) / 1000000.0d); + } + + return; + } else { + if (isSpoutFullSleep) { + JStormUtils.sleepMs(1); + } + emptyCpuCounter.start(); + // just return, no sleep + } + } + + @Override + public void run() { + + throw new RuntimeException("Should implement this function"); + } + + /** + * Handle acker message + * + * @see com.lmax.disruptor.EventHandler#onEvent(java.lang.Object, long, + * boolean) + */ + @Override + public void onEvent(Object event, long sequence, boolean endOfBatch) + throws Exception { + long start = System.nanoTime(); + try { + + if (event == null) { + return; + } + + Runnable runnable = null; + if (event instanceof Tuple) { + runnable = processTupleEvent((Tuple) event); + } else if (event instanceof BatchTuple) { + for (Tuple tuple : ((BatchTuple) event).getTuples()) { + runnable = processTupleEvent(tuple); + if (runnable != null) { + runnable.run(); + runnable = null; + } + } + } else if (event instanceof TimerTrigger.TimerEvent) { + processTimerEvent((TimerTrigger.TimerEvent) event); + return; + } else if (event instanceof IAckMsg) { + + runnable = (Runnable) event; + } else if (event instanceof Runnable) { + + runnable = (Runnable) event; + } else { + + LOG.warn("Receive one unknow event-" + event.toString() + " " + + idStr); + return; + } + + if (runnable != null) + runnable.run(); + + } catch (Throwable e) { + if (taskStatus.isShutdown() == false) { + LOG.info("Unknow excpetion ", e); + report_error.report(e); + } + } finally { + long end = System.nanoTime(); + ackerTimer.update((end - start) / 1000000.0d); + } + } + + private Runnable processTupleEvent(Tuple event) { + Runnable runnable; + Tuple tuple = (Tuple) event; + Object id = tuple.getValue(0); + Object obj = pending.remove((Long) id); + + if (obj == null) { + if (isDebug) { + LOG.info("Pending map no entry:" + id); + } + runnable = null; + } else { + TupleInfo tupleInfo = (TupleInfo) obj; + + String stream_id = tuple.getSourceStreamId(); + + if (stream_id.equals(Acker.ACKER_ACK_STREAM_ID)) { + + runnable = + new AckSpoutMsg(spout, tupleInfo, task_stats, isDebug); + } else if (stream_id.equals(Acker.ACKER_FAIL_STREAM_ID)) { + runnable = + new FailSpoutMsg(id, spout, tupleInfo, task_stats, + isDebug); + } else { + LOG.warn("Receive one unknow source Tuple " + idStr); + runnable = null; + } + } + + task_stats.recv_tuple(tuple.getSourceComponent(), + tuple.getSourceStreamId()); + return runnable; + } + + public AsyncLoopThread getAckerRunnableThread() { + return ackerRunnableThread; + } + + private void processTimerEvent(TimerTrigger.TimerEvent event) { + switch (event.getOpCode()) { + case TimerConstants.ROTATING_MAP: { + Map timeoutMap = pending.rotate(); + for (java.util.Map.Entry entry : timeoutMap + .entrySet()) { + TupleInfo tupleInfo = entry.getValue(); + FailSpoutMsg fail = + new FailSpoutMsg(entry.getKey(), spout, + (TupleInfo) tupleInfo, task_stats, isDebug); + fail.run(); + } + break; + } + case TimerConstants.TASK_HEARTBEAT: { + Integer taskId = (Integer) event.getMsg(); + TaskHeartbeatRunable.updateTaskHbStats(taskId, task); + break; + } + default: { + LOG.warn("Receive unsupported timer event, opcode=" + + event.getOpCode()); + break; + } + } + } + + protected void processControlEvent() { + Object event = controlQueue.poll(); + + if (event != null) { + if (event instanceof TimerTrigger.TimerEvent) { + processTimerEvent((TimerTrigger.TimerEvent) event); + } else { + LOG.warn("Received unknown control event, " + + event.getClass().getName()); + } + } + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/SpoutTimeoutCallBack.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/SpoutTimeoutCallBack.java new file mode 100755 index 000000000..968831b46 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/execute/spout/SpoutTimeoutCallBack.java @@ -0,0 +1,74 @@ +/** + * 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.task.execute.spout; + +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.utils.DisruptorQueue; + +import com.alibaba.jstorm.task.TaskBaseMetric; +import com.alibaba.jstorm.task.comm.TupleInfo; +import com.alibaba.jstorm.utils.ExpiredCallback; +import com.alibaba.jstorm.utils.JStormUtils; + +public class SpoutTimeoutCallBack implements ExpiredCallback { + private static Logger LOG = LoggerFactory + .getLogger(SpoutTimeoutCallBack.class); + + private DisruptorQueue disruptorEventQueue; + private backtype.storm.spout.ISpout spout; + private Map storm_conf; + private TaskBaseMetric task_stats; + private boolean isDebug; + + public SpoutTimeoutCallBack(DisruptorQueue disruptorEventQueue, + backtype.storm.spout.ISpout _spout, Map _storm_conf, + TaskBaseMetric stat) { + this.storm_conf = _storm_conf; + this.disruptorEventQueue = disruptorEventQueue; + this.spout = _spout; + this.task_stats = stat; + this.isDebug = + JStormUtils.parseBoolean(storm_conf.get(Config.TOPOLOGY_DEBUG), + false); + } + + /** + * pending.put(root_id, JStormUtils.mk_list(message_id, TupleInfo, ms)); + */ + @Override + public void expire(K key, V val) { + if (val == null) { + return; + } + try { + TupleInfo tupleInfo = (TupleInfo) val; + FailSpoutMsg fail = + new FailSpoutMsg(key, spout, (TupleInfo) tupleInfo, + task_stats, isDebug); + + disruptorEventQueue.publish(fail); + } catch (Exception e) { + LOG.error("expire error", e); + } + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/GrouperType.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/GrouperType.java new file mode 100755 index 000000000..9fe9eec5e --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/GrouperType.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.task.group; + +/** + * Grouping type + * + * @author yannian + * + */ +public enum GrouperType { + global, fields, all, shuffle, none, custom_obj, custom_serialized, direct, local_or_shuffle, localFirst +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/MkCustomGrouper.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/MkCustomGrouper.java new file mode 100755 index 000000000..bb6ad9c3b --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/MkCustomGrouper.java @@ -0,0 +1,49 @@ +/** + * 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.task.group; + +import java.util.List; + +import backtype.storm.generated.GlobalStreamId; +import backtype.storm.grouping.CustomStreamGrouping; +import backtype.storm.task.TopologyContext; + +/** + * user defined grouping method + * + * @author Longda/yannian + * + */ +public class MkCustomGrouper { + private CustomStreamGrouping grouping; + + private int myTaskId; + + public MkCustomGrouper(TopologyContext context, + CustomStreamGrouping _grouping, GlobalStreamId stream, + List targetTask, int myTaskId) { + this.myTaskId = myTaskId; + this.grouping = _grouping; + this.grouping.prepare(context, stream, targetTask); + + } + + public List grouper(List values) { + return this.grouping.chooseTasks(myTaskId, values); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/MkFieldsGrouper.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/MkFieldsGrouper.java new file mode 100755 index 000000000..3bf651895 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/MkFieldsGrouper.java @@ -0,0 +1,60 @@ +/** + * 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.task.group; + +import java.util.Iterator; +import java.util.List; + +import backtype.storm.tuple.Fields; + +import com.alibaba.jstorm.utils.JStormUtils; + +/** + * field grouping + * + * @author yannian + * + */ +public class MkFieldsGrouper { + private Fields out_fields; + private Fields group_fields; + private List out_tasks; + + public MkFieldsGrouper(Fields _out_fields, Fields _group_fields, + List _out_tasks) { + + for (Iterator it = _group_fields.iterator(); it.hasNext();) { + String groupField = it.next(); + + // if groupField isn't in _out_fields, it would throw Exception + _out_fields.fieldIndex(groupField); + } + + this.out_fields = _out_fields; + this.group_fields = _group_fields; + this.out_tasks = _out_tasks; + + } + + public List grouper(List values) { + int hashcode = + this.out_fields.select(this.group_fields, values).hashCode(); + int group = Math.abs(hashcode % this.out_tasks.size()); + return JStormUtils.mk_list(out_tasks.get(group)); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/MkGrouper.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/MkGrouper.java new file mode 100755 index 000000000..5408afd1c --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/MkGrouper.java @@ -0,0 +1,203 @@ +/** + * 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.task.group; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Random; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.generated.GlobalStreamId; +import backtype.storm.generated.Grouping; +import backtype.storm.generated.JavaObject; +import backtype.storm.grouping.CustomStreamGrouping; +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Fields; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.daemon.worker.WorkerData; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.RandomRange; +import com.alibaba.jstorm.utils.Thrift; + +/** + * Grouper, get which task should be send to for one tuple + * + * @author yannian + * + */ +public class MkGrouper { + private static final Logger LOG = LoggerFactory.getLogger(MkGrouper.class); + + private TopologyContext topology_context; + // this component output fields + private Fields out_fields; + private Grouping thrift_grouping; + private Grouping._Fields fields; + private GrouperType grouptype; + private List out_tasks; + private List local_tasks; + private String streamId; + + // grouping method + private RandomRange randomrange; + private Random random; + private MkShuffer shuffer; + private MkCustomGrouper custom_grouper; + private MkFieldsGrouper fields_grouper; + private MkLocalShuffer local_shuffer_grouper; + private MkLocalFirst localFirst; + + public MkGrouper(TopologyContext _topology_context, Fields _out_fields, + Grouping _thrift_grouping, List _outTasks, + String streamId, WorkerData workerData) { + this.topology_context = _topology_context; + this.out_fields = _out_fields; + this.thrift_grouping = _thrift_grouping; + this.streamId = streamId; + + this.out_tasks = new ArrayList(); + this.out_tasks.addAll(_outTasks); + Collections.sort(this.out_tasks); + + this.local_tasks = _topology_context.getThisWorkerTasks(); + this.fields = Thrift.groupingType(thrift_grouping); + this.grouptype = this.parseGroupType(workerData); + + String id = _topology_context.getThisTaskId() + ":" + streamId; + LOG.info(id + " grouptype is " + grouptype + ", out_tasks is " + + out_tasks + ", local_tasks" + local_tasks); + + } + + public GrouperType gettype() { + return grouptype; + } + + private GrouperType parseGroupType(WorkerData workerData) { + + GrouperType grouperType = null; + + if (Grouping._Fields.FIELDS.equals(fields)) { + + if (Thrift.isGlobalGrouping(thrift_grouping)) { + + // global grouping, just send tuple to first task + grouperType = GrouperType.global; + } else { + + List fields_group = + Thrift.fieldGrouping(thrift_grouping); + Fields fields = new Fields(fields_group); + + fields_grouper = + new MkFieldsGrouper(out_fields, fields, out_tasks); + + // hashcode by fields + grouperType = GrouperType.fields; + } + + } else if (Grouping._Fields.ALL.equals(fields)) { + // send to every task + grouperType = GrouperType.all; + } else if (Grouping._Fields.SHUFFLE.equals(fields)) { + grouperType = GrouperType.shuffle; + shuffer = new MkShuffer(out_tasks, workerData); + } else if (Grouping._Fields.NONE.equals(fields)) { + // random send one task + this.random = new Random(); + grouperType = GrouperType.none; + } else if (Grouping._Fields.CUSTOM_OBJECT.equals(fields)) { + // user custom grouping by JavaObject + JavaObject jobj = thrift_grouping.get_custom_object(); + CustomStreamGrouping g = Thrift.instantiateJavaObject(jobj); + int myTaskId = topology_context.getThisTaskId(); + String componentId = topology_context.getComponentId(myTaskId); + GlobalStreamId stream = new GlobalStreamId(componentId, streamId); + custom_grouper = + new MkCustomGrouper(topology_context, g, stream, out_tasks, + myTaskId); + grouperType = GrouperType.custom_obj; + } else if (Grouping._Fields.CUSTOM_SERIALIZED.equals(fields)) { + // user custom group by serialized Object + byte[] obj = thrift_grouping.get_custom_serialized(); + CustomStreamGrouping g = + (CustomStreamGrouping) Utils.javaDeserialize(obj); + int myTaskId = topology_context.getThisTaskId(); + String componentId = topology_context.getComponentId(myTaskId); + GlobalStreamId stream = new GlobalStreamId(componentId, streamId); + custom_grouper = + new MkCustomGrouper(topology_context, g, stream, out_tasks, + myTaskId); + grouperType = GrouperType.custom_serialized; + } else if (Grouping._Fields.DIRECT.equals(fields)) { + // directly send to a special task + 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, workerData); + } else if (Grouping._Fields.LOCAL_FIRST.equals(fields)) { + grouperType = GrouperType.localFirst; + localFirst = new MkLocalFirst(local_tasks, out_tasks, workerData); + } + + return grouperType; + } + + /** + * get which task should tuple be sent to + * + * @param values + * @return + */ + public List grouper(List values) { + if (GrouperType.global.equals(grouptype)) { + // send to task which taskId is 0 + return JStormUtils.mk_list(out_tasks.get(0)); + } else if (GrouperType.fields.equals(grouptype)) { + // field grouping + return fields_grouper.grouper(values); + } else if (GrouperType.all.equals(grouptype)) { + // send to every task + return out_tasks; + } else if (GrouperType.shuffle.equals(grouptype)) { + // random, but the random is different from none + 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)); + } else if (GrouperType.custom_obj.equals(grouptype)) { + return custom_grouper.grouper(values); + } else if (GrouperType.custom_serialized.equals(grouptype)) { + return custom_grouper.grouper(values); + } else if (GrouperType.local_or_shuffle.equals(grouptype)) { + return local_shuffer_grouper.grouper(values); + } else if (GrouperType.localFirst.equals(grouptype)) { + return localFirst.grouper(values); + } else { + LOG.warn("Unsupportted group type"); + } + + return new ArrayList(); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/MkLocalFirst.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/MkLocalFirst.java new file mode 100644 index 000000000..56f917564 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/MkLocalFirst.java @@ -0,0 +1,144 @@ +/** + * 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.task.group; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.utils.DisruptorQueue; + +import com.alibaba.jstorm.daemon.worker.WorkerData; +import com.alibaba.jstorm.utils.IntervalCheck; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.RandomRange; + +/** + * + * + * @author zhongyan.feng + * @version + */ +public class MkLocalFirst extends Shuffer { + private static final Logger LOG = LoggerFactory + .getLogger(MkLocalFirst.class); + + private List allOutTasks = new ArrayList(); + private List localOutTasks = new ArrayList(); + private List remoteOutTasks = new ArrayList(); + private RandomRange randomrange; + private RandomRange remoteRandomRange; + private boolean isLocalWorkerAvail; + private WorkerData workerData; + private IntervalCheck intervalCheck; + + public MkLocalFirst(List workerTasks, List allOutTasks, + WorkerData workerData) { + super(workerData); + + intervalCheck = new IntervalCheck(); + intervalCheck.setInterval(10); + + this.allOutTasks.addAll(allOutTasks); + this.workerData = workerData; + + List localWorkerOutTasks = new ArrayList(); + + for (Integer outTask : allOutTasks) { + if (workerTasks.contains(outTask)) { + localWorkerOutTasks.add(outTask); + } + } + + remoteOutTasks.addAll(allOutTasks); + if (localWorkerOutTasks.size() != 0) { + isLocalWorkerAvail = true; + localOutTasks.addAll(localWorkerOutTasks); + remoteOutTasks.removeAll(localWorkerOutTasks); + } else { + isLocalWorkerAvail = false; + } + randomrange = new RandomRange(localOutTasks.size()); + remoteRandomRange = new RandomRange(remoteOutTasks.size()); + + LOG.info("Local out tasks:" + localOutTasks + ", Remote out tasks:" + remoteOutTasks); + } + + @Override + protected int getActiveTask(RandomRange randomrange, List outTasks) { + int index = randomrange.nextInt(); + int size = outTasks.size(); + int i = 0; + + for (i = 0; i < size; i++) { + Integer taskId = outTasks.get(index); + boolean taskStatus = workerData.isOutboundTaskActive(taskId); + DisruptorQueue exeQueue = + (workerData.getInnerTaskTransfer().get(taskId)); + float queueLoadRatio = exeQueue != null ? exeQueue.pctFull() : 0; + if (taskStatus && queueLoadRatio < 1.0) + break; + else + index = randomrange.nextInt(); + } + + return (i < size ? index : -1); + } + + private List intraGroup(List values) { + if (localOutTasks.size() == 0) + return null; + + int index = getActiveTask(randomrange, localOutTasks); + if (index == -1) { + return null; + } + return JStormUtils.mk_list(localOutTasks.get(index)); + } + + private List interGroup(List values) { + int index = getActiveTask(remoteRandomRange, remoteOutTasks); + if (index == -1) { + index = randomrange.nextInt(); + } + return JStormUtils.mk_list(remoteOutTasks.get(index)); + } + + + public List grouper(List values) { + List ret; + ret = intraGroup(values); + if (ret == null) + ret = interGroup(values); + return ret; + } + + /** + * @param args + */ + public static void main(String[] args) { + // TODO Auto-generated method stub + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/MkLocalShuffer.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/MkLocalShuffer.java new file mode 100755 index 000000000..324e1e63c --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/MkLocalShuffer.java @@ -0,0 +1,73 @@ +/** + * 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.task.group; + +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 extends Shuffer { + + private List outTasks; + private RandomRange randomrange; + private boolean isLocal; + + public MkLocalShuffer(List workerTasks, List allOutTasks, + WorkerData workerData) { + super(workerData); + List localOutTasks = new ArrayList(); + + for (Integer outTask : allOutTasks) { + if (workerTasks.contains(outTask)) { + localOutTasks.add(outTask); + } + } + + if (localOutTasks.size() != 0) { + this.outTasks = localOutTasks; + isLocal = true; + } else { + this.outTasks = new ArrayList(); + this.outTasks.addAll(allOutTasks); + isLocal = false; + } + + 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-core/src/main/java/com/alibaba/jstorm/task/group/MkShuffer.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/MkShuffer.java new file mode 100755 index 000000000..be166fd29 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/MkShuffer.java @@ -0,0 +1,55 @@ +/** + * 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.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-core/src/main/java/com/alibaba/jstorm/task/group/Shuffer.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/Shuffer.java new file mode 100644 index 000000000..acad674e1 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/group/Shuffer.java @@ -0,0 +1,50 @@ +/** + * 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.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(); + int i = 0; + + for (i = 0; i < size; i++) { + if (workerData.isOutboundTaskActive(Integer.valueOf(outTasks + .get(index)))) + break; + else + index = randomrange.nextInt(); + } + + return (i < size ? index : -1); + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/heartbeat/TaskHeartbeat.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/heartbeat/TaskHeartbeat.java new file mode 100755 index 000000000..532f55370 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/heartbeat/TaskHeartbeat.java @@ -0,0 +1,98 @@ +/** + * 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.task.heartbeat; + +import java.io.Serializable; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; + +/** + * Task heartbeat, this Object will be updated to ZK timely + * + * @author yannian + * + */ +public class TaskHeartbeat implements Serializable { + + private static final long serialVersionUID = -6369195955255963810L; + private Integer timeSecs; + private Integer uptimeSecs; + + public TaskHeartbeat(int timeSecs, int uptimeSecs) { + this.timeSecs = timeSecs; + this.uptimeSecs = uptimeSecs; + } + + public int getTimeSecs() { + return timeSecs; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } + + public void setTimeSecs(int timeSecs) { + this.timeSecs = timeSecs; + } + + public int getUptimeSecs() { + return uptimeSecs; + } + + public void setUptimeSecs(int uptimeSecs) { + this.uptimeSecs = uptimeSecs; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = + prime * result + ((timeSecs == null) ? 0 : timeSecs.hashCode()); + result = + prime * result + + ((uptimeSecs == null) ? 0 : uptimeSecs.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + TaskHeartbeat other = (TaskHeartbeat) obj; + if (timeSecs == null) { + if (other.timeSecs != null) + return false; + } else if (!timeSecs.equals(other.timeSecs)) + return false; + if (uptimeSecs == null) { + if (other.uptimeSecs != null) + return false; + } else if (!uptimeSecs.equals(other.uptimeSecs)) + return false; + return true; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/heartbeat/TaskHeartbeatRunable.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/heartbeat/TaskHeartbeatRunable.java new file mode 100644 index 000000000..be66911c0 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/heartbeat/TaskHeartbeatRunable.java @@ -0,0 +1,191 @@ +/** + * 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.task.heartbeat; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.LinkedBlockingDeque; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; + +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.cluster.StormClusterState; +import com.alibaba.jstorm.daemon.worker.WorkerData; +import com.alibaba.jstorm.schedule.Assignment.AssignmentType; +import com.alibaba.jstorm.task.Task; +import com.alibaba.jstorm.task.UptimeComputer; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.TimeUtils; + +/** + * Task hearbeat + * + * @author yannian + * + */ +public class TaskHeartbeatRunable extends RunnableCallback { + private static final Logger LOG = LoggerFactory + .getLogger(TaskHeartbeatRunable.class); + + private StormClusterState zkCluster; + private String topology_id; + private UptimeComputer uptime; + private Map storm_conf; + private Integer frequence; + private Map taskAssignTsMap = new HashMap(); + + private static Map taskStatsMap = + new HashMap(); + private static LinkedBlockingDeque eventQueue = + new LinkedBlockingDeque(); + + public static void registerTaskStats(int taskId, TaskStats taskStats) { + Event event = new Event(Event.REGISTER_TYPE, taskId, taskStats); + eventQueue.offer(event); + } + + public static void unregisterTaskStats(int taskId) { + Event event = new Event(Event.UNREGISTER_TYPE, taskId, null); + eventQueue.offer(event); + } + + public static void updateTaskHbStats(int taskId, Task taskData) { + Event event = new Event(Event.TASK_HEARTBEAT_TYPE, taskId, taskData); + eventQueue.offer(event); + } + + public TaskHeartbeatRunable(WorkerData workerData) { + + this.zkCluster = workerData.getZkCluster(); + this.topology_id = workerData.getTopologyId(); + this.uptime = new UptimeComputer(); + this.storm_conf = workerData.getStormConf(); + + String key = Config.TASK_HEARTBEAT_FREQUENCY_SECS; + Object time = storm_conf.get(key); + frequence = JStormUtils.parseInt(time, 10); + + } + + public void handle() throws InterruptedException { + Event event = eventQueue.take(); + while (event != null) { + switch (event.getType()) { + case Event.TASK_HEARTBEAT_TYPE: { + updateTaskHbStats(event); + break; + } + case Event.REGISTER_TYPE: { + Event regEvent = event; + taskStatsMap.put(event.getTaskId(), regEvent.getEventValue()); + taskAssignTsMap.put(event.getTaskId(), + System.currentTimeMillis()); + break; + } + case Event.UNREGISTER_TYPE: { + taskStatsMap.remove(event.getTaskId()); + taskAssignTsMap.remove(event.getTaskId()); + break; + } + default: { + LOG.warn("Unknown event type received:" + event.getType()); + break; + } + } + + event = eventQueue.take(); + } + } + + @Override + public void run() { + try { + handle(); + } catch (InterruptedException e) { + LOG.info(e.getMessage()); + } + } + + @Override + public Object getResult() { + return frequence; + } + + public void updateTaskHbStats(Event event) { + Integer currtime = TimeUtils.current_time_secs(); + Event taskHbEvent = event; + int taskId = taskHbEvent.getTaskId(); + String idStr = " " + topology_id + ":" + taskId + " "; + + try { + + TaskHeartbeat hb = new TaskHeartbeat(currtime, uptime.uptime()); + zkCluster.task_heartbeat(topology_id, taskId, hb); + + LOG.info("update task hearbeat ts " + currtime + " for" + idStr); + + // Check if assignment is changed. e.g scale-out + Task task = taskHbEvent.getEventValue(); + Long timeStamp = taskAssignTsMap.get(taskId); + if (timeStamp != null) { + if (timeStamp < task.getWorkerAssignmentTs() && + task.getWorkerAssignmentType().equals(AssignmentType.Assign)) { + LOG.info("Start to update the task data for task-" + taskId); + task.updateTaskData(); + taskAssignTsMap.put(taskId, task.getWorkerAssignmentTs()); + } + } + } catch (Exception e) { + // TODO Auto-generated catch block + String errMsg = "Failed to update heartbeat to ZK " + idStr + "\n"; + LOG.error(errMsg, e); + } + } + + private static class Event { + public static final int REGISTER_TYPE = 0; + public static final int UNREGISTER_TYPE = 1; + public static final int TASK_HEARTBEAT_TYPE = 2; + private final int type; + private final int taskId; + private final T value; + + public Event(int type, int taskId, T value) { + this.type = type; + this.taskId = taskId; + this.value = value; + } + + public int getType() { + return type; + } + + public int getTaskId() { + return taskId; + } + + public T getEventValue() { + return value; + } + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/task/heartbeat/TaskStats.java b/jstorm-core/src/main/java/com/alibaba/jstorm/task/heartbeat/TaskStats.java new file mode 100755 index 000000000..2f6b08a19 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/task/heartbeat/TaskStats.java @@ -0,0 +1,38 @@ +/** + * 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.task.heartbeat; + +import com.alibaba.jstorm.task.TaskBaseMetric; + +public class TaskStats { + private TaskBaseMetric taskStats; + private String componentType; + + public TaskStats(String componentType, TaskBaseMetric taskStats) { + this.componentType = componentType; + this.taskStats = taskStats; + } + + public TaskBaseMetric getTaskStat() { + return taskStats; + } + + public String getComponentType() { + return componentType; + } +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/DisruptorQueue.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/DisruptorQueue.java new file mode 100755 index 000000000..4a4a72b42 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/DisruptorQueue.java @@ -0,0 +1,168 @@ +/** + * 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.utils; +// +//import java.util.ArrayList; +//import java.util.List; +//import java.util.concurrent.Executor; +//import java.util.concurrent.atomic.AtomicBoolean; +// +//import org.apache.commons.lang.mutable.MutableObject; +// +//import com.lmax.disruptor.EventFactory; +//import com.lmax.disruptor.ExceptionHandler; +//import com.lmax.disruptor.FatalExceptionHandler; +//import com.lmax.disruptor.RingBuffer; +//import com.lmax.disruptor.Sequence; +//import com.lmax.disruptor.SequenceBarrier; +//import com.lmax.disruptor.Sequencer; +//import com.lmax.disruptor.WaitStrategy; +//import com.lmax.disruptor.WorkHandler; +//import com.lmax.disruptor.WorkProcessor; +//import com.lmax.disruptor.util.Util; +// +//public class DisruptorQueue { +// private final RingBuffer ringBuffer; +// private final SequenceBarrier sequenceBarrier; +// private final ExceptionHandler exceptionHandler; +// private final List workProcessors; +// private final Sequence workSequence; +// private final AtomicBoolean started = new AtomicBoolean(false); +// +// public DisruptorQueue(boolean isMultiProducer, int bufferSize, +// WaitStrategy waitStrategy) { +// if (isMultiProducer) { +// ringBuffer = RingBuffer.createMultiProducer( +// new ObjectEventFactory(), bufferSize, waitStrategy); +// } else { +// ringBuffer = RingBuffer.createSingleProducer( +// new ObjectEventFactory(), bufferSize, waitStrategy); +// } +// +// sequenceBarrier = ringBuffer.newBarrier(); +// exceptionHandler = new FatalExceptionHandler(); +// workProcessors = new ArrayList(); +// workSequence = new Sequence(Sequencer.INITIAL_CURSOR_VALUE); +// } +// +// public void register(WorkHandler handler) { +// WorkProcessor workProcessor = new WorkProcessor(ringBuffer, +// sequenceBarrier, new HandleWraper(handler), exceptionHandler, +// workSequence); +// +// ringBuffer.addGatingSequences(workProcessor.getSequence()); +// +// workProcessors.add(workProcessor); +// } +// +// void cleanup() { +// +// } +// +// /** +// * Start the worker pool processing events in sequence. +// * +// * @param executor +// * providing threads for running the workers. +// * @return the {@link RingBuffer} used for the work queue. +// * @throws IllegalStateException +// * if the pool has already been started and not halted yet +// */ +// public void start() { +// if (!started.compareAndSet(false, true)) { +// throw new IllegalStateException( +// "WorkerPool has already been started and cannot be restarted until halted."); +// } +// +// final long cursor = ringBuffer.getCursor(); +// workSequence.set(cursor); +// +// for (WorkProcessor processor : workProcessors) { +// processor.getSequence().set(cursor); +// new Thread(processor).start(); +// } +// +// return; +// } +// +// public Sequence[] getWorkerSequences() { +// final Sequence[] sequences = new Sequence[workProcessors.size()]; +// for (int i = 0, size = workProcessors.size(); i < size; i++) { +// sequences[i] = workProcessors.get(i).getSequence(); +// } +// +// return sequences; +// } +// +// /** +// * Wait for the {@link RingBuffer} to drain of published events then halt +// * the workers. +// */ +// public void drainAndHalt() { +// Sequence[] workerSequences = getWorkerSequences(); +// while (ringBuffer.getCursor() > Util +// .getMinimumSequence(workerSequences)) { +// Thread.yield(); +// } +// +// for (WorkProcessor processor : workProcessors) { +// processor.halt(); +// } +// +// started.set(false); +// } +// +// /** +// * Halt all workers immediately at the end of their current cycle. +// */ +// public void halt() { +// for (WorkProcessor processor : workProcessors) { +// processor.halt(); +// } +// +// started.set(false); +// } +// +// public void offer(T o) { +// long sequence = ringBuffer.next(); +// ringBuffer.get(sequence).setValue(o); +// ringBuffer.publish(sequence); +// } +// +// public static class ObjectEventFactory implements +// EventFactory { +// +// public MutableObject newInstance() { +// return new MutableObject(); +// } +// } +// +// public static class HandleWraper implements WorkHandler { +// private WorkHandler handler; +// +// public HandleWraper(WorkHandler handler) { +// this.handler = handler; +// } +// +// public void onEvent(MutableObject event) throws Exception { +// // TODO Auto-generated method stub +// handler.onEvent((T) event.getValue()); +// } +// +// } +// } diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/DisruptorRunable.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/DisruptorRunable.java new file mode 100755 index 000000000..17b78858d --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/DisruptorRunable.java @@ -0,0 +1,113 @@ +/** + * 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.utils; + +import java.util.concurrent.atomic.AtomicBoolean; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.utils.DisruptorQueue; + +import com.alibaba.jstorm.callback.AsyncLoopRunnable; +import com.alibaba.jstorm.callback.RunnableCallback; +import com.alibaba.jstorm.common.metric.QueueGauge; +import com.alibaba.jstorm.common.metric.Timer; +import com.alibaba.jstorm.metric.JStormHealthCheck; +import com.alibaba.jstorm.metric.JStormMetrics; +import com.alibaba.jstorm.metric.MetricDef; +import com.lmax.disruptor.EventHandler; + +//import com.alibaba.jstorm.message.zeroMq.ISendConnection; + +/** + * + * Disruptor Consumer thread + * + * @author yannian + * + */ +public abstract class DisruptorRunable extends RunnableCallback implements + EventHandler { + private final static Logger LOG = LoggerFactory + .getLogger(DisruptorRunable.class); + + protected DisruptorQueue queue; + protected String idStr; + protected Timer timer; + protected AtomicBoolean shutdown = AsyncLoopRunnable.getShutdown(); + + public DisruptorRunable(DisruptorQueue queue, String idStr) { + this.queue = queue; + this.timer = + JStormMetrics.registerWorkerTimer(idStr + MetricDef.TIME_TYPE); + this.idStr = idStr; + + QueueGauge queueGauge = + new QueueGauge(idStr + MetricDef.QUEUE_TYPE, queue); + JStormMetrics.registerWorkerGauge(queueGauge, idStr + + MetricDef.QUEUE_TYPE); + + JStormHealthCheck.registerWorkerHealthCheck(idStr, queueGauge); + } + + public abstract void handleEvent(Object event, boolean endOfBatch) + throws Exception; + + /** + * This function need to be implements + * + * @see com.lmax.disruptor.EventHandler#onEvent(java.lang.Object, long, + * boolean) + */ + @Override + public void onEvent(Object event, long sequence, boolean endOfBatch) + throws Exception { + if (event == null) { + return; + } + + long start = System.nanoTime(); + try { + handleEvent(event, endOfBatch); + } finally { + long end = System.nanoTime(); + timer.update((end - start)/1000000.0d); + } + } + + @Override + public void run() { + LOG.info("Successfully start thread " + idStr); + queue.consumerStarted(); + + while (shutdown.get() == false) { + queue.consumeBatchWhenAvailable(this); + + } + + LOG.info("Successfully exit thread " + idStr); + } + + @Override + public void shutdown() { + JStormMetrics.unregisterWorkerMetric(idStr + MetricDef.QUEUE_TYPE); + JStormHealthCheck.unregisterWorkerHealthCheck(idStr); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/EPlatform.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/EPlatform.java new file mode 100755 index 000000000..8c62d6fda --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/EPlatform.java @@ -0,0 +1,50 @@ +/** + * 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.utils; + +public enum EPlatform { + Any("any"), + Linux("Linux"), + Mac_OS("Mac OS"), + Mac_OS_X("Mac OS X"), + Windows("Windows"), + OS2("OS/2"), + Solaris("Solaris"), + SunOS("SunOS"), + MPEiX("MPE/iX"), + HP_UX("HP-UX"), + AIX("AIX"), + OS390("OS/390"), + FreeBSD("FreeBSD"), + Irix("Irix"), + Digital_Unix("Digital Unix"), + NetWare_411("NetWare"), + OSF1("OSF1"), + OpenVMS("OpenVMS"), + Others("Others"); + + private EPlatform(String desc){ + this.description = desc; + } + + public String toString(){ + return description; + } + + private String description; +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/EventSampler.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/EventSampler.java new file mode 100755 index 000000000..a9bbc17c4 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/EventSampler.java @@ -0,0 +1,116 @@ +/** + * 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.utils; + +import java.util.Random; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +/** + * + * statistics tuples: sampling event + * + * + * @author yannian/Longda + * + */ +public class EventSampler { + private volatile int freq; + private AtomicInteger i = new AtomicInteger(0); + private volatile int target; + private Random r = new Random(); + + public EventSampler(int freq) { + this.freq = freq; + this.target = r.nextInt(freq); + + if (freq / 4 > 1) { + intervalCheck.setInterval(freq / 4); + } + } + + /** + * select 1/freq + * + * @return + */ + public boolean countCheck() { + i.incrementAndGet(); + if (i.get() > freq) { + target = r.nextInt(freq); + i.set(0); + } + if (i.get() == target) { + return true; + } + return false; + } + + private AtomicInteger counter = new AtomicInteger(0); + private AtomicLong sum = new AtomicLong(0); + private IntervalCheck intervalCheck = new IntervalCheck(); + + public Integer tpsCheck() { + int send = counter.incrementAndGet(); + + Double pastSeconds = intervalCheck.checkAndGet(); + if (pastSeconds != null) { + counter.set(0); + + return Integer.valueOf((int) (send / pastSeconds)); + + } + + return null; + } + + public Integer timesCheck() { + int send = counter.incrementAndGet(); + + Double pastSeconds = intervalCheck.checkAndGet(); + if (pastSeconds != null) { + counter.set(0); + + return send; + + } + + return null; + } + + public Pair avgCheck(long one) { + int send = counter.incrementAndGet(); + long total = sum.addAndGet(one); + + Double pastSeconds = intervalCheck.checkAndGet(); + if (pastSeconds != null) { + counter.set(0); + sum.set(0); + + Double avg = Double.valueOf(0); + if (send != 0) { + avg = ((double) total) / send; + } + + return new Pair(send, avg); + + } + + return null; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/ExpiredCallback.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/ExpiredCallback.java new file mode 100755 index 000000000..5f9450535 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/ExpiredCallback.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.utils; + +public interface ExpiredCallback { + public void expire(K key, V val); +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/FailedAssignTopologyException.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/FailedAssignTopologyException.java new file mode 100755 index 000000000..db5143203 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/FailedAssignTopologyException.java @@ -0,0 +1,39 @@ +/** + * 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.utils; + +public class FailedAssignTopologyException extends RuntimeException { + + private static final long serialVersionUID = 6921061096898899476L; + + public FailedAssignTopologyException() { + super(); + } + + public FailedAssignTopologyException(String msg) { + super(msg); + } + + public FailedAssignTopologyException(String msg, Throwable cause) { + super(msg, cause); + } + + public FailedAssignTopologyException(Throwable cause) { + super(cause); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/FileAttribute.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/FileAttribute.java new file mode 100755 index 000000000..e33167a91 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/FileAttribute.java @@ -0,0 +1,135 @@ +/** + * 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.utils; + +import java.io.Serializable; +import java.util.Date; +import java.util.HashMap; +import java.util.Map; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; +import org.json.simple.JSONAware; + +//import com.alibaba.fastjson.JSONAware; + +public class FileAttribute implements Serializable, JSONAware { + + /** */ + private static final long serialVersionUID = -5131640995402822835L; + + private String fileName; + private String isDir; + private String modifyTime; + private String size; + + public static final String FILE_NAME_FIELD = "fileName"; + public static final String IS_DIR_FIELD = "isDir"; + public static final String MODIFY_TIME_FIELD = "modifyTime"; + public static final String SIZE_FIELD = "size"; + + public String getFileName() { + return fileName; + } + + public void setFileName(String fileName) { + this.fileName = fileName; + } + + public String getIsDir() { + return isDir; + } + + public void setIsDir(String isDir) { + this.isDir = isDir; + } + + public String getModifyTime() { + return modifyTime; + } + + public void setModifyTime(String modifyTime) { + this.modifyTime = modifyTime; + } + + public String getSize() { + return size; + } + + public void setSize(String size) { + this.size = size; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } + + @Override + public String toJSONString() { + Map map = new HashMap(); + + map.put(FILE_NAME_FIELD, fileName); + map.put(IS_DIR_FIELD, isDir); + map.put(MODIFY_TIME_FIELD, modifyTime); + map.put(SIZE_FIELD, size); + return JStormUtils.to_json(map); + } + + public static FileAttribute fromJSONObject(Map jobj) { + if (jobj == null) { + return null; + } + + FileAttribute attribute = new FileAttribute(); + + attribute.setFileName((String) jobj.get(FILE_NAME_FIELD)); + attribute.setIsDir((String) jobj.get(IS_DIR_FIELD)); + attribute.setModifyTime((String) jobj.get(MODIFY_TIME_FIELD)); + attribute.setSize((String) jobj.get(SIZE_FIELD)); + + return attribute; + } + + public static void main(String[] args) { + Map map = new HashMap(); + + FileAttribute attribute = new FileAttribute(); + attribute.setFileName("test"); + attribute.setIsDir("true"); + attribute.setModifyTime(new Date().toString()); + attribute.setSize("4096"); + + map.put("test", attribute); + + System.out.println("Before:" + map); + + String jsonString = JStormUtils.to_json(map); + + Map map2 = + (Map) JStormUtils.from_json(jsonString); + + Map jObject = map2.get("test"); + + FileAttribute attribute2 = FileAttribute.fromJSONObject(jObject); + + System.out.println("attribute2:" + attribute2); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/HttpserverUtils.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/HttpserverUtils.java new file mode 100755 index 000000000..20c1f7a42 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/HttpserverUtils.java @@ -0,0 +1,46 @@ +/** + * 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.utils; + +public class HttpserverUtils { + + public static final String HTTPSERVER_CONTEXT_PATH_LOGVIEW = "/logview"; + + public static final String HTTPSERVER_LOGVIEW_PARAM_CMD = "cmd"; + + public static final String HTTPSERVER_LOGVIEW_PARAM_CMD_LIST = "listDir"; + + public static final String HTTPSERVER_LOGVIEW_PARAM_CMD_SHOW = "showLog"; + + 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"; + + public static final String HTTPSERVER_LOGVIEW_PARAM_DIR = "dir"; + + public static final String HTTPSERVER_LOGVIEW_PARAM_WORKER_PORT = + "workerPort"; + + public static final String HTTPSERVER_LOGVIEW_PARAM_SIZE_FORMAT = "%016d\n"; + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/IntervalCheck.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/IntervalCheck.java new file mode 100644 index 000000000..992659cad --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/IntervalCheck.java @@ -0,0 +1,83 @@ +/** + * 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.utils; + +import java.io.Serializable; + +public class IntervalCheck implements Serializable { + + /** + * + */ + private static final long serialVersionUID = 8952971673547362883L; + + long lastCheck = System.currentTimeMillis(); + + // default interval is 1 second + long interval = 1000; + + /* + * if last check time is before interval seconds, return true, otherwise + * return false + */ + public boolean check() { + return checkAndGet() != null; + } + + /** + * + * @return + */ + public Double checkAndGet() { + long now = System.currentTimeMillis(); + + synchronized (this) { + if (now >= interval + lastCheck) { + double pastSecond = ((double) (now - lastCheck)) / 1000; + lastCheck = now; + return pastSecond; + } + } + + return null; + } + + public long getInterval() { + return interval/1000; + } + + public long getIntervalMs() { + return interval; + } + + public void setInterval(long interval) { + this.interval = interval * 1000; + } + + public void setIntervalMs(long interval) { + this.interval = interval; + } + + public void adjust(long addTimeMillis) { + lastCheck += addTimeMillis; + } + + public void start() { + lastCheck = System.currentTimeMillis(); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/JStormServerUtils.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/JStormServerUtils.java new file mode 100755 index 000000000..600068861 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/JStormServerUtils.java @@ -0,0 +1,153 @@ +/** + * 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.utils; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import org.apache.commons.io.FileUtils; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.GenericOptionsParser; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.StormConfig; + +/** + * storm utils + * + * + * @author yannian/Longda/Xin.Zhou/Xin.Li + * + */ +public class JStormServerUtils { + + private static final Logger LOG = LoggerFactory + .getLogger(JStormServerUtils.class); + + public static void downloadCodeFromMaster(Map conf, String localRoot, + String masterCodeDir, String topologyId, boolean isSupervisor) + throws IOException, TException { + FileUtils.forceMkdir(new File(localRoot)); + FileUtils.forceMkdir(new File(StormConfig.stormlib_path(localRoot))); + + String localStormjarPath = StormConfig.stormjar_path(localRoot); + String masterStormjarPath = StormConfig.stormjar_path(masterCodeDir); + Utils.downloadFromMaster(conf, masterStormjarPath, localStormjarPath); + + String localStormcodePath = StormConfig.stormcode_path(localRoot); + String masterStormcodePath = StormConfig.stormcode_path(masterCodeDir); + Utils.downloadFromMaster(conf, masterStormcodePath, localStormcodePath); + + String localStormConfPath = StormConfig.stormconf_path(localRoot); + String masterStormConfPath = StormConfig.stormconf_path(masterCodeDir); + Utils.downloadFromMaster(conf, masterStormConfPath, localStormConfPath); + + Map stormConf = + (Map) StormConfig.readLocalObject(topologyId, + localStormConfPath); + + if (stormConf == null) + throw new IOException("Get topology conf error: " + topologyId); + + List libs = + (List) stormConf + .get(GenericOptionsParser.TOPOLOGY_LIB_NAME); + if (libs == null) + return; + for (String libName : libs) { + String localStromLibPath = + StormConfig.stormlib_path(localRoot, libName); + String masterStormLibPath = + StormConfig.stormlib_path(masterCodeDir, libName); + Utils.downloadFromMaster(conf, masterStormLibPath, + localStromLibPath); + } + } + + public static void createPid(String dir) throws Exception { + File file = new File(dir); + + if (file.exists() == false) { + file.mkdirs(); + } else if (file.isDirectory() == false) { + throw new RuntimeException("pid dir:" + dir + " isn't directory"); + } + + String[] existPids = file.list(); + + // touch pid before + String pid = JStormUtils.process_pid(); + String pidPath = dir + File.separator + pid; + PathUtils.touch(pidPath); + LOG.info("Successfully touch pid " + pidPath); + + for (String existPid : existPids) { + try { + JStormUtils.kill(Integer.valueOf(existPid)); + PathUtils.rmpath(dir + File.separator + existPid); + } catch (Exception e) { + LOG.warn(e.getMessage(), e); + } + } + + } + + public static void startTaobaoJvmMonitor() { + // JmonitorBootstrap bootstrap = JmonitorBootstrap.getInstance(); + // bootstrap.start(); + } + + public static boolean isOnePending(Map conf) { + Object pending = conf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING); + if (pending == null) { + return false; + } + + int pendingNum = JStormUtils.parseInt(pending); + if (pendingNum == 1) { + return true; + } else { + return false; + } + } + + public static String getName(String componentId, int taskId) { + return componentId + ":" + taskId; + } + + public static String getHostName(Map conf) { + String hostName = ConfigExtension.getSupervisorHost(conf); + if (hostName == null) { + hostName = NetWorkUtils.hostname(); + } + + if (ConfigExtension.isSupervisorUseIp(conf)) { + hostName = NetWorkUtils.ip(); + } + + return hostName; + } + +}; diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/JStormUtils.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/JStormUtils.java new file mode 100644 index 000000000..983f579f4 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/JStormUtils.java @@ -0,0 +1,1279 @@ +/** + * 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.utils; + +import java.io.BufferedOutputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.PrintStream; +import java.io.PrintWriter; +import java.io.StringWriter; +import java.lang.management.ManagementFactory; +import java.lang.management.MemoryMXBean; +import java.lang.management.MemoryUsage; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.zip.ZipEntry; +import java.util.zip.ZipFile; + +import javax.management.ObjectName; + +import org.apache.commons.exec.CommandLine; +import org.apache.commons.exec.DefaultExecutor; +import org.apache.commons.exec.ExecuteException; +import org.apache.commons.exec.ExecuteResultHandler; +import org.apache.commons.exec.PumpStreamHandler; +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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 + * + * @author yannian/Longda/Xin.Zhou/Xin.Li + * + */ +public class JStormUtils { + private static final Logger LOG = LoggerFactory + .getLogger(JStormUtils.class); + + public static long SIZE_1_K = 1024; + public static long SIZE_1_M = SIZE_1_K * 1024; + public static long SIZE_1_G = SIZE_1_M * 1024; + public static long SIZE_1_T = SIZE_1_G * 1024; + public static long SIZE_1_P = SIZE_1_T * 1024; + + public static final int MIN_1 = 60; + 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 String getErrorInfo(String baseInfo, Exception e) { + try { + StringWriter sw = new StringWriter(); + PrintWriter pw = new PrintWriter(sw); + e.printStackTrace(pw); + return baseInfo + "\r\n" + sw.toString() + "\r\n"; + } catch (Exception e2) { + return baseInfo; + } + } + + public static String getErrorInfo(Throwable error) { + try { + StringWriter sw = new StringWriter(); + PrintWriter pw = new PrintWriter(sw); + error.printStackTrace(pw); + return sw.toString(); + } catch (Exception e1) { + return ""; + } + } + + /** + * filter the map + * + * @param filter + * @param all + * @return + */ + public static Map select_keys_pred(Set filter, Map all) { + Map filterMap = new HashMap(); + + for (Entry entry : all.entrySet()) { + if (!filter.contains(entry.getKey())) { + filterMap.put(entry.getKey(), entry.getValue()); + } + } + + return filterMap; + } + + public static byte[] barr(byte v) { + byte[] byteArray = new byte[1]; + byteArray[0] = v; + + return byteArray; + } + + public static byte[] barr(Short v) { + byte[] byteArray = new byte[Short.SIZE / 8]; + for (int i = 0; i < byteArray.length; i++) { + int off = (byteArray.length - 1 - i) * 8; + byteArray[i] = (byte) ((v >> off) & 0xFF); + } + return byteArray; + } + + public static byte[] barr(Integer v) { + byte[] byteArray = new byte[Integer.SIZE / 8]; + for (int i = 0; i < byteArray.length; i++) { + int off = (byteArray.length - 1 - i) * 8; + byteArray[i] = (byte) ((v >> off) & 0xFF); + } + return byteArray; + } + + // for test + public static int byteToInt2(byte[] b) { + + int iOutcome = 0; + byte bLoop; + + for (int i = 0; i < 4; i++) { + bLoop = b[i]; + int off = (b.length - 1 - i) * 8; + iOutcome += (bLoop & 0xFF) << off; + + } + + 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) { + } + if (localMode && val == 0) { + // throw new RuntimeException(msg); + } else { + haltProcess(val); + } + } + + /** + * "{:a 1 :b 1 :c 2} -> {1 [:a :b] 2 :c}" + * + * @param map + * @return + */ + public static HashMap> reverse_map(Map map) { + HashMap> rtn = new HashMap>(); + if (map == null) { + return rtn; + } + for (Entry entry : map.entrySet()) { + K key = entry.getKey(); + V val = entry.getValue(); + List list = rtn.get(val); + if (list == null) { + list = new ArrayList(); + rtn.put(entry.getValue(), list); + } + list.add(key); + + } + + return rtn; + } + + /** + * Gets the pid of this JVM, because Java doesn't provide a real way to do + * this. + * + * @return + */ + public static String process_pid() { + String name = ManagementFactory.getRuntimeMXBean().getName(); + String[] split = name.split("@"); + if (split.length != 2) { + throw new RuntimeException("Got unexpected process name: " + name); + } + + return split[0]; + } + + public static void exec_command(String command) throws ExecuteException, + IOException { + String[] cmdlist = command.split(" "); + CommandLine cmd = new CommandLine(cmdlist[0]); + for (int i = 1; i < cmdlist.length; i++) { + cmd.addArgument(cmdlist[i]); + } + + DefaultExecutor exec = new DefaultExecutor(); + exec.execute(cmd); + } + + /** + * Extra dir from the jar to destdir + * + * @param jarpath + * @param dir + * @param destdir + */ + public static void extract_dir_from_jar(String jarpath, String dir, + String destdir) { + String cmd = "unzip -qq " + jarpath + " " + dir + "/** -d " + destdir; + try { + exec_command(cmd); + } catch (Exception e) { + LOG.warn("No " + dir + " from " + jarpath + " by cmd:" + cmd + + "!\n" + e.getMessage()); + } + + } + + public static void ensure_process_killed(Integer pid) { + // in this function, just kill the process 5 times + // make sure the process be killed definitely + for (int i = 0; i < 5; i++) { + try { + exec_command("kill -9 " + pid); + LOG.info("kill -9 process " + pid); + sleepMs(100); + } catch (ExecuteException e) { + LOG.info("Error when trying to kill " + pid + + ". Process has been killed"); + } catch (Exception e) { + LOG.info("Error when trying to kill " + pid + ".Exception ", e); + } + } + } + + public static void process_killed(Integer pid) { + try { + exec_command("kill " + pid); + LOG.info("kill process " + pid); + } catch (ExecuteException e) { + LOG.info("Error when trying to kill " + pid + + ". Process has been killed. "); + } catch (Exception e) { + LOG.info("Error when trying to kill " + pid + ".Exception ", e); + } + } + + public static void kill(Integer pid) { + process_killed(pid); + + sleepMs(5 * 1000); + + ensure_process_killed(pid); + } + + public static void kill_signal(Integer pid, String signal) { + String cmd = "kill " + signal + " " + pid; + try { + exec_command(cmd); + LOG.info(cmd); + } catch (ExecuteException e) { + LOG.info("Error when run " + cmd + ". Process has been killed. "); + } catch (Exception e) { + LOG.info("Error when run " + cmd + ". Exception ", e); + } + } + + /** + * This function is only for linux + * + * @param pid + * @return + */ + public static boolean isProcDead(String pid) { + if (OSInfo.isLinux() == 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; + } + + public static Double getCpuUsage() { + if (OSInfo.isLinux() == false) { + return 0.0; + } + + Double value = 0.0; + String output = null; + try { + String pid = JStormUtils.process_pid(); + output = SystemOperation.exec("top -b -n 1 | grep " + pid); + String subStr = output.substring(output.indexOf("S") + 1); + for (int i = 0; i < subStr.length(); i++) { + char ch = subStr.charAt(i); + if (ch != ' ') { + subStr = subStr.substring(i); + break; + } + } + String usedCpu = subStr.substring(0, subStr.indexOf(" ")); + value = Double.valueOf(usedCpu); + } catch (Exception e) { + LOG.warn("Failed to get cpu usage ratio."); + if (output != null) + LOG.warn("Output string is \"" + output + "\""); + value = 0.0; + } + + return value; + } + + public static Double getMemUsage() { + if (OSInfo.isLinux() == true) { + try { + Double value = 0.0; + String pid = JStormUtils.process_pid(); + String output = SystemOperation.exec("top -b -n 1 | grep " + pid); + + int m = 0; + String[] strArray = output.split(" "); + for (int i = 0; i < strArray.length; i++) { + String info = strArray[i]; + if (info.trim().length() == 0){ + continue; + } + if(m == 5) { + // memory + String unit = info.substring(info.length() - 1); + + if(unit.equalsIgnoreCase("g")) { + value = Double.parseDouble(info.substring(0, info.length() - 1)); + value *= 1000000000; + } else if(unit.equalsIgnoreCase("m")) { + value = Double.parseDouble(info.substring(0, info.length() - 1)); + value *= 1000000; + } else { + value = Double.parseDouble(info); + } + return value; + } + if(m == 8) { + // cpu usage + + } + if(m == 9) { + // memory ratio + + } + m++; + } + } catch (Exception e) { + LOG.warn("Failed to get memory usage ."); + + } + } + + // this will be incorrect + MemoryMXBean memoryMXBean = ManagementFactory.getMemoryMXBean(); + MemoryUsage memoryUsage = memoryMXBean.getHeapMemoryUsage(); + + return Double.valueOf(memoryUsage.getUsed()); + } + + /** + * If it is backend, please set resultHandler, such as + * DefaultExecuteResultHandler If it is frontend, + * ByteArrayOutputStream.toString get the result + * + * This function don't care whether the command is successfully or not + * + * @param command + * @param environment + * @param workDir + * @param resultHandler + * @return + * @throws IOException + */ + public static ByteArrayOutputStream launchProcess(String command, + final Map environment, final String workDir, + ExecuteResultHandler resultHandler) throws IOException { + + String[] cmdlist = command.split(" "); + + CommandLine cmd = new CommandLine(cmdlist[0]); + for (String cmdItem : cmdlist) { + if (StringUtils.isBlank(cmdItem) == false) { + cmd.addArgument(cmdItem); + } + } + + DefaultExecutor executor = new DefaultExecutor(); + + executor.setExitValue(0); + if (StringUtils.isBlank(workDir) == false) { + executor.setWorkingDirectory(new File(workDir)); + } + + ByteArrayOutputStream out = new ByteArrayOutputStream(); + + PumpStreamHandler streamHandler = new PumpStreamHandler(out, out); + if (streamHandler != null) { + executor.setStreamHandler(streamHandler); + } + + try { + if (resultHandler == null) { + executor.execute(cmd, environment); + } else { + executor.execute(cmd, environment, resultHandler); + } + } catch (ExecuteException e) { + + // @@@@ + // failed to run command + } + + return out; + + } + + protected static java.lang.Process launchProcess(final String[] cmdlist, + final Map environment) throws IOException { + ArrayList buff = new ArrayList(); + for (String tok : cmdlist) { + if (!tok.isEmpty()) { + buff.add(tok); + } + } + + ProcessBuilder builder = new ProcessBuilder(buff); + builder.redirectErrorStream(true); + Map process_evn = builder.environment(); + for (Entry entry : environment.entrySet()) { + process_evn.put(entry.getKey(), entry.getValue()); + } + + return builder.start(); + } + + /** + * @@@ it should use DefaultExecutor to start a process, but some little + * problem have been found, such as exitCode/output string so still use + * the old method to start process + * + * @param command + * @param environment + * @param backend + * @return + * @throws IOException + */ + public static java.lang.Process launch_process(final String command, + final Map environment, boolean backend) + throws IOException { + + if (backend == true) { + new Thread(new Runnable() { + + @Override + public void run() { + String[] cmdlist = + (new String("nohup " + command + " &")).split(" "); + try { + launchProcess(cmdlist, environment); + } catch (IOException e) { + LOG.error( + "Failed to run " + command + ":" + e.getCause(), + e); + } + } + }).start(); + return null; + } else { + String[] cmdlist = command.split(" "); + return launchProcess(cmdlist, environment); + } + } + + public static String current_classpath() { + return System.getProperty("java.class.path"); + } + + // public static String add_to_classpath(String classpath, String[] paths) { + // for (String path : paths) { + // classpath += ":" + path; + // } + // return classpath; + // } + + public static String to_json(Map m) { + return Utils.to_json(m); + } + + public static Object from_json(String json) { + return Utils.from_json(json); + } + + public static HashMap multi_set(List list) { + HashMap rtn = new HashMap(); + for (V v : list) { + int cnt = 1; + if (rtn.containsKey(v)) { + cnt += rtn.get(v); + } + rtn.put(v, cnt); + } + return rtn; + } + + /** + * + * if the list exist repeat string, return the repeated string + * + * this function will be used to check wheter bolt or spout exist same id + * + * @param sets + * @return + */ + public static List getRepeat(List list) { + + List rtn = new ArrayList(); + Set idSet = new HashSet(); + + for (String id : list) { + if (idSet.contains(id)) { + rtn.add(id); + } else { + idSet.add(id); + } + } + + return rtn; + } + + /** + * balance all T + * + * @param + * @param splitup + * @return + */ + public static List interleave_all(List> splitup) { + ArrayList rtn = new ArrayList(); + int maxLength = 0; + for (List e : splitup) { + int len = e.size(); + if (maxLength < len) { + maxLength = len; + } + } + + for (int i = 0; i < maxLength; i++) { + for (List e : splitup) { + if (e.size() > i) { + rtn.add(e.get(i)); + } + } + } + + return rtn; + } + + public static Long bit_xor_vals(Object... vals) { + Long rtn = 0l; + for (Object n : vals) { + rtn = bit_xor(rtn, n); + } + + return rtn; + } + + public static Long bit_xor_vals(java.util.List vals) { + Long rtn = 0l; + for (T n : vals) { + rtn = bit_xor(rtn, n); + } + + return rtn; + } + + public static Long bit_xor_vals_sets(java.util.Set vals) { + Long rtn = 0l; + for (T n : vals) { + rtn = bit_xor(rtn, n); + } + return rtn; + } + + public static Long bit_xor(Object a, Object b) { + Long rtn = 0l; + + if (a instanceof Long && b instanceof Long) { + rtn = ((Long) a) ^ ((Long) b); + return rtn; + } else if (b instanceof Set) { + Long bs = bit_xor_vals_sets((Set) b); + return bit_xor(a, bs); + } else if (a instanceof Set) { + Long as = bit_xor_vals_sets((Set) a); + return bit_xor(as, b); + } else { + Long ai = Long.parseLong(String.valueOf(a)); + Long bi = Long.parseLong(String.valueOf(b)); + rtn = ai ^ bi; + return rtn; + } + + } + + public static List mk_list(V... args) { + ArrayList rtn = new ArrayList(); + for (V o : args) { + rtn.add(o); + } + return rtn; + } + + public static List mk_list(java.util.Set args) { + ArrayList rtn = new ArrayList(); + if (args != null) { + for (V o : args) { + rtn.add(o); + } + } + return rtn; + } + + public static List mk_list(Collection args) { + ArrayList rtn = new ArrayList(); + if (args != null) { + for (V o : args) { + rtn.add(o); + } + } + return rtn; + } + + public static V[] mk_arr(V... args) { + return args; + } + + public static Long parseLong(Object o) { + if (o == null) { + return null; + } + + if (o instanceof String) { + return Long.valueOf(String.valueOf(o)); + } else if (o instanceof Integer) { + Integer value = (Integer) o; + return Long.valueOf((Integer) value); + } else if (o instanceof Long) { + return (Long) o; + } else { + throw new RuntimeException("Invalid value " + + 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) { + + if (o == null) { + return defaultValue; + } + + if (o instanceof String) { + return Long.valueOf(String.valueOf(o)); + } else if (o instanceof Integer) { + Integer value = (Integer) o; + return Long.valueOf((Integer) value); + } else if (o instanceof Long) { + return (Long) o; + } else { + return defaultValue; + } + } + + public static Integer parseInt(Object o) { + if (o == null) { + return null; + } + + if (o instanceof String) { + return Integer.parseInt(String.valueOf(o)); + } else if (o instanceof Long) { + long value = (Long) o; + return Integer.valueOf((int) value); + } else if (o instanceof Integer) { + return (Integer) o; + } else { + throw new RuntimeException("Invalid value " + + o.getClass().getName() + " " + o); + } + } + + public static Integer parseInt(Object o, int defaultValue) { + if (o == null) { + return defaultValue; + } + + if (o instanceof String) { + return Integer.parseInt(String.valueOf(o)); + } else if (o instanceof Long) { + long value = (Long) o; + return Integer.valueOf((int) value); + } else if (o instanceof Integer) { + return (Integer) o; + } else { + return defaultValue; + } + } + + public static boolean parseBoolean(Object o, boolean defaultValue) { + if (o == null) { + return defaultValue; + } + + if (o instanceof String) { + return Boolean.valueOf((String) o); + } else if (o instanceof Boolean) { + return (Boolean) o; + } else { + return defaultValue; + } + } + + public static Set listToSet(List list) { + if (list == null) { + return null; + } + + Set set = new HashSet(); + set.addAll(list); + return set; + } + + /** + * Check whether the zipfile contain the resources + * + * @param zipfile + * @param resources + * @return + */ + public static boolean zipContainsDir(String zipfile, String resources) { + + Enumeration entries = null; + try { + entries = (new ZipFile(zipfile)).entries(); + while (entries != null && entries.hasMoreElements()) { + ZipEntry ze = entries.nextElement(); + String name = ze.getName(); + if (name.startsWith(resources + "/")) { + return true; + } + } + } catch (IOException e) { + // TODO Auto-generated catch block + // e.printStackTrace(); + LOG.error(e + "zipContainsDir error"); + } + + return false; + } + + public static Object add(Object oldValue, Object newValue) { + if (oldValue == null) { + return newValue; + } else if (newValue == null) { + return oldValue; + } + + if (oldValue instanceof Long) { + return (Long) oldValue + (Long) newValue; + } else if (oldValue instanceof Double) { + return (Double) oldValue + (Double) newValue; + } else if (oldValue instanceof Integer) { + return (Integer) oldValue + (Integer) newValue; + } else if (oldValue instanceof Float) { + return (Float) oldValue + (Float) newValue; + } else if (oldValue instanceof Short) { + return (Short) oldValue + (Short) newValue; + } else if (oldValue instanceof BigInteger) { + return ((BigInteger) oldValue).add((BigInteger) newValue); + } else if (oldValue instanceof Number) { + return ((Number) oldValue).doubleValue() + + ((Number) newValue).doubleValue(); + } else { + return null; + } + } + + public static Object mergeList(List list) { + Object ret = null; + + for (Object value : list) { + ret = add(ret, value); + } + + return ret; + } + + public static List mergeList(List result, Object add) { + if (add instanceof Collection) { + for (Object o : (Collection) add) { + result.add(o); + } + } else if (add instanceof Set) { + for (Object o : (Collection) add) { + result.add(o); + } + } else { + result.add(add); + } + + return result; + } + + public static List distinctList(List input) { + List retList = new ArrayList(); + + for (Object object : input) { + if (retList.contains(object)) { + continue; + } else { + retList.add(object); + } + + } + + return retList; + } + + public static Map mergeMapList(List> list) { + Map ret = new HashMap(); + + for (Map listEntry : list) { + if (listEntry == null) { + continue; + } + for (Entry mapEntry : listEntry.entrySet()) { + K key = mapEntry.getKey(); + V value = mapEntry.getValue(); + + V retValue = (V) add(ret.get(key), value); + + ret.put(key, retValue); + } + } + + return ret; + } + + public static String formatSimpleDouble(Double value) { + try { + java.text.DecimalFormat form = + new java.text.DecimalFormat("##0.000"); + String s = form.format(value); + return s; + } catch (Exception e) { + return "0.000"; + } + + } + + public static double formatDoubleDecPoint2(Double value) { + try { + java.text.DecimalFormat form = new java.text.DecimalFormat("##.00"); + String s = form.format(value); + return Double.valueOf(s); + } catch (Exception e) { + return 0.0; + } + } + + public static double formatDoubleDecPoint4(Double value) { + try { + java.text.DecimalFormat form = + new java.text.DecimalFormat("###.0000"); + String s = form.format(value); + return Double.valueOf(s); + } catch (Exception e) { + return 0.0; + } + } + + public static Double convertToDouble(Object value) { + Double ret; + + if (value == null) { + ret = null; + } else { + if (value instanceof Integer) { + ret = ((Integer) value).doubleValue(); + } else if (value instanceof Long) { + ret = ((Long) value).doubleValue(); + } else if (value instanceof Float) { + ret = ((Float) value).doubleValue(); + } else if (value instanceof Double) { + ret = (Double) value; + } else { + ret = null; + } + } + + return ret; + } + + public static String formatValue(Object value) { + if (value == null) { + return "0"; + } + + if (value instanceof Long) { + return String.valueOf((Long) value); + } else if (value instanceof Double) { + return formatSimpleDouble((Double) value); + } else { + return String.valueOf(value); + } + } + + public static void sleepMs(long ms) { + try { + Thread.sleep(ms); + } catch (InterruptedException e) { + + } + } + + public static void sleepNs(int ns) { + try { + Thread.sleep(0, ns); + } catch (InterruptedException e) { + + } + } + + public static String HEXES = "0123456789ABCDEF"; + + public static String toPrintableString(byte[] buf) { + if (buf == null) { + return null; + } + + StringBuilder sb = new StringBuilder(); + int index = 0; + for (byte b : buf) { + if (index % 10 == 0) { + sb.append("\n"); + } + index++; + + sb.append(HEXES.charAt((b & 0xF0) >> 4)); + sb.append(HEXES.charAt((b & 0x0F))); + sb.append(" "); + + } + + return sb.toString(); + } + + /** + * @@@ Todo + * + * @return + */ + public static Long getPhysicMemorySize() { + Object object; + try { + object = + ManagementFactory.getPlatformMBeanServer().getAttribute( + new ObjectName("java.lang", "type", + "OperatingSystem"), + "TotalPhysicalMemorySize"); + } catch (Exception e) { + LOG.warn("Failed to get system physical memory size,", e); + return null; + } + + Long ret = (Long) object; + + return ret; + } + + public static String genLogName(String topology, Integer port) { + return topology + "-worker-" + port + ".log"; + } + + // public static String getLog4jFileName(org.apache.log4j.Logger + // log4jLogger) throws Exception{ + // Enumeration enumAppender = + // log4jLogger.getAllAppenders(); + // org.apache.log4j.FileAppender fileAppender = null; + // while (enumAppender.hasMoreElements()) { + // org.apache.log4j.Appender appender = enumAppender.nextElement(); + // if (appender instanceof org.apache.log4j.FileAppender) { + // fileAppender = (org.apache.log4j.FileAppender) appender; + // break; + // } + // } + // if (fileAppender != null) { + // return fileAppender.getFile(); + // + // } + // + // return null; + // } + + public static String getLogFileName() { + try { + Logger rootLogger = + LoggerFactory.getLogger(org.slf4j.Logger.ROOT_LOGGER_NAME); + if (rootLogger instanceof ch.qos.logback.classic.Logger) { + ch.qos.logback.classic.Logger logbackLogger = + (ch.qos.logback.classic.Logger) rootLogger; + // Logger framework is Logback + for (Iterator> index = + logbackLogger.iteratorForAppenders(); index.hasNext();) { + ch.qos.logback.core.Appender appender = + index.next(); + if (appender instanceof ch.qos.logback.core.FileAppender) { + ch.qos.logback.core.FileAppender fileAppender = + (ch.qos.logback.core.FileAppender) appender; + return fileAppender.getFile(); + } + } + } + // else if (rootLogger instanceof org.slf4j.impl.Log4jLoggerAdapter) + // { + // // slf4j-log4j + // org.slf4j.impl.Log4jLoggerAdapter log4jAdapter = + // (org.slf4j.impl.Log4jLoggerAdapter) rootLogger; + // try { + // Field field = log4jAdapter.getClass().getDeclaredField( + // "logger"); + // field.setAccessible(true); + // Object obj = field.get(log4jAdapter); + // if (obj instanceof org.apache.log4j.spi.RootLogger) { + // return getLog4jFileName((org.apache.log4j.spi.RootLogger) obj); + // } + // } catch (Exception e) { + // e.printStackTrace(); + // } + // + // } else if (rootLogger instanceof org.apache.log4j.Logger) { + // return getLog4jFileName((org.apache.log4j.Logger) rootLogger); + // } + + } catch (Throwable e) { + LOG.info("Failed to get root logger file name", e.getMessage()); + return null; + } + return null; + } + + public static String getLogDir() { + String file = JStormUtils.getLogFileName(); + if (file != null) { + if (file.lastIndexOf(File.separator) < 0) + return ""; + return file.substring(0, file.lastIndexOf(File.separator)); + } + + String stormHome = System.getProperty("jstorm.home"); + if (stormHome == null) { + return "." + File.separator + "logs"; + } else { + return stormHome + File.separator + "logs"; + } + } + + public static void redirectOutput(String file) throws Exception { + + System.out.println("Redirect output to " + file); + + FileOutputStream workerOut = new FileOutputStream(new File(file)); + + PrintStream ps = + new PrintStream(new BufferedOutputStream(workerOut), true); + System.setOut(ps); + System.setErr(ps); + + LOG.info("Successfully redirect System.out to " + file); + + } + + public static RunnableCallback getDefaultKillfn() { + + return new AsyncLoopDefaultKill(); + } + + public static TreeMap integer_divided(int sum, + int num_pieces) { + return Utils.integerDivided(sum, num_pieces); + } + + public static HashMap filter_val(RunnableCallback fn, + Map amap) { + HashMap rtn = new HashMap(); + + for (Entry entry : amap.entrySet()) { + V value = entry.getValue(); + Object result = fn.execute(value); + + if (result == (Boolean) true) { + rtn.put(entry.getKey(), value); + } + } + 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; + } + + public static byte[] longToBytes(long x) { + ByteBuffer buffer = ByteBuffer.allocate(Long.SIZE); + buffer.putLong(x); + return buffer.array(); + } + + public static long bytesToLong(byte[] bytes) { + ByteBuffer buffer = ByteBuffer.allocate(Long.SIZE); + buffer.put(bytes); + buffer.flip();// need flip + return buffer.getLong(); + } + + public static Object createDisruptorWaitStrategy(Map conf) { + String waitStrategy = + (String) conf.get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY); + Object ret; + + if (waitStrategy.indexOf("TimeoutBlockingWaitStrategy") != -1) { + long timeout = + parseLong(conf.get(Config.TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT), + 10); + ret = Utils.newInstance(waitStrategy, timeout, TimeUnit.MILLISECONDS); + } else { + ret = Utils.newInstance(waitStrategy); + } + + return ret; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/LoadConf.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/LoadConf.java new file mode 100644 index 000000000..d082fcc5e --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/LoadConf.java @@ -0,0 +1,141 @@ +package com.alibaba.jstorm.utils; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.net.URL; +import java.util.ArrayList; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.yaml.snakeyaml.Yaml; +import org.yaml.snakeyaml.constructor.SafeConstructor; + +public class LoadConf { + private static final Logger LOG = LoggerFactory.getLogger(LoadConf.class); + + public static List findResources(String name) { + try { + Enumeration resources = Thread.currentThread().getContextClassLoader().getResources(name); + List ret = new ArrayList(); + while (resources.hasMoreElements()) { + ret.add(resources.nextElement()); + } + return ret; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** + * + * @param name + * @param mustExist -- if this is true, the file must exist, otherwise throw exception + * @param canMultiple -- if this is false and there is multiple conf, it will throw exception + * @return + */ + public static Map findAndReadYaml(String name, boolean mustExist, boolean canMultiple) { + InputStream in = null; + boolean confFileEmpty = false; + try { + in = getConfigFileInputStream(name, canMultiple); + if (null != in) { + Yaml yaml = new Yaml(new SafeConstructor()); + Map ret = (Map) yaml.load(new InputStreamReader(in)); + if (null != ret) { + return new HashMap(ret); + } else { + confFileEmpty = true; + } + } + + if (mustExist) { + if (confFileEmpty) + throw new RuntimeException("Config file " + name + " doesn't have any valid storm configs"); + else + throw new RuntimeException("Could not find config file on classpath " + name); + } else { + return new HashMap(); + } + } catch (IOException e) { + throw new RuntimeException(e); + } finally { + if (null != in) { + try { + in.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } + } + + public static InputStream getConfigFileInputStream(String configFilePath, boolean canMultiple) throws IOException { + if (null == configFilePath) { + throw new IOException("Could not find config file, name not specified"); + } + + HashSet resources = new HashSet(findResources(configFilePath)); + if (resources.isEmpty()) { + File configFile = new File(configFilePath); + if (configFile.exists()) { + return new FileInputStream(configFile); + } + } else if (resources.size() > 1 && canMultiple == false) { + throw new IOException("Found multiple " + configFilePath + + " resources. You're probably bundling the Storm jars with your topology jar. " + resources); + } else { + LOG.info("Using " + configFilePath + " from resources"); + URL resource = resources.iterator().next(); + return resource.openStream(); + } + return null; + } + + public static InputStream getConfigFileInputStream(String configFilePath) throws IOException { + return getConfigFileInputStream(configFilePath, true); + } + + public static Map LoadYaml(String confPath) { + + return findAndReadYaml(confPath, true, true); + + } + + public static Map LoadProperty(String prop) { + + InputStream in = null; + Properties properties = new Properties(); + + try { + in = getConfigFileInputStream(prop); + properties.load(in); + } catch (FileNotFoundException e) { + throw new RuntimeException("No such file " + prop); + } catch (Exception e1) { + throw new RuntimeException("Failed to read config file"); + } finally { + if (null != in) { + try { + in.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } + + Map ret = new HashMap(); + ret.putAll(properties); + return ret; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/NetWorkUtils.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/NetWorkUtils.java new file mode 100755 index 000000000..8bca59975 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/NetWorkUtils.java @@ -0,0 +1,180 @@ +/** + * 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.utils; + +import java.io.IOException; +import java.net.InetAddress; +import java.net.ServerSocket; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Network utilis + * + * @author yannian + * + */ +public class NetWorkUtils { + private static Logger LOG = LoggerFactory.getLogger(NetWorkUtils.class); + + public static String hostname() { + String hostname = null; + try { + hostname = InetAddress.getLocalHost().getCanonicalHostName(); + } catch (UnknownHostException e) { + LOG.error("local_hostname", e); + } + return hostname; + } + + public static String ip() { + String hostname = null; + try { + hostname = InetAddress.getLocalHost().getHostAddress(); + } catch (UnknownHostException e) { + LOG.error("local_hostname", e); + } + return hostname; + } + + /** + * Check whether the port is available to binding + * + * @param port + * @return -1 means not available, others means available + * @throws IOException + */ + public static int tryPort(int port) throws IOException { + ServerSocket socket = new ServerSocket(port); + int rtn = socket.getLocalPort(); + socket.close(); + return rtn; + } + + /** + * get one available port + * + * @return -1 means failed, others means one availablePort + */ + public static int getAvailablePort() { + return availablePort(0); + } + + /** + * Check whether the port is available to binding + * + * @param prefered + * @return -1 means not available, others means available + */ + public static int availablePort(int prefered) { + int rtn = -1; + try { + rtn = tryPort(prefered); + } catch (IOException e) { + + } + return rtn; + } + + public static String host2Ip(String host) { + InetAddress address = null; + try { + address = InetAddress.getByName(host); + } catch (UnknownHostException e) { + LOG.warn("NetWorkUtil can't transfer hostname(" + host + + ") to ip, return hostname", e); + return host; + } + return address.getHostAddress(); + } + + public static List host2Ip(List servers) { + if (servers == null || servers.size() == 0) { + return new ArrayList(); + } + + Set ret = new HashSet(); + for (String server : servers) { + if (StringUtils.isBlank(server)) { + continue; + } + + InetAddress ia; + try { + ia = InetAddress.getByName(server); + } catch (UnknownHostException e) { + // TODO Auto-generated catch block + LOG.info("Fail to get address of ", server); + continue; + } + if (ia.isLoopbackAddress() || ia.isAnyLocalAddress()) { + ret.add(NetWorkUtils.ip()); + }else { + ret.add(ia.getHostAddress()); + } + } + + + return JStormUtils.mk_list(ret); + } + + public static String ip2Host(String ip) { + InetAddress address = null; + try { + address = InetAddress.getByName(ip); + } catch (UnknownHostException e) { + LOG.warn("NetWorkUtil can't transfer ip(" + ip + + ") to hostname, return ip", e); + return 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); + + } + + public static void main(String[] args) { + List servers = new ArrayList(); + servers.add("localhost"); + + System.out.println(host2Ip(servers)); + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/OSInfo.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/OSInfo.java new file mode 100644 index 000000000..d4f6e0f41 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/OSInfo.java @@ -0,0 +1,157 @@ +/** + * 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.utils; + +public class OSInfo { + + private static String OS = System.getProperty("os.name").toLowerCase(); + + private static OSInfo _instance = new OSInfo(); + + private EPlatform platform; + + private OSInfo(){} + + public static boolean isLinux(){ + return OS.indexOf("linux")>=0; + } + + public static boolean isMacOS(){ + return OS.indexOf("mac")>=0&&OS.indexOf("os")>0&&OS.indexOf("x")<0; + } + + public static boolean isMacOSX(){ + return OS.indexOf("mac")>=0&&OS.indexOf("os")>0&&OS.indexOf("x")>0; + } + + public static boolean isMac() { + return OS.indexOf("mac")>=0&&OS.indexOf("os")>0; + } + + public static boolean isWindows(){ + return OS.indexOf("windows")>=0; + } + + public static boolean isOS2(){ + return OS.indexOf("os/2")>=0; + } + + public static boolean isSolaris(){ + return OS.indexOf("solaris")>=0; + } + + public static boolean isSunOS(){ + return OS.indexOf("sunos")>=0; + } + + public static boolean isMPEiX(){ + return OS.indexOf("mpe/ix")>=0; + } + + public static boolean isHPUX(){ + return OS.indexOf("hp-ux")>=0; + } + + public static boolean isAix(){ + return OS.indexOf("aix")>=0; + } + + public static boolean isOS390(){ + return OS.indexOf("os/390")>=0; + } + + public static boolean isFreeBSD(){ + return OS.indexOf("freebsd")>=0; + } + + public static boolean isIrix(){ + return OS.indexOf("irix")>=0; + } + + public static boolean isDigitalUnix(){ + return OS.indexOf("digital")>=0&&OS.indexOf("unix")>0; + } + + public static boolean isNetWare(){ + return OS.indexOf("netware")>=0; + } + + public static boolean isOSF1(){ + return OS.indexOf("osf1")>=0; + } + + public static boolean isOpenVMS(){ + return OS.indexOf("openvms")>=0; + } + + /** + * Get OS name + * @return OS name + */ + public static EPlatform getOSname(){ + if(isAix()){ + _instance.platform = EPlatform.AIX; + }else if (isDigitalUnix()) { + _instance.platform = EPlatform.Digital_Unix; + }else if (isFreeBSD()) { + _instance.platform = EPlatform.FreeBSD; + }else if (isHPUX()) { + _instance.platform = EPlatform.HP_UX; + }else if (isIrix()) { + _instance.platform = EPlatform.Irix; + }else if (isLinux()) { + _instance.platform = EPlatform.Linux; + }else if (isMacOS()) { + _instance.platform = EPlatform.Mac_OS; + }else if (isMacOSX()) { + _instance.platform = EPlatform.Mac_OS_X; + }else if (isMPEiX()) { + _instance.platform = EPlatform.MPEiX; + }else if (isNetWare()) { + _instance.platform = EPlatform.NetWare_411; + }else if (isOpenVMS()) { + _instance.platform = EPlatform.OpenVMS; + }else if (isOS2()) { + _instance.platform = EPlatform.OS2; + }else if (isOS390()) { + _instance.platform = EPlatform.OS390; + }else if (isOSF1()) { + _instance.platform = EPlatform.OSF1; + }else if (isSolaris()) { + _instance.platform = EPlatform.Solaris; + }else if (isSunOS()) { + _instance.platform = EPlatform.SunOS; + }else if (isWindows()) { + _instance.platform = EPlatform.Windows; + }else{ + _instance.platform = EPlatform.Others; + } + return _instance.platform; + } + /** + * @param args + */ + public static void main(String[] args) { + System.out.println( System.getProperty("os.name") ); + System.out.println( System.getProperty("os.version") ); + System.out.println( System.getProperty("os.arch") ); + + System.out.println(OSInfo.getOSname()); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/OlderFileFilter.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/OlderFileFilter.java new file mode 100755 index 000000000..13b1d98e3 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/OlderFileFilter.java @@ -0,0 +1,46 @@ +/** + * 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.utils; + +import java.io.File; +import java.io.FileFilter; + +/** + * filter the older file, skip the files' modify time which is less sec than now + * + * @author lixin + * + */ +public class OlderFileFilter implements FileFilter { + + private int seconds; + + public OlderFileFilter(int seconds) { + this.seconds = seconds; + } + + @Override + public boolean accept(File pathname) { + + long current_time = System.currentTimeMillis(); + + return (pathname.isFile() && (pathname.lastModified() + seconds * 1000 <= current_time)) + || pathname.isDirectory(); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/Pair.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/Pair.java new file mode 100644 index 000000000..49d35d6c0 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/Pair.java @@ -0,0 +1,53 @@ +/** + * 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.utils; + +public class Pair { + private F first; + private S second; + + public Pair(F first, S second) { + this.first = first; + this.second = second; + } + + public F getFirst() { + return first; + } + + public void setFirst(F first) { + this.first = first; + } + + public S getSecond() { + return second; + } + + public void setSecond(S second) { + this.second = second; + } + + @Override + public String toString(){ + StringBuilder sb = new StringBuilder(); + sb.append("first:"+ first); + sb.append(":"); + sb.append("sencond:"+ second); + return sb.toString(); + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/PathUtils.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/PathUtils.java new file mode 100755 index 000000000..939b81b09 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/PathUtils.java @@ -0,0 +1,148 @@ +/** + * 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.utils; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.io.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * + * @author yannian + * + */ +public class PathUtils { + static Logger LOG = LoggerFactory.getLogger(PathUtils.class); + + public static final String SEPERATOR = "/"; + + /** + * split path as list + * + * @param path + * @return + */ + public static List tokenize_path(String path) { + String[] toks = path.split(SEPERATOR); + java.util.ArrayList rtn = new ArrayList(); + for (String str : toks) { + if (!str.isEmpty()) { + rtn.add(str); + } + } + return rtn; + } + + public static String toks_to_path(List toks) { + StringBuffer buff = new StringBuffer(); + buff.append(SEPERATOR); + int size = toks.size(); + for (int i = 0; i < size; i++) { + buff.append(toks.get(i)); + if (i < (size - 1)) { + buff.append(SEPERATOR); + } + + } + return buff.toString(); + } + + public static String normalize_path(String path) { + String rtn = toks_to_path(tokenize_path(path)); + return rtn; + } + + public static String parent_path(String path) { + List toks = tokenize_path(path); + int size = toks.size(); + if (size > 0) { + toks.remove(size - 1); + } + return toks_to_path(toks); + } + + public static String full_path(String parent, String name) { + return normalize_path(parent + SEPERATOR + name); + } + + public static boolean exists_file(String path) { + return (new File(path)).exists(); + } + + public static void rmr(String path) throws IOException { + LOG.debug("Rmr path " + path); + if (exists_file(path)) { + FileUtils.forceDelete(new File(path)); + } + + } + + public static void local_mkdirs(String path) throws IOException { + LOG.debug("Making dirs at" + path); + FileUtils.forceMkdir(new File(path)); + } + + public static void rmpath(String path) { + LOG.debug("Removing path " + path); + boolean isdelete = (new File(path)).delete(); + if (!isdelete) { + throw new RuntimeException("Failed to delete " + path); + } + } + + public static void touch(String path) throws IOException { + LOG.debug("Touching file at" + path); + boolean success = (new File(path)).createNewFile(); + if (!success) { + throw new RuntimeException("Failed to touch " + path); + } + } + + public static List read_dir_contents(String dir) { + ArrayList rtn = new ArrayList(); + if (exists_file(dir)) { + File[] list = (new File(dir)).listFiles(); + for (File f : list) { + rtn.add(f.getName()); + } + } + 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-core/src/main/java/com/alibaba/jstorm/utils/RandomRange.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/RandomRange.java new file mode 100755 index 000000000..e3be73f20 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/RandomRange.java @@ -0,0 +1,74 @@ +/** + * 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.utils; + +import java.util.ArrayList; + +/** + * Shuffle the Range, This class is used in shuffle grouping, it is better than + * random, which can't make sure balance. + * + * @author yannian + * + */ +public class RandomRange { + private ArrayList rr; + private Integer amt; + + public RandomRange(int amt) { + this.amt = amt; + this.rr = rotating_random_range(amt); + } + + public Integer nextInt() { + return this.acquire_random_range_id(); + } + + private ArrayList rotating_random_range(int amt) { + + ArrayList range = new ArrayList(); + for (int i = 0; i < amt; i++) { + range.add(i); + } + + ArrayList rtn = new ArrayList(); + for (int i = 0; i < amt; i++) { + int index = (int) (Math.random() * range.size()); + rtn.add(range.remove(index)); + } + + return rtn; + } + + private synchronized int acquire_random_range_id() { + int ret = this.rr.remove(0); + if (this.rr.size() == 0) { + this.rr.addAll(rotating_random_range(this.amt)); + } + return ret; + } + + public static void main(String[] args) { + RandomRange test = new RandomRange(10); + + for (int i = 0; i < 10; i++) { + System.out.println(test.acquire_random_range_id()); + } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/RotatingMap.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/RotatingMap.java new file mode 100755 index 000000000..454e987f1 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/RotatingMap.java @@ -0,0 +1,149 @@ +/** + * 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.utils; + +import java.util.Deque; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingDeque; + +/** + * RotatingMap must be used under thread-safe environment + * + * Expires keys that have not been updated in the configured number of seconds. + * The algorithm used will take between expirationSecs and expirationSecs * (1 + + * 1 / (numBuckets-1)) to actually expire the message. + * + * get, put, remove, containsKey, and size take O(numBuckets) time to run. + * + */ +public class RotatingMap implements TimeOutMap { + // this default ensures things expire at most 50% past the expiration time + private static final int DEFAULT_NUM_BUCKETS = 3; + + private Deque> _buckets; + + private ExpiredCallback _callback; + + private final Object lock = new Object(); + + public RotatingMap(int numBuckets, ExpiredCallback callback, + boolean isSingleThread) { + if (numBuckets < 2) { + throw new IllegalArgumentException("numBuckets must be >= 2"); + } + if (isSingleThread == true) { + _buckets = new LinkedList>(); + } else { + _buckets = new LinkedBlockingDeque>(); + } + + for (int i = 0; i < numBuckets; i++) { + _buckets.add(new ConcurrentHashMap()); + } + + _callback = callback; + } + + public RotatingMap(ExpiredCallback callback) { + this(DEFAULT_NUM_BUCKETS, callback, false); + } + + public RotatingMap(int numBuckets) { + this(numBuckets, null, false); + } + + public Map rotate() { + Map dead = _buckets.removeLast(); + _buckets.addFirst(new ConcurrentHashMap()); + if (_callback != null) { + for (Entry entry : dead.entrySet()) { + _callback.expire(entry.getKey(), entry.getValue()); + } + } + return dead; + } + + @Override + public boolean containsKey(K key) { + for (Map bucket : _buckets) { + if (bucket.containsKey(key)) { + return true; + } + } + return false; + } + + @Override + public V get(K key) { + for (Map bucket : _buckets) { + if (bucket.containsKey(key)) { + return bucket.get(key); + } + } + return null; + } + + @Override + public void putHead(K key, V value) { + _buckets.peekFirst().put(key, value); + } + + @Override + public void put(K key, V value) { + Iterator> it = _buckets.iterator(); + Map bucket = it.next(); + bucket.put(key, value); + while (it.hasNext()) { + bucket = it.next(); + bucket.remove(key); + } + } + + /** + * Remove item from Rotate + * + * On the side of performance, scanning from header is faster On the side of + * logic, it should scan from the end to first. + * + * @param key + * @return + */ + @Override + public Object remove(K key) { + for (Map bucket : _buckets) { + Object value = bucket.remove(key); + if (value != null) { + return value; + } + } + return null; + } + + @Override + public int size() { + int size = 0; + for (Map bucket : _buckets) { + size += bucket.size(); + } + return size; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/RunCounter.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/RunCounter.java new file mode 100755 index 000000000..24b8035f3 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/RunCounter.java @@ -0,0 +1,100 @@ +/** + * 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.utils; + +import java.io.Serializable; +import java.util.concurrent.atomic.AtomicLong; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class RunCounter implements Serializable { + + private static final long serialVersionUID = 2177944366059817622L; + private static final Logger LOG = LoggerFactory.getLogger(RunCounter.class); + private AtomicLong total = new AtomicLong(0); + private AtomicLong times = new AtomicLong(0); + private AtomicLong values = new AtomicLong(0); + + private IntervalCheck intervalCheck; + + private final String id; + + public RunCounter() { + this("", RunCounter.class); + } + + public RunCounter(String id) { + this(id, RunCounter.class); + } + + public RunCounter(Class tclass) { + this(tclass.getName(), tclass); + + } + + public RunCounter(String id, Class tclass) { + this.id = id; + + intervalCheck = new IntervalCheck(); + intervalCheck.setInterval(60); + } + + public Double count(long value) { + long totalValue = total.incrementAndGet(); + long timesValue = times.incrementAndGet(); + long v = values.addAndGet(value); + + Double pass = intervalCheck.checkAndGet(); + if (pass != null) { + times.set(0); + values.set(0); + + Double tps = timesValue / pass; + + StringBuilder sb = new StringBuilder(); + sb.append(id); + sb.append(", tps:" + tps); + sb.append(", avg:" + ((double) v) / timesValue); + sb.append(", total:" + totalValue); + LOG.info(sb.toString()); + + return tps; + } + + return null; + } + + public void cleanup() { + + LOG.info(id + ", total:" + total); + } + + public IntervalCheck getIntervalCheck() { + return intervalCheck; + } + + /** + * @param args + */ + public static void main(String[] args) { + // TODO Auto-generated method stub + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/SmartThread.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/SmartThread.java new file mode 100755 index 000000000..1c37c1a9d --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/SmartThread.java @@ -0,0 +1,35 @@ +/** + * 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.utils; + +/** + * + * @author yannian + * + */ +public interface SmartThread { + public void start(); + + public void join() throws InterruptedException;; + + public void interrupt(); + + public Boolean isSleeping(); + + public void cleanup(); +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/SystemOperation.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/SystemOperation.java new file mode 100644 index 000000000..ba7547ba6 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/SystemOperation.java @@ -0,0 +1,76 @@ +/** + * 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.utils; + +import java.io.IOException; + +import org.apache.commons.io.IOUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SystemOperation { + + public static final Logger LOG = LoggerFactory + .getLogger(SystemOperation.class); + + public static boolean isRoot() throws IOException { + String result = SystemOperation.exec("echo $EUID").substring(0, 1); + return Integer.valueOf(result.substring(0, result.length())).intValue() == 0 ? true + : false; + }; + + public static void mount(String name, String target, String type, + String data) throws IOException { + StringBuilder sb = new StringBuilder(); + sb.append("mount -t ").append(type).append(" -o ").append(data) + .append(" ").append(name).append(" ").append(target); + SystemOperation.exec(sb.toString()); + } + + public static void umount(String name) throws IOException { + StringBuilder sb = new StringBuilder(); + sb.append("umount ").append(name); + SystemOperation.exec(sb.toString()); + } + + public static String exec(String cmd) throws IOException { + LOG.debug("Shell cmd: " + cmd); + Process process = + new ProcessBuilder(new String[] { "/bin/bash", "-c", cmd }) + .start(); + try { + process.waitFor(); + String output = IOUtils.toString(process.getInputStream()); + String errorOutput = IOUtils.toString(process.getErrorStream()); + LOG.debug("Shell Output: " + output); + if (errorOutput.length() != 0) { + LOG.error("Shell Error Output: " + errorOutput); + throw new IOException(errorOutput); + } + return output; + } catch (InterruptedException ie) { + throw new IOException(ie.toString()); + } + + } + + public static void main(String[] args) throws IOException { + SystemOperation.mount("test", "/cgroup/cpu", "cgroup", "cpu"); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/Thrift.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/Thrift.java new file mode 100755 index 000000000..c55751c54 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/Thrift.java @@ -0,0 +1,501 @@ +/** + * 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.utils; + +import java.lang.reflect.Constructor; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.generated.Bolt; +import backtype.storm.generated.ComponentCommon; +import backtype.storm.generated.ComponentObject; +import backtype.storm.generated.GlobalStreamId; +import backtype.storm.generated.Grouping; +import backtype.storm.generated.JavaObject; +import backtype.storm.generated.JavaObjectArg; +import backtype.storm.generated.NullStruct; +import backtype.storm.generated.StormTopology; +import backtype.storm.generated.StormTopology._Fields; +import backtype.storm.generated.StreamInfo; +import backtype.storm.generated.TopologyInitialStatus; +import backtype.storm.grouping.CustomStreamGrouping; +import backtype.storm.task.IBolt; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.cluster.StormStatus; +import com.alibaba.jstorm.daemon.nimbus.StatusType; + +/** + * Thrift utils + * + * 2012-03-28 + * + * @author yannian + * + */ +public class Thrift { + private static Logger LOG = LoggerFactory.getLogger(Thrift.class); + + public static StormStatus topologyInitialStatusToStormStatus( + TopologyInitialStatus tStatus) { + if (tStatus.equals(TopologyInitialStatus.ACTIVE)) { + return new StormStatus(StatusType.active); + } else { + return new StormStatus(StatusType.inactive); + } + } + + public static CustomStreamGrouping instantiateJavaObject(JavaObject obj) { + + List args = obj.get_args_list(); + Class[] paraTypes = new Class[args.size()]; + Object[] paraValues = new Object[args.size()]; + for (int i = 0; i < args.size(); i++) { + JavaObjectArg arg = args.get(i); + paraValues[i] = arg.getFieldValue(); + + if (arg.getSetField().equals(JavaObjectArg._Fields.INT_ARG)) { + paraTypes[i] = Integer.class; + } else if (arg.getSetField().equals(JavaObjectArg._Fields.LONG_ARG)) { + paraTypes[i] = Long.class; + } else if (arg.getSetField().equals( + JavaObjectArg._Fields.STRING_ARG)) { + paraTypes[i] = String.class; + } else if (arg.getSetField().equals(JavaObjectArg._Fields.BOOL_ARG)) { + paraTypes[i] = Boolean.class; + } else if (arg.getSetField().equals( + JavaObjectArg._Fields.BINARY_ARG)) { + paraTypes[i] = ByteBuffer.class; + } else if (arg.getSetField().equals( + JavaObjectArg._Fields.DOUBLE_ARG)) { + paraTypes[i] = Double.class; + } else { + paraTypes[i] = Object.class; + } + } + + try { + Class clas = Class.forName(obj.get_full_class_name()); + Constructor cons = clas.getConstructor(paraTypes); + return (CustomStreamGrouping) cons.newInstance(paraValues); + } catch (Exception e) { + LOG.error("instantiate_java_object fail", e); + } + + return null; + + } + + public static Grouping._Fields groupingType(Grouping grouping) { + return grouping.getSetField(); + } + + public static List fieldGrouping(Grouping grouping) { + if (!Grouping._Fields.FIELDS.equals(groupingType(grouping))) { + throw new IllegalArgumentException( + "Tried to get grouping fields from non fields grouping"); + } + + return grouping.get_fields(); + } + + public static boolean isGlobalGrouping(Grouping grouping) { + if (Grouping._Fields.FIELDS.equals(groupingType(grouping))) { + return fieldGrouping(grouping).isEmpty(); + } + + return false; + } + + public static int parallelismHint(ComponentCommon component_common) { + int phint = component_common.get_parallelism_hint(); + if (!component_common.is_set_parallelism_hint()) { + phint = 1; + } + return phint; + } + + public static StreamInfo directOutputFields(List fields) { + return new StreamInfo(fields, true); + } + + public static StreamInfo outputFields(List fields) { + return new StreamInfo(fields, false); + } + + public static Grouping mkFieldsGrouping(List fields) { + return Grouping.fields(fields); + } + + public static Grouping mkDirectGrouping() { + return Grouping.direct(new NullStruct()); + } + + private static ComponentCommon mkComponentcommon( + Map inputs, + HashMap output_spec, Integer parallelism_hint) { + ComponentCommon ret = new ComponentCommon(inputs, output_spec); + if (parallelism_hint != null) { + ret.set_parallelism_hint(parallelism_hint); + } + return ret; + } + + public static Bolt mkBolt(Map inputs, IBolt bolt, + HashMap output, Integer p) { + ComponentCommon common = mkComponentcommon(inputs, output, p); + byte[] boltSer = Utils.serialize(bolt); + ComponentObject component = ComponentObject.serialized_java(boltSer); + return new Bolt(component, common); + } + + public static StormTopology._Fields[] STORM_TOPOLOGY_FIELDS = null; + public static StormTopology._Fields[] SPOUT_FIELDS = { + StormTopology._Fields.SPOUTS, StormTopology._Fields.STATE_SPOUTS }; + static { + Set<_Fields> keys = StormTopology.metaDataMap.keySet(); + STORM_TOPOLOGY_FIELDS = new StormTopology._Fields[keys.size()]; + keys.toArray(STORM_TOPOLOGY_FIELDS); + } + + // + // private static HashMap mk_acker_inputs2( + // Map> inputs) { + // HashMap rtn = new HashMap(); + // for (Entry> e : inputs.entrySet()) { + // Object stream_id = e.getKey(); + // GlobalStreamId SID = null; + // if (stream_id instanceof Object[]) { + // String[] l = (String[]) stream_id; + // SID = new GlobalStreamId(l[0], l[1]); + // } else if (stream_id instanceof List) { + // List l = (List) stream_id; + // SID = new GlobalStreamId(l.get(0), l.get(1)); + // } else { + // SID = new GlobalStreamId((String) stream_id, + // Utils.DEFAULT_STREAM_ID); + // } + // + // rtn.put(SID, mk_fields_grouping(e.getValue())); + // } + // + // return rtn; + // } + // + // public static Bolt mk_ackerBolt(Map> inputs, + // IBolt bolt, HashMap> output, Integer p) { + // HashMap commonInputs=mk_inputs(inputs); + // ComponentCommon common = mk_component_common(commonInputs,output, p); + // return new Bolt(ComponentObject.serialized_java(Utils.serialize(bolt)), + // common); + // } + + // public static Map + // mk_output_StreamInfo(HashMap> output_spec) { + // Map> o_spec = output_spec; + // + // Map rtn = new HashMap(); + // + // for (Entry> e : o_spec.entrySet()) { + // List val = e.getValue(); + // rtn.put(e.getKey(), new StreamInfo((List) val, false)); + // + // } + // + // return rtn; + // } + + // public static Map + // mk_output_StreamInfo2(HashMap output_spec) { + // return output_spec; + // } + + // private static ComponentCommon mk_component_common( + // Map inputs, HashMap> + // output_spec, + // Integer parallelism_hint) { + // ComponentCommon ret = new ComponentCommon(inputs, + // mk_output_StreamInfo(output_spec)); + // if (parallelism_hint != null) { + // ret.set_parallelism_hint(parallelism_hint); + // } + // + // return ret; + // } + + // public static Object[] nimbus_client_and_conn(String host, Integer port) + // throws TTransportException { + // TFramedTransport transport = new TFramedTransport(new TSocket(host, + // port)); + // TBinaryProtocol prot = new TBinaryProtocol(transport); + // Nimbus.Client client = new Nimbus.Client(prot); + // transport.open(); + // Object[] rtn = { client, transport }; + // return rtn; + // } + // public static SpoutSpec mk_spout_spec(Object spout, Object outputs, + // Integer p) { + // Map inputs = new HashMap(); + // return new SpoutSpec(ComponentObject.serialized_java(Utils + // .serialize(spout)), mk_plain_component_common(inputs, outputs, + // p)); + // } + + // public static SpoutSpec mk_spout_spec(Object spout, Object outputs) { + // return mk_spout_spec(spout, outputs, null); + // } + + // public static Grouping mk_shuffle_grouping() { + // return Grouping.shuffle(new NullStruct()); + // } + + // public static Grouping mk_global_grouping() { + // return mk_fields_grouping(new ArrayList()); + // } + + // public static Grouping mk_all_grouping() { + // return Grouping.all(new NullStruct()); + // } + + // public static Grouping mk_none_grouping() { + // return Grouping.none(new NullStruct()); + // } + + // public static Object deserialized_component_object(ComponentObject obj) { + // if (obj.getSetField().equals(ComponentObject._Fields.SERIALIZED_JAVA)) { + // throw new RuntimeException( + // "Cannot deserialize non-java-serialized object"); + // } + // + // return Utils.deserialize(obj.get_serialized_java()); + // } + + // public static ComponentObject serialize_component_object(Object obj) { + // return ComponentObject.serialized_java(Utils.serialize(obj)); + // } + + // public static Grouping mk_grouping(Object grouping_spec) { + // if (grouping_spec == null) { + // return mk_none_grouping(); + // } + // + // if (grouping_spec instanceof Grouping) { + // return (Grouping) grouping_spec; + // } + // + // if (grouping_spec instanceof CustomStreamGrouping) { + // return Grouping.custom_serialized(Utils.serialize(grouping_spec)); + // } + // + // if (grouping_spec instanceof JavaObject) { + // return Grouping.custom_object((JavaObject) grouping_spec); + // } + // + // if (grouping_spec instanceof List) { + // return mk_fields_grouping((List) grouping_spec); + // } + // + // // + // if (GroupingConstants.shuffle.equals(grouping_spec)) { + // return mk_shuffle_grouping(); + // } + // + // if (GroupingConstants.none.equals(grouping_spec)) { + // return mk_none_grouping(); + // } + // + // if (GroupingConstants.all.equals(grouping_spec)) { + // return mk_all_grouping(); + // } + // + // if ("global".equals(grouping_spec)) { + // return mk_global_grouping(); + // } + // + // if (GroupingConstants.direct.equals(grouping_spec)) { + // return mk_direct_grouping(); + // } + // + // throw new IllegalArgumentException(grouping_spec + // + " is not a valid grouping"); + // } + + // public static Bolt mk_bolt_spec(Map> inputs, + // IBolt bolt, Object output) { + // return mk_bolt_spec(inputs, bolt, output, null); + // } + + // + // public class BoltSpecObj { + // public IRichBolt obj; + // public Map conf; + // public Map> inputs; + // public Integer p; + // } + // public static BoltSpecObj mk_bolt_spec(Map> inputs, + // IRichBolt bolt, Object... args) { + // Integer parallelism_hint = null; + // Integer p = null; + // Map conf = null; + // if (args.length >= 1) { + // parallelism_hint = (Integer) args[0]; + // } + // if (args.length >= 2) { + // p = (Integer) args[1]; + // } + // if (args.length >= 3) { + // conf = (Map) args[2]; + // } + // + // return mk_bolt_spec(inputs, bolt, parallelism_hint, p, conf); + // } + // public class SpoutSpecObj { + // public IRichSpout obj; + // public Map conf; + // public Integer p; + // } + // public static SpoutSpecObj mk_spout_spec(Map> + // inputs, + // IRichBolt bolt, Object... args) { + // Integer parallelism_hint = null; + // Integer p = null; + // Map conf = null; + // if (args.length >= 1) { + // parallelism_hint = (Integer) args[0]; + // } + // if (args.length >= 2) { + // p = (Integer) args[1]; + // } + // if (args.length >= 3) { + // conf = (Map) args[2]; + // } + // + // return mk_spout_spec(inputs, bolt, parallelism_hint, p, conf); + // } + // + // public static SpoutSpecObj mk_spout_spec(IRichSpout bolt, + // Integer parallelism_hint, Integer p, Map conf) { + // if (p != null) { + // parallelism_hint = p; + // } + // SpoutSpecObj rtn = new SpoutSpecObj(); + // rtn.p = parallelism_hint; + // rtn.obj = bolt; + // rtn.conf = conf; + // return rtn; + // } + // + // public static void add_inputs(BoltDeclarer declarer, + // Map> inputs) { + // HashMap getinputs = mk_inputs(inputs); + // for (Entry e : getinputs.entrySet()) { + // declarer.grouping(e.getKey(), e.getValue()); + // } + // } + + // public static StormTopology mk_topology( + // HashMap spout_map, + // HashMap bolt_map) { + // TopologyBuilder builder = new TopologyBuilder(); + // for (Entry e : spout_map.entrySet()) { + // String id = e.getKey(); + // SpoutSpecObj spout_obj = e.getValue(); + // SpoutDeclarer dec = builder + // .setSpout(id, spout_obj.obj, spout_obj.p); + // if (spout_obj.conf != null) { + // dec.addConfigurations(spout_obj.conf); + // } + // } + // + // for (Entry e : bolt_map.entrySet()) { + // String id = e.getKey(); + // BoltSpecObj bolt = e.getValue(); + // BoltDeclarer dec = builder.setBolt(id, bolt.obj, bolt.p); + // if (bolt.conf != null) { + // dec.addConfigurations(bolt.conf); + // } + // add_inputs(dec, bolt.inputs); + // } + // + // return builder.createTopology(); + // } + + // public static BoltSpecObj mk_bolt_spec(Map> inputs, + // IRichBolt bolt, Integer parallelism_hint, Integer p, Map conf) { + // if (p != null) { + // parallelism_hint = p; + // } + // BoltSpecObj rtn = new BoltSpecObj(); + // rtn.inputs = inputs; + // rtn.p = parallelism_hint; + // rtn.obj = bolt; + // rtn.conf = conf; + // return rtn; + // } + + // public static BoltSpecObj mk_shellbolt_spec( + // Map> inputs, String command, String script, + // Object output_spec, Object... args) { + // Integer parallelism_hint = null; + // Integer p = null; + // Map conf = null; + // if (args.length >= 1) { + // parallelism_hint = (Integer) args[0]; + // } + // if (args.length >= 2) { + // p = (Integer) args[1]; + // } + // if (args.length >= 3) { + // conf = (Map) args[2]; + // } + // + // return mk_shellbolt_spec(inputs, command, script, output_spec, + // parallelism_hint, p, conf); + // } + + // public static BoltSpecObj mk_shellbolt_spec( + // Map> inputs, String command, String script, + // Object output_spec, Integer parallelism_hint, Integer p, Map conf) { + // return mk_bolt_spec(inputs, new RichShellBolt(script, command, + // mk_output_spec(output_spec)), parallelism_hint, p, conf); + // } + + // public static Bolt mk_spout_spec(Map inputs, + // IComponent bolt, Object output) { + // return mk_spout_spec(inputs, bolt, output, null); + // } + + // public static Bolt mk_spout_spec(Map inputs, + // IComponent bolt, Object output, Integer p) { + // ComponentCommon common = mk_plain_component_common(inputs, output, p); + // + // return new Bolt(ComponentObject.serialized_java(Utils.serialize(bolt)), + // common); + // } + + // public static String COORD_STREAM = Constants.COORDINATED_STREAM_ID; + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/TimeCacheMap.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/TimeCacheMap.java new file mode 100755 index 000000000..c56e30790 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/TimeCacheMap.java @@ -0,0 +1,173 @@ +/** + * 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.utils; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.Map; +import java.util.Map.Entry; + +/** + * Expires keys that have not been updated in the configured number of seconds. + * The algorithm used will take between expirationSecs and expirationSecs * (1 + + * 1 / (numBuckets-1)) to actually expire the message. + * + * get, put, remove, containsKey, and size take O(numBuckets) time to run. + * + * + */ +public class TimeCacheMap implements TimeOutMap { + // this default ensures things expire at most 50% past the expiration time + private static final int DEFAULT_NUM_BUCKETS = 3; + + private LinkedList> _buckets; + + private final Object _lock = new Object(); + private Thread _cleaner; + private ExpiredCallback _callback; + + public TimeCacheMap(int expirationSecs, int numBuckets, + ExpiredCallback callback) { + if (numBuckets < 2) { + throw new IllegalArgumentException("numBuckets must be >= 2"); + } + _buckets = new LinkedList>(); + for (int i = 0; i < numBuckets; i++) { + _buckets.add(new HashMap()); + } + + _callback = callback; + final long expirationMillis = expirationSecs * 1000L; + final long sleepTime = expirationMillis / (numBuckets - 1); + _cleaner = new Thread(new Runnable() { + public void run() { + + while (true) { + Map dead = null; + JStormUtils.sleepMs(sleepTime); + synchronized (_lock) { + dead = _buckets.removeLast(); + _buckets.addFirst(new HashMap()); + } + if (_callback != null) { + for (Entry entry : dead.entrySet()) { + _callback.expire(entry.getKey(), entry.getValue()); + } + } + } + } + }); + _cleaner.setDaemon(true); + _cleaner.start(); + } + + public TimeCacheMap(int expirationSecs, ExpiredCallback callback) { + this(expirationSecs, DEFAULT_NUM_BUCKETS, callback); + } + + public TimeCacheMap(int expirationSecs) { + this(expirationSecs, DEFAULT_NUM_BUCKETS); + } + + public TimeCacheMap(int expirationSecs, int numBuckets) { + this(expirationSecs, numBuckets, null); + } + + @Override + public boolean containsKey(K key) { + synchronized (_lock) { + for (HashMap bucket : _buckets) { + if (bucket.containsKey(key)) { + return true; + } + } + return false; + } + } + + @Override + public V get(K key) { + synchronized (_lock) { + for (HashMap bucket : _buckets) { + if (bucket.containsKey(key)) { + return bucket.get(key); + } + } + return null; + } + } + + @Override + public void putHead(K key, V value) { + synchronized (_lock) { + _buckets.getFirst().put(key, value); + } + } + + @Override + public void put(K key, V value) { + synchronized (_lock) { + Iterator> it = _buckets.iterator(); + HashMap bucket = it.next(); + bucket.put(key, value); + while (it.hasNext()) { + bucket = it.next(); + bucket.remove(key); + } + } + } + + @Override + public Object remove(K key) { + synchronized (_lock) { + for (HashMap bucket : _buckets) { + if (bucket.containsKey(key)) { + return bucket.remove(key); + } + } + return null; + } + } + + @Override + public int size() { + synchronized (_lock) { + int size = 0; + for (HashMap bucket : _buckets) { + size += bucket.size(); + } + return size; + } + } + + public void cleanup() { + _cleaner.interrupt(); + } + + public Map buildMap() { + Map ret = new HashMap(); + synchronized (_lock) { + for (HashMap bucket : _buckets) { + ret.putAll(bucket); + } + return ret; + } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/TimeCacheQueue.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/TimeCacheQueue.java new file mode 100755 index 000000000..84683104c --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/TimeCacheQueue.java @@ -0,0 +1,186 @@ +/** + * 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.utils; + +import java.util.Iterator; +import java.util.LinkedList; +import java.util.concurrent.LinkedBlockingDeque; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Expires keys that have not been updated in the configured number of seconds. + * The algorithm used will take between expirationSecs and expirationSecs * (1 + + * 1 / (numBuckets-1)) to actually expire the message. + * + * get, put, remove, containsKey, and size take O(numBuckets) time to run. + * + * The advantage of this design is that the expiration thread only locks the + * object for O(1) time, meaning the object is essentially always available for + * poll/offer + */ +public class TimeCacheQueue { + // this default ensures things expire at most 50% past the expiration time + public static final int DEFAULT_NUM_BUCKETS = 3; + + public static interface ExpiredCallback { + public void expire(K entry); + } + + public static class DefaultExpiredCallback implements ExpiredCallback { + protected static final Logger LOG = LoggerFactory + .getLogger(TimeCacheQueue.DefaultExpiredCallback.class); + + protected String queueName; + + public DefaultExpiredCallback(String queueName) { + this.queueName = queueName; + } + + public void expire(K entry) { + LOG.info("TimeCacheQueue " + queueName + " entry:" + entry + + ", timeout"); + } + } + + protected LinkedList> _buckets; + + protected final Object _lock = new Object(); + protected Thread _cleaner; + protected ExpiredCallback _callback; + + public TimeCacheQueue(int expirationSecs, int numBuckets, + ExpiredCallback callback) { + if (numBuckets < 2) { + throw new IllegalArgumentException("numBuckets must be >= 2"); + } + _buckets = new LinkedList>(); + for (int i = 0; i < numBuckets; i++) { + _buckets.add(new LinkedBlockingDeque()); + } + + _callback = callback; + final long expirationMillis = expirationSecs * 1000L; + final long sleepTime = expirationMillis / (numBuckets - 1); + _cleaner = new Thread(new Runnable() { + public void run() { + try { + while (true) { + LinkedBlockingDeque dead = null; + + Thread.sleep(sleepTime); + + synchronized (_lock) { + dead = _buckets.removeLast(); + _buckets.addFirst(new LinkedBlockingDeque()); + } + if (_callback != null) { + for (K entry : dead) { + _callback.expire(entry); + } + } + } + } catch (InterruptedException ex) { + + } + } + }); + _cleaner.setDaemon(true); + _cleaner.start(); + } + + public TimeCacheQueue(int expirationSecs, ExpiredCallback callback) { + this(expirationSecs, DEFAULT_NUM_BUCKETS, callback); + } + + public TimeCacheQueue(int expirationSecs) { + this(expirationSecs, DEFAULT_NUM_BUCKETS, null); + } + + public TimeCacheQueue(int expirationSecs, int numBuckets) { + this(expirationSecs, numBuckets, null); + } + + public boolean containsKey(K entry) { + synchronized (_lock) { + for (LinkedBlockingDeque bucket : _buckets) { + if (bucket.contains(entry)) { + return true; + } + } + return false; + } + } + + public K poll() { + synchronized (_lock) { + Iterator> itor = + _buckets.descendingIterator(); + while (itor.hasNext()) { + LinkedBlockingDeque bucket = itor.next(); + K entry = bucket.poll(); + if (entry != null) { + return entry; + } + } + + return null; + } + } + + public void offer(K entry) { + synchronized (_lock) { + LinkedBlockingDeque bucket = _buckets.getFirst(); + + bucket.offer(entry); + } + } + + public void remove(K entry) { + synchronized (_lock) { + for (LinkedBlockingDeque bucket : _buckets) { + if (bucket.contains(entry)) { + bucket.remove(entry); + return; + } + } + return; + } + } + + public int size() { + synchronized (_lock) { + int size = 0; + for (LinkedBlockingDeque bucket : _buckets) { + size += bucket.size(); + } + return size; + } + } + + @Override + protected void finalize() throws Throwable { + try { + _cleaner.interrupt(); + } finally { + super.finalize(); + } + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/TimeFormat.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/TimeFormat.java new file mode 100755 index 000000000..a5c189fed --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/TimeFormat.java @@ -0,0 +1,219 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.utils; + +import java.text.SimpleDateFormat; +import java.util.Calendar; +import java.util.Date; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * + * @author longda + * + */ +public class TimeFormat { + public static Logger log = LoggerFactory.getLogger(TimeFormat.class); + + public static final long ONE_SECOND_MILLISECONDS = 1000; + + public static final long ONE_MINUTE_SECONDS = 60; + + public static final long ONE_HOUR_MINUTES = 60; + + public static final long ONE_DAY_HOURS = 24; + + public static final long ONE_MINUTE_MILLISECONDS = ONE_MINUTE_SECONDS + * ONE_SECOND_MILLISECONDS; + + public static final long ONE_HOUR_MILLISECONDS = ONE_HOUR_MINUTES + * ONE_MINUTE_MILLISECONDS; + + public static final long ONE_DAY_MILLISECONDS = ONE_DAY_HOURS + * ONE_HOUR_MILLISECONDS; + + public static Date convertDate(String dateStr, String format) { + Date date = null; + try { + if (format != null) { + SimpleDateFormat simpleDateFormat = + new SimpleDateFormat(format); + date = simpleDateFormat.parse(dateStr); + } else { + date = new Date(dateStr); + } + + } catch (Exception ex) { + log.error("Failed to convert " + dateStr + " to Date, format:" + + format); + return null; + } + return date; + } + + public static String convertStr(Date date, String format) { + String ret = null; + try { + + SimpleDateFormat sdf = new SimpleDateFormat(format); + + ret = sdf.format(date); + + } catch (Exception e) { + log.error("Failed to convert " + date + " to String, format:" + + format); + return null; + } + return ret; + } + + public static Date getYear(String dateStr) { + return convertDate(dateStr, "yyyy"); + } + + public static String getYear(Date date) { + return convertStr(date, "yyyy"); + } + + public static Date getMonth(String dateStr) { + return convertDate(dateStr, "yyyyMM"); + } + + public static String getMonth(Date date) { + return convertStr(date, "yyyyMM"); + } + + public static Date getDay(String dateStr) { + return convertDate(dateStr, "yyyyMMdd"); + } + + public static String getDay(Date date) { + return convertStr(date, "yyyyMMdd"); + } + + public static Date getHour(String dateStr) { + return convertDate(dateStr, "yyyyMMddHH"); + } + + public static String getHour(Date date) { + return convertStr(date, "yyyyMMddHH"); + } + + public static Date getMinute(String dateStr) { + return convertDate(dateStr, "yyyyMMddHHmm"); + } + + public static String getMinute(Date date) { + return convertStr(date, "yyyyMMddHHmm"); + } + + public static Date getSecond(String dateStr) { + return convertDate(dateStr, "yyyyMMddHHmmss"); + } + + public static String getSecond(Date date) { + return convertStr(date, "yyyyMMddHHmmss"); + } + + public static String getHourMin(String dateStr) { + Date date = convertDate(dateStr, null); + if (date == null) { + return null; + } + + return getHourMin(date); + } + + public static String getHourMin(Date date) { + String output = null; + try { + SimpleDateFormat sdf = new SimpleDateFormat("HH:mm"); + output = sdf.format(date); + } catch (Exception e) { + return null; + } + return output; + } + + public static Date getToday() { + Date now = new Date(); + + String todayStr = getDay(now); + + return getDay(todayStr); + } + + public static Date getYesterday() { + Date now = new Date(); + + Calendar yesterdayCal = Calendar.getInstance(); + yesterdayCal.setTime(now); + yesterdayCal.add(Calendar.DATE, -1); + + String yesterdayStr = getDay(yesterdayCal.getTime()); + + return getDay(yesterdayStr); + } + + /** + * get the days number pass from 1970-00-00 + * + * @return + */ + public static long getDayNum(Date date) { + long passMs = date.getTime() + (8 * 1000 * 60 * 60); + + return (passMs / 1000 / 60 / 60 / 24); + } + + /** + * @param args + */ + public static void main(String[] args) { + // TODO Auto-generated method stub + + Date date = new Date(); + + String dateStr = getDay(date); + + Date newDate = getDay(dateStr); + + System.out.println("new date:" + newDate); + + Date current = new Date(); + Calendar tomorrow = Calendar.getInstance(); + + tomorrow.setTime(current); + tomorrow.add(Calendar.DATE, 1); + tomorrow.set(Calendar.AM_PM, Calendar.AM); + tomorrow.set(Calendar.HOUR, 2); + tomorrow.set(Calendar.MINUTE, 0); + Date startTime = tomorrow.getTime(); + + long hourdiff = + (startTime.getTime() - current.getTime()) + / ONE_HOUR_MILLISECONDS; + + System.out.println("Current:" + current + ", tomorrow" + startTime + + ", diff hour" + hourdiff); + + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/TimeOutMap.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/TimeOutMap.java new file mode 100755 index 000000000..76bd75629 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/TimeOutMap.java @@ -0,0 +1,34 @@ +/** + * 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.utils; + +public interface TimeOutMap { + + public boolean containsKey(K key); + + public V get(K key); + + public void putHead(K key, V value); + + public void put(K key, V value); + + public Object remove(K key); + + public int size(); + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/utils/TimeUtils.java b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/TimeUtils.java new file mode 100755 index 000000000..8c9bd3db4 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/utils/TimeUtils.java @@ -0,0 +1,51 @@ +/** + * 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.utils; + +import backtype.storm.utils.Time; + +/** + * Time utils + * + * @author yannian + * + */ +public class TimeUtils { + + /** + * Take care of int overflow + * + * @return + */ + public static int current_time_secs() { + return (int) (Time.currentTimeMillis() / 1000); + } + + /** + * Take care of int overflow + * + * @return + */ + public static int time_delta(int time_secs) { + return current_time_secs() - time_secs; + } + + public static long time_delta_ms(long time_ms) { + return System.currentTimeMillis() - time_ms; + } +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/zk/Factory.java b/jstorm-core/src/main/java/com/alibaba/jstorm/zk/Factory.java new file mode 100755 index 000000000..d6a712691 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/zk/Factory.java @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.zk; + +import java.io.IOException; +import java.net.InetSocketAddress; + +import org.apache.zookeeper.server.NIOServerCnxnFactory; +import org.apache.zookeeper.server.ZooKeeperServer; + +public class Factory extends NIOServerCnxnFactory { + + public Factory(InetSocketAddress addr, int maxcc) throws IOException { + super(); + this.configure(addr, maxcc); + } + + public ZooKeeperServer getZooKeeperServer() { + return zkServer; + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/zk/ZkCreateModes.java b/jstorm-core/src/main/java/com/alibaba/jstorm/zk/ZkCreateModes.java new file mode 100755 index 000000000..89d4fe36c --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/zk/ZkCreateModes.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.zk; + +import java.util.HashMap; + +import org.apache.zookeeper.CreateMode; + +public class ZkCreateModes { + + private static HashMap map; + + static { + map = new HashMap(); + map.put(CreateMode.EPHEMERAL, ":ephemeral"); + map.put(CreateMode.PERSISTENT, ":persistent"); + } + + public static String getStateName(CreateMode mode) { + return map.get(mode); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/zk/ZkEventTypes.java b/jstorm-core/src/main/java/com/alibaba/jstorm/zk/ZkEventTypes.java new file mode 100755 index 000000000..09c25a5f7 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/zk/ZkEventTypes.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.zk; + +import java.util.HashMap; + +import org.apache.zookeeper.Watcher; + +public class ZkEventTypes { + + private static HashMap map; + + static { + map = new HashMap(); + + map.put(Watcher.Event.EventType.None, ":none"); + map.put(Watcher.Event.EventType.NodeCreated, ":node-created"); + map.put(Watcher.Event.EventType.NodeDeleted, ":node-deleted"); + map.put(Watcher.Event.EventType.NodeDataChanged, ":node-data-changed"); + map.put(Watcher.Event.EventType.NodeChildrenChanged, + ":node-children-changed"); + + } + + public static String getStateName(Watcher.Event.EventType type) { + return map.get(type); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/zk/ZkKeeperStates.java b/jstorm-core/src/main/java/com/alibaba/jstorm/zk/ZkKeeperStates.java new file mode 100755 index 000000000..67b87bf0e --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/zk/ZkKeeperStates.java @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.zk; + +import java.util.HashMap; + +import org.apache.zookeeper.Watcher; + +public class ZkKeeperStates { + + private static HashMap map; + + static { + map = new HashMap(); + + map.put(Watcher.Event.KeeperState.AuthFailed, ":auth-failed"); + map.put(Watcher.Event.KeeperState.SyncConnected, ":connected"); + map.put(Watcher.Event.KeeperState.Disconnected, ":disconnected"); + map.put(Watcher.Event.KeeperState.Expired, ":expired"); + } + + public static String getStateName(Watcher.Event.KeeperState state) { + return map.get(state); + } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/zk/ZkTool.java b/jstorm-core/src/main/java/com/alibaba/jstorm/zk/ZkTool.java new file mode 100755 index 000000000..b7267811a --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/zk/ZkTool.java @@ -0,0 +1,218 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.zk; + +import java.util.List; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.cluster.Cluster; +import com.alibaba.jstorm.cluster.ClusterState; +import com.alibaba.jstorm.cluster.DistributedClusterState; +import com.google.common.collect.Maps; + +public class ZkTool { + private static Logger LOG = LoggerFactory.getLogger(ZkTool.class); + + public static final String READ_CMD = "read"; + + public static final String RM_CMD = "rm"; + + public static void usage() { + LOG.info("Read ZK node's data, please do as following:"); + LOG.info(ZkTool.class.getName() + " read zkpath"); + + LOG.info("\nDelete topology backup assignment, please do as following:"); + LOG.info(ZkTool.class.getName() + " rm topologyname"); + } + + public static String getData(DistributedClusterState zkClusterState, + String path) throws Exception { + byte[] data = zkClusterState.get_data(path, false); + if (data == null || data.length == 0) { + return null; + } + + Object obj = Utils.deserialize(data, null); + + return obj.toString(); + } + + public static void readData(String path) { + + DistributedClusterState zkClusterState = null; + + try { + conf.put(Config.STORM_ZOOKEEPER_ROOT, "/"); + + zkClusterState = new DistributedClusterState(conf); + + String data = getData(zkClusterState, path); + if (data == null) { + LOG.info("No data of " + path); + } + + StringBuilder sb = new StringBuilder(); + + sb.append("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!\n\n"); + sb.append("Zk node " + path + "\n"); + sb.append("Readable data:" + data + "\n"); + sb.append("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!\n\n"); + + LOG.info(sb.toString()); + + } catch (Exception e) { + if (zkClusterState == null) { + LOG.error("Failed to connect ZK ", e); + } else { + LOG.error("Failed to read data " + path + "\n", e); + } + } finally { + if (zkClusterState != null) { + zkClusterState.close(); + } + } + } + + public static void rmBakTopology(String topologyName) { + + DistributedClusterState zkClusterState = null; + + try { + + zkClusterState = new DistributedClusterState(conf); + + String path = Cluster.ASSIGNMENTS_BAK_SUBTREE; + List bakTopologys = + zkClusterState.get_children(path, false); + + for (String tid : bakTopologys) { + if (tid.equals(topologyName)) { + LOG.info("Find backup " + topologyName); + + String topologyPath = + Cluster.assignment_bak_path(topologyName); + zkClusterState.delete_node(topologyPath); + + LOG.info("Successfully delete topology " + topologyName + + " backup Assignment"); + + return; + } + } + + LOG.info("No backup topology " + topologyName + " Assignment"); + + } catch (Exception e) { + if (zkClusterState == null) { + LOG.error("Failed to connect ZK ", e); + } else { + LOG.error("Failed to delete old topology " + topologyName + + "\n", e); + } + } finally { + if (zkClusterState != null) { + zkClusterState.close(); + } + } + + } + + private static Map conf; + + /** + * @param args + * @throws Exception + */ + public static void main(String[] args) throws Exception { + // TODO Auto-generated method stub + + if (args.length < 2) { + LOG.info("Invalid parameter"); + usage(); + return; + } + + conf = Utils.readStormConfig(); + + if (args[0].equalsIgnoreCase(READ_CMD)) { + + readData(args[1]); + + } else if (args[0].equalsIgnoreCase(RM_CMD)) { + rmBakTopology(args[1]); + } + + } + + /*******************************************************************/ + + @SuppressWarnings("rawtypes") + public static ClusterState mk_distributed_cluster_state(Map _conf) + throws Exception { + return new DistributedClusterState(_conf); + } + + public static Map get_followers(ClusterState cluster_state) + throws Exception { + Map ret = Maps.newHashMap(); + List followers = + cluster_state.get_children(Cluster.NIMBUS_SLAVE_SUBTREE, false); + if (followers == null || followers.size() == 0) { + return ret; + } + for (String follower : followers) { + if (follower != null) { + String uptime = + new String(cluster_state.get_data( + Cluster.NIMBUS_SLAVE_SUBTREE + + Cluster.ZK_SEPERATOR + follower, + false)); + ret.put(follower, uptime); + } + } + return ret; + } + + // public static List get_follower_hosts(ClusterState cluster_state) + // throws Exception { + // List followers = cluster_state.get_children( + // ZkConstant.NIMBUS_SLAVE_SUBTREE, false); + // if (followers == null || followers.size() == 0) { + // return Lists.newArrayList(); + // } + // return followers; + // } + // + // public static List get_follower_hbs(ClusterState cluster_state) + // throws Exception { + // List ret = Lists.newArrayList(); + // List followers = get_follower_hosts(cluster_state); + // for (String follower : followers) { + // ret.add(new String(cluster_state.get_data(ZkConstant.NIMBUS_SLAVE_SUBTREE + // + ZkConstant.ZK_SEPERATOR + follower, false))); + // } + // return ret; + // } + +} diff --git a/jstorm-core/src/main/java/com/alibaba/jstorm/zk/Zookeeper.java b/jstorm-core/src/main/java/com/alibaba/jstorm/zk/Zookeeper.java new file mode 100755 index 000000000..ed9989d87 --- /dev/null +++ b/jstorm-core/src/main/java/com/alibaba/jstorm/zk/Zookeeper.java @@ -0,0 +1,237 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.zk; + +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.List; +import java.util.Map; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.api.CuratorEvent; +import org.apache.curator.framework.api.CuratorEventType; +import org.apache.curator.framework.api.CuratorListener; +import org.apache.curator.framework.api.UnhandledErrorListener; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.callback.DefaultWatcherCallBack; +import com.alibaba.jstorm.callback.WatcherCallBack; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.PathUtils; + +/** + * ZK simple wrapper + * + * @author yannian + * + */ +public class Zookeeper { + + private static Logger LOG = LoggerFactory.getLogger(Zookeeper.class); + + public CuratorFramework mkClient(Map conf, List servers, + Object port, String root) { + return mkClient(conf, servers, port, root, new DefaultWatcherCallBack()); + } + + /** + * connect ZK, register Watch/unhandle Watch + * + * @return + */ + public CuratorFramework mkClient(Map conf, List servers, + Object port, String root, final WatcherCallBack watcher) { + + CuratorFramework fk = Utils.newCurator(conf, servers, port, root); + + fk.getCuratorListenable().addListener(new CuratorListener() { + @Override + public void eventReceived(CuratorFramework _fk, CuratorEvent e) + throws Exception { + if (e.getType().equals(CuratorEventType.WATCHED)) { + WatchedEvent event = e.getWatchedEvent(); + + watcher.execute(event.getState(), event.getType(), + event.getPath()); + } + + } + }); + + fk.getUnhandledErrorListenable().addListener( + new UnhandledErrorListener() { + @Override + public void unhandledError(String msg, Throwable error) { + String errmsg = + "Unrecoverable Zookeeper error, halting process: " + + msg; + LOG.error(errmsg, error); + JStormUtils.halt_process(1, + "Unrecoverable Zookeeper error"); + + } + }); + fk.start(); + return fk; + } + + public String createNode(CuratorFramework zk, String path, byte[] data, + org.apache.zookeeper.CreateMode mode) throws Exception { + + String npath = PathUtils.normalize_path(path); + + return zk.create().withMode(mode).withACL(ZooDefs.Ids.OPEN_ACL_UNSAFE) + .forPath(npath, data); + } + + public String createNode(CuratorFramework zk, String path, byte[] data) + throws Exception { + return createNode(zk, path, data, + org.apache.zookeeper.CreateMode.PERSISTENT); + } + + public boolean existsNode(CuratorFramework zk, String path, boolean watch) + throws Exception { + Stat stat = null; + if (watch) { + stat = + zk.checkExists().watched() + .forPath(PathUtils.normalize_path(path)); + } else { + stat = zk.checkExists().forPath(PathUtils.normalize_path(path)); + } + return stat != null; + } + + public void deleteNode(CuratorFramework zk, String path) throws Exception { + zk.delete().forPath(PathUtils.normalize_path(path)); + } + + public void mkdirs(CuratorFramework zk, String path) throws Exception { + + String npath = PathUtils.normalize_path(path); + + // the node is "/" + if (npath.equals("/")) { + return; + } + + // the node exist + if (existsNode(zk, npath, false)) { + return; + } + + mkdirs(zk, PathUtils.parent_path(npath)); + try { + createNode(zk, npath, JStormUtils.barr((byte) 7), + org.apache.zookeeper.CreateMode.PERSISTENT); + } catch (KeeperException e) { + ;// this can happen when multiple clients doing mkdir at same + // time + LOG.warn("zookeeper mkdirs for path" + path, e); + + } + + } + + public byte[] getData(CuratorFramework zk, String path, boolean watch) + throws Exception { + String npath = PathUtils.normalize_path(path); + try { + if (existsNode(zk, npath, watch)) { + if (watch) { + return zk.getData().watched().forPath(npath); + } else { + return zk.getData().forPath(npath); + } + } + } catch (KeeperException e) { + LOG.error("zookeeper getdata for path" + path, e); + } + + return null; + } + + public List getChildren(CuratorFramework zk, String path, + boolean watch) throws Exception { + + String npath = PathUtils.normalize_path(path); + + if (watch) { + return zk.getChildren().watched().forPath(npath); + } else { + return zk.getChildren().forPath(npath); + } + } + + public Stat setData(CuratorFramework zk, String path, byte[] data) + throws Exception { + String npath = PathUtils.normalize_path(path); + return zk.setData().forPath(npath, data); + } + + public boolean exists(CuratorFramework zk, String path, boolean watch) + throws Exception { + return existsNode(zk, path, watch); + } + + public void deletereRcursive(CuratorFramework zk, String path) + throws Exception { + + String npath = PathUtils.normalize_path(path); + + if (existsNode(zk, npath, false)) { + + List childs = getChildren(zk, npath, false); + + for (String child : childs) { + + String childFullPath = PathUtils.full_path(npath, child); + + deletereRcursive(zk, childFullPath); + } + + deleteNode(zk, npath); + } + } + + public static Factory mkInprocessZookeeper(String localdir, int port) + throws IOException, InterruptedException { + LOG.info("Starting inprocess zookeeper at port " + port + " and dir " + + localdir); + File localfile = new File(localdir); + ZooKeeperServer zk = new ZooKeeperServer(localfile, localfile, 2000); + Factory factory = new Factory(new InetSocketAddress(port), 0); + factory.startup(zk); + return factory; + } + + public void shutdownInprocessZookeeper(Factory handle) { + handle.shutdown(); + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/JoinType.java b/jstorm-core/src/main/java/storm/trident/JoinType.java new file mode 100755 index 000000000..a90869aa5 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/JoinType.java @@ -0,0 +1,30 @@ +/** + * 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 storm.trident; + +import java.util.Arrays; +import java.util.List; + +public enum JoinType { + INNER, + OUTER; + + public static List mixed(JoinType... types) { + return Arrays.asList(types); + } +} \ No newline at end of file diff --git a/jstorm-client/src/main/java/storm/trident/Stream.java b/jstorm-core/src/main/java/storm/trident/Stream.java old mode 100644 new mode 100755 similarity index 94% rename from jstorm-client/src/main/java/storm/trident/Stream.java rename to jstorm-core/src/main/java/storm/trident/Stream.java index 629cad03a..f4c515e6d --- a/jstorm-client/src/main/java/storm/trident/Stream.java +++ b/jstorm-core/src/main/java/storm/trident/Stream.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident; import backtype.storm.generated.Grouping; @@ -77,7 +94,7 @@ public Stream partitionBy(Fields fields) { } public Stream partition(CustomStreamGrouping partitioner) { - return partition(Grouping.custom_serialized(Utils.serialize(partitioner))); + return partition(Grouping.custom_serialized(Utils.javaSerialize(partitioner))); } public Stream shuffle() { diff --git a/jstorm-core/src/main/java/storm/trident/TridentState.java b/jstorm-core/src/main/java/storm/trident/TridentState.java new file mode 100755 index 000000000..056622c7a --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/TridentState.java @@ -0,0 +1,40 @@ +/** + * 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 storm.trident; + +import storm.trident.planner.Node; + + +public class TridentState { + TridentTopology _topology; + Node _node; + + protected TridentState(TridentTopology topology, Node node) { + _topology = topology; + _node = node; + } + + public Stream newValuesStream() { + return new Stream(_topology, _node.name, _node); + } + + public TridentState parallelismHint(int parallelism) { + _node.parallelismHint = parallelism; + return this; + } +} diff --git a/jstorm-client/src/main/java/storm/trident/TridentTopology.java b/jstorm-core/src/main/java/storm/trident/TridentTopology.java old mode 100644 new mode 100755 similarity index 97% rename from jstorm-client/src/main/java/storm/trident/TridentTopology.java rename to jstorm-core/src/main/java/storm/trident/TridentTopology.java index 7b4b00d01..3ed23867d --- a/jstorm-client/src/main/java/storm/trident/TridentTopology.java +++ b/jstorm-core/src/main/java/storm/trident/TridentTopology.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident; import backtype.storm.Config; @@ -11,6 +28,8 @@ import backtype.storm.topology.IRichSpout; import backtype.storm.tuple.Fields; import backtype.storm.utils.Utils; + +import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -647,7 +666,7 @@ private static Integer getFixedParallelism(Set groups) { private static boolean isIdentityPartition(PartitionNode n) { Grouping g = n.thriftGrouping; if(g.is_set_custom_serialized()) { - CustomStreamGrouping csg = (CustomStreamGrouping) Utils.deserialize(g.get_custom_serialized()); + CustomStreamGrouping csg = (CustomStreamGrouping) Utils.javaDeserialize(g.get_custom_serialized(), Serializable.class); return csg instanceof IdentityGrouping; } return false; @@ -706,7 +725,7 @@ private static Set externalGroupOutputs(Group g) { private static PartitionNode makeIdentityPartition(Node basis) { return new PartitionNode(basis.streamId, basis.name, basis.allOutputFields, - Grouping.custom_serialized(Utils.serialize(new IdentityGrouping()))); + Grouping.custom_serialized(Utils.javaSerialize(new IdentityGrouping()))); } diff --git a/jstorm-core/src/main/java/storm/trident/drpc/ReturnResultsReducer.java b/jstorm-core/src/main/java/storm/trident/drpc/ReturnResultsReducer.java new file mode 100755 index 000000000..849fb104d --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/drpc/ReturnResultsReducer.java @@ -0,0 +1,121 @@ +/** + * 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 storm.trident.drpc; + +import backtype.storm.Config; +import backtype.storm.drpc.DRPCInvocationsClient; +import backtype.storm.generated.DistributedRPCInvocations; +import backtype.storm.generated.AuthorizationException; +import backtype.storm.utils.ServiceRegistry; +import backtype.storm.utils.Utils; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.thrift.TException; +import org.apache.thrift.transport.TTransportException; +import org.json.simple.JSONValue; +import storm.trident.drpc.ReturnResultsReducer.ReturnResultsState; +import storm.trident.operation.MultiReducer; +import storm.trident.operation.TridentCollector; +import storm.trident.operation.TridentMultiReducerContext; +import storm.trident.tuple.TridentTuple; + + +public class ReturnResultsReducer implements MultiReducer { + public static class ReturnResultsState { + List results = new ArrayList(); + String returnInfo; + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this); + } + } + boolean local; + Map conf; + Map _clients = new HashMap(); + + + @Override + public void prepare(Map conf, TridentMultiReducerContext context) { + this.conf = conf; + local = conf.get(Config.STORM_CLUSTER_MODE).equals("local"); + } + + @Override + public ReturnResultsState init(TridentCollector collector) { + return new ReturnResultsState(); + } + + @Override + public void execute(ReturnResultsState state, int streamIndex, TridentTuple input, TridentCollector collector) { + if(streamIndex==0) { + state.returnInfo = input.getString(0); + } else { + state.results.add(input); + } + } + + @Override + public void complete(ReturnResultsState state, TridentCollector collector) { + // only one of the multireducers will receive the tuples + if(state.returnInfo!=null) { + String result = JSONValue.toJSONString(state.results); + Map retMap = (Map) JSONValue.parse(state.returnInfo); + final String host = (String) retMap.get("host"); + final int port = Utils.getInt(retMap.get("port")); + String id = (String) retMap.get("id"); + DistributedRPCInvocations.Iface client; + if(local) { + client = (DistributedRPCInvocations.Iface) ServiceRegistry.getService(host); + } else { + List server = new ArrayList() {{ + add(host); + add(port); + }}; + + if(!_clients.containsKey(server)) { + try { + _clients.put(server, new DRPCInvocationsClient(conf, host, port)); + } catch (TTransportException ex) { + throw new RuntimeException(ex); + } + } + client = _clients.get(server); + } + + try { + client.result(id, result); + } catch (AuthorizationException aze) { + collector.reportError(aze); + } catch(TException e) { + collector.reportError(e); + } + } + } + + @Override + public void cleanup() { + for(DRPCInvocationsClient c: _clients.values()) { + c.close(); + } + } + +} diff --git a/jstorm-client/src/main/java/storm/trident/fluent/ChainedAggregatorDeclarer.java b/jstorm-core/src/main/java/storm/trident/fluent/ChainedAggregatorDeclarer.java old mode 100644 new mode 100755 similarity index 89% rename from jstorm-client/src/main/java/storm/trident/fluent/ChainedAggregatorDeclarer.java rename to jstorm-core/src/main/java/storm/trident/fluent/ChainedAggregatorDeclarer.java index de8fe9c01..8040e8b5c --- a/jstorm-client/src/main/java/storm/trident/fluent/ChainedAggregatorDeclarer.java +++ b/jstorm-core/src/main/java/storm/trident/fluent/ChainedAggregatorDeclarer.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.fluent; import backtype.storm.tuple.Fields; diff --git a/jstorm-core/src/main/java/storm/trident/fluent/ChainedFullAggregatorDeclarer.java b/jstorm-core/src/main/java/storm/trident/fluent/ChainedFullAggregatorDeclarer.java new file mode 100755 index 000000000..0631a6bc5 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/fluent/ChainedFullAggregatorDeclarer.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.fluent; + +import backtype.storm.tuple.Fields; +import storm.trident.operation.Aggregator; +import storm.trident.operation.CombinerAggregator; +import storm.trident.operation.ReducerAggregator; + +public interface ChainedFullAggregatorDeclarer extends IChainedAggregatorDeclarer { + ChainedFullAggregatorDeclarer aggregate(Aggregator agg, Fields functionFields); + ChainedFullAggregatorDeclarer aggregate(Fields inputFields, Aggregator agg, Fields functionFields); + ChainedFullAggregatorDeclarer aggregate(CombinerAggregator agg, Fields functionFields); + ChainedFullAggregatorDeclarer aggregate(Fields inputFields, CombinerAggregator agg, Fields functionFields); + ChainedFullAggregatorDeclarer aggregate(ReducerAggregator agg, Fields functionFields); + ChainedFullAggregatorDeclarer aggregate(Fields inputFields, ReducerAggregator agg, Fields functionFields); +} diff --git a/jstorm-core/src/main/java/storm/trident/fluent/ChainedPartitionAggregatorDeclarer.java b/jstorm-core/src/main/java/storm/trident/fluent/ChainedPartitionAggregatorDeclarer.java new file mode 100755 index 000000000..df8578d7e --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/fluent/ChainedPartitionAggregatorDeclarer.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.fluent; + +import backtype.storm.tuple.Fields; +import storm.trident.operation.Aggregator; +import storm.trident.operation.CombinerAggregator; +import storm.trident.operation.ReducerAggregator; + +public interface ChainedPartitionAggregatorDeclarer extends IChainedAggregatorDeclarer { + ChainedPartitionAggregatorDeclarer partitionAggregate(Aggregator agg, Fields functionFields); + ChainedPartitionAggregatorDeclarer partitionAggregate(Fields inputFields, Aggregator agg, Fields functionFields); + ChainedPartitionAggregatorDeclarer partitionAggregate(CombinerAggregator agg, Fields functionFields); + ChainedPartitionAggregatorDeclarer partitionAggregate(Fields inputFields, CombinerAggregator agg, Fields functionFields); + ChainedPartitionAggregatorDeclarer partitionAggregate(ReducerAggregator agg, Fields functionFields); + ChainedPartitionAggregatorDeclarer partitionAggregate(Fields inputFields, ReducerAggregator agg, Fields functionFields); +} diff --git a/jstorm-core/src/main/java/storm/trident/fluent/GlobalAggregationScheme.java b/jstorm-core/src/main/java/storm/trident/fluent/GlobalAggregationScheme.java new file mode 100755 index 000000000..6fb3660dd --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/fluent/GlobalAggregationScheme.java @@ -0,0 +1,26 @@ +/** + * 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 storm.trident.fluent; + +import storm.trident.operation.impl.SingleEmitAggregator.BatchToPartition; + + +public interface GlobalAggregationScheme { + IAggregatableStream aggPartition(S stream); // how to partition for second stage of aggregation + BatchToPartition singleEmitPartitioner(); // return null if it's not single emit +} diff --git a/jstorm-client/src/main/java/storm/trident/fluent/GroupedStream.java b/jstorm-core/src/main/java/storm/trident/fluent/GroupedStream.java old mode 100644 new mode 100755 similarity index 88% rename from jstorm-client/src/main/java/storm/trident/fluent/GroupedStream.java rename to jstorm-core/src/main/java/storm/trident/fluent/GroupedStream.java index ad1e121f9..2d223fb0f --- a/jstorm-client/src/main/java/storm/trident/fluent/GroupedStream.java +++ b/jstorm-core/src/main/java/storm/trident/fluent/GroupedStream.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.fluent; import backtype.storm.tuple.Fields; diff --git a/jstorm-core/src/main/java/storm/trident/fluent/IAggregatableStream.java b/jstorm-core/src/main/java/storm/trident/fluent/IAggregatableStream.java new file mode 100755 index 000000000..9308f0513 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/fluent/IAggregatableStream.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.fluent; + +import backtype.storm.tuple.Fields; +import storm.trident.Stream; +import storm.trident.operation.Aggregator; +import storm.trident.operation.Function; +import storm.trident.operation.impl.SingleEmitAggregator.BatchToPartition; + +public interface IAggregatableStream { + IAggregatableStream each(Fields inputFields, Function function, Fields functionFields); + IAggregatableStream partitionAggregate(Fields inputFields, Aggregator agg, Fields functionFields); + Stream toStream(); + Fields getOutputFields(); +} diff --git a/jstorm-core/src/main/java/storm/trident/fluent/IChainedAggregatorDeclarer.java b/jstorm-core/src/main/java/storm/trident/fluent/IChainedAggregatorDeclarer.java new file mode 100755 index 000000000..001ea6835 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/fluent/IChainedAggregatorDeclarer.java @@ -0,0 +1,24 @@ +/** + * 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 storm.trident.fluent; + +import storm.trident.Stream; + +public interface IChainedAggregatorDeclarer { + Stream chainEnd(); +} diff --git a/jstorm-core/src/main/java/storm/trident/fluent/UniqueIdGen.java b/jstorm-core/src/main/java/storm/trident/fluent/UniqueIdGen.java new file mode 100755 index 000000000..37bb0adc4 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/fluent/UniqueIdGen.java @@ -0,0 +1,34 @@ +/** + * 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 storm.trident.fluent; + +public class UniqueIdGen { + int _streamCounter = 0; + + public String getUniqueStreamId() { + _streamCounter++; + return "s" + _streamCounter; + } + + int _stateCounter = 0; + + public String getUniqueStateId() { + _stateCounter++; + return "state" + _stateCounter; + } +} diff --git a/jstorm-client/src/main/java/storm/trident/graph/GraphGrouper.java b/jstorm-core/src/main/java/storm/trident/graph/GraphGrouper.java old mode 100644 new mode 100755 similarity index 79% rename from jstorm-client/src/main/java/storm/trident/graph/GraphGrouper.java rename to jstorm-core/src/main/java/storm/trident/graph/GraphGrouper.java index b10726904..94db07767 --- a/jstorm-client/src/main/java/storm/trident/graph/GraphGrouper.java +++ b/jstorm-core/src/main/java/storm/trident/graph/GraphGrouper.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.graph; import java.util.Collection; diff --git a/jstorm-core/src/main/java/storm/trident/graph/Group.java b/jstorm-core/src/main/java/storm/trident/graph/Group.java new file mode 100755 index 000000000..8ed0023af --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/graph/Group.java @@ -0,0 +1,87 @@ +/** + * 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 storm.trident.graph; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import org.jgrapht.DirectedGraph; +import storm.trident.planner.Node; +import storm.trident.util.IndexedEdge; +import storm.trident.util.TridentUtils; + + +public class Group { + public Set nodes = new HashSet(); + private DirectedGraph graph; + private String id; + + public Group(DirectedGraph graph, List nodes) { + init(graph); + this.nodes.addAll(nodes); + this.graph = graph; + } + + public Group(DirectedGraph graph, Node n) { + this(graph, Arrays.asList(n)); + } + + public Group(Group g1, Group g2) { + init(g1.graph); + nodes.addAll(g1.nodes); + nodes.addAll(g2.nodes); + } + + private void init(DirectedGraph graph) { + this.graph = graph; + this.id = UUID.randomUUID().toString(); + } + + public Set outgoingNodes() { + Set ret = new HashSet(); + for(Node n: nodes) { + ret.addAll(TridentUtils.getChildren(graph, n)); + } + return ret; + } + + public Set incomingNodes() { + Set ret = new HashSet(); + for(Node n: nodes) { + ret.addAll(TridentUtils.getParents(graph, n)); + } + return ret; + } + + @Override + public int hashCode() { + return id.hashCode(); + } + + @Override + public boolean equals(Object o) { + return id.equals(((Group) o).id); + } + + @Override + public String toString() { + return nodes.toString(); + } +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/Aggregator.java b/jstorm-core/src/main/java/storm/trident/operation/Aggregator.java new file mode 100755 index 000000000..7b49f02a0 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/Aggregator.java @@ -0,0 +1,26 @@ +/** + * 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 storm.trident.operation; + +import storm.trident.tuple.TridentTuple; + +public interface Aggregator extends Operation { + T init(Object batchId, TridentCollector collector); + void aggregate(T val, TridentTuple tuple, TridentCollector collector); + void complete(T val, TridentCollector collector); +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/Assembly.java b/jstorm-core/src/main/java/storm/trident/operation/Assembly.java new file mode 100755 index 000000000..55fed8f85 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/Assembly.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.operation; + +import storm.trident.Stream; + + +public interface Assembly { + Stream apply(Stream input); +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/BaseAggregator.java b/jstorm-core/src/main/java/storm/trident/operation/BaseAggregator.java new file mode 100755 index 000000000..6d582787c --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/BaseAggregator.java @@ -0,0 +1,23 @@ +/** + * 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 storm.trident.operation; + + +public abstract class BaseAggregator extends BaseOperation implements Aggregator { + +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/BaseFilter.java b/jstorm-core/src/main/java/storm/trident/operation/BaseFilter.java new file mode 100755 index 000000000..260cf61f6 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/BaseFilter.java @@ -0,0 +1,23 @@ +/** + * 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 storm.trident.operation; + + +public abstract class BaseFilter extends BaseOperation implements Filter { + +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/BaseFunction.java b/jstorm-core/src/main/java/storm/trident/operation/BaseFunction.java new file mode 100755 index 000000000..0d45450d2 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/BaseFunction.java @@ -0,0 +1,23 @@ +/** + * 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 storm.trident.operation; + + +public abstract class BaseFunction extends BaseOperation implements Function { + +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/BaseMultiReducer.java b/jstorm-core/src/main/java/storm/trident/operation/BaseMultiReducer.java new file mode 100755 index 000000000..608c64bd6 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/BaseMultiReducer.java @@ -0,0 +1,33 @@ +/** + * 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 storm.trident.operation; + +import java.util.Map; + +public abstract class BaseMultiReducer implements MultiReducer { + + @Override + public void prepare(Map conf, TridentMultiReducerContext context) { + } + + + @Override + public void cleanup() { + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/BaseOperation.java b/jstorm-core/src/main/java/storm/trident/operation/BaseOperation.java new file mode 100755 index 000000000..766d51dff --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/BaseOperation.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.operation; + +import java.util.Map; + +public class BaseOperation implements Operation { + + @Override + public void prepare(Map conf, TridentOperationContext context) { + } + + @Override + public void cleanup() { + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/CombinerAggregator.java b/jstorm-core/src/main/java/storm/trident/operation/CombinerAggregator.java new file mode 100755 index 000000000..1af43c585 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/CombinerAggregator.java @@ -0,0 +1,29 @@ +/** + * 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 storm.trident.operation; + +import java.io.Serializable; +import storm.trident.tuple.TridentTuple; + +// doesn't manipulate tuples (lists of stuff) so that things like aggregating into +// cassandra is cleaner (don't need lists everywhere, just store the single value there) +public interface CombinerAggregator extends Serializable { + T init(TridentTuple tuple); + T combine(T val1, T val2); + T zero(); +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/EachOperation.java b/jstorm-core/src/main/java/storm/trident/operation/EachOperation.java new file mode 100755 index 000000000..431538152 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/EachOperation.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.operation; + +public interface EachOperation extends Operation { + +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/Filter.java b/jstorm-core/src/main/java/storm/trident/operation/Filter.java new file mode 100755 index 000000000..abda33482 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/Filter.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.operation; + +import storm.trident.tuple.TridentTuple; + + +public interface Filter extends EachOperation { + boolean isKeep(TridentTuple tuple); +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/Function.java b/jstorm-core/src/main/java/storm/trident/operation/Function.java new file mode 100755 index 000000000..2532e002f --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/Function.java @@ -0,0 +1,24 @@ +/** + * 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 storm.trident.operation; + +import storm.trident.tuple.TridentTuple; + +public interface Function extends EachOperation { + void execute(TridentTuple tuple, TridentCollector collector); +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/GroupedMultiReducer.java b/jstorm-core/src/main/java/storm/trident/operation/GroupedMultiReducer.java new file mode 100755 index 000000000..213bfca74 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/GroupedMultiReducer.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.operation; + +import java.io.Serializable; +import java.util.Map; +import storm.trident.tuple.TridentTuple; + + +public interface GroupedMultiReducer extends Serializable { + void prepare(Map conf, TridentMultiReducerContext context); + T init(TridentCollector collector, TridentTuple group); + void execute(T state, int streamIndex, TridentTuple group, TridentTuple input, TridentCollector collector); + void complete(T state, TridentTuple group, TridentCollector collector); + void cleanup(); +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/MultiReducer.java b/jstorm-core/src/main/java/storm/trident/operation/MultiReducer.java new file mode 100755 index 000000000..ecbd90983 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/MultiReducer.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.operation; + +import java.io.Serializable; +import java.util.Map; +import storm.trident.tuple.TridentTuple; + + +public interface MultiReducer extends Serializable { + void prepare(Map conf, TridentMultiReducerContext context); + T init(TridentCollector collector); + void execute(T state, int streamIndex, TridentTuple input, TridentCollector collector); + void complete(T state, TridentCollector collector); + void cleanup(); +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/Operation.java b/jstorm-core/src/main/java/storm/trident/operation/Operation.java new file mode 100755 index 000000000..f945a4c18 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/Operation.java @@ -0,0 +1,26 @@ +/** + * 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 storm.trident.operation; + +import java.io.Serializable; +import java.util.Map; + +public interface Operation extends Serializable { + void prepare(Map conf, TridentOperationContext context); + void cleanup(); +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/ReducerAggregator.java b/jstorm-core/src/main/java/storm/trident/operation/ReducerAggregator.java new file mode 100755 index 000000000..5cd34b4dd --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/ReducerAggregator.java @@ -0,0 +1,26 @@ +/** + * 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 storm.trident.operation; + +import java.io.Serializable; +import storm.trident.tuple.TridentTuple; + +public interface ReducerAggregator extends Serializable { + T init(); + T reduce(T curr, TridentTuple tuple); +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/TridentCollector.java b/jstorm-core/src/main/java/storm/trident/operation/TridentCollector.java new file mode 100755 index 000000000..120dca6f2 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/TridentCollector.java @@ -0,0 +1,26 @@ +/** + * 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 storm.trident.operation; + +import java.util.List; + + +public interface TridentCollector { + void emit(List values); + void reportError(Throwable t); +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/TridentMultiReducerContext.java b/jstorm-core/src/main/java/storm/trident/operation/TridentMultiReducerContext.java new file mode 100755 index 000000000..14876d846 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/TridentMultiReducerContext.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.operation; + +import backtype.storm.tuple.Fields; +import java.util.List; +import storm.trident.tuple.TridentTuple; +import storm.trident.tuple.TridentTupleView.ProjectionFactory; + + +public class TridentMultiReducerContext { + List _factories; + + public TridentMultiReducerContext(List factories) { + _factories = factories; + } + + public ProjectionFactory makeProjectionFactory(int streamIndex, Fields fields) { + return new ProjectionFactory(_factories.get(streamIndex), fields); + } +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/TridentOperationContext.java b/jstorm-core/src/main/java/storm/trident/operation/TridentOperationContext.java new file mode 100755 index 000000000..36726b2be --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/TridentOperationContext.java @@ -0,0 +1,65 @@ +/** + * 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 storm.trident.operation; + +import backtype.storm.metric.api.CombinedMetric; +import backtype.storm.metric.api.ICombiner; +import backtype.storm.metric.api.IMetric; +import backtype.storm.metric.api.IReducer; +import backtype.storm.metric.api.ReducedMetric; +import backtype.storm.task.IMetricsContext; +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Fields; +import storm.trident.tuple.TridentTuple; +import storm.trident.tuple.TridentTupleView.ProjectionFactory; + +public class TridentOperationContext implements IMetricsContext{ + TridentTuple.Factory _factory; + TopologyContext _topoContext; + + public TridentOperationContext(TopologyContext topoContext, TridentTuple.Factory factory) { + _factory = factory; + _topoContext = topoContext; + } + + public TridentOperationContext(TridentOperationContext parent, TridentTuple.Factory factory) { + this(parent._topoContext, factory); + } + + public ProjectionFactory makeProjectionFactory(Fields fields) { + return new ProjectionFactory(_factory, fields); + } + + public int numPartitions() { + return _topoContext.getComponentTasks(_topoContext.getThisComponentId()).size(); + } + + public int getPartitionIndex() { + return _topoContext.getThisTaskIndex(); + } + + public T registerMetric(String name, T metric, int timeBucketSizeInSecs) { + return _topoContext.registerMetric(name, metric, timeBucketSizeInSecs); + } + public ReducedMetric registerMetric(String name, IReducer reducer, int timeBucketSizeInSecs) { + return _topoContext.registerMetric(name, new ReducedMetric(reducer), timeBucketSizeInSecs); + } + public CombinedMetric registerMetric(String name, ICombiner combiner, int timeBucketSizeInSecs) { + return _topoContext.registerMetric(name, new CombinedMetric(combiner), timeBucketSizeInSecs); + } +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/builtin/Count.java b/jstorm-core/src/main/java/storm/trident/operation/builtin/Count.java new file mode 100755 index 000000000..a7449fdeb --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/builtin/Count.java @@ -0,0 +1,41 @@ +/** + * 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 storm.trident.operation.builtin; + +import storm.trident.operation.CombinerAggregator; +import storm.trident.tuple.TridentTuple; + + +public class Count implements CombinerAggregator { + + @Override + public Long init(TridentTuple tuple) { + return 1L; + } + + @Override + public Long combine(Long val1, Long val2) { + return val1 + val2; + } + + @Override + public Long zero() { + return 0L; + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/builtin/Debug.java b/jstorm-core/src/main/java/storm/trident/operation/builtin/Debug.java new file mode 100755 index 000000000..3bdeb3fb9 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/builtin/Debug.java @@ -0,0 +1,39 @@ +/** + * 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 storm.trident.operation.builtin; + +import storm.trident.operation.BaseFilter; +import storm.trident.tuple.TridentTuple; + +public class Debug extends BaseFilter { + private final String name; + + public Debug() { + name = "DEBUG: "; + } + + public Debug(String name) { + this.name = "DEBUG(" + name + "): "; + } + + @Override + public boolean isKeep(TridentTuple tuple) { + System.out.println(name + tuple.toString()); + return true; + } +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/builtin/Equals.java b/jstorm-core/src/main/java/storm/trident/operation/builtin/Equals.java new file mode 100755 index 000000000..6091227e8 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/builtin/Equals.java @@ -0,0 +1,38 @@ +/** + * 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 storm.trident.operation.builtin; + +import storm.trident.operation.BaseFilter; +import storm.trident.tuple.TridentTuple; + + +public class Equals extends BaseFilter { + + @Override + public boolean isKeep(TridentTuple tuple) { + for(int i=0; i { + @Override + public List batchRetrieve(ReadOnlyMapState map, List keys) { + return map.multiGet((List) keys); + } + + @Override + public void execute(TridentTuple tuple, Object result, TridentCollector collector) { + collector.emit(new Values(result)); + } +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/builtin/Negate.java b/jstorm-core/src/main/java/storm/trident/operation/builtin/Negate.java new file mode 100755 index 000000000..bd7827959 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/builtin/Negate.java @@ -0,0 +1,48 @@ +/** + * 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 storm.trident.operation.builtin; + +import java.util.Map; +import storm.trident.operation.Filter; +import storm.trident.operation.TridentOperationContext; +import storm.trident.tuple.TridentTuple; + +public class Negate implements Filter { + + Filter _delegate; + + public Negate(Filter delegate) { + _delegate = delegate; + } + + @Override + public boolean isKeep(TridentTuple tuple) { + return !_delegate.isKeep(tuple); + } + + @Override + public void prepare(Map conf, TridentOperationContext context) { + _delegate.prepare(conf, context); + } + + @Override + public void cleanup() { + _delegate.cleanup(); + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/builtin/SnapshotGet.java b/jstorm-core/src/main/java/storm/trident/operation/builtin/SnapshotGet.java new file mode 100755 index 000000000..6d24ae68c --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/builtin/SnapshotGet.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.operation.builtin; + +import backtype.storm.tuple.Values; +import java.util.ArrayList; +import java.util.List; +import storm.trident.operation.TridentCollector; +import storm.trident.state.BaseQueryFunction; +import storm.trident.state.snapshot.ReadOnlySnapshottable; +import storm.trident.tuple.TridentTuple; + +public class SnapshotGet extends BaseQueryFunction { + + @Override + public List batchRetrieve(ReadOnlySnapshottable state, List args) { + List ret = new ArrayList(args.size()); + Object snapshot = state.get(); + for(int i=0; i { + + @Override + public Number init(TridentTuple tuple) { + return (Number) tuple.getValue(0); + } + + @Override + public Number combine(Number val1, Number val2) { + return Numbers.add(val1, val2); + } + + @Override + public Number zero() { + return 0; + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/builtin/TupleCollectionGet.java b/jstorm-core/src/main/java/storm/trident/operation/builtin/TupleCollectionGet.java new file mode 100755 index 000000000..52dd63315 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/builtin/TupleCollectionGet.java @@ -0,0 +1,46 @@ +/** + * 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 storm.trident.operation.builtin; + +import storm.trident.state.ITupleCollection; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import storm.trident.operation.TridentCollector; +import storm.trident.state.BaseQueryFunction; +import storm.trident.state.State; +import storm.trident.tuple.TridentTuple; + +public class TupleCollectionGet extends BaseQueryFunction>> { + + @Override + public List>> batchRetrieve(State state, List args) { + List>> ret = new ArrayList(args.size()); + for(int i=0; i> tuplesIterator, TridentCollector collector) { + while(tuplesIterator.hasNext()) { + collector.emit(tuplesIterator.next()); + } + } +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/impl/CaptureCollector.java b/jstorm-core/src/main/java/storm/trident/operation/impl/CaptureCollector.java new file mode 100755 index 000000000..72d7e5dec --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/impl/CaptureCollector.java @@ -0,0 +1,42 @@ +/** + * 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 storm.trident.operation.impl; + +import java.util.ArrayList; +import java.util.List; +import storm.trident.operation.TridentCollector; + +public class CaptureCollector implements TridentCollector { + public List> captured = new ArrayList(); + + TridentCollector _coll; + + public void setCollector(TridentCollector coll) { + _coll = coll; + } + + @Override + public void emit(List values) { + this.captured.add(values); + } + + @Override + public void reportError(Throwable t) { + _coll.reportError(t); + } +} diff --git a/jstorm-client/src/main/java/storm/trident/operation/impl/ChainedAggregatorImpl.java b/jstorm-core/src/main/java/storm/trident/operation/impl/ChainedAggregatorImpl.java old mode 100644 new mode 100755 similarity index 82% rename from jstorm-client/src/main/java/storm/trident/operation/impl/ChainedAggregatorImpl.java rename to jstorm-core/src/main/java/storm/trident/operation/impl/ChainedAggregatorImpl.java index f8bd0010a..de58d646e --- a/jstorm-client/src/main/java/storm/trident/operation/impl/ChainedAggregatorImpl.java +++ b/jstorm-core/src/main/java/storm/trident/operation/impl/ChainedAggregatorImpl.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.operation.impl; import backtype.storm.tuple.Fields; diff --git a/jstorm-core/src/main/java/storm/trident/operation/impl/ChainedResult.java b/jstorm-core/src/main/java/storm/trident/operation/impl/ChainedResult.java new file mode 100755 index 000000000..29069b62f --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/impl/ChainedResult.java @@ -0,0 +1,53 @@ +/** + * 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 storm.trident.operation.impl; + +import org.apache.commons.lang.builder.ToStringBuilder; +import storm.trident.operation.TridentCollector; + + +//for ChainedAggregator +public class ChainedResult { + Object[] objs; + TridentCollector[] collectors; + + public ChainedResult(TridentCollector collector, int size) { + objs = new Object[size]; + collectors = new TridentCollector[size]; + for(int i=0; i1) { + for(TridentCollector c: collectors) { + ((CaptureCollector) c).setCollector(collector); + } + } + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(objs); + } +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/impl/CombinerAggStateUpdater.java b/jstorm-core/src/main/java/storm/trident/operation/impl/CombinerAggStateUpdater.java new file mode 100755 index 000000000..bfe9dc31a --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/impl/CombinerAggStateUpdater.java @@ -0,0 +1,56 @@ +/** + * 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 storm.trident.operation.impl; + +import backtype.storm.tuple.Values; +import java.util.List; +import java.util.Map; +import storm.trident.operation.CombinerAggregator; +import storm.trident.operation.TridentCollector; +import storm.trident.operation.TridentOperationContext; +import storm.trident.state.CombinerValueUpdater; +import storm.trident.state.StateUpdater; +import storm.trident.state.snapshot.Snapshottable; +import storm.trident.tuple.TridentTuple; + +public class CombinerAggStateUpdater implements StateUpdater { + CombinerAggregator _agg; + + public CombinerAggStateUpdater(CombinerAggregator agg) { + _agg = agg; + } + + + @Override + public void updateState(Snapshottable state, List tuples, TridentCollector collector) { + if(tuples.size()!=1) { + throw new IllegalArgumentException("Combiner state updater should receive a single tuple. Received: " + tuples.toString()); + } + Object newVal = state.update(new CombinerValueUpdater(_agg, tuples.get(0).getValue(0))); + collector.emit(new Values(newVal)); + } + + @Override + public void prepare(Map conf, TridentOperationContext context) { + } + + @Override + public void cleanup() { + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/impl/CombinerAggregatorCombineImpl.java b/jstorm-core/src/main/java/storm/trident/operation/impl/CombinerAggregatorCombineImpl.java new file mode 100755 index 000000000..76722c42f --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/impl/CombinerAggregatorCombineImpl.java @@ -0,0 +1,61 @@ +/** + * 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 storm.trident.operation.impl; + +import backtype.storm.tuple.Values; +import java.util.Map; +import storm.trident.operation.Aggregator; +import storm.trident.operation.CombinerAggregator; +import storm.trident.operation.TridentCollector; +import storm.trident.operation.TridentOperationContext; +import storm.trident.tuple.TridentTuple; + +public class CombinerAggregatorCombineImpl implements Aggregator { + CombinerAggregator _agg; + + public CombinerAggregatorCombineImpl(CombinerAggregator agg) { + _agg = agg; + } + + public void prepare(Map conf, TridentOperationContext context) { + + } + + public Result init(Object batchId, TridentCollector collector) { + Result ret = new Result(); + ret.obj = _agg.zero(); + return ret; + } + + public void aggregate(Result val, TridentTuple tuple, TridentCollector collector) { + Object v = tuple.getValue(0); + if(val.obj==null) { + val.obj = v; + } else { + val.obj = _agg.combine(val.obj, v); + } + } + + public void complete(Result val, TridentCollector collector) { + collector.emit(new Values(val.obj)); + } + + public void cleanup() { + + } +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/impl/CombinerAggregatorInitImpl.java b/jstorm-core/src/main/java/storm/trident/operation/impl/CombinerAggregatorInitImpl.java new file mode 100755 index 000000000..71f1178bf --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/impl/CombinerAggregatorInitImpl.java @@ -0,0 +1,49 @@ +/** + * 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 storm.trident.operation.impl; + +import backtype.storm.tuple.Values; +import java.util.Map; +import storm.trident.operation.CombinerAggregator; +import storm.trident.operation.Function; +import storm.trident.operation.TridentCollector; +import storm.trident.operation.TridentOperationContext; +import storm.trident.tuple.TridentTuple; + +public class CombinerAggregatorInitImpl implements Function { + + CombinerAggregator _agg; + + public CombinerAggregatorInitImpl(CombinerAggregator agg) { + _agg = agg; + } + + @Override + public void execute(TridentTuple tuple, TridentCollector collector) { + collector.emit(new Values(_agg.init(tuple))); + } + + @Override + public void prepare(Map conf, TridentOperationContext context) { + } + + @Override + public void cleanup() { + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/impl/FilterExecutor.java b/jstorm-core/src/main/java/storm/trident/operation/impl/FilterExecutor.java new file mode 100755 index 000000000..5ed2883a2 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/impl/FilterExecutor.java @@ -0,0 +1,53 @@ +/** + * 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 storm.trident.operation.impl; + +import java.util.Map; +import storm.trident.operation.Filter; +import storm.trident.operation.Function; +import storm.trident.operation.TridentCollector; +import storm.trident.operation.TridentOperationContext; +import storm.trident.tuple.TridentTuple; + +// works by emitting null to the collector. since the planner knows this is an ADD node with +// no new output fields, it just passes the tuple forward +public class FilterExecutor implements Function { + Filter _filter; + + public FilterExecutor(Filter filter) { + _filter = filter; + } + + @Override + public void execute(TridentTuple tuple, TridentCollector collector) { + if(_filter.isKeep(tuple)) { + collector.emit(null); + } + } + + @Override + public void prepare(Map conf, TridentOperationContext context) { + _filter.prepare(conf, context); + } + + @Override + public void cleanup() { + _filter.cleanup(); + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/impl/GlobalBatchToPartition.java b/jstorm-core/src/main/java/storm/trident/operation/impl/GlobalBatchToPartition.java new file mode 100755 index 000000000..aa27b6d86 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/impl/GlobalBatchToPartition.java @@ -0,0 +1,29 @@ +/** + * 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 storm.trident.operation.impl; + + +public class GlobalBatchToPartition implements SingleEmitAggregator.BatchToPartition { + + @Override + public int partitionIndex(Object batchId, int numPartitions) { + // TODO: take away knowledge of storm's internals here + return 0; + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/impl/GroupCollector.java b/jstorm-core/src/main/java/storm/trident/operation/impl/GroupCollector.java new file mode 100755 index 000000000..d549c5986 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/impl/GroupCollector.java @@ -0,0 +1,48 @@ +/** + * 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 storm.trident.operation.impl; + +import java.util.List; +import storm.trident.operation.TridentCollector; +import storm.trident.tuple.ComboList; + +public class GroupCollector implements TridentCollector { + public List currGroup; + + ComboList.Factory _factory; + TridentCollector _collector; + + public GroupCollector(TridentCollector collector, ComboList.Factory factory) { + _factory = factory; + _collector = collector; + } + + @Override + public void emit(List values) { + List[] delegates = new List[2]; + delegates[0] = currGroup; + delegates[1] = values; + _collector.emit(_factory.create(delegates)); + } + + @Override + public void reportError(Throwable t) { + _collector.reportError(t); + } + +} diff --git a/jstorm-client/src/main/java/storm/trident/operation/impl/GroupedAggregator.java b/jstorm-core/src/main/java/storm/trident/operation/impl/GroupedAggregator.java old mode 100644 new mode 100755 similarity index 76% rename from jstorm-client/src/main/java/storm/trident/operation/impl/GroupedAggregator.java rename to jstorm-core/src/main/java/storm/trident/operation/impl/GroupedAggregator.java index d78de7014..caa5dc30d --- a/jstorm-client/src/main/java/storm/trident/operation/impl/GroupedAggregator.java +++ b/jstorm-core/src/main/java/storm/trident/operation/impl/GroupedAggregator.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.operation.impl; import backtype.storm.tuple.Fields; diff --git a/jstorm-client/src/main/java/storm/trident/operation/impl/GroupedMultiReducerExecutor.java b/jstorm-core/src/main/java/storm/trident/operation/impl/GroupedMultiReducerExecutor.java old mode 100644 new mode 100755 similarity index 78% rename from jstorm-client/src/main/java/storm/trident/operation/impl/GroupedMultiReducerExecutor.java rename to jstorm-core/src/main/java/storm/trident/operation/impl/GroupedMultiReducerExecutor.java index 26159629e..3f31b3ff3 --- a/jstorm-client/src/main/java/storm/trident/operation/impl/GroupedMultiReducerExecutor.java +++ b/jstorm-core/src/main/java/storm/trident/operation/impl/GroupedMultiReducerExecutor.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.operation.impl; import backtype.storm.tuple.Fields; diff --git a/jstorm-core/src/main/java/storm/trident/operation/impl/IdentityMultiReducer.java b/jstorm-core/src/main/java/storm/trident/operation/impl/IdentityMultiReducer.java new file mode 100755 index 000000000..b7ecc74af --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/impl/IdentityMultiReducer.java @@ -0,0 +1,51 @@ +/** + * 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 storm.trident.operation.impl; + +import java.util.Map; +import storm.trident.operation.MultiReducer; +import storm.trident.operation.TridentCollector; +import storm.trident.operation.TridentMultiReducerContext; +import storm.trident.tuple.TridentTuple; + + +public class IdentityMultiReducer implements MultiReducer { + + @Override + public void prepare(Map conf, TridentMultiReducerContext context) { + } + + @Override + public Object init(TridentCollector collector) { + return null; + } + + @Override + public void execute(Object state, int streamIndex, TridentTuple input, TridentCollector collector) { + collector.emit(input); + } + + @Override + public void complete(Object state, TridentCollector collector) { + } + + @Override + public void cleanup() { + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/impl/IndexHashBatchToPartition.java b/jstorm-core/src/main/java/storm/trident/operation/impl/IndexHashBatchToPartition.java new file mode 100755 index 000000000..ce054ba3a --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/impl/IndexHashBatchToPartition.java @@ -0,0 +1,29 @@ +/** + * 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 storm.trident.operation.impl; + +import storm.trident.partition.IndexHashGrouping; + +public class IndexHashBatchToPartition implements SingleEmitAggregator.BatchToPartition { + + @Override + public int partitionIndex(Object batchId, int numPartitions) { + return IndexHashGrouping.objectToIndex(batchId, numPartitions); + } + +} diff --git a/jstorm-client/src/main/java/storm/trident/operation/impl/JoinerMultiReducer.java b/jstorm-core/src/main/java/storm/trident/operation/impl/JoinerMultiReducer.java old mode 100644 new mode 100755 similarity index 85% rename from jstorm-client/src/main/java/storm/trident/operation/impl/JoinerMultiReducer.java rename to jstorm-core/src/main/java/storm/trident/operation/impl/JoinerMultiReducer.java index 963751e5a..1561b1e02 --- a/jstorm-client/src/main/java/storm/trident/operation/impl/JoinerMultiReducer.java +++ b/jstorm-core/src/main/java/storm/trident/operation/impl/JoinerMultiReducer.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.operation.impl; import backtype.storm.tuple.Fields; diff --git a/jstorm-core/src/main/java/storm/trident/operation/impl/ReducerAggStateUpdater.java b/jstorm-core/src/main/java/storm/trident/operation/impl/ReducerAggStateUpdater.java new file mode 100755 index 000000000..fd87b0754 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/impl/ReducerAggStateUpdater.java @@ -0,0 +1,53 @@ +/** + * 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 storm.trident.operation.impl; + +import backtype.storm.tuple.Values; +import java.util.List; +import java.util.Map; +import storm.trident.operation.ReducerAggregator; +import storm.trident.operation.TridentCollector; +import storm.trident.operation.TridentOperationContext; +import storm.trident.state.ReducerValueUpdater; +import storm.trident.state.StateUpdater; +import storm.trident.state.snapshot.Snapshottable; +import storm.trident.tuple.TridentTuple; + +public class ReducerAggStateUpdater implements StateUpdater { + ReducerAggregator _agg; + + public ReducerAggStateUpdater(ReducerAggregator agg) { + _agg = agg; + } + + + @Override + public void updateState(Snapshottable state, List tuples, TridentCollector collector) { + Object newVal = state.update(new ReducerValueUpdater(_agg, tuples)); + collector.emit(new Values(newVal)); + } + + @Override + public void prepare(Map conf, TridentOperationContext context) { + } + + @Override + public void cleanup() { + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/impl/ReducerAggregatorImpl.java b/jstorm-core/src/main/java/storm/trident/operation/impl/ReducerAggregatorImpl.java new file mode 100755 index 000000000..cf8da4252 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/impl/ReducerAggregatorImpl.java @@ -0,0 +1,56 @@ +/** + * 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 storm.trident.operation.impl; + +import backtype.storm.tuple.Values; +import java.util.Map; +import storm.trident.operation.Aggregator; +import storm.trident.operation.ReducerAggregator; +import storm.trident.operation.TridentCollector; +import storm.trident.operation.TridentOperationContext; +import storm.trident.tuple.TridentTuple; + +public class ReducerAggregatorImpl implements Aggregator { + ReducerAggregator _agg; + + public ReducerAggregatorImpl(ReducerAggregator agg) { + _agg = agg; + } + + public void prepare(Map conf, TridentOperationContext context) { + + } + + public Result init(Object batchId, TridentCollector collector) { + Result ret = new Result(); + ret.obj = _agg.init(); + return ret; + } + + public void aggregate(Result val, TridentTuple tuple, TridentCollector collector) { + val.obj = _agg.reduce(val.obj, tuple); + } + + public void complete(Result val, TridentCollector collector) { + collector.emit(new Values(val.obj)); + } + + public void cleanup() { + + } +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/impl/Result.java b/jstorm-core/src/main/java/storm/trident/operation/impl/Result.java new file mode 100755 index 000000000..ed5dc9fbc --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/impl/Result.java @@ -0,0 +1,27 @@ +/** + * 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 storm.trident.operation.impl; + +public class Result { + public Object obj; + + @Override + public String toString() { + return "" + obj; + } +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/impl/SingleEmitAggregator.java b/jstorm-core/src/main/java/storm/trident/operation/impl/SingleEmitAggregator.java new file mode 100755 index 000000000..9a7fbc894 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/impl/SingleEmitAggregator.java @@ -0,0 +1,95 @@ +/** + * 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 storm.trident.operation.impl; + +import java.io.Serializable; +import java.util.Map; +import storm.trident.operation.Aggregator; +import storm.trident.operation.TridentCollector; +import storm.trident.operation.TridentOperationContext; +import storm.trident.operation.impl.SingleEmitAggregator.SingleEmitState; +import storm.trident.tuple.TridentTuple; + + +public class SingleEmitAggregator implements Aggregator { + public static interface BatchToPartition extends Serializable { + int partitionIndex(Object batchId, int numPartitions); + } + + static class SingleEmitState { + boolean received = false; + Object state; + Object batchId; + + public SingleEmitState(Object batchId) { + this.batchId = batchId; + } + } + + Aggregator _agg; + BatchToPartition _batchToPartition; + + public SingleEmitAggregator(Aggregator agg, BatchToPartition batchToPartition) { + _agg = agg; + _batchToPartition = batchToPartition; + } + + + @Override + public SingleEmitState init(Object batchId, TridentCollector collector) { + return new SingleEmitState(batchId); + } + + @Override + public void aggregate(SingleEmitState val, TridentTuple tuple, TridentCollector collector) { + if(!val.received) { + val.state = _agg.init(val.batchId, collector); + val.received = true; + } + _agg.aggregate(val.state, tuple, collector); + } + + @Override + public void complete(SingleEmitState val, TridentCollector collector) { + if(!val.received) { + if(this.myPartitionIndex == _batchToPartition.partitionIndex(val.batchId, this.totalPartitions)) { + val.state = _agg.init(val.batchId, collector); + _agg.complete(val.state, collector); + } + } else { + _agg.complete(val.state, collector); + } + } + + int myPartitionIndex; + int totalPartitions; + + @Override + public void prepare(Map conf, TridentOperationContext context) { + _agg.prepare(conf, context); + this.myPartitionIndex = context.getPartitionIndex(); + this.totalPartitions = context.numPartitions(); + } + + @Override + public void cleanup() { + _agg.cleanup(); + } + + +} diff --git a/jstorm-core/src/main/java/storm/trident/operation/impl/TrueFilter.java b/jstorm-core/src/main/java/storm/trident/operation/impl/TrueFilter.java new file mode 100755 index 000000000..0fe3abb13 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/operation/impl/TrueFilter.java @@ -0,0 +1,40 @@ +/** + * 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 storm.trident.operation.impl; + +import java.util.Map; +import storm.trident.operation.Filter; +import storm.trident.operation.TridentOperationContext; +import storm.trident.tuple.TridentTuple; + +public class TrueFilter implements Filter { + + @Override + public boolean isKeep(TridentTuple tuple) { + return true; + } + + @Override + public void prepare(Map conf, TridentOperationContext context) { + } + + @Override + public void cleanup() { + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/partition/GlobalGrouping.java b/jstorm-core/src/main/java/storm/trident/partition/GlobalGrouping.java new file mode 100755 index 000000000..54fa84486 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/partition/GlobalGrouping.java @@ -0,0 +1,45 @@ +/** + * 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 storm.trident.partition; + +import backtype.storm.generated.GlobalStreamId; +import backtype.storm.grouping.CustomStreamGrouping; +import backtype.storm.task.WorkerTopologyContext; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +public class GlobalGrouping implements CustomStreamGrouping { + + List target; + + + @Override + public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List targets) { + List sorted = new ArrayList(targets); + Collections.sort(sorted); + target = Arrays.asList(sorted.get(0)); + } + + @Override + public List chooseTasks(int i, List list) { + return target; + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/partition/IdentityGrouping.java b/jstorm-core/src/main/java/storm/trident/partition/IdentityGrouping.java new file mode 100755 index 000000000..30f48adf6 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/partition/IdentityGrouping.java @@ -0,0 +1,61 @@ +/** + * 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 storm.trident.partition; + +import backtype.storm.generated.GlobalStreamId; +import backtype.storm.grouping.CustomStreamGrouping; +import backtype.storm.task.WorkerTopologyContext; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + + +public class IdentityGrouping implements CustomStreamGrouping { + + List ret = new ArrayList(); + Map> _precomputed = new HashMap(); + + @Override + public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List tasks) { + List sourceTasks = new ArrayList(context.getComponentTasks(stream.get_componentId())); + Collections.sort(sourceTasks); + if(sourceTasks.size()!=tasks.size()) { + throw new RuntimeException("Can only do an identity grouping when source and target have same number of tasks"); + } + tasks = new ArrayList(tasks); + Collections.sort(tasks); + for(int i=0; i chooseTasks(int task, List values) { + List ret = _precomputed.get(task); + if(ret==null) { + throw new RuntimeException("Tuple emitted by task that's not part of this component. Should be impossible"); + } + return ret; + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/partition/IndexHashGrouping.java b/jstorm-core/src/main/java/storm/trident/partition/IndexHashGrouping.java new file mode 100755 index 000000000..55c7a0230 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/partition/IndexHashGrouping.java @@ -0,0 +1,53 @@ +/** + * 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 storm.trident.partition; + +import backtype.storm.generated.GlobalStreamId; +import backtype.storm.grouping.CustomStreamGrouping; +import backtype.storm.task.WorkerTopologyContext; +import java.util.Arrays; +import java.util.List; + +public class IndexHashGrouping implements CustomStreamGrouping { + public static int objectToIndex(Object val, int numPartitions) { + if(val==null) return 0; + else { + return Math.abs(val.hashCode()) % numPartitions; + } + } + + int _index; + List _targets; + + public IndexHashGrouping(int index) { + _index = index; + } + + + @Override + public void prepare(WorkerTopologyContext context, GlobalStreamId stream, List targetTasks) { + _targets = targetTasks; + } + + @Override + public List chooseTasks(int fromTask, List values) { + int i = objectToIndex(values.get(_index), _targets.size()); + return Arrays.asList(_targets.get(i)); + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/planner/BridgeReceiver.java b/jstorm-core/src/main/java/storm/trident/planner/BridgeReceiver.java new file mode 100755 index 000000000..16bafa3a0 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/planner/BridgeReceiver.java @@ -0,0 +1,38 @@ +/** + * 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 storm.trident.planner; + +import backtype.storm.coordination.BatchOutputCollector; +import storm.trident.tuple.ConsList; +import storm.trident.tuple.TridentTuple; + + +public class BridgeReceiver implements TupleReceiver { + + BatchOutputCollector _collector; + + public BridgeReceiver(BatchOutputCollector collector) { + _collector = collector; + } + + @Override + public void execute(ProcessorContext context, String streamId, TridentTuple tuple) { + _collector.emit(streamId, new ConsList(context.batchId, tuple)); + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/planner/Node.java b/jstorm-core/src/main/java/storm/trident/planner/Node.java new file mode 100755 index 000000000..6284cb9d5 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/planner/Node.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.planner; + +import backtype.storm.tuple.Fields; +import java.io.Serializable; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; + + +public class Node implements Serializable { + private static AtomicInteger INDEX = new AtomicInteger(0); + + private String nodeId; + + public String name = null; + public Fields allOutputFields; + public String streamId; + public Integer parallelismHint = null; + public NodeStateInfo stateInfo = null; + public int creationIndex; + + public Node(String streamId, String name, Fields allOutputFields) { + this.nodeId = UUID.randomUUID().toString(); + this.allOutputFields = allOutputFields; + this.streamId = streamId; + this.name = name; + this.creationIndex = INDEX.incrementAndGet(); + } + + @Override + public boolean equals(Object o) { + return nodeId.equals(((Node) o).nodeId); + } + + @Override + public int hashCode() { + return nodeId.hashCode(); + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, ToStringStyle.MULTI_LINE_STYLE); + } + + +} diff --git a/jstorm-core/src/main/java/storm/trident/planner/NodeStateInfo.java b/jstorm-core/src/main/java/storm/trident/planner/NodeStateInfo.java new file mode 100755 index 000000000..177c0a5eb --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/planner/NodeStateInfo.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.planner; + +import java.io.Serializable; +import storm.trident.state.StateSpec; + +public class NodeStateInfo implements Serializable { + public String id; + public StateSpec spec; + + public NodeStateInfo(String id, StateSpec spec) { + this.id = id; + this.spec = spec; + } +} diff --git a/jstorm-core/src/main/java/storm/trident/planner/PartitionNode.java b/jstorm-core/src/main/java/storm/trident/planner/PartitionNode.java new file mode 100755 index 000000000..4f10c25ba --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/planner/PartitionNode.java @@ -0,0 +1,52 @@ +/** + * 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 storm.trident.planner; + +import backtype.storm.generated.Grouping; +import backtype.storm.tuple.Fields; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.ArrayList; +import java.util.List; +import storm.trident.util.TridentUtils; + + +public class PartitionNode extends Node { + public transient Grouping thriftGrouping; + + //has the streamid/outputFields of the node it's doing the partitioning on + public PartitionNode(String streamId, String name, Fields allOutputFields, Grouping grouping) { + super(streamId, name, allOutputFields); + this.thriftGrouping = grouping; + } + + private void writeObject(ObjectOutputStream oos) throws IOException { + oos.defaultWriteObject(); + byte[] ser = TridentUtils.thriftSerialize(thriftGrouping); + oos.writeInt(ser.length); + oos.write(ser); + } + + private void readObject(ObjectInputStream ois) throws ClassNotFoundException, IOException { + ois.defaultReadObject(); + byte[] ser = new byte[ois.readInt()]; + ois.readFully(ser); + this.thriftGrouping = TridentUtils.thriftDeserialize(Grouping.class, ser); + } +} diff --git a/jstorm-core/src/main/java/storm/trident/planner/ProcessorContext.java b/jstorm-core/src/main/java/storm/trident/planner/ProcessorContext.java new file mode 100755 index 000000000..4b89c4275 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/planner/ProcessorContext.java @@ -0,0 +1,29 @@ +/** + * 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 storm.trident.planner; + + +public class ProcessorContext { + public Object batchId; + public Object[] state; + + public ProcessorContext(Object batchId, Object[] state) { + this.batchId = batchId; + this.state = state; + } +} diff --git a/jstorm-core/src/main/java/storm/trident/planner/ProcessorNode.java b/jstorm-core/src/main/java/storm/trident/planner/ProcessorNode.java new file mode 100755 index 000000000..02f846391 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/planner/ProcessorNode.java @@ -0,0 +1,33 @@ +/** + * 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 storm.trident.planner; + +import backtype.storm.tuple.Fields; + +public class ProcessorNode extends Node { + + public boolean committer; // for partitionpersist + public TridentProcessor processor; + public Fields selfOutFields; + + public ProcessorNode(String streamId, String name, Fields allOutputFields, Fields selfOutFields, TridentProcessor processor) { + super(streamId, name, allOutputFields); + this.processor = processor; + this.selfOutFields = selfOutFields; + } +} diff --git a/jstorm-core/src/main/java/storm/trident/planner/SpoutNode.java b/jstorm-core/src/main/java/storm/trident/planner/SpoutNode.java new file mode 100755 index 000000000..6353a5cf8 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/planner/SpoutNode.java @@ -0,0 +1,39 @@ +/** + * 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 storm.trident.planner; + +import backtype.storm.tuple.Fields; + + +public class SpoutNode extends Node { + public static enum SpoutType { + DRPC, + BATCH + } + + public Object spout; + public String txId; //where state is stored in zookeeper (only for batch spout types) + public SpoutType type; + + public SpoutNode(String streamId, Fields allOutputFields, String txid, Object spout, SpoutType type) { + super(streamId, null, allOutputFields); + this.txId = txid; + this.spout = spout; + this.type = type; + } +} diff --git a/jstorm-client/src/main/java/storm/trident/planner/SubtopologyBolt.java b/jstorm-core/src/main/java/storm/trident/planner/SubtopologyBolt.java old mode 100644 new mode 100755 similarity index 91% rename from jstorm-client/src/main/java/storm/trident/planner/SubtopologyBolt.java rename to jstorm-core/src/main/java/storm/trident/planner/SubtopologyBolt.java index 596c15df3..85d0708b0 --- a/jstorm-client/src/main/java/storm/trident/planner/SubtopologyBolt.java +++ b/jstorm-core/src/main/java/storm/trident/planner/SubtopologyBolt.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.planner; import backtype.storm.coordination.BatchOutputCollector; diff --git a/jstorm-core/src/main/java/storm/trident/planner/TridentProcessor.java b/jstorm-core/src/main/java/storm/trident/planner/TridentProcessor.java new file mode 100755 index 000000000..159391354 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/planner/TridentProcessor.java @@ -0,0 +1,40 @@ +/** + * 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 storm.trident.planner; + +import backtype.storm.task.TopologyContext; +import java.io.Serializable; +import java.util.Map; +import storm.trident.planner.processor.TridentContext; +import storm.trident.tuple.TridentTuple.Factory; + +public interface TridentProcessor extends Serializable, TupleReceiver { + + // imperative that don't emit any tuples from here, since output factory cannot be gotten until + // preparation is done, therefore, receivers won't be ready to receive tuples yet + // can't emit tuples from here anyway, since it's not within a batch context (which is only + // startBatch, execute, and finishBatch + void prepare(Map conf, TopologyContext context, TridentContext tridentContext); + void cleanup(); + + void startBatch(ProcessorContext processorContext); + + void finishBatch(ProcessorContext processorContext); + + Factory getOutputFactory(); +} diff --git a/jstorm-core/src/main/java/storm/trident/planner/TupleReceiver.java b/jstorm-core/src/main/java/storm/trident/planner/TupleReceiver.java new file mode 100755 index 000000000..4fa3bc801 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/planner/TupleReceiver.java @@ -0,0 +1,27 @@ +/** + * 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 storm.trident.planner; + +import storm.trident.tuple.TridentTuple; + + +public interface TupleReceiver { + //streaId indicates where tuple came from + void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple); + +} diff --git a/jstorm-core/src/main/java/storm/trident/planner/processor/AggregateProcessor.java b/jstorm-core/src/main/java/storm/trident/planner/processor/AggregateProcessor.java new file mode 100755 index 000000000..e9dff1d58 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/planner/processor/AggregateProcessor.java @@ -0,0 +1,84 @@ +/** + * 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 storm.trident.planner.processor; + +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Fields; +import java.util.List; +import java.util.Map; +import storm.trident.operation.Aggregator; +import storm.trident.operation.TridentOperationContext; +import storm.trident.planner.ProcessorContext; +import storm.trident.planner.TridentProcessor; +import storm.trident.tuple.TridentTuple; +import storm.trident.tuple.TridentTuple.Factory; +import storm.trident.tuple.TridentTupleView.ProjectionFactory; + + +public class AggregateProcessor implements TridentProcessor { + Aggregator _agg; + TridentContext _context; + FreshCollector _collector; + Fields _inputFields; + ProjectionFactory _projection; + + public AggregateProcessor(Fields inputFields, Aggregator agg) { + _agg = agg; + _inputFields = inputFields; + } + + @Override + public void prepare(Map conf, TopologyContext context, TridentContext tridentContext) { + List parents = tridentContext.getParentTupleFactories(); + if(parents.size()!=1) { + throw new RuntimeException("Aggregate operation can only have one parent"); + } + _context = tridentContext; + _collector = new FreshCollector(tridentContext); + _projection = new ProjectionFactory(parents.get(0), _inputFields); + _agg.prepare(conf, new TridentOperationContext(context, _projection)); + } + + @Override + public void cleanup() { + _agg.cleanup(); + } + + @Override + public void startBatch(ProcessorContext processorContext) { + _collector.setContext(processorContext); + processorContext.state[_context.getStateIndex()] = _agg.init(processorContext.batchId, _collector); + } + + @Override + public void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple) { + _collector.setContext(processorContext); + _agg.aggregate(processorContext.state[_context.getStateIndex()], _projection.create(tuple), _collector); + } + + @Override + public void finishBatch(ProcessorContext processorContext) { + _collector.setContext(processorContext); + _agg.complete(processorContext.state[_context.getStateIndex()], _collector); + } + + @Override + public Factory getOutputFactory() { + return _collector.getOutputFactory(); + } +} diff --git a/jstorm-core/src/main/java/storm/trident/planner/processor/AppendCollector.java b/jstorm-core/src/main/java/storm/trident/planner/processor/AppendCollector.java new file mode 100755 index 000000000..ea1f1b6da --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/planner/processor/AppendCollector.java @@ -0,0 +1,62 @@ +/** + * 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 storm.trident.planner.processor; + +import java.util.List; +import storm.trident.operation.TridentCollector; +import storm.trident.planner.ProcessorContext; +import storm.trident.planner.TupleReceiver; +import storm.trident.tuple.TridentTuple; +import storm.trident.tuple.TridentTuple.Factory; +import storm.trident.tuple.TridentTupleView; +import storm.trident.tuple.TridentTupleView.OperationOutputFactory; + + +public class AppendCollector implements TridentCollector { + OperationOutputFactory _factory; + TridentContext _triContext; + TridentTuple tuple; + ProcessorContext context; + + public AppendCollector(TridentContext context) { + _triContext = context; + _factory = new OperationOutputFactory(context.getParentTupleFactories().get(0), context.getSelfOutputFields()); + } + + public void setContext(ProcessorContext pc, TridentTuple t) { + this.context = pc; + this.tuple = t; + } + + @Override + public void emit(List values) { + TridentTuple toEmit = _factory.create((TridentTupleView) tuple, values); + for(TupleReceiver r: _triContext.getReceivers()) { + r.execute(context, _triContext.getOutStreamId(), toEmit); + } + } + + @Override + public void reportError(Throwable t) { + _triContext.getDelegateCollector().reportError(t); + } + + public Factory getOutputFactory() { + return _factory; + } +} diff --git a/jstorm-core/src/main/java/storm/trident/planner/processor/EachProcessor.java b/jstorm-core/src/main/java/storm/trident/planner/processor/EachProcessor.java new file mode 100755 index 000000000..68ad2a9e8 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/planner/processor/EachProcessor.java @@ -0,0 +1,80 @@ +/** + * 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 storm.trident.planner.processor; + +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Fields; +import java.util.List; +import java.util.Map; +import storm.trident.operation.Function; +import storm.trident.operation.TridentOperationContext; +import storm.trident.planner.ProcessorContext; +import storm.trident.planner.TridentProcessor; +import storm.trident.tuple.TridentTuple; +import storm.trident.tuple.TridentTuple.Factory; +import storm.trident.tuple.TridentTupleView.ProjectionFactory; + + +public class EachProcessor implements TridentProcessor { + Function _function; + TridentContext _context; + AppendCollector _collector; + Fields _inputFields; + ProjectionFactory _projection; + + public EachProcessor(Fields inputFields, Function function) { + _function = function; + _inputFields = inputFields; + } + + @Override + public void prepare(Map conf, TopologyContext context, TridentContext tridentContext) { + List parents = tridentContext.getParentTupleFactories(); + if(parents.size()!=1) { + throw new RuntimeException("Each operation can only have one parent"); + } + _context = tridentContext; + _collector = new AppendCollector(tridentContext); + _projection = new ProjectionFactory(parents.get(0), _inputFields); + _function.prepare(conf, new TridentOperationContext(context, _projection)); + } + + @Override + public void cleanup() { + _function.cleanup(); + } + + @Override + public void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple) { + _collector.setContext(processorContext, tuple); + _function.execute(_projection.create(tuple), _collector); + } + + @Override + public void startBatch(ProcessorContext processorContext) { + } + + @Override + public void finishBatch(ProcessorContext processorContext) { + } + + @Override + public Factory getOutputFactory() { + return _collector.getOutputFactory(); + } +} diff --git a/jstorm-core/src/main/java/storm/trident/planner/processor/FreshCollector.java b/jstorm-core/src/main/java/storm/trident/planner/processor/FreshCollector.java new file mode 100755 index 000000000..6c7cf3b0a --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/planner/processor/FreshCollector.java @@ -0,0 +1,59 @@ +/** + * 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 storm.trident.planner.processor; + +import java.util.List; +import storm.trident.operation.TridentCollector; +import storm.trident.planner.ProcessorContext; +import storm.trident.planner.TupleReceiver; +import storm.trident.tuple.TridentTuple; +import storm.trident.tuple.TridentTuple.Factory; +import storm.trident.tuple.TridentTupleView.FreshOutputFactory; + + +public class FreshCollector implements TridentCollector { + FreshOutputFactory _factory; + TridentContext _triContext; + ProcessorContext context; + + public FreshCollector(TridentContext context) { + _triContext = context; + _factory = new FreshOutputFactory(context.getSelfOutputFields()); + } + + public void setContext(ProcessorContext pc) { + this.context = pc; + } + + @Override + public void emit(List values) { + TridentTuple toEmit = _factory.create(values); + for(TupleReceiver r: _triContext.getReceivers()) { + r.execute(context, _triContext.getOutStreamId(), toEmit); + } + } + + @Override + public void reportError(Throwable t) { + _triContext.getDelegateCollector().reportError(t); + } + + public Factory getOutputFactory() { + return _factory; + } +} diff --git a/jstorm-client/src/main/java/storm/trident/planner/processor/MultiReducerProcessor.java b/jstorm-core/src/main/java/storm/trident/planner/processor/MultiReducerProcessor.java old mode 100644 new mode 100755 similarity index 78% rename from jstorm-client/src/main/java/storm/trident/planner/processor/MultiReducerProcessor.java rename to jstorm-core/src/main/java/storm/trident/planner/processor/MultiReducerProcessor.java index 1998e1a27..6777d2f65 --- a/jstorm-client/src/main/java/storm/trident/planner/processor/MultiReducerProcessor.java +++ b/jstorm-core/src/main/java/storm/trident/planner/processor/MultiReducerProcessor.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.planner.processor; import backtype.storm.task.TopologyContext; diff --git a/jstorm-client/src/main/java/storm/trident/planner/processor/PartitionPersistProcessor.java b/jstorm-core/src/main/java/storm/trident/planner/processor/PartitionPersistProcessor.java old mode 100644 new mode 100755 similarity index 81% rename from jstorm-client/src/main/java/storm/trident/planner/processor/PartitionPersistProcessor.java rename to jstorm-core/src/main/java/storm/trident/planner/processor/PartitionPersistProcessor.java index 5ab235746..e50e6d254 --- a/jstorm-client/src/main/java/storm/trident/planner/processor/PartitionPersistProcessor.java +++ b/jstorm-core/src/main/java/storm/trident/planner/processor/PartitionPersistProcessor.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.planner.processor; import backtype.storm.task.TopologyContext; diff --git a/jstorm-core/src/main/java/storm/trident/planner/processor/ProjectedProcessor.java b/jstorm-core/src/main/java/storm/trident/planner/processor/ProjectedProcessor.java new file mode 100755 index 000000000..cfa560ce4 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/planner/processor/ProjectedProcessor.java @@ -0,0 +1,73 @@ +/** + * 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 storm.trident.planner.processor; + +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Fields; +import java.util.Map; +import storm.trident.planner.ProcessorContext; +import storm.trident.planner.TridentProcessor; +import storm.trident.planner.TupleReceiver; +import storm.trident.tuple.TridentTuple; +import storm.trident.tuple.TridentTuple.Factory; +import storm.trident.tuple.TridentTupleView.ProjectionFactory; + + +public class ProjectedProcessor implements TridentProcessor { + Fields _projectFields; + ProjectionFactory _factory; + TridentContext _context; + + public ProjectedProcessor(Fields projectFields) { + _projectFields = projectFields; + } + + @Override + public void prepare(Map conf, TopologyContext context, TridentContext tridentContext) { + if(tridentContext.getParentTupleFactories().size()!=1) { + throw new RuntimeException("Projection processor can only have one parent"); + } + _context = tridentContext; + _factory = new ProjectionFactory(tridentContext.getParentTupleFactories().get(0), _projectFields); + } + + @Override + public void cleanup() { + } + + @Override + public void startBatch(ProcessorContext processorContext) { + } + + @Override + public void execute(ProcessorContext processorContext, String streamId, TridentTuple tuple) { + TridentTuple toEmit = _factory.create(tuple); + for(TupleReceiver r: _context.getReceivers()) { + r.execute(processorContext, _context.getOutStreamId(), toEmit); + } + } + + @Override + public void finishBatch(ProcessorContext processorContext) { + } + + @Override + public Factory getOutputFactory() { + return _factory; + } +} diff --git a/jstorm-client/src/main/java/storm/trident/planner/processor/StateQueryProcessor.java b/jstorm-core/src/main/java/storm/trident/planner/processor/StateQueryProcessor.java old mode 100644 new mode 100755 similarity index 80% rename from jstorm-client/src/main/java/storm/trident/planner/processor/StateQueryProcessor.java rename to jstorm-core/src/main/java/storm/trident/planner/processor/StateQueryProcessor.java index 878fb6cee..367ba5dfb --- a/jstorm-client/src/main/java/storm/trident/planner/processor/StateQueryProcessor.java +++ b/jstorm-core/src/main/java/storm/trident/planner/processor/StateQueryProcessor.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.planner.processor; import backtype.storm.task.TopologyContext; diff --git a/jstorm-core/src/main/java/storm/trident/planner/processor/TridentContext.java b/jstorm-core/src/main/java/storm/trident/planner/processor/TridentContext.java new file mode 100755 index 000000000..d887bcc4e --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/planner/processor/TridentContext.java @@ -0,0 +1,76 @@ +/** + * 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 storm.trident.planner.processor; + +import backtype.storm.coordination.BatchOutputCollector; +import backtype.storm.tuple.Fields; +import java.util.List; +import storm.trident.planner.TupleReceiver; +import storm.trident.tuple.TridentTuple.Factory; + + +public class TridentContext { + Fields selfFields; + List parentFactories; + List parentStreams; + List receivers; + String outStreamId; + int stateIndex; + BatchOutputCollector collector; + + public TridentContext(Fields selfFields, List parentFactories, + List parentStreams, List receivers, + String outStreamId, int stateIndex, BatchOutputCollector collector) { + this.selfFields = selfFields; + this.parentFactories = parentFactories; + this.parentStreams = parentStreams; + this.receivers = receivers; + this.outStreamId = outStreamId; + this.stateIndex = stateIndex; + this.collector = collector; + } + + public List getParentTupleFactories() { + return parentFactories; + } + + public Fields getSelfOutputFields() { + return selfFields; + } + + public List getParentStreams() { + return parentStreams; + } + + public List getReceivers() { + return receivers; + } + + public String getOutStreamId() { + return outStreamId; + } + + public int getStateIndex() { + return stateIndex; + } + + //for reporting errors + public BatchOutputCollector getDelegateCollector() { + return collector; + } +} diff --git a/jstorm-core/src/main/java/storm/trident/spout/BatchSpoutExecutor.java b/jstorm-core/src/main/java/storm/trident/spout/BatchSpoutExecutor.java new file mode 100755 index 000000000..69c0ed195 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/spout/BatchSpoutExecutor.java @@ -0,0 +1,92 @@ +/** + * 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 storm.trident.spout; + +import backtype.storm.task.TopologyContext; +import storm.trident.topology.TransactionAttempt; +import backtype.storm.tuple.Fields; +import java.util.Map; +import storm.trident.operation.TridentCollector; + +public class BatchSpoutExecutor implements ITridentSpout { + public static class EmptyCoordinator implements BatchCoordinator { + @Override + public Object initializeTransaction(long txid, Object prevMetadata, Object currMetadata) { + return null; + } + + @Override + public void close() { + } + + @Override + public void success(long txid) { + } + + @Override + public boolean isReady(long txid) { + return true; + } + } + + public class BatchSpoutEmitter implements Emitter { + + @Override + public void emitBatch(TransactionAttempt tx, Object coordinatorMeta, TridentCollector collector) { + _spout.emitBatch(tx.getTransactionId(), collector); + } + + @Override + public void success(TransactionAttempt tx) { + _spout.ack(tx.getTransactionId()); + } + + @Override + public void close() { + _spout.close(); + } + } + + IBatchSpout _spout; + + public BatchSpoutExecutor(IBatchSpout spout) { + _spout = spout; + } + + @Override + public BatchCoordinator getCoordinator(String txStateId, Map conf, TopologyContext context) { + return new EmptyCoordinator(); + } + + @Override + public Emitter getEmitter(String txStateId, Map conf, TopologyContext context) { + _spout.open(conf, context); + return new BatchSpoutEmitter(); + } + + @Override + public Map getComponentConfiguration() { + return _spout.getComponentConfiguration(); + } + + @Override + public Fields getOutputFields() { + return _spout.getOutputFields(); + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/spout/IBatchID.java b/jstorm-core/src/main/java/storm/trident/spout/IBatchID.java new file mode 100755 index 000000000..972b327b0 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/spout/IBatchID.java @@ -0,0 +1,24 @@ +/** + * 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 storm.trident.spout; + + +public interface IBatchID { + Object getId(); + int getAttemptId(); +} diff --git a/jstorm-core/src/main/java/storm/trident/spout/IBatchSpout.java b/jstorm-core/src/main/java/storm/trident/spout/IBatchSpout.java new file mode 100755 index 000000000..b07508272 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/spout/IBatchSpout.java @@ -0,0 +1,33 @@ +/** + * 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 storm.trident.spout; + +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Fields; +import java.io.Serializable; +import java.util.Map; +import storm.trident.operation.TridentCollector; + +public interface IBatchSpout extends Serializable { + void open(Map conf, TopologyContext context); + void emitBatch(long batchId, TridentCollector collector); + void ack(long batchId); + void close(); + Map getComponentConfiguration(); + Fields getOutputFields(); +} diff --git a/jstorm-core/src/main/java/storm/trident/spout/ICommitterTridentSpout.java b/jstorm-core/src/main/java/storm/trident/spout/ICommitterTridentSpout.java new file mode 100755 index 000000000..8495354cd --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/spout/ICommitterTridentSpout.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.spout; + +import backtype.storm.task.TopologyContext; +import storm.trident.topology.TransactionAttempt; +import java.util.Map; + +public interface ICommitterTridentSpout extends ITridentSpout { + public interface Emitter extends ITridentSpout.Emitter { + void commit(TransactionAttempt attempt); + } + + @Override + public Emitter getEmitter(String txStateId, Map conf, TopologyContext context); +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/storm/trident/spout/IOpaquePartitionedTridentSpout.java b/jstorm-core/src/main/java/storm/trident/spout/IOpaquePartitionedTridentSpout.java new file mode 100755 index 000000000..43710dd41 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/spout/IOpaquePartitionedTridentSpout.java @@ -0,0 +1,62 @@ +/** + * 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 storm.trident.spout; + +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Fields; +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import storm.trident.operation.TridentCollector; +import storm.trident.topology.TransactionAttempt; + +/** + * This defines a transactional spout which does *not* necessarily + * replay the same batch every time it emits a batch for a transaction id. + * + */ +public interface IOpaquePartitionedTridentSpout extends Serializable { + public interface Coordinator { + boolean isReady(long txid); + Partitions getPartitionsForBatch(); + void close(); + } + + public interface Emitter { + /** + * Emit a batch of tuples for a partition/transaction. + * + * Return the metadata describing this batch that will be used as lastPartitionMeta + * for defining the parameters of the next batch. + */ + M emitPartitionBatch(TransactionAttempt tx, TridentCollector collector, Partition partition, M lastPartitionMeta); + + /** + * This method is called when this task is responsible for a new set of partitions. Should be used + * to manage things like connections to brokers. + */ + void refreshPartitions(List partitionResponsibilities); + List getOrderedPartitions(Partitions allPartitionInfo); + void close(); + } + + Emitter getEmitter(Map conf, TopologyContext context); + Coordinator getCoordinator(Map conf, TopologyContext context); + Map getComponentConfiguration(); + Fields getOutputFields(); +} \ No newline at end of file diff --git a/jstorm-client/src/main/java/storm/trident/spout/IPartitionedTridentSpout.java b/jstorm-core/src/main/java/storm/trident/spout/IPartitionedTridentSpout.java old mode 100644 new mode 100755 similarity index 76% rename from jstorm-client/src/main/java/storm/trident/spout/IPartitionedTridentSpout.java rename to jstorm-core/src/main/java/storm/trident/spout/IPartitionedTridentSpout.java index c27bdc9ef..b5ea455b6 --- a/jstorm-client/src/main/java/storm/trident/spout/IPartitionedTridentSpout.java +++ b/jstorm-core/src/main/java/storm/trident/spout/IPartitionedTridentSpout.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.spout; import backtype.storm.task.TopologyContext; diff --git a/jstorm-core/src/main/java/storm/trident/spout/ISpoutPartition.java b/jstorm-core/src/main/java/storm/trident/spout/ISpoutPartition.java new file mode 100755 index 000000000..491ae3412 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/spout/ISpoutPartition.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.spout; + +public interface ISpoutPartition { + /** + * This is used as a Zookeeper node path for storing metadata. + */ + String getId(); +} diff --git a/jstorm-client/src/main/java/storm/trident/spout/ITridentSpout.java b/jstorm-core/src/main/java/storm/trident/spout/ITridentSpout.java old mode 100644 new mode 100755 similarity index 79% rename from jstorm-client/src/main/java/storm/trident/spout/ITridentSpout.java rename to jstorm-core/src/main/java/storm/trident/spout/ITridentSpout.java index 37d8c114d..2637b54f3 --- a/jstorm-client/src/main/java/storm/trident/spout/ITridentSpout.java +++ b/jstorm-core/src/main/java/storm/trident/spout/ITridentSpout.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.spout; import backtype.storm.task.TopologyContext; diff --git a/jstorm-client/src/main/java/storm/trident/spout/OpaquePartitionedTridentSpoutExecutor.java b/jstorm-core/src/main/java/storm/trident/spout/OpaquePartitionedTridentSpoutExecutor.java old mode 100644 new mode 100755 similarity index 90% rename from jstorm-client/src/main/java/storm/trident/spout/OpaquePartitionedTridentSpoutExecutor.java rename to jstorm-core/src/main/java/storm/trident/spout/OpaquePartitionedTridentSpoutExecutor.java index 0e02205a6..d1b3fe82c --- a/jstorm-client/src/main/java/storm/trident/spout/OpaquePartitionedTridentSpoutExecutor.java +++ b/jstorm-core/src/main/java/storm/trident/spout/OpaquePartitionedTridentSpoutExecutor.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.spout; diff --git a/jstorm-client/src/main/java/storm/trident/spout/PartitionedTridentSpoutExecutor.java b/jstorm-core/src/main/java/storm/trident/spout/PartitionedTridentSpoutExecutor.java old mode 100644 new mode 100755 similarity index 87% rename from jstorm-client/src/main/java/storm/trident/spout/PartitionedTridentSpoutExecutor.java rename to jstorm-core/src/main/java/storm/trident/spout/PartitionedTridentSpoutExecutor.java index 484d2750c..f96efca0f --- a/jstorm-client/src/main/java/storm/trident/spout/PartitionedTridentSpoutExecutor.java +++ b/jstorm-core/src/main/java/storm/trident/spout/PartitionedTridentSpoutExecutor.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.spout; import backtype.storm.task.TopologyContext; diff --git a/jstorm-client/src/main/java/storm/trident/spout/RichSpoutBatchExecutor.java b/jstorm-core/src/main/java/storm/trident/spout/RichSpoutBatchExecutor.java old mode 100644 new mode 100755 similarity index 86% rename from jstorm-client/src/main/java/storm/trident/spout/RichSpoutBatchExecutor.java rename to jstorm-core/src/main/java/storm/trident/spout/RichSpoutBatchExecutor.java index f6a5a5911..345a5a0cf --- a/jstorm-client/src/main/java/storm/trident/spout/RichSpoutBatchExecutor.java +++ b/jstorm-core/src/main/java/storm/trident/spout/RichSpoutBatchExecutor.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.spout; import backtype.storm.Config; @@ -123,6 +140,7 @@ private void fail(long batchId) { @Override public void close() { + _spout.close(); } } diff --git a/jstorm-core/src/main/java/storm/trident/spout/RichSpoutBatchId.java b/jstorm-core/src/main/java/storm/trident/spout/RichSpoutBatchId.java new file mode 100755 index 000000000..f8ff3c070 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/spout/RichSpoutBatchId.java @@ -0,0 +1,49 @@ +/** + * 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 storm.trident.spout; + +public class RichSpoutBatchId implements IBatchID { + long _id; + + public RichSpoutBatchId(long id) { + _id = id; + } + + @Override + public Object getId() { + // this is to distinguish from TransactionAttempt + return this; + } + + @Override + public int getAttemptId() { + return 0; // each drpc request is always a single attempt + } + + @Override + public int hashCode() { + return ((Long) _id).hashCode(); + } + + @Override + public boolean equals(Object o) { + if(!(o instanceof RichSpoutBatchId)) return false; + RichSpoutBatchId other = (RichSpoutBatchId) o; + return _id == other._id; + } +} diff --git a/jstorm-core/src/main/java/storm/trident/spout/RichSpoutBatchIdSerializer.java b/jstorm-core/src/main/java/storm/trident/spout/RichSpoutBatchIdSerializer.java new file mode 100755 index 000000000..2f0b9b93e --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/spout/RichSpoutBatchIdSerializer.java @@ -0,0 +1,38 @@ +/** + * 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 storm.trident.spout; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; + + +public class RichSpoutBatchIdSerializer extends Serializer { + + @Override + public void write(Kryo kryo, Output output, RichSpoutBatchId id) { + output.writeLong(id._id); + } + + @Override + public RichSpoutBatchId read(Kryo kryo, Input input, Class type) { + long l = input.readLong(); + return new RichSpoutBatchId(l); + } +} diff --git a/jstorm-client/src/main/java/storm/trident/spout/RichSpoutBatchTriggerer.java b/jstorm-core/src/main/java/storm/trident/spout/RichSpoutBatchTriggerer.java old mode 100644 new mode 100755 similarity index 87% rename from jstorm-client/src/main/java/storm/trident/spout/RichSpoutBatchTriggerer.java rename to jstorm-core/src/main/java/storm/trident/spout/RichSpoutBatchTriggerer.java index 39878a11a..728d51e98 --- a/jstorm-client/src/main/java/storm/trident/spout/RichSpoutBatchTriggerer.java +++ b/jstorm-core/src/main/java/storm/trident/spout/RichSpoutBatchTriggerer.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.spout; import backtype.storm.Config; diff --git a/jstorm-core/src/main/java/storm/trident/spout/TridentSpoutCoordinator.java b/jstorm-core/src/main/java/storm/trident/spout/TridentSpoutCoordinator.java new file mode 100755 index 000000000..a936e1901 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/spout/TridentSpoutCoordinator.java @@ -0,0 +1,94 @@ +/** + * 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 storm.trident.spout; + +import backtype.storm.Config; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.BasicOutputCollector; +import backtype.storm.topology.IBasicBolt; +import backtype.storm.topology.OutputFieldsDeclarer; +import storm.trident.topology.TransactionAttempt; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; +import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import storm.trident.topology.MasterBatchCoordinator; +import storm.trident.topology.state.RotatingTransactionalState; +import storm.trident.topology.state.TransactionalState; + + +public class TridentSpoutCoordinator implements IBasicBolt { + public static final Logger LOG = LoggerFactory.getLogger(TridentSpoutCoordinator.class); + private static final String META_DIR = "meta"; + + ITridentSpout _spout; + ITridentSpout.BatchCoordinator _coord; + RotatingTransactionalState _state; + TransactionalState _underlyingState; + String _id; + + + public TridentSpoutCoordinator(String id, ITridentSpout spout) { + _spout = spout; + _id = id; + } + + @Override + public void prepare(Map conf, TopologyContext context) { + _coord = _spout.getCoordinator(_id, conf, context); + _underlyingState = TransactionalState.newCoordinatorState(conf, _id); + _state = new RotatingTransactionalState(_underlyingState, META_DIR); + } + + @Override + public void execute(Tuple tuple, BasicOutputCollector collector) { + TransactionAttempt attempt = (TransactionAttempt) tuple.getValue(0); + + if(tuple.getSourceStreamId().equals(MasterBatchCoordinator.SUCCESS_STREAM_ID)) { + _state.cleanupBefore(attempt.getTransactionId()); + _coord.success(attempt.getTransactionId()); + } else { + long txid = attempt.getTransactionId(); + Object prevMeta = _state.getPreviousState(txid); + Object meta = _coord.initializeTransaction(txid, prevMeta, _state.getState(txid)); + _state.overrideState(txid, meta); + collector.emit(MasterBatchCoordinator.BATCH_STREAM_ID, new Values(attempt, meta)); + } + + } + + @Override + public void cleanup() { + _coord.close(); + _underlyingState.close(); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declareStream(MasterBatchCoordinator.BATCH_STREAM_ID, new Fields("tx", "metadata")); + } + + @Override + public Map getComponentConfiguration() { + Config ret = new Config(); + ret.setMaxTaskParallelism(1); + return ret; + } +} diff --git a/jstorm-client/src/main/java/storm/trident/spout/TridentSpoutExecutor.java b/jstorm-core/src/main/java/storm/trident/spout/TridentSpoutExecutor.java old mode 100644 new mode 100755 similarity index 80% rename from jstorm-client/src/main/java/storm/trident/spout/TridentSpoutExecutor.java rename to jstorm-core/src/main/java/storm/trident/spout/TridentSpoutExecutor.java index 10b40aeff..22b304a65 --- a/jstorm-client/src/main/java/storm/trident/spout/TridentSpoutExecutor.java +++ b/jstorm-core/src/main/java/storm/trident/spout/TridentSpoutExecutor.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.spout; import backtype.storm.coordination.BatchOutputCollector; @@ -11,7 +28,8 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; -import org.apache.log4j.Logger; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import storm.trident.operation.TridentCollector; import storm.trident.topology.BatchInfo; import storm.trident.topology.ITridentBatchBolt; @@ -21,7 +39,7 @@ public class TridentSpoutExecutor implements ITridentBatchBolt { public static String ID_FIELD = "$tx"; - public static Logger LOG = Logger.getLogger(TridentSpoutExecutor.class); + public static Logger LOG = LoggerFactory.getLogger(TridentSpoutExecutor.class); AddIdCollector _collector; ITridentSpout _spout; diff --git a/jstorm-core/src/main/java/storm/trident/state/BaseQueryFunction.java b/jstorm-core/src/main/java/storm/trident/state/BaseQueryFunction.java new file mode 100755 index 000000000..a4c01274b --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/BaseQueryFunction.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.state; + +import storm.trident.operation.BaseOperation; + + +public abstract class BaseQueryFunction extends BaseOperation implements QueryFunction { + +} diff --git a/jstorm-core/src/main/java/storm/trident/state/BaseStateUpdater.java b/jstorm-core/src/main/java/storm/trident/state/BaseStateUpdater.java new file mode 100755 index 000000000..e52669699 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/BaseStateUpdater.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.state; + +import storm.trident.operation.BaseOperation; + + +public abstract class BaseStateUpdater extends BaseOperation implements StateUpdater { + +} diff --git a/jstorm-core/src/main/java/storm/trident/state/CombinerValueUpdater.java b/jstorm-core/src/main/java/storm/trident/state/CombinerValueUpdater.java new file mode 100755 index 000000000..e22c66138 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/CombinerValueUpdater.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.state; + +import storm.trident.operation.CombinerAggregator; + +public class CombinerValueUpdater implements ValueUpdater { + Object arg; + CombinerAggregator agg; + + public CombinerValueUpdater(CombinerAggregator agg, Object arg) { + this.agg = agg; + this.arg = arg; + } + + @Override + public Object update(Object stored) { + if(stored==null) return arg; + else return agg.combine(stored, arg); + } +} diff --git a/jstorm-core/src/main/java/storm/trident/state/ITupleCollection.java b/jstorm-core/src/main/java/storm/trident/state/ITupleCollection.java new file mode 100755 index 000000000..8cc09bde1 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/ITupleCollection.java @@ -0,0 +1,26 @@ +/** + * 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 storm.trident.state; + +import java.util.Iterator; +import java.util.List; + +/* Container of a collection of tuples */ +public interface ITupleCollection { + public Iterator> getTuples(); +} diff --git a/jstorm-core/src/main/java/storm/trident/state/JSONNonTransactionalSerializer.java b/jstorm-core/src/main/java/storm/trident/state/JSONNonTransactionalSerializer.java new file mode 100755 index 000000000..bc3cc2f04 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/JSONNonTransactionalSerializer.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.state; + +import java.io.UnsupportedEncodingException; +import org.json.simple.JSONValue; + + +public class JSONNonTransactionalSerializer implements Serializer { + + @Override + public byte[] serialize(Object obj) { + try { + return JSONValue.toJSONString(obj).getBytes("UTF-8"); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } + + @Override + public Object deserialize(byte[] b) { + try { + return JSONValue.parse(new String(b, "UTF-8")); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/state/JSONOpaqueSerializer.java b/jstorm-core/src/main/java/storm/trident/state/JSONOpaqueSerializer.java new file mode 100755 index 000000000..1eb669de0 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/JSONOpaqueSerializer.java @@ -0,0 +1,52 @@ +/** + * 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 storm.trident.state; + +import java.io.UnsupportedEncodingException; +import java.util.ArrayList; +import java.util.List; +import org.json.simple.JSONValue; + + +public class JSONOpaqueSerializer implements Serializer { + + @Override + public byte[] serialize(OpaqueValue obj) { + List toSer = new ArrayList(3); + toSer.add(obj.currTxid); + toSer.add(obj.curr); + toSer.add(obj.prev); + try { + return JSONValue.toJSONString(toSer).getBytes("UTF-8"); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } + + @Override + public OpaqueValue deserialize(byte[] b) { + try { + String s = new String(b, "UTF-8"); + List deser = (List) JSONValue.parse(s); + return new OpaqueValue((Long) deser.get(0), deser.get(1), deser.get(2)); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/state/JSONTransactionalSerializer.java b/jstorm-core/src/main/java/storm/trident/state/JSONTransactionalSerializer.java new file mode 100755 index 000000000..89895a715 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/JSONTransactionalSerializer.java @@ -0,0 +1,50 @@ +/** + * 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 storm.trident.state; + +import java.io.UnsupportedEncodingException; +import java.util.ArrayList; +import java.util.List; +import org.json.simple.JSONValue; + + +public class JSONTransactionalSerializer implements Serializer { + @Override + public byte[] serialize(TransactionalValue obj) { + List toSer = new ArrayList(2); + toSer.add(obj.getTxid()); + toSer.add(obj.getVal()); + try { + return JSONValue.toJSONString(toSer).getBytes("UTF-8"); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } + + @Override + public TransactionalValue deserialize(byte[] b) { + try { + String s = new String(b, "UTF-8"); + List deser = (List) JSONValue.parse(s); + return new TransactionalValue((Long) deser.get(0), deser.get(1)); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/state/OpaqueValue.java b/jstorm-core/src/main/java/storm/trident/state/OpaqueValue.java new file mode 100755 index 000000000..bcc523afc --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/OpaqueValue.java @@ -0,0 +1,75 @@ +/** + * 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 storm.trident.state; + +import org.apache.commons.lang.builder.ToStringBuilder; + +public class OpaqueValue { + Long currTxid; + T prev; + T curr; + + public OpaqueValue(Long currTxid, T val, T prev) { + this.curr = val; + this.currTxid = currTxid; + this.prev = prev; + } + + public OpaqueValue(Long currTxid, T val) { + this(currTxid, val, null); + } + + public OpaqueValue update(Long batchTxid, T newVal) { + T prev; + if(batchTxid==null || (this.currTxid < batchTxid)) { + prev = this.curr; + } else if(batchTxid.equals(this.currTxid)){ + prev = this.prev; + } else { + throw new RuntimeException("Current batch (" + batchTxid + ") is behind state's batch: " + this.toString()); + } + return new OpaqueValue(batchTxid, newVal, prev); + } + + public T get(Long batchTxid) { + if(batchTxid==null || (this.currTxid < batchTxid)) { + return curr; + } else if(batchTxid.equals(this.currTxid)){ + return prev; + } else { + throw new RuntimeException("Current batch (" + batchTxid + ") is behind state's batch: " + this.toString()); + } + } + + public T getCurr() { + return curr; + } + + public Long getCurrTxid() { + return currTxid; + } + + public T getPrev() { + return prev; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this); + } +} diff --git a/jstorm-core/src/main/java/storm/trident/state/QueryFunction.java b/jstorm-core/src/main/java/storm/trident/state/QueryFunction.java new file mode 100755 index 000000000..41cbcdbec --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/QueryFunction.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.state; + +import java.util.List; +import storm.trident.operation.EachOperation; +import storm.trident.operation.TridentCollector; +import storm.trident.tuple.TridentTuple; + +public interface QueryFunction extends EachOperation { + List batchRetrieve(S state, List args); + void execute(TridentTuple tuple, T result, TridentCollector collector); +} diff --git a/jstorm-core/src/main/java/storm/trident/state/ReadOnlyState.java b/jstorm-core/src/main/java/storm/trident/state/ReadOnlyState.java new file mode 100755 index 000000000..a57d7f0c7 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/ReadOnlyState.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.state; + +public class ReadOnlyState implements State { + + @Override + public void beginCommit(Long txid) { + throw new UnsupportedOperationException("This state is read-only and does not support updates"); + } + + @Override + public void commit(Long txid) { + throw new UnsupportedOperationException("This state is read-only and does not support updates"); + } +} diff --git a/jstorm-core/src/main/java/storm/trident/state/ReducerValueUpdater.java b/jstorm-core/src/main/java/storm/trident/state/ReducerValueUpdater.java new file mode 100755 index 000000000..cbda82bbf --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/ReducerValueUpdater.java @@ -0,0 +1,41 @@ +/** + * 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 storm.trident.state; + +import java.util.List; +import storm.trident.operation.ReducerAggregator; +import storm.trident.tuple.TridentTuple; + +public class ReducerValueUpdater implements ValueUpdater { + List tuples; + ReducerAggregator agg; + + public ReducerValueUpdater(ReducerAggregator agg, List tuples) { + this.agg = agg; + this.tuples = tuples; + } + + @Override + public Object update(Object stored) { + Object ret = (stored == null) ? this.agg.init() : stored; + for(TridentTuple t: tuples) { + ret = this.agg.reduce(ret, t); + } + return ret; + } +} diff --git a/jstorm-core/src/main/java/storm/trident/state/Serializer.java b/jstorm-core/src/main/java/storm/trident/state/Serializer.java new file mode 100755 index 000000000..4c1c85857 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/Serializer.java @@ -0,0 +1,26 @@ +/** + * 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 storm.trident.state; + +import java.io.Serializable; + + +public interface Serializer extends Serializable { + byte[] serialize(T obj); + T deserialize(byte[] b); +} diff --git a/jstorm-core/src/main/java/storm/trident/state/State.java b/jstorm-core/src/main/java/storm/trident/state/State.java new file mode 100755 index 000000000..e27596787 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/State.java @@ -0,0 +1,39 @@ +/** + * 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 storm.trident.state; + +/** + * There's 3 different kinds of state: + * + * 1. non-transactional: ignores commits, updates are permanent. no rollback. a cassandra incrementing state would be like this + * 2. repeat-transactional: idempotent as long as all batches for a txid are identical + * 3. opaque-transactional: the most general kind of state. updates are always done + * based on the previous version of the value if the current commit = latest stored commit + * Idempotent even if the batch for a txid can change. + * + * repeat transactional is idempotent for transactional spouts + * opaque transactional is idempotent for opaque or transactional spouts + * + * Trident should log warnings when state is idempotent but updates will not be idempotent + * because of spout + */ +// retrieving is encapsulated in Retrieval interface +public interface State { + void beginCommit(Long txid); // can be null for things like partitionPersist occuring off a DRPC stream + void commit(Long txid); +} diff --git a/jstorm-core/src/main/java/storm/trident/state/StateFactory.java b/jstorm-core/src/main/java/storm/trident/state/StateFactory.java new file mode 100755 index 000000000..a896fb085 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/StateFactory.java @@ -0,0 +1,26 @@ +/** + * 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 storm.trident.state; + +import backtype.storm.task.IMetricsContext; +import java.io.Serializable; +import java.util.Map; + +public interface StateFactory extends Serializable { + State makeState(Map conf, IMetricsContext metrics, int partitionIndex, int numPartitions); +} diff --git a/jstorm-core/src/main/java/storm/trident/state/StateSpec.java b/jstorm-core/src/main/java/storm/trident/state/StateSpec.java new file mode 100755 index 000000000..c72c82289 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/StateSpec.java @@ -0,0 +1,30 @@ +/** + * 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 storm.trident.state; + +import java.io.Serializable; + + +public class StateSpec implements Serializable { + public StateFactory stateFactory; + public Integer requiredNumPartitions = null; + + public StateSpec(StateFactory stateFactory) { + this.stateFactory = stateFactory; + } +} diff --git a/jstorm-core/src/main/java/storm/trident/state/StateType.java b/jstorm-core/src/main/java/storm/trident/state/StateType.java new file mode 100755 index 000000000..a6660eae2 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/StateType.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.state; + + +public enum StateType { + NON_TRANSACTIONAL, + TRANSACTIONAL, + OPAQUE +} diff --git a/jstorm-core/src/main/java/storm/trident/state/StateUpdater.java b/jstorm-core/src/main/java/storm/trident/state/StateUpdater.java new file mode 100755 index 000000000..d702cae88 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/StateUpdater.java @@ -0,0 +1,33 @@ +/** + * 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 storm.trident.state; + +import java.util.List; +import storm.trident.operation.Operation; +import storm.trident.operation.TridentCollector; +import storm.trident.tuple.TridentTuple; + + +public interface StateUpdater extends Operation { + // maybe it needs a start phase (where it can do a retrieval, an update phase, and then a finish phase...? + // shouldn't really be a one-at-a-time interface, since we have all the tuples already? + // TOOD: used for the new values stream + // the list is needed to be able to get reduceragg and combineragg persistentaggregate + // for grouped streams working efficiently + void updateState(S state, List tuples, TridentCollector collector); +} diff --git a/jstorm-core/src/main/java/storm/trident/state/TransactionalValue.java b/jstorm-core/src/main/java/storm/trident/state/TransactionalValue.java new file mode 100755 index 000000000..83c596259 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/TransactionalValue.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.state; + +import org.apache.commons.lang.builder.ToStringBuilder; + + +public class TransactionalValue { + T val; + Long txid; + + public TransactionalValue(Long txid, T val) { + this.val = val; + this.txid = txid; + } + + public T getVal() { + return val; + } + + public Long getTxid() { + return txid; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this); + } +} diff --git a/jstorm-core/src/main/java/storm/trident/state/ValueUpdater.java b/jstorm-core/src/main/java/storm/trident/state/ValueUpdater.java new file mode 100755 index 000000000..66e406d10 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/ValueUpdater.java @@ -0,0 +1,23 @@ +/** + * 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 storm.trident.state; + + +public interface ValueUpdater { + T update(T stored); +} diff --git a/jstorm-core/src/main/java/storm/trident/state/map/CachedBatchReadsMap.java b/jstorm-core/src/main/java/storm/trident/state/map/CachedBatchReadsMap.java new file mode 100755 index 000000000..a748c0dc5 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/map/CachedBatchReadsMap.java @@ -0,0 +1,80 @@ +/** + * 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 storm.trident.state.map; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import storm.trident.state.ValueUpdater; + + +public class CachedBatchReadsMap { + public static class RetVal { + public boolean cached; + public T val; + + public RetVal(T v, boolean c) { + val = v; + cached = c; + } + } + + Map, T> _cached = new HashMap, T>(); + + public IBackingMap _delegate; + + public CachedBatchReadsMap(IBackingMap delegate) { + _delegate = delegate; + } + + public void reset() { + _cached.clear(); + } + + public List> multiGet(List> keys) { + // TODO: can optimize further by only querying backing map for keys not in the cache + List vals = _delegate.multiGet(keys); + List> ret = new ArrayList(vals.size()); + for(int i=0; i key = keys.get(i); + if(_cached.containsKey(key)) { + ret.add(new RetVal(_cached.get(key), true)); + } else { + ret.add(new RetVal(vals.get(i), false)); + } + } + return ret; + } + + public void multiPut(List> keys, List vals) { + _delegate.multiPut(keys, vals); + cache(keys, vals); + } + + private void cache(List> keys, List vals) { + for(int i=0; i key = keys.get(i); + T val = vals.get(i); + _cached.put(key, val); + } + } + + + +} diff --git a/jstorm-core/src/main/java/storm/trident/state/map/CachedMap.java b/jstorm-core/src/main/java/storm/trident/state/map/CachedMap.java new file mode 100755 index 000000000..ecc495d6d --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/map/CachedMap.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 storm.trident.state.map; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import storm.trident.util.LRUMap; + +/** + * Useful to layer over a map that communicates with a database. you generally layer opaque map over this over your database store + * @author nathan + * @param + */ +public class CachedMap implements IBackingMap { + LRUMap, T> _cache; + IBackingMap _delegate; + + public CachedMap(IBackingMap delegate, int cacheSize) { + _cache = new LRUMap, T>(cacheSize); + _delegate = delegate; + } + + @Override + public List multiGet(List> keys) { + Map, T> results = new HashMap, T>(); + List> toGet = new ArrayList>(); + for(List key: keys) { + if(_cache.containsKey(key)) { + results.put(key, _cache.get(key)); + } else { + toGet.add(key); + } + } + + List fetchedVals = _delegate.multiGet(toGet); + for(int i=0; i key = toGet.get(i); + T val = fetchedVals.get(i); + _cache.put(key, val); + results.put(key, val); + } + + List ret = new ArrayList(keys.size()); + for(List key: keys) { + ret.add(results.get(key)); + } + return ret; + } + + @Override + public void multiPut(List> keys, List values) { + cache(keys, values); + _delegate.multiPut(keys, values); + } + + private void cache(List> keys, List values) { + for(int i=0; i { + List multiGet(List> keys); + void multiPut(List> keys, List vals); +} diff --git a/jstorm-client/src/main/java/storm/trident/state/map/MapCombinerAggStateUpdater.java b/jstorm-core/src/main/java/storm/trident/state/map/MapCombinerAggStateUpdater.java old mode 100644 new mode 100755 similarity index 75% rename from jstorm-client/src/main/java/storm/trident/state/map/MapCombinerAggStateUpdater.java rename to jstorm-core/src/main/java/storm/trident/state/map/MapCombinerAggStateUpdater.java index 338c9ac8b..80a5ffe36 --- a/jstorm-client/src/main/java/storm/trident/state/map/MapCombinerAggStateUpdater.java +++ b/jstorm-core/src/main/java/storm/trident/state/map/MapCombinerAggStateUpdater.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.state.map; import backtype.storm.tuple.Fields; diff --git a/jstorm-core/src/main/java/storm/trident/state/map/MapReducerAggStateUpdater.java b/jstorm-core/src/main/java/storm/trident/state/map/MapReducerAggStateUpdater.java new file mode 100755 index 000000000..dbef36588 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/map/MapReducerAggStateUpdater.java @@ -0,0 +1,92 @@ +/** + * 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 storm.trident.state.map; + +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import storm.trident.operation.ReducerAggregator; +import storm.trident.operation.TridentCollector; +import storm.trident.operation.TridentOperationContext; +import storm.trident.state.ReducerValueUpdater; +import storm.trident.state.StateUpdater; +import storm.trident.state.ValueUpdater; +import storm.trident.tuple.ComboList; +import storm.trident.tuple.TridentTuple; +import storm.trident.tuple.TridentTupleView.ProjectionFactory; + +public class MapReducerAggStateUpdater implements StateUpdater { + ReducerAggregator _agg; + Fields _groupFields; + Fields _inputFields; + ProjectionFactory _groupFactory; + ProjectionFactory _inputFactory; + ComboList.Factory _factory; + + + public MapReducerAggStateUpdater(ReducerAggregator agg, Fields groupFields, Fields inputFields) { + _agg = agg; + _groupFields = groupFields; + _inputFields = inputFields; + _factory = new ComboList.Factory(groupFields.size(), 1); + } + + + @Override + public void updateState(MapState map, List tuples, TridentCollector collector) { + Map, List> grouped = new HashMap(); + + List> groups = new ArrayList>(tuples.size()); + List values = new ArrayList(tuples.size()); + for(TridentTuple t: tuples) { + List group = _groupFactory.create(t); + List groupTuples = grouped.get(group); + if(groupTuples==null) { + groupTuples = new ArrayList(); + grouped.put(group, groupTuples); + } + groupTuples.add(_inputFactory.create(t)); + } + List> uniqueGroups = new ArrayList(grouped.keySet()); + List updaters = new ArrayList(uniqueGroups.size()); + for(List group: uniqueGroups) { + updaters.add(new ReducerValueUpdater(_agg, grouped.get(group))); + } + List results = map.multiUpdate(uniqueGroups, updaters); + + for(int i=0; i group = uniqueGroups.get(i); + Object result = results.get(i); + collector.emit(_factory.create(new List[] {group, new Values(result) })); + } + } + + @Override + public void prepare(Map conf, TridentOperationContext context) { + _groupFactory = context.makeProjectionFactory(_groupFields); + _inputFactory = context.makeProjectionFactory(_inputFields); + } + + @Override + public void cleanup() { + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/state/map/MapState.java b/jstorm-core/src/main/java/storm/trident/state/map/MapState.java new file mode 100755 index 000000000..2681fb679 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/map/MapState.java @@ -0,0 +1,26 @@ +/** + * 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 storm.trident.state.map; + +import java.util.List; +import storm.trident.state.ValueUpdater; + +public interface MapState extends ReadOnlyMapState { + List multiUpdate(List> keys, List updaters); + void multiPut(List> keys, List vals); +} diff --git a/jstorm-core/src/main/java/storm/trident/state/map/MicroBatchIBackingMap.java b/jstorm-core/src/main/java/storm/trident/state/map/MicroBatchIBackingMap.java new file mode 100755 index 000000000..7c6998441 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/map/MicroBatchIBackingMap.java @@ -0,0 +1,85 @@ +/** + * 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 storm.trident.state.map; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; + +public class MicroBatchIBackingMap implements IBackingMap { + IBackingMap _delegate; + Options _options; + + + public static class Options implements Serializable { + public int maxMultiGetBatchSize = 0; // 0 means delegate batch size = trident batch size. + public int maxMultiPutBatchSize = 0; + } + + public MicroBatchIBackingMap(final Options options, final IBackingMap delegate) { + _options = options; + _delegate = delegate; + assert options.maxMultiPutBatchSize >= 0; + assert options.maxMultiGetBatchSize >= 0; + } + + @Override + public void multiPut(final List> keys, final List values) { + int thisBatchSize; + if(_options.maxMultiPutBatchSize == 0) { thisBatchSize = keys.size(); } + else { thisBatchSize = _options.maxMultiPutBatchSize; } + + LinkedList> keysTodo = new LinkedList>(keys); + LinkedList valuesTodo = new LinkedList(values); + + while(!keysTodo.isEmpty()) { + List> keysBatch = new ArrayList>(thisBatchSize); + List valuesBatch = new ArrayList(thisBatchSize); + for(int i=0; i multiGet(final List> keys) { + int thisBatchSize; + if(_options.maxMultiGetBatchSize == 0) { thisBatchSize = keys.size(); } + else { thisBatchSize = _options.maxMultiGetBatchSize; } + + LinkedList> keysTodo = new LinkedList>(keys); + + List ret = new ArrayList(keys.size()); + + while(!keysTodo.isEmpty()) { + List> keysBatch = new ArrayList>(thisBatchSize); + for(int i=0; i retSubset = _delegate.multiGet(keysBatch); + ret.addAll(retSubset); + } + + return ret; + } +} diff --git a/jstorm-core/src/main/java/storm/trident/state/map/NonTransactionalMap.java b/jstorm-core/src/main/java/storm/trident/state/map/NonTransactionalMap.java new file mode 100755 index 000000000..ef85532c0 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/map/NonTransactionalMap.java @@ -0,0 +1,67 @@ +/** + * 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 storm.trident.state.map; + +import storm.trident.state.ValueUpdater; + +import java.util.ArrayList; +import java.util.List; + + +public class NonTransactionalMap implements MapState { + public static MapState build(IBackingMap backing) { + return new NonTransactionalMap(backing); + } + + IBackingMap _backing; + + protected NonTransactionalMap(IBackingMap backing) { + _backing = backing; + } + + @Override + public List multiGet(List> keys) { + return _backing.multiGet(keys); + } + + @Override + public List multiUpdate(List> keys, List updaters) { + List curr = _backing.multiGet(keys); + List ret = new ArrayList(curr.size()); + for(int i=0; i updater = updaters.get(i); + ret.add(updater.update(currVal)); + } + _backing.multiPut(keys, ret); + return ret; + } + + @Override + public void multiPut(List> keys, List vals) { + _backing.multiPut(keys, vals); + } + + @Override + public void beginCommit(Long txid) { + } + + @Override + public void commit(Long txid) { + } +} diff --git a/jstorm-client/src/main/java/storm/trident/state/map/OpaqueMap.java b/jstorm-core/src/main/java/storm/trident/state/map/OpaqueMap.java old mode 100644 new mode 100755 similarity index 79% rename from jstorm-client/src/main/java/storm/trident/state/map/OpaqueMap.java rename to jstorm-core/src/main/java/storm/trident/state/map/OpaqueMap.java index f646d66a9..12f308364 --- a/jstorm-client/src/main/java/storm/trident/state/map/OpaqueMap.java +++ b/jstorm-core/src/main/java/storm/trident/state/map/OpaqueMap.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.state.map; import storm.trident.state.OpaqueValue; diff --git a/jstorm-core/src/main/java/storm/trident/state/map/ReadOnlyMapState.java b/jstorm-core/src/main/java/storm/trident/state/map/ReadOnlyMapState.java new file mode 100755 index 000000000..40039d35e --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/map/ReadOnlyMapState.java @@ -0,0 +1,26 @@ +/** + * 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 storm.trident.state.map; + +import java.util.List; +import storm.trident.state.State; + +public interface ReadOnlyMapState extends State { + // certain states might only accept one-tuple keys - those should just throw an error + List multiGet(List> keys); +} diff --git a/jstorm-core/src/main/java/storm/trident/state/map/RemovableMapState.java b/jstorm-core/src/main/java/storm/trident/state/map/RemovableMapState.java new file mode 100755 index 000000000..b9153546a --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/map/RemovableMapState.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.state.map; + +import java.util.List; +import storm.trident.state.State; + +public interface RemovableMapState extends State { + void multiRemove(List> keys); +} diff --git a/jstorm-core/src/main/java/storm/trident/state/map/SnapshottableMap.java b/jstorm-core/src/main/java/storm/trident/state/map/SnapshottableMap.java new file mode 100755 index 000000000..3d7c24ce1 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/map/SnapshottableMap.java @@ -0,0 +1,76 @@ +/** + * 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 storm.trident.state.map; + +import java.util.Arrays; +import java.util.List; +import storm.trident.state.ValueUpdater; +import storm.trident.state.snapshot.Snapshottable; + + +public class SnapshottableMap implements MapState, Snapshottable { + MapState _delegate; + List> _keys; + + public SnapshottableMap(MapState delegate, List snapshotKey) { + _delegate = delegate; + _keys = Arrays.asList(snapshotKey); + } + + @Override + public List multiGet(List> keys) { + return _delegate.multiGet(keys); + } + + @Override + public List multiUpdate(List> keys, List updaters) { + return _delegate.multiUpdate(keys, updaters); + } + + @Override + public void multiPut(List> keys, List vals) { + _delegate.multiPut(keys, vals); + } + + @Override + public void beginCommit(Long txid) { + _delegate.beginCommit(txid); + } + + @Override + public void commit(Long txid) { + _delegate.commit(txid); + } + + @Override + public T get() { + return multiGet(_keys).get(0); + } + + @Override + public T update(ValueUpdater updater) { + List updaters = Arrays.asList(updater); + return multiUpdate(_keys, updaters).get(0); + } + + @Override + public void set(T o) { + multiPut(_keys, Arrays.asList(o)); + } + +} diff --git a/jstorm-client/src/main/java/storm/trident/state/map/TransactionalMap.java b/jstorm-core/src/main/java/storm/trident/state/map/TransactionalMap.java old mode 100644 new mode 100755 similarity index 79% rename from jstorm-client/src/main/java/storm/trident/state/map/TransactionalMap.java rename to jstorm-core/src/main/java/storm/trident/state/map/TransactionalMap.java index 1f44910f6..c32766580 --- a/jstorm-client/src/main/java/storm/trident/state/map/TransactionalMap.java +++ b/jstorm-core/src/main/java/storm/trident/state/map/TransactionalMap.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.state.map; import storm.trident.state.TransactionalValue; diff --git a/jstorm-core/src/main/java/storm/trident/state/snapshot/ReadOnlySnapshottable.java b/jstorm-core/src/main/java/storm/trident/state/snapshot/ReadOnlySnapshottable.java new file mode 100755 index 000000000..645779030 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/snapshot/ReadOnlySnapshottable.java @@ -0,0 +1,24 @@ +/** + * 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 storm.trident.state.snapshot; + +import storm.trident.state.State; + +public interface ReadOnlySnapshottable extends State { + T get(); +} diff --git a/jstorm-core/src/main/java/storm/trident/state/snapshot/Snapshottable.java b/jstorm-core/src/main/java/storm/trident/state/snapshot/Snapshottable.java new file mode 100755 index 000000000..506284fe4 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/state/snapshot/Snapshottable.java @@ -0,0 +1,27 @@ +/** + * 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 storm.trident.state.snapshot; + +import storm.trident.state.ValueUpdater; + + +// used by Stream#persistentAggregate +public interface Snapshottable extends ReadOnlySnapshottable { + T update(ValueUpdater updater); + void set(T o); +} diff --git a/jstorm-core/src/main/java/storm/trident/testing/CountAsAggregator.java b/jstorm-core/src/main/java/storm/trident/testing/CountAsAggregator.java new file mode 100755 index 000000000..5c4bf2f97 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/testing/CountAsAggregator.java @@ -0,0 +1,47 @@ +/** + * 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 storm.trident.testing; + +import backtype.storm.tuple.Values; +import storm.trident.operation.BaseAggregator; +import storm.trident.operation.TridentCollector; +import storm.trident.tuple.TridentTuple; + + +public class CountAsAggregator extends BaseAggregator { + + static class State { + long count = 0; + } + + @Override + public State init(Object batchId, TridentCollector collector) { + return new State(); + } + + @Override + public void aggregate(State state, TridentTuple tuple, TridentCollector collector) { + state.count++; + } + + @Override + public void complete(State state, TridentCollector collector) { + collector.emit(new Values(state.count)); + } + +} diff --git a/jstorm-client/src/main/java/storm/trident/testing/FeederBatchSpout.java b/jstorm-core/src/main/java/storm/trident/testing/FeederBatchSpout.java old mode 100644 new mode 100755 similarity index 87% rename from jstorm-client/src/main/java/storm/trident/testing/FeederBatchSpout.java rename to jstorm-core/src/main/java/storm/trident/testing/FeederBatchSpout.java index 557115367..03357a8b5 --- a/jstorm-client/src/main/java/storm/trident/testing/FeederBatchSpout.java +++ b/jstorm-core/src/main/java/storm/trident/testing/FeederBatchSpout.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.testing; import backtype.storm.task.TopologyContext; diff --git a/jstorm-core/src/main/java/storm/trident/testing/FeederCommitterBatchSpout.java b/jstorm-core/src/main/java/storm/trident/testing/FeederCommitterBatchSpout.java new file mode 100755 index 000000000..7c1d9fe9d --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/testing/FeederCommitterBatchSpout.java @@ -0,0 +1,96 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.testing; + +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Fields; +import java.util.List; +import java.util.Map; +import storm.trident.operation.TridentCollector; +import storm.trident.spout.ICommitterTridentSpout; +import storm.trident.spout.ITridentSpout; +import storm.trident.topology.TransactionAttempt; + + +public class FeederCommitterBatchSpout implements ICommitterTridentSpout, IFeeder { + + FeederBatchSpout _spout; + + public FeederCommitterBatchSpout(List fields) { + _spout = new FeederBatchSpout(fields); + } + + public void setWaitToEmit(boolean trueIfWait) { + _spout.setWaitToEmit(trueIfWait); + } + + static class CommitterEmitter implements ICommitterTridentSpout.Emitter { + ITridentSpout.Emitter _emitter; + + + public CommitterEmitter(ITridentSpout.Emitter e) { + _emitter = e; + } + + @Override + public void commit(TransactionAttempt attempt) { + } + + @Override + public void emitBatch(TransactionAttempt tx, Object coordinatorMeta, TridentCollector collector) { + _emitter.emitBatch(tx, coordinatorMeta, collector); + } + + @Override + public void success(TransactionAttempt tx) { + _emitter.success(tx); + } + + @Override + public void close() { + _emitter.close(); + } + + } + + @Override + public Emitter getEmitter(String txStateId, Map conf, TopologyContext context) { + return new CommitterEmitter(_spout.getEmitter(txStateId, conf, context)); + } + + @Override + public BatchCoordinator getCoordinator(String txStateId, Map conf, TopologyContext context) { + return _spout.getCoordinator(txStateId, conf, context); + } + + @Override + public Fields getOutputFields() { + return _spout.getOutputFields(); + } + + @Override + public Map getComponentConfiguration() { + return _spout.getComponentConfiguration(); + } + + @Override + public void feed(Object tuples) { + _spout.feed(tuples); + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/testing/FixedBatchSpout.java b/jstorm-core/src/main/java/storm/trident/testing/FixedBatchSpout.java new file mode 100755 index 000000000..0125f7696 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/testing/FixedBatchSpout.java @@ -0,0 +1,97 @@ +/** + * 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 storm.trident.testing; + +import backtype.storm.Config; +import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Fields; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.HashMap; + +import storm.trident.operation.TridentCollector; +import storm.trident.spout.IBatchSpout; + + +public class FixedBatchSpout implements IBatchSpout { + + Fields fields; + List[] outputs; + int maxBatchSize; + HashMap>> batches = new HashMap>>(); + + public FixedBatchSpout(Fields fields, int maxBatchSize, List... outputs) { + this.fields = fields; + this.outputs = outputs; + this.maxBatchSize = maxBatchSize; + } + + int index = 0; + boolean cycle = false; + + public void setCycle(boolean cycle) { + this.cycle = cycle; + } + + @Override + public void open(Map conf, TopologyContext context) { + index = 0; + } + + @Override + public void emitBatch(long batchId, TridentCollector collector) { + List> batch = this.batches.get(batchId); + if(batch == null){ + batch = new ArrayList>(); + if(index>=outputs.length && cycle) { + index = 0; + } + for(int i=0; index < outputs.length && i < maxBatchSize; index++, i++) { + batch.add(outputs[index]); + } + this.batches.put(batchId, batch); + } + for(List list : batch){ + collector.emit(list); + } + } + + @Override + public void ack(long batchId) { + this.batches.remove(batchId); + } + + @Override + public void close() { + } + + @Override + public Map getComponentConfiguration() { + Config conf = new Config(); + conf.setMaxTaskParallelism(1); + return conf; + } + + @Override + public Fields getOutputFields() { + return fields; + } + +} \ No newline at end of file diff --git a/jstorm-core/src/main/java/storm/trident/testing/IFeeder.java b/jstorm-core/src/main/java/storm/trident/testing/IFeeder.java new file mode 100755 index 000000000..c3cc97429 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/testing/IFeeder.java @@ -0,0 +1,23 @@ +/** + * 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 storm.trident.testing; + + +public interface IFeeder { + void feed(Object tuples); +} diff --git a/jstorm-client/src/main/java/storm/trident/testing/LRUMemoryMapState.java b/jstorm-core/src/main/java/storm/trident/testing/LRUMemoryMapState.java old mode 100644 new mode 100755 similarity index 81% rename from jstorm-client/src/main/java/storm/trident/testing/LRUMemoryMapState.java rename to jstorm-core/src/main/java/storm/trident/testing/LRUMemoryMapState.java index 51c8ffb9f..32e851acf --- a/jstorm-client/src/main/java/storm/trident/testing/LRUMemoryMapState.java +++ b/jstorm-core/src/main/java/storm/trident/testing/LRUMemoryMapState.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.testing; import backtype.storm.task.IMetricsContext; @@ -86,7 +103,9 @@ public static void clearAll() { Long currTx; public LRUMemoryMapStateBacking(int cacheSize, String id) { - _dbs.putIfAbsent(id, new LRUMap, Object>(cacheSize)); + if (!_dbs.containsKey(id)) { + _dbs.put(id, new LRUMap, Object>(cacheSize)); + } this.db = (Map, T>) _dbs.get(id); } diff --git a/jstorm-core/src/main/java/storm/trident/testing/MemoryBackingMap.java b/jstorm-core/src/main/java/storm/trident/testing/MemoryBackingMap.java new file mode 100755 index 000000000..25207ff97 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/testing/MemoryBackingMap.java @@ -0,0 +1,47 @@ +/** + * 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 storm.trident.testing; + +import storm.trident.state.map.IBackingMap; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class MemoryBackingMap implements IBackingMap { + Map _vals = new HashMap(); + + @Override + public List multiGet(List> keys) { + List ret = new ArrayList(); + for(List key: keys) { + ret.add(_vals.get(key)); + } + return ret; + } + + @Override + public void multiPut(List> keys, List vals) { + for(int i=0; i, T>) _dbs.get(id); } diff --git a/jstorm-core/src/main/java/storm/trident/testing/Split.java b/jstorm-core/src/main/java/storm/trident/testing/Split.java new file mode 100755 index 000000000..4fd9cb63f --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/testing/Split.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.testing; + +import backtype.storm.tuple.Values; +import storm.trident.operation.BaseFunction; +import storm.trident.operation.TridentCollector; +import storm.trident.tuple.TridentTuple; + +public class Split extends BaseFunction { + + @Override + public void execute(TridentTuple tuple, TridentCollector collector) { + for(String word: tuple.getString(0).split(" ")) { + if(word.length() > 0) { + collector.emit(new Values(word)); + } + } + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/testing/StringLength.java b/jstorm-core/src/main/java/storm/trident/testing/StringLength.java new file mode 100755 index 000000000..3765bf74b --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/testing/StringLength.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.testing; + +import backtype.storm.tuple.Values; +import storm.trident.operation.BaseFunction; +import storm.trident.operation.TridentCollector; +import storm.trident.tuple.TridentTuple; + +public class StringLength extends BaseFunction { + + @Override + public void execute(TridentTuple tuple, TridentCollector collector) { + collector.emit(new Values(tuple.getString(0).length())); + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/testing/TrueFilter.java b/jstorm-core/src/main/java/storm/trident/testing/TrueFilter.java new file mode 100755 index 000000000..16db96ed2 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/testing/TrueFilter.java @@ -0,0 +1,30 @@ +/** + * 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 storm.trident.testing; + +import storm.trident.operation.BaseFilter; +import storm.trident.tuple.TridentTuple; + +public class TrueFilter extends BaseFilter { + + @Override + public boolean isKeep(TridentTuple tuple) { + return true; + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/testing/TuplifyArgs.java b/jstorm-core/src/main/java/storm/trident/testing/TuplifyArgs.java new file mode 100755 index 000000000..e53025bb5 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/testing/TuplifyArgs.java @@ -0,0 +1,37 @@ +/** + * 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 storm.trident.testing; + +import java.util.List; +import org.json.simple.JSONValue; +import storm.trident.operation.BaseFunction; +import storm.trident.operation.TridentCollector; +import storm.trident.tuple.TridentTuple; + +public class TuplifyArgs extends BaseFunction { + + @Override + public void execute(TridentTuple input, TridentCollector collector) { + String args = input.getString(0); + List> tuples = (List) JSONValue.parse(args); + for(List tuple: tuples) { + collector.emit(tuple); + } + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/topology/BatchInfo.java b/jstorm-core/src/main/java/storm/trident/topology/BatchInfo.java new file mode 100755 index 000000000..a5601803b --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/topology/BatchInfo.java @@ -0,0 +1,33 @@ +/** + * 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 storm.trident.topology; + +import storm.trident.spout.IBatchID; + + +public class BatchInfo { + public IBatchID batchId; + public Object state; + public String batchGroup; + + public BatchInfo(String batchGroup, IBatchID batchId, Object state) { + this.batchGroup = batchGroup; + this.batchId = batchId; + this.state = state; + } +} diff --git a/jstorm-core/src/main/java/storm/trident/topology/ITridentBatchBolt.java b/jstorm-core/src/main/java/storm/trident/topology/ITridentBatchBolt.java new file mode 100755 index 000000000..a8327588b --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/topology/ITridentBatchBolt.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package storm.trident.topology; + +import backtype.storm.coordination.BatchOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.IComponent; +import backtype.storm.tuple.Tuple; +import java.util.Map; + +public interface ITridentBatchBolt extends IComponent { + void prepare(Map conf, TopologyContext context, BatchOutputCollector collector); + void execute(BatchInfo batchInfo, Tuple tuple); + void finishBatch(BatchInfo batchInfo); + Object initBatchState(String batchGroup, Object batchId); + void cleanup(); +} diff --git a/jstorm-core/src/main/java/storm/trident/topology/MasterBatchCoordinator.java b/jstorm-core/src/main/java/storm/trident/topology/MasterBatchCoordinator.java new file mode 100755 index 000000000..78cdd8d98 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/topology/MasterBatchCoordinator.java @@ -0,0 +1,289 @@ +/** + * 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 storm.trident.topology; + +import backtype.storm.Config; +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.OutputFieldsDeclarer; +import backtype.storm.topology.base.BaseRichSpout; +import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; +import backtype.storm.utils.WindowedTimeThrottler; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.TreeMap; +import java.util.Random; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import storm.trident.spout.ITridentSpout; +import storm.trident.topology.state.TransactionalState; + +public class MasterBatchCoordinator extends BaseRichSpout { + public static final Logger LOG = LoggerFactory.getLogger(MasterBatchCoordinator.class); + + public static final long INIT_TXID = 1L; + + + public static final String BATCH_STREAM_ID = "$batch"; + public static final String COMMIT_STREAM_ID = "$commit"; + public static final String SUCCESS_STREAM_ID = "$success"; + + private static final String CURRENT_TX = "currtx"; + private static final String CURRENT_ATTEMPTS = "currattempts"; + + private List _states = new ArrayList(); + + TreeMap _activeTx = new TreeMap(); + TreeMap _attemptIds; + + private SpoutOutputCollector _collector; + Long _currTransaction; + int _maxTransactionActive; + + List _coordinators = new ArrayList(); + + + List _managedSpoutIds; + List _spouts; + WindowedTimeThrottler _throttler; + + boolean _active = true; + + public MasterBatchCoordinator(List spoutIds, List spouts) { + if(spoutIds.isEmpty()) { + throw new IllegalArgumentException("Must manage at least one spout"); + } + _managedSpoutIds = spoutIds; + _spouts = spouts; + } + + public List getManagedSpoutIds(){ + return _managedSpoutIds; + } + + @Override + public void activate() { + _active = true; + } + + @Override + public void deactivate() { + _active = false; + } + + @Override + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + _throttler = new WindowedTimeThrottler((Number)conf.get(Config.TOPOLOGY_TRIDENT_BATCH_EMIT_INTERVAL_MILLIS), 1); + for(String spoutId: _managedSpoutIds) { + _states.add(TransactionalState.newCoordinatorState(conf, spoutId)); + } + _currTransaction = getStoredCurrTransaction(); + + _collector = collector; + Number active = (Number) conf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING); + if(active==null) { + _maxTransactionActive = 1; + } else { + _maxTransactionActive = active.intValue(); + } + _attemptIds = getStoredCurrAttempts(_currTransaction, _maxTransactionActive); + + + for(int i=0; i<_spouts.size(); i++) { + String txId = _managedSpoutIds.get(i); + _coordinators.add(_spouts.get(i).getCoordinator(txId, conf, context)); + } + } + + @Override + public void close() { + for(TransactionalState state: _states) { + state.close(); + } + } + + @Override + public void nextTuple() { + sync(); + } + + @Override + public void ack(Object msgId) { + TransactionAttempt tx = (TransactionAttempt) msgId; + TransactionStatus status = _activeTx.get(tx.getTransactionId()); + if(status!=null && tx.equals(status.attempt)) { + if(status.status==AttemptStatus.PROCESSING) { + status.status = AttemptStatus.PROCESSED; + } else if(status.status==AttemptStatus.COMMITTING) { + _activeTx.remove(tx.getTransactionId()); + _attemptIds.remove(tx.getTransactionId()); + _collector.emit(SUCCESS_STREAM_ID, new Values(tx)); + _currTransaction = nextTransactionId(tx.getTransactionId()); + for(TransactionalState state: _states) { + state.setData(CURRENT_TX, _currTransaction); + } + } + sync(); + } + } + + @Override + public void fail(Object msgId) { + TransactionAttempt tx = (TransactionAttempt) msgId; + TransactionStatus stored = _activeTx.remove(tx.getTransactionId()); + if(stored!=null && tx.equals(stored.attempt)) { + _activeTx.tailMap(tx.getTransactionId()).clear(); + sync(); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + // in partitioned example, in case an emitter task receives a later transaction than it's emitted so far, + // when it sees the earlier txid it should know to emit nothing + declarer.declareStream(BATCH_STREAM_ID, new Fields("tx")); + declarer.declareStream(COMMIT_STREAM_ID, new Fields("tx")); + declarer.declareStream(SUCCESS_STREAM_ID, new Fields("tx")); + } + + private void sync() { + // note that sometimes the tuples active may be less than max_spout_pending, e.g. + // max_spout_pending = 3 + // tx 1, 2, 3 active, tx 2 is acked. there won't be a commit for tx 2 (because tx 1 isn't committed yet), + // and there won't be a batch for tx 4 because there's max_spout_pending tx active + TransactionStatus maybeCommit = _activeTx.get(_currTransaction); + if(maybeCommit!=null && maybeCommit.status == AttemptStatus.PROCESSED) { + maybeCommit.status = AttemptStatus.COMMITTING; + _collector.emit(COMMIT_STREAM_ID, new Values(maybeCommit.attempt), maybeCommit.attempt); + } + + if(_active) { + if(_activeTx.size() < _maxTransactionActive) { + Long curr = _currTransaction; + for(int i=0; i<_maxTransactionActive; i++) { + if(!_activeTx.containsKey(curr) && isReady(curr)) { + // by using a monotonically increasing attempt id, downstream tasks + // can be memory efficient by clearing out state for old attempts + // as soon as they see a higher attempt id for a transaction + Integer attemptId = _attemptIds.get(curr); + if(attemptId==null) { + attemptId = 0; + } else { + attemptId++; + } + _attemptIds.put(curr, attemptId); + for(TransactionalState state: _states) { + state.setData(CURRENT_ATTEMPTS, _attemptIds); + } + + TransactionAttempt attempt = new TransactionAttempt(curr, attemptId); + _activeTx.put(curr, new TransactionStatus(attempt)); + _collector.emit(BATCH_STREAM_ID, new Values(attempt), attempt); + _throttler.markEvent(); + } + curr = nextTransactionId(curr); + } + } + } + } + + private boolean isReady(long txid) { + if(_throttler.isThrottled()) return false; + //TODO: make this strategy configurable?... right now it goes if anyone is ready + for(ITridentSpout.BatchCoordinator coord: _coordinators) { + if(coord.isReady(txid)) return true; + } + return false; + } + + @Override + public Map getComponentConfiguration() { + Config ret = new Config(); + ret.setMaxTaskParallelism(1); + ret.registerSerialization(TransactionAttempt.class); + return ret; + } + + private static enum AttemptStatus { + PROCESSING, + PROCESSED, + COMMITTING + } + + private static class TransactionStatus { + TransactionAttempt attempt; + AttemptStatus status; + + public TransactionStatus(TransactionAttempt attempt) { + this.attempt = attempt; + this.status = AttemptStatus.PROCESSING; + } + + @Override + public String toString() { + return attempt.toString() + " <" + status.toString() + ">"; + } + } + + + private Long nextTransactionId(Long id) { + return id + 1; + } + + private Long getStoredCurrTransaction() { + Long ret = INIT_TXID; + for(TransactionalState state: _states) { + Long curr = (Long) state.getData(CURRENT_TX); + if(curr!=null && curr.compareTo(ret) > 0) { + ret = curr; + } + } + return ret; + } + + private TreeMap getStoredCurrAttempts(long currTransaction, int maxBatches) { + TreeMap ret = new TreeMap(); + for(TransactionalState state: _states) { + Map attempts = (Map) state.getData(CURRENT_ATTEMPTS); + if(attempts==null) attempts = new HashMap(); + for(Entry e: attempts.entrySet()) { + // this is because json doesn't allow numbers as keys... + // TODO: replace json with a better form of encoding + Number txidObj; + if(e.getKey() instanceof String) { + txidObj = Long.parseLong((String) e.getKey()); + } else { + txidObj = (Number) e.getKey(); + } + long txid = ((Number) txidObj).longValue(); + int attemptId = ((Number) e.getValue()).intValue(); + Integer curr = ret.get(txid); + if(curr==null || attemptId > curr) { + ret.put(txid, attemptId); + } + } + } + ret.headMap(currTransaction).clear(); + ret.tailMap(currTransaction + maxBatches - 1).clear(); + return ret; + } +} diff --git a/jstorm-core/src/main/java/storm/trident/topology/TransactionAttempt.java b/jstorm-core/src/main/java/storm/trident/topology/TransactionAttempt.java new file mode 100755 index 000000000..8bcd6a698 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/topology/TransactionAttempt.java @@ -0,0 +1,66 @@ +/** + * 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 storm.trident.topology; + +import storm.trident.spout.IBatchID; + + +public class TransactionAttempt implements IBatchID { + Long _txid; + int _attemptId; + + + // for kryo compatibility + public TransactionAttempt() { + + } + + public TransactionAttempt(Long txid, int attemptId) { + _txid = txid; + _attemptId = attemptId; + } + + public Long getTransactionId() { + return _txid; + } + + public Object getId() { + return _txid; + } + + public int getAttemptId() { + return _attemptId; + } + + @Override + public int hashCode() { + return _txid.hashCode(); + } + + @Override + public boolean equals(Object o) { + if(!(o instanceof TransactionAttempt)) return false; + TransactionAttempt other = (TransactionAttempt) o; + return _txid.equals(other._txid) && _attemptId == other._attemptId; + } + + @Override + public String toString() { + return "" + _txid + ":" + _attemptId; + } +} \ No newline at end of file diff --git a/jstorm-client/src/main/java/storm/trident/topology/TridentBoltExecutor.java b/jstorm-core/src/main/java/storm/trident/topology/TridentBoltExecutor.java old mode 100644 new mode 100755 similarity index 93% rename from jstorm-client/src/main/java/storm/trident/topology/TridentBoltExecutor.java rename to jstorm-core/src/main/java/storm/trident/topology/TridentBoltExecutor.java index 71807bb9a..a23e5559e --- a/jstorm-client/src/main/java/storm/trident/topology/TridentBoltExecutor.java +++ b/jstorm-core/src/main/java/storm/trident/topology/TridentBoltExecutor.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.topology; import backtype.storm.Config; @@ -17,8 +34,8 @@ import backtype.storm.tuple.Tuple; import backtype.storm.tuple.Values; import backtype.storm.utils.RotatingMap; +import backtype.storm.utils.TupleUtils; import backtype.storm.utils.Utils; - import java.io.Serializable; import java.util.Arrays; import java.util.Collection; @@ -27,9 +44,7 @@ import java.util.List; import java.util.Map; import java.util.Set; - import org.apache.commons.lang.builder.ToStringBuilder; - import storm.trident.spout.IBatchID; public class TridentBoltExecutor implements IRichBolt { @@ -54,34 +69,15 @@ public static CoordType all() { return new CoordType(false); } - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + (singleCount ? 1231 : 1237); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - CoordType other = (CoordType) obj; - if (singleCount != other.singleCount) - return false; - return true; - } + @Override + public boolean equals(Object o) { + return singleCount == ((CoordType) o).singleCount; + } @Override public String toString() { return ""; } - - } public static class CoordSpec implements Serializable { @@ -304,7 +300,7 @@ private void checkFinish(TrackedBatch tracked, Tuple tuple, TupleType type) { @Override public void execute(Tuple tuple) { - if(tuple.getSourceStreamId().equals(Constants.SYSTEM_TICK_STREAM_ID)) { + if(TupleUtils.isTick(tuple)) { long now = System.currentTimeMillis(); if(now - _lastRotate > _messageTimeoutMs) { _batches.rotate(); diff --git a/jstorm-client/src/main/java/storm/trident/topology/TridentTopologyBuilder.java b/jstorm-core/src/main/java/storm/trident/topology/TridentTopologyBuilder.java old mode 100644 new mode 100755 similarity index 94% rename from jstorm-client/src/main/java/storm/trident/topology/TridentTopologyBuilder.java rename to jstorm-core/src/main/java/storm/trident/topology/TridentTopologyBuilder.java index 1e75e00fa..8e4ed4848 --- a/jstorm-client/src/main/java/storm/trident/topology/TridentTopologyBuilder.java +++ b/jstorm-core/src/main/java/storm/trident/topology/TridentTopologyBuilder.java @@ -1,9 +1,27 @@ +/** + * 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 storm.trident.topology; import backtype.storm.generated.GlobalStreamId; import backtype.storm.generated.Grouping; import backtype.storm.generated.StormTopology; import backtype.storm.grouping.CustomStreamGrouping; +import backtype.storm.grouping.PartialKeyGrouping; import backtype.storm.topology.BaseConfigurationDeclarer; import backtype.storm.topology.BoltDeclarer; import backtype.storm.topology.IRichSpout; @@ -11,14 +29,17 @@ import backtype.storm.topology.SpoutDeclarer; import backtype.storm.topology.TopologyBuilder; import backtype.storm.tuple.Fields; + 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 org.apache.commons.lang.builder.ToStringBuilder; import org.apache.commons.lang.builder.ToStringStyle; + import storm.trident.spout.BatchSpoutExecutor; import storm.trident.spout.IBatchSpout; import storm.trident.spout.ICommitterTridentSpout; @@ -141,8 +162,7 @@ public StormTopology buildTopology() { BoltDeclarer scd = builder.setBolt(spoutCoordinator(id), new TridentSpoutCoordinator(c.commitStateId, (ITridentSpout) c.spout)) .globalGrouping(masterCoordinator(c.batchGroupId), MasterBatchCoordinator.BATCH_STREAM_ID) - .globalGrouping(masterCoordinator(c.batchGroupId), MasterBatchCoordinator.SUCCESS_STREAM_ID) - .globalGrouping(masterCoordinator(c.batchGroupId), MasterBatchCoordinator.COMMIT_STREAM_ID); + .globalGrouping(masterCoordinator(c.batchGroupId), MasterBatchCoordinator.SUCCESS_STREAM_ID); for(Map m: c.componentConfs) { scd.addConfigurations(m); @@ -178,6 +198,11 @@ public StormTopology buildTopology() { d.addConfigurations(conf); } } + + for(String batch: batchesToCommitIds.keySet()) { + List commitIds = batchesToCommitIds.get(batch); + builder.setSpout(masterCoordinator(batch), new MasterBatchCoordinator(commitIds, batchesToSpouts.get(batch))); + } for(String id: _bolts.keySet()) { Component c = _bolts.get(id); @@ -221,12 +246,6 @@ public StormTopology buildTopology() { d.allGrouping(masterCoordinator(b), MasterBatchCoordinator.COMMIT_STREAM_ID); } } - - for(String batch: batchesToCommitIds.keySet()) { - List commitIds = batchesToCommitIds.get(batch); - boolean batchCommit = false; - builder.setSpout(masterCoordinator(batch), new MasterBatchCoordinator(commitIds, batchesToSpouts.get(batch))); - } return builder.createTopology(); } @@ -506,16 +525,17 @@ public String getStream() { } @Override - public BoltDeclarer localFirstGrouping(final String component) { + public BoltDeclarer localFirstGrouping(final String componentId) { + // TODO Auto-generated method stub addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { - declarer.localFirstGrouping(component); + declarer.localFirstGrouping(componentId); } @Override public String getComponent() { - return component; + return componentId; } @Override @@ -527,16 +547,17 @@ public String getStream() { } @Override - public BoltDeclarer localFirstGrouping(final String component, final String streamId) { + public BoltDeclarer localFirstGrouping(final String componentId, final String streamId) { + // TODO Auto-generated method stub addDeclaration(new InputDeclaration() { @Override public void declare(InputDeclarer declarer) { - declarer.localFirstGrouping(component, streamId); + declarer.localFirstGrouping(componentId, streamId); } @Override public String getComponent() { - return component; + return componentId; } @Override @@ -672,7 +693,17 @@ public String getStream() { }); return this; } - + + @Override + public BoltDeclarer partialKeyGrouping(String componentId, Fields fields) { + return customGrouping(componentId, new PartialKeyGrouping(fields)); + } + + @Override + public BoltDeclarer partialKeyGrouping(String componentId, String streamId, Fields fields) { + return customGrouping(componentId, streamId, new PartialKeyGrouping(fields)); + } + @Override public BoltDeclarer customGrouping(final String component, final CustomStreamGrouping grouping) { addDeclaration(new InputDeclaration() { @@ -746,6 +777,6 @@ public BoltDeclarer addConfigurations(Map conf) { return this; } - + } } diff --git a/jstorm-core/src/main/java/storm/trident/topology/state/RotatingTransactionalState.java b/jstorm-core/src/main/java/storm/trident/topology/state/RotatingTransactionalState.java new file mode 100755 index 000000000..9abafd67c --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/topology/state/RotatingTransactionalState.java @@ -0,0 +1,147 @@ +/** + * 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 storm.trident.topology.state; + +import backtype.storm.utils.Utils; +import org.apache.zookeeper.KeeperException; + +import java.util.HashSet; +import java.util.List; +import java.util.SortedMap; +import java.util.TreeMap; + +public class RotatingTransactionalState { + public static interface StateInitializer { + Object init(long txid, Object lastState); + } + + private TransactionalState _state; + private String _subdir; + + private TreeMap _curr = new TreeMap(); + + public RotatingTransactionalState(TransactionalState state, String subdir) { + _state = state; + _subdir = subdir; + state.mkdir(subdir); + sync(); + } + + + public Object getLastState() { + if(_curr.isEmpty()) return null; + else return _curr.lastEntry().getValue(); + } + + public void overrideState(long txid, Object state) { + _state.setData(txPath(txid), state); + _curr.put(txid, state); + } + + public void removeState(long txid) { + if(_curr.containsKey(txid)) { + _curr.remove(txid); + _state.delete(txPath(txid)); + } + } + + public Object getState(long txid) { + return _curr.get(txid); + } + + public Object getState(long txid, StateInitializer init) { + if(!_curr.containsKey(txid)) { + SortedMap prevMap = _curr.headMap(txid); + SortedMap afterMap = _curr.tailMap(txid); + + Long prev = null; + if(!prevMap.isEmpty()) prev = prevMap.lastKey(); + + Object data; + if(afterMap.isEmpty()) { + Object prevData; + if(prev!=null) { + prevData = _curr.get(prev); + } else { + prevData = null; + } + data = init.init(txid, prevData); + } else { + data = null; + } + _curr.put(txid, data); + _state.setData(txPath(txid), data); + } + return _curr.get(txid); + } + + public Object getPreviousState(long txid) { + SortedMap prevMap = _curr.headMap(txid); + if(prevMap.isEmpty()) return null; + else return prevMap.get(prevMap.lastKey()); + } + + public boolean hasCache(long txid) { + return _curr.containsKey(txid); + } + + /** + * Returns null if it was created, the value otherwise. + */ + public Object getStateOrCreate(long txid, StateInitializer init) { + if(_curr.containsKey(txid)) { + return _curr.get(txid); + } else { + getState(txid, init); + return null; + } + } + + public void cleanupBefore(long txid) { + SortedMap toDelete = _curr.headMap(txid); + for(long tx: new HashSet(toDelete.keySet())) { + _curr.remove(tx); + try { + _state.delete(txPath(tx)); + } catch(RuntimeException e) { + // Ignore NoNodeExists exceptions because when sync() it may populate _curr with stale data since + // zookeeper reads are eventually consistent. + if(!Utils.exceptionCauseIsInstanceOf(KeeperException.NoNodeException.class, e)) { + throw e; + } + } + } + } + + private void sync() { + List txids = _state.list(_subdir); + for(String txid_s: txids) { + Object data = _state.getData(txPath(txid_s)); + _curr.put(Long.parseLong(txid_s), data); + } + } + + private String txPath(long tx) { + return txPath("" + tx); + } + + private String txPath(String tx) { + return _subdir + "/" + tx; + } + +} diff --git a/jstorm-core/src/main/java/storm/trident/topology/state/TestTransactionalState.java b/jstorm-core/src/main/java/storm/trident/topology/state/TestTransactionalState.java new file mode 100755 index 000000000..ff3edb61a --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/topology/state/TestTransactionalState.java @@ -0,0 +1,47 @@ +/** + * 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 storm.trident.topology.state; + +import java.util.List; +import java.util.Map; + +import backtype.storm.utils.ZookeeperAuthInfo; +import org.apache.curator.framework.CuratorFramework; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.data.ACL; + +/** + * Facilitates testing of non-public methods in the parent class. + */ +public class TestTransactionalState extends TransactionalState { + + /** + * Matching constructor in absence of a default constructor in the parent + * class. + */ + protected TestTransactionalState(Map conf, String id, String subroot) { + super(conf, id, subroot); + } + + public static void createNode(CuratorFramework curator, + String rootDir, byte[] data, List acls, CreateMode mode) + throws Exception { + TransactionalState.createNode(curator, rootDir, data, acls, mode); + } +} diff --git a/jstorm-core/src/main/java/storm/trident/topology/state/TransactionalState.java b/jstorm-core/src/main/java/storm/trident/topology/state/TransactionalState.java new file mode 100755 index 000000000..5fba1a22d --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/topology/state/TransactionalState.java @@ -0,0 +1,171 @@ +/** + * 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 storm.trident.topology.state; + + +import backtype.storm.Config; +import backtype.storm.utils.Utils; +import backtype.storm.utils.ZookeeperAuthInfo; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.api.ProtectACLCreateModePathAndBytesable; +import org.apache.curator.framework.api.PathAndBytesable; + +import java.io.UnsupportedEncodingException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.data.Id; +import org.json.simple.JSONValue; + +public class TransactionalState { + CuratorFramework _curator; + List _zkAcls = null; + + public static TransactionalState newUserState(Map conf, String id) { + return new TransactionalState(conf, id, "user"); + } + + public static TransactionalState newCoordinatorState(Map conf, String id) { + return new TransactionalState(conf, id, "coordinator"); + } + + protected TransactionalState(Map conf, String id, String subroot) { + try { + conf = new HashMap(conf); + String transactionalRoot = (String)conf.get(Config.TRANSACTIONAL_ZOOKEEPER_ROOT); + String rootDir = transactionalRoot + "/" + id + "/" + subroot; + List servers = (List) getWithBackup(conf, Config.TRANSACTIONAL_ZOOKEEPER_SERVERS, Config.STORM_ZOOKEEPER_SERVERS); + Object port = getWithBackup(conf, Config.TRANSACTIONAL_ZOOKEEPER_PORT, Config.STORM_ZOOKEEPER_PORT); + ZookeeperAuthInfo auth = new ZookeeperAuthInfo(conf); + CuratorFramework initter = Utils.newCuratorStarted(conf, servers, port, auth); + _zkAcls = Utils.getWorkerACL(conf); + try { + TransactionalState.createNode(initter, transactionalRoot, null, null, null); + } catch (KeeperException.NodeExistsException e) { + } + try { + TransactionalState.createNode(initter, rootDir, null, _zkAcls, null); + } catch (KeeperException.NodeExistsException e) { + } + initter.close(); + + _curator = Utils.newCuratorStarted(conf, servers, port, rootDir, auth); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + protected static String forPath(PathAndBytesable builder, + String path, byte[] data) throws Exception { + return (data == null) + ? builder.forPath(path) + : builder.forPath(path, data); + } + + protected static void createNode(CuratorFramework curator, String path, + byte[] data, List acls, CreateMode mode) throws Exception { + ProtectACLCreateModePathAndBytesable builder = + curator.create().creatingParentsIfNeeded(); + + if (acls == null) { + if (mode == null ) { + TransactionalState.forPath(builder, path, data); + } else { + TransactionalState.forPath(builder.withMode(mode), path, data); + } + return; + } + + TransactionalState.forPath(builder.withACL(acls), path, data); + } + + public void setData(String path, Object obj) { + path = "/" + path; + byte[] ser; + try { + ser = JSONValue.toJSONString(obj).getBytes("UTF-8"); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + try { + if(_curator.checkExists().forPath(path)!=null) { + _curator.setData().forPath(path, ser); + } else { + TransactionalState.createNode(_curator, path, ser, _zkAcls, + CreateMode.PERSISTENT); + } + } catch(Exception e) { + throw new RuntimeException(e); + } + } + + public void delete(String path) { + path = "/" + path; + try { + _curator.delete().forPath(path); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public List list(String path) { + path = "/" + path; + try { + if(_curator.checkExists().forPath(path)==null) { + return new ArrayList(); + } else { + return _curator.getChildren().forPath(path); + } + } catch(Exception e) { + throw new RuntimeException(e); + } + } + + public void mkdir(String path) { + setData(path, 7); + } + + public Object getData(String path) { + path = "/" + path; + try { + if(_curator.checkExists().forPath(path)!=null) { + return JSONValue.parse(new String(_curator.getData().forPath(path), "UTF-8")); + } else { + return null; + } + } catch(Exception e) { + throw new RuntimeException(e); + } + } + + public void close() { + _curator.close(); + } + + private Object getWithBackup(Map amap, Object primary, Object backup) { + Object ret = amap.get(primary); + if(ret==null) return amap.get(backup); + return ret; + } +} diff --git a/jstorm-core/src/main/java/storm/trident/tuple/ComboList.java b/jstorm-core/src/main/java/storm/trident/tuple/ComboList.java new file mode 100755 index 000000000..006f1680e --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/tuple/ComboList.java @@ -0,0 +1,92 @@ +/** + * 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 storm.trident.tuple; + +import java.io.Serializable; +import java.util.AbstractList; +import java.util.List; +import org.apache.commons.lang.builder.ToStringBuilder; + + +public class ComboList extends AbstractList { + public static class Factory implements Serializable { + Pointer[] index; + int[] sizes; + + public Factory(int... sizes) { + this.sizes = sizes; + int total = 0; + for(int size: sizes) { + total+=size; + } + index = new Pointer[total]; + int i=0; + int j=0; + for(int size: sizes) { + for(int z=0; z { + List _elems; + Object _first; + + public ConsList(Object o, List elems) { + _elems = elems; + _first = o; + } + + @Override + public Object get(int i) { + if(i==0) return _first; + else { + return _elems.get(i - 1); + } + } + + @Override + public int size() { + return _elems.size() + 1; + } +} diff --git a/jstorm-core/src/main/java/storm/trident/tuple/TridentTuple.java b/jstorm-core/src/main/java/storm/trident/tuple/TridentTuple.java new file mode 100755 index 000000000..3dd7e8f30 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/tuple/TridentTuple.java @@ -0,0 +1,34 @@ +/** + * 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 storm.trident.tuple; + +import backtype.storm.tuple.ITuple; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; + +public interface TridentTuple extends ITuple, List { + + public static interface Factory extends Serializable { + Map getFieldIndex(); + List getOutputFields(); + int numDelegates(); + } + +} diff --git a/jstorm-client/src/main/java/storm/trident/tuple/TridentTupleView.java b/jstorm-core/src/main/java/storm/trident/tuple/TridentTupleView.java old mode 100644 new mode 100755 similarity index 92% rename from jstorm-client/src/main/java/storm/trident/tuple/TridentTupleView.java rename to jstorm-core/src/main/java/storm/trident/tuple/TridentTupleView.java index 17f3e3fb7..0be0c3fb0 --- a/jstorm-client/src/main/java/storm/trident/tuple/TridentTupleView.java +++ b/jstorm-core/src/main/java/storm/trident/tuple/TridentTupleView.java @@ -1,3 +1,20 @@ +/** + * 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 storm.trident.tuple; import backtype.storm.tuple.Fields; diff --git a/jstorm-core/src/main/java/storm/trident/tuple/ValuePointer.java b/jstorm-core/src/main/java/storm/trident/tuple/ValuePointer.java new file mode 100755 index 000000000..93470d78a --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/tuple/ValuePointer.java @@ -0,0 +1,60 @@ +/** + * 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 storm.trident.tuple; + +import backtype.storm.tuple.Fields; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.commons.lang.builder.ToStringBuilder; + +public class ValuePointer { + public static Map buildFieldIndex(ValuePointer[] pointers) { + Map ret = new HashMap(); + for(ValuePointer ptr: pointers) { + ret.put(ptr.field, ptr); + } + return ret; + } + + public static ValuePointer[] buildIndex(Fields fieldsOrder, Map pointers) { + if(fieldsOrder.size()!=pointers.size()) { + throw new IllegalArgumentException("Fields order must be same length as pointers map"); + } + ValuePointer[] ret = new ValuePointer[pointers.size()]; + for(int i=0; i implements Comparable, Serializable { + public T source; + public T target; + public int index; + + public IndexedEdge(T source, T target, int index) { + this.source = source; + this.target = target; + this.index = index; + } + + @Override + public int hashCode() { + return 13* source.hashCode() + 7 * target.hashCode() + index; + } + + @Override + public boolean equals(Object o) { + IndexedEdge other = (IndexedEdge) o; + return source.equals(other.source) && target.equals(other.target) && index == other.index; + } + + @Override + public int compareTo(Object t) { + IndexedEdge other = (IndexedEdge) t; + return index - other.index; + } +} diff --git a/jstorm-core/src/main/java/storm/trident/util/LRUMap.java b/jstorm-core/src/main/java/storm/trident/util/LRUMap.java new file mode 100755 index 000000000..986538420 --- /dev/null +++ b/jstorm-core/src/main/java/storm/trident/util/LRUMap.java @@ -0,0 +1,35 @@ +/** + * 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 storm.trident.util; + +import java.util.LinkedHashMap; +import java.util.Map; + +public class LRUMap extends LinkedHashMap { + private int _maxSize; + + public LRUMap(int maxSize) { + super(maxSize + 1, 1.0f, true); + _maxSize = maxSize; + } + + @Override + protected boolean removeEldestEntry(final Map.Entry eldest) { + return size() > _maxSize; + } +} diff --git a/jstorm-client/src/main/java/storm/trident/util/TridentUtils.java b/jstorm-core/src/main/java/storm/trident/util/TridentUtils.java old mode 100644 new mode 100755 similarity index 79% rename from jstorm-client/src/main/java/storm/trident/util/TridentUtils.java rename to jstorm-core/src/main/java/storm/trident/util/TridentUtils.java index 00597219e..0cdec0aec --- a/jstorm-client/src/main/java/storm/trident/util/TridentUtils.java +++ b/jstorm-core/src/main/java/storm/trident/util/TridentUtils.java @@ -1,23 +1,40 @@ +/** + * 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 storm.trident.util; +import backtype.storm.generated.StreamInfo; +import backtype.storm.topology.IComponent; +import backtype.storm.topology.OutputFieldsGetter; +import backtype.storm.tuple.Fields; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; - -import org.apache.thrift7.TBase; -import org.apache.thrift7.TDeserializer; -import org.apache.thrift7.TException; -import org.apache.thrift7.TSerializer; +import java.util.logging.Level; +import java.util.logging.Logger; +import org.apache.thrift.TBase; +import org.apache.thrift.TDeserializer; +import org.apache.thrift.TException; +import org.apache.thrift.TSerializer; import org.jgrapht.DirectedGraph; -import backtype.storm.generated.StreamInfo; -import backtype.storm.topology.IComponent; -import backtype.storm.topology.OutputFieldsGetter; -import backtype.storm.tuple.Fields; - public class TridentUtils { public static Fields fieldsUnion(Fields... fields) { Set ret = new HashSet(); diff --git a/jstorm-client/src/main/py/__init__.py b/jstorm-core/src/main/py/__init__.py old mode 100644 new mode 100755 similarity index 100% rename from jstorm-client/src/main/py/__init__.py rename to jstorm-core/src/main/py/__init__.py diff --git a/jstorm-core/src/main/py/storm/DistributedRPC-remote b/jstorm-core/src/main/py/storm/DistributedRPC-remote new file mode 100755 index 000000000..282b132fe --- /dev/null +++ b/jstorm-core/src/main/py/storm/DistributedRPC-remote @@ -0,0 +1,94 @@ +#!/usr/bin/env python +# +# Autogenerated by Thrift Compiler (0.9.2) +# +# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING +# +# options string: py:utf8strings +# + +import sys +import pprint +from urlparse import urlparse +from thrift.transport import TTransport +from thrift.transport import TSocket +from thrift.transport import TSSLSocket +from thrift.transport import THttpClient +from thrift.protocol import TBinaryProtocol + +from storm import DistributedRPC +from storm.ttypes import * + +if len(sys.argv) <= 1 or sys.argv[1] == '--help': + print('') + print('Usage: ' + sys.argv[0] + ' [-h host[:port]] [-u url] [-f[ramed]] [-s[sl]] function [arg1 [arg2...]]') + print('') + print('Functions:') + print(' string execute(string functionName, string funcArgs)') + print('') + sys.exit(0) + +pp = pprint.PrettyPrinter(indent = 2) +host = 'localhost' +port = 9090 +uri = '' +framed = False +ssl = False +http = False +argi = 1 + +if sys.argv[argi] == '-h': + parts = sys.argv[argi+1].split(':') + host = parts[0] + if len(parts) > 1: + port = int(parts[1]) + argi += 2 + +if sys.argv[argi] == '-u': + url = urlparse(sys.argv[argi+1]) + parts = url[1].split(':') + host = parts[0] + if len(parts) > 1: + port = int(parts[1]) + else: + port = 80 + uri = url[2] + if url[4]: + uri += '?%s' % url[4] + http = True + argi += 2 + +if sys.argv[argi] == '-f' or sys.argv[argi] == '-framed': + framed = True + argi += 1 + +if sys.argv[argi] == '-s' or sys.argv[argi] == '-ssl': + ssl = True + argi += 1 + +cmd = sys.argv[argi] +args = sys.argv[argi+1:] + +if http: + transport = THttpClient.THttpClient(host, port, uri) +else: + socket = TSSLSocket.TSSLSocket(host, port, validate=False) if ssl else TSocket.TSocket(host, port) + if framed: + transport = TTransport.TFramedTransport(socket) + else: + transport = TTransport.TBufferedTransport(socket) +protocol = TBinaryProtocol.TBinaryProtocol(transport) +client = DistributedRPC.Client(protocol) +transport.open() + +if cmd == 'execute': + if len(args) != 2: + print('execute requires 2 args') + sys.exit(1) + pp.pprint(client.execute(args[0],args[1],)) + +else: + print('Unrecognized method %s' % cmd) + sys.exit(1) + +transport.close() diff --git a/jstorm-client/src/main/py/storm/DistributedRPC.py b/jstorm-core/src/main/py/storm/DistributedRPC.py old mode 100644 new mode 100755 similarity index 92% rename from jstorm-client/src/main/py/storm/DistributedRPC.py rename to jstorm-core/src/main/py/storm/DistributedRPC.py index a7e6ef9d2..c3248083e --- a/jstorm-client/src/main/py/storm/DistributedRPC.py +++ b/jstorm-core/src/main/py/storm/DistributedRPC.py @@ -1,10 +1,12 @@ # -# Autogenerated by Thrift Compiler (0.7.0) +# Autogenerated by Thrift Compiler (0.9.2) # # DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING # +# options string: py:utf8strings +# -from thrift.Thrift import * +from thrift.Thrift import TType, TMessageType, TException, TApplicationException from ttypes import * from thrift.Thrift import TProcessor from thrift.transport import TTransport @@ -50,16 +52,17 @@ def send_execute(self, functionName, funcArgs): self._oprot.writeMessageEnd() self._oprot.trans.flush() - def recv_execute(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() + def recv_execute(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() + x.read(iprot) + iprot.readMessageEnd() raise x result = execute_result() - result.read(self._iprot) - self._iprot.readMessageEnd() + result.read(iprot) + iprot.readMessageEnd() if result.success is not None: return result.success if result.e is not None: @@ -118,9 +121,6 @@ class execute_args: (2, TType.STRING, 'funcArgs', None, None, ), # 2 ) - def __hash__(self): - return 0 + hash(self.functionName) + hash(self.funcArgs) - def __init__(self, functionName=None, funcArgs=None,): self.functionName = functionName self.funcArgs = funcArgs @@ -169,6 +169,12 @@ def validate(self): return + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.functionName) + value = (value * 31) ^ hash(self.funcArgs) + return value + def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] @@ -192,9 +198,6 @@ class execute_result: (1, TType.STRUCT, 'e', (DRPCExecutionException, DRPCExecutionException.thrift_spec), None, ), # 1 ) - def __hash__(self): - return 0 + hash(self.success) + hash(self.e) - def __init__(self, success=None, e=None,): self.success = success self.e = e @@ -244,6 +247,12 @@ def validate(self): return + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + value = (value * 31) ^ hash(self.e) + return value + def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] diff --git a/jstorm-core/src/main/py/storm/DistributedRPCInvocations-remote b/jstorm-core/src/main/py/storm/DistributedRPCInvocations-remote new file mode 100755 index 000000000..93cf153b0 --- /dev/null +++ b/jstorm-core/src/main/py/storm/DistributedRPCInvocations-remote @@ -0,0 +1,108 @@ +#!/usr/bin/env python +# +# Autogenerated by Thrift Compiler (0.9.2) +# +# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING +# +# options string: py:utf8strings +# + +import sys +import pprint +from urlparse import urlparse +from thrift.transport import TTransport +from thrift.transport import TSocket +from thrift.transport import TSSLSocket +from thrift.transport import THttpClient +from thrift.protocol import TBinaryProtocol + +from storm import DistributedRPCInvocations +from storm.ttypes import * + +if len(sys.argv) <= 1 or sys.argv[1] == '--help': + print('') + print('Usage: ' + sys.argv[0] + ' [-h host[:port]] [-u url] [-f[ramed]] [-s[sl]] function [arg1 [arg2...]]') + print('') + print('Functions:') + print(' void result(string id, string result)') + print(' DRPCRequest fetchRequest(string functionName)') + print(' void failRequest(string id)') + print('') + sys.exit(0) + +pp = pprint.PrettyPrinter(indent = 2) +host = 'localhost' +port = 9090 +uri = '' +framed = False +ssl = False +http = False +argi = 1 + +if sys.argv[argi] == '-h': + parts = sys.argv[argi+1].split(':') + host = parts[0] + if len(parts) > 1: + port = int(parts[1]) + argi += 2 + +if sys.argv[argi] == '-u': + url = urlparse(sys.argv[argi+1]) + parts = url[1].split(':') + host = parts[0] + if len(parts) > 1: + port = int(parts[1]) + else: + port = 80 + uri = url[2] + if url[4]: + uri += '?%s' % url[4] + http = True + argi += 2 + +if sys.argv[argi] == '-f' or sys.argv[argi] == '-framed': + framed = True + argi += 1 + +if sys.argv[argi] == '-s' or sys.argv[argi] == '-ssl': + ssl = True + argi += 1 + +cmd = sys.argv[argi] +args = sys.argv[argi+1:] + +if http: + transport = THttpClient.THttpClient(host, port, uri) +else: + socket = TSSLSocket.TSSLSocket(host, port, validate=False) if ssl else TSocket.TSocket(host, port) + if framed: + transport = TTransport.TFramedTransport(socket) + else: + transport = TTransport.TBufferedTransport(socket) +protocol = TBinaryProtocol.TBinaryProtocol(transport) +client = DistributedRPCInvocations.Client(protocol) +transport.open() + +if cmd == 'result': + if len(args) != 2: + print('result requires 2 args') + sys.exit(1) + pp.pprint(client.result(args[0],args[1],)) + +elif cmd == 'fetchRequest': + if len(args) != 1: + print('fetchRequest requires 1 args') + sys.exit(1) + pp.pprint(client.fetchRequest(args[0],)) + +elif cmd == 'failRequest': + if len(args) != 1: + print('failRequest requires 1 args') + sys.exit(1) + pp.pprint(client.failRequest(args[0],)) + +else: + print('Unrecognized method %s' % cmd) + sys.exit(1) + +transport.close() diff --git a/jstorm-client/src/main/py/storm/DistributedRPCInvocations.py b/jstorm-core/src/main/py/storm/DistributedRPCInvocations.py old mode 100644 new mode 100755 similarity index 92% rename from jstorm-client/src/main/py/storm/DistributedRPCInvocations.py rename to jstorm-core/src/main/py/storm/DistributedRPCInvocations.py index 4f951a943..7693a2f38 --- a/jstorm-client/src/main/py/storm/DistributedRPCInvocations.py +++ b/jstorm-core/src/main/py/storm/DistributedRPCInvocations.py @@ -1,10 +1,12 @@ # -# Autogenerated by Thrift Compiler (0.7.0) +# Autogenerated by Thrift Compiler (0.9.2) # # DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING # +# options string: py:utf8strings +# -from thrift.Thrift import * +from thrift.Thrift import TType, TMessageType, TException, TApplicationException from ttypes import * from thrift.Thrift import TProcessor from thrift.transport import TTransport @@ -64,16 +66,17 @@ def send_result(self, id, result): self._oprot.writeMessageEnd() self._oprot.trans.flush() - def recv_result(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() + def recv_result(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() + x.read(iprot) + iprot.readMessageEnd() raise x result = result_result() - result.read(self._iprot) - self._iprot.readMessageEnd() + result.read(iprot) + iprot.readMessageEnd() return def fetchRequest(self, functionName): @@ -92,16 +95,17 @@ def send_fetchRequest(self, functionName): self._oprot.writeMessageEnd() self._oprot.trans.flush() - def recv_fetchRequest(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() + def recv_fetchRequest(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() + x.read(iprot) + iprot.readMessageEnd() raise x result = fetchRequest_result() - result.read(self._iprot) - self._iprot.readMessageEnd() + result.read(iprot) + iprot.readMessageEnd() if result.success is not None: return result.success raise TApplicationException(TApplicationException.MISSING_RESULT, "fetchRequest failed: unknown result"); @@ -122,16 +126,17 @@ def send_failRequest(self, id): self._oprot.writeMessageEnd() self._oprot.trans.flush() - def recv_failRequest(self, ): - (fname, mtype, rseqid) = self._iprot.readMessageBegin() + def recv_failRequest(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() if mtype == TMessageType.EXCEPTION: x = TApplicationException() - x.read(self._iprot) - self._iprot.readMessageEnd() + x.read(iprot) + iprot.readMessageEnd() raise x result = failRequest_result() - result.read(self._iprot) - self._iprot.readMessageEnd() + result.read(iprot) + iprot.readMessageEnd() return @@ -207,9 +212,6 @@ class result_args: (2, TType.STRING, 'result', None, None, ), # 2 ) - def __hash__(self): - return 0 + hash(self.id) + hash(self.result) - def __init__(self, id=None, result=None,): self.id = id self.result = result @@ -258,6 +260,12 @@ def validate(self): return + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.id) + value = (value * 31) ^ hash(self.result) + return value + def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] @@ -274,9 +282,6 @@ class result_result: thrift_spec = ( ) - def __hash__(self): - return 0 - def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) @@ -303,6 +308,10 @@ def validate(self): return + def __hash__(self): + value = 17 + return value + def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] @@ -325,9 +334,6 @@ class fetchRequest_args: (1, TType.STRING, 'functionName', None, None, ), # 1 ) - def __hash__(self): - return 0 + hash(self.functionName) - def __init__(self, functionName=None,): self.functionName = functionName @@ -366,6 +372,11 @@ def validate(self): return + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.functionName) + return value + def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] @@ -387,9 +398,6 @@ class fetchRequest_result: (0, TType.STRUCT, 'success', (DRPCRequest, DRPCRequest.thrift_spec), None, ), # 0 ) - def __hash__(self): - return 0 + hash(self.success) - def __init__(self, success=None,): self.success = success @@ -429,6 +437,11 @@ def validate(self): return + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + return value + def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] @@ -451,9 +464,6 @@ class failRequest_args: (1, TType.STRING, 'id', None, None, ), # 1 ) - def __hash__(self): - return 0 + hash(self.id) - def __init__(self, id=None,): self.id = id @@ -492,6 +502,11 @@ def validate(self): return + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.id) + return value + def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] @@ -508,9 +523,6 @@ class failRequest_result: thrift_spec = ( ) - def __hash__(self): - return 0 - def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) @@ -537,6 +549,10 @@ def validate(self): return + def __hash__(self): + value = 17 + return value + def __repr__(self): L = ['%s=%r' % (key, value) for key, value in self.__dict__.iteritems()] diff --git a/jstorm-core/src/main/py/storm/Nimbus-remote b/jstorm-core/src/main/py/storm/Nimbus-remote new file mode 100644 index 000000000..27c9a6fd3 --- /dev/null +++ b/jstorm-core/src/main/py/storm/Nimbus-remote @@ -0,0 +1,304 @@ +#!/usr/bin/env python +# +# Autogenerated by Thrift Compiler (0.9.2) +# +# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING +# +# options string: py:utf8strings +# + +import sys +import pprint +from urlparse import urlparse +from thrift.transport import TTransport +from thrift.transport import TSocket +from thrift.transport import TSSLSocket +from thrift.transport import THttpClient +from thrift.protocol import TBinaryProtocol + +from storm import Nimbus +from storm.ttypes import * + +if len(sys.argv) <= 1 or sys.argv[1] == '--help': + print('') + print('Usage: ' + sys.argv[0] + ' [-h host[:port]] [-u url] [-f[ramed]] [-s[sl]] function [arg1 [arg2...]]') + print('') + print('Functions:') + print(' void submitTopology(string name, string uploadedJarLocation, string jsonConf, StormTopology topology)') + print(' void submitTopologyWithOpts(string name, string uploadedJarLocation, string jsonConf, StormTopology topology, SubmitOptions options)') + print(' void killTopology(string name)') + print(' void killTopologyWithOpts(string name, KillOptions options)') + print(' void activate(string name)') + print(' void deactivate(string name)') + print(' void rebalance(string name, RebalanceOptions options)') + print(' void metricMonitor(string name, MonitorOptions options)') + print(' void restart(string name, string jsonConf)') + print(' void beginLibUpload(string libName)') + print(' string beginFileUpload()') + print(' void uploadChunk(string location, string chunk)') + print(' void finishFileUpload(string location)') + print(' string beginFileDownload(string file)') + print(' string downloadChunk(string id)') + print(' void finishFileDownload(string id)') + print(' string getNimbusConf()') + print(' string getTopologyConf(string id)') + print(' string getTopologyId(string topologyName)') + print(' ClusterSummary getClusterInfo()') + print(' SupervisorWorkers getSupervisorWorkers(string host)') + print(' TopologyInfo getTopologyInfo(string id)') + print(' TopologyInfo getTopologyInfoByName(string topologyName)') + print(' StormTopology getTopology(string id)') + print(' StormTopology getUserTopology(string id)') + print(' void workerUploadMetric(WorkerUploadMetrics uploadMetrics)') + print(' TopologyMetric getTopologyMetric(string topologyName)') + print(' NettyMetric getNettyMetric(string topologyName, i32 pos)') + print(' NettyMetric getServerNettyMetric(string topologyName, string serverName)') + print(' string getVersion()') + print(' void updateConf(string name, string conf)') + print('') + sys.exit(0) + +pp = pprint.PrettyPrinter(indent = 2) +host = 'localhost' +port = 9090 +uri = '' +framed = False +ssl = False +http = False +argi = 1 + +if sys.argv[argi] == '-h': + parts = sys.argv[argi+1].split(':') + host = parts[0] + if len(parts) > 1: + port = int(parts[1]) + argi += 2 + +if sys.argv[argi] == '-u': + url = urlparse(sys.argv[argi+1]) + parts = url[1].split(':') + host = parts[0] + if len(parts) > 1: + port = int(parts[1]) + else: + port = 80 + uri = url[2] + if url[4]: + uri += '?%s' % url[4] + http = True + argi += 2 + +if sys.argv[argi] == '-f' or sys.argv[argi] == '-framed': + framed = True + argi += 1 + +if sys.argv[argi] == '-s' or sys.argv[argi] == '-ssl': + ssl = True + argi += 1 + +cmd = sys.argv[argi] +args = sys.argv[argi+1:] + +if http: + transport = THttpClient.THttpClient(host, port, uri) +else: + socket = TSSLSocket.TSSLSocket(host, port, validate=False) if ssl else TSocket.TSocket(host, port) + if framed: + transport = TTransport.TFramedTransport(socket) + else: + transport = TTransport.TBufferedTransport(socket) +protocol = TBinaryProtocol.TBinaryProtocol(transport) +client = Nimbus.Client(protocol) +transport.open() + +if cmd == 'submitTopology': + if len(args) != 4: + print('submitTopology requires 4 args') + sys.exit(1) + pp.pprint(client.submitTopology(args[0],args[1],args[2],eval(args[3]),)) + +elif cmd == 'submitTopologyWithOpts': + if len(args) != 5: + print('submitTopologyWithOpts requires 5 args') + sys.exit(1) + pp.pprint(client.submitTopologyWithOpts(args[0],args[1],args[2],eval(args[3]),eval(args[4]),)) + +elif cmd == 'killTopology': + if len(args) != 1: + print('killTopology requires 1 args') + sys.exit(1) + pp.pprint(client.killTopology(args[0],)) + +elif cmd == 'killTopologyWithOpts': + if len(args) != 2: + print('killTopologyWithOpts requires 2 args') + sys.exit(1) + pp.pprint(client.killTopologyWithOpts(args[0],eval(args[1]),)) + +elif cmd == 'activate': + if len(args) != 1: + print('activate requires 1 args') + sys.exit(1) + pp.pprint(client.activate(args[0],)) + +elif cmd == 'deactivate': + if len(args) != 1: + print('deactivate requires 1 args') + sys.exit(1) + pp.pprint(client.deactivate(args[0],)) + +elif cmd == 'rebalance': + if len(args) != 2: + print('rebalance requires 2 args') + sys.exit(1) + pp.pprint(client.rebalance(args[0],eval(args[1]),)) + +elif cmd == 'metricMonitor': + if len(args) != 2: + print('metricMonitor requires 2 args') + sys.exit(1) + pp.pprint(client.metricMonitor(args[0],eval(args[1]),)) + +elif cmd == 'restart': + if len(args) != 2: + print('restart requires 2 args') + sys.exit(1) + pp.pprint(client.restart(args[0],args[1],)) + +elif cmd == 'beginLibUpload': + if len(args) != 1: + print('beginLibUpload requires 1 args') + sys.exit(1) + pp.pprint(client.beginLibUpload(args[0],)) + +elif cmd == 'beginFileUpload': + if len(args) != 0: + print('beginFileUpload requires 0 args') + sys.exit(1) + pp.pprint(client.beginFileUpload()) + +elif cmd == 'uploadChunk': + if len(args) != 2: + print('uploadChunk requires 2 args') + sys.exit(1) + pp.pprint(client.uploadChunk(args[0],args[1],)) + +elif cmd == 'finishFileUpload': + if len(args) != 1: + print('finishFileUpload requires 1 args') + sys.exit(1) + pp.pprint(client.finishFileUpload(args[0],)) + +elif cmd == 'beginFileDownload': + if len(args) != 1: + print('beginFileDownload requires 1 args') + sys.exit(1) + pp.pprint(client.beginFileDownload(args[0],)) + +elif cmd == 'downloadChunk': + if len(args) != 1: + print('downloadChunk requires 1 args') + sys.exit(1) + pp.pprint(client.downloadChunk(args[0],)) + +elif cmd == 'finishFileDownload': + if len(args) != 1: + print('finishFileDownload requires 1 args') + sys.exit(1) + pp.pprint(client.finishFileDownload(args[0],)) + +elif cmd == 'getNimbusConf': + if len(args) != 0: + print('getNimbusConf requires 0 args') + sys.exit(1) + pp.pprint(client.getNimbusConf()) + +elif cmd == 'getTopologyConf': + if len(args) != 1: + print('getTopologyConf requires 1 args') + sys.exit(1) + pp.pprint(client.getTopologyConf(args[0],)) + +elif cmd == 'getTopologyId': + if len(args) != 1: + print('getTopologyId requires 1 args') + sys.exit(1) + pp.pprint(client.getTopologyId(args[0],)) + +elif cmd == 'getClusterInfo': + if len(args) != 0: + print('getClusterInfo requires 0 args') + sys.exit(1) + pp.pprint(client.getClusterInfo()) + +elif cmd == 'getSupervisorWorkers': + if len(args) != 1: + print('getSupervisorWorkers requires 1 args') + sys.exit(1) + pp.pprint(client.getSupervisorWorkers(args[0],)) + +elif cmd == 'getTopologyInfo': + if len(args) != 1: + print('getTopologyInfo requires 1 args') + sys.exit(1) + pp.pprint(client.getTopologyInfo(args[0],)) + +elif cmd == 'getTopologyInfoByName': + if len(args) != 1: + print('getTopologyInfoByName requires 1 args') + sys.exit(1) + pp.pprint(client.getTopologyInfoByName(args[0],)) + +elif cmd == 'getTopology': + if len(args) != 1: + print('getTopology requires 1 args') + sys.exit(1) + pp.pprint(client.getTopology(args[0],)) + +elif cmd == 'getUserTopology': + if len(args) != 1: + print('getUserTopology requires 1 args') + sys.exit(1) + pp.pprint(client.getUserTopology(args[0],)) + +elif cmd == 'workerUploadMetric': + if len(args) != 1: + print('workerUploadMetric requires 1 args') + sys.exit(1) + pp.pprint(client.workerUploadMetric(eval(args[0]),)) + +elif cmd == 'getTopologyMetric': + if len(args) != 1: + print('getTopologyMetric requires 1 args') + sys.exit(1) + pp.pprint(client.getTopologyMetric(args[0],)) + +elif cmd == 'getNettyMetric': + if len(args) != 2: + print('getNettyMetric requires 2 args') + sys.exit(1) + pp.pprint(client.getNettyMetric(args[0],eval(args[1]),)) + +elif cmd == 'getServerNettyMetric': + if len(args) != 2: + print('getServerNettyMetric requires 2 args') + sys.exit(1) + pp.pprint(client.getServerNettyMetric(args[0],args[1],)) + +elif cmd == 'getVersion': + if len(args) != 0: + print('getVersion requires 0 args') + sys.exit(1) + pp.pprint(client.getVersion()) + +elif cmd == 'updateConf': + if len(args) != 2: + print('updateConf requires 2 args') + sys.exit(1) + pp.pprint(client.updateConf(args[0],args[1],)) + +else: + print('Unrecognized method %s' % cmd) + sys.exit(1) + +transport.close() diff --git a/jstorm-core/src/main/py/storm/Nimbus.py b/jstorm-core/src/main/py/storm/Nimbus.py new file mode 100644 index 000000000..41a5a3a87 --- /dev/null +++ b/jstorm-core/src/main/py/storm/Nimbus.py @@ -0,0 +1,5971 @@ +# +# Autogenerated by Thrift Compiler (0.9.2) +# +# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING +# +# options string: py:utf8strings +# + +from thrift.Thrift import TType, TMessageType, TException, TApplicationException +from ttypes import * +from thrift.Thrift import TProcessor +from thrift.transport import TTransport +from thrift.protocol import TBinaryProtocol, TProtocol +try: + from thrift.protocol import fastbinary +except: + fastbinary = None + + +class Iface: + def submitTopology(self, name, uploadedJarLocation, jsonConf, topology): + """ + Parameters: + - name + - uploadedJarLocation + - jsonConf + - topology + """ + pass + + def submitTopologyWithOpts(self, name, uploadedJarLocation, jsonConf, topology, options): + """ + Parameters: + - name + - uploadedJarLocation + - jsonConf + - topology + - options + """ + pass + + def killTopology(self, name): + """ + Parameters: + - name + """ + pass + + def killTopologyWithOpts(self, name, options): + """ + Parameters: + - name + - options + """ + pass + + def activate(self, name): + """ + Parameters: + - name + """ + pass + + def deactivate(self, name): + """ + Parameters: + - name + """ + pass + + def rebalance(self, name, options): + """ + Parameters: + - name + - options + """ + pass + + def metricMonitor(self, name, options): + """ + Parameters: + - name + - options + """ + pass + + def restart(self, name, jsonConf): + """ + Parameters: + - name + - jsonConf + """ + pass + + def beginLibUpload(self, libName): + """ + Parameters: + - libName + """ + pass + + def beginFileUpload(self): + pass + + def uploadChunk(self, location, chunk): + """ + Parameters: + - location + - chunk + """ + pass + + def finishFileUpload(self, location): + """ + Parameters: + - location + """ + pass + + def beginFileDownload(self, file): + """ + Parameters: + - file + """ + pass + + def downloadChunk(self, id): + """ + Parameters: + - id + """ + pass + + def finishFileDownload(self, id): + """ + Parameters: + - id + """ + pass + + def getNimbusConf(self): + pass + + def getTopologyConf(self, id): + """ + Parameters: + - id + """ + pass + + def getTopologyId(self, topologyName): + """ + Parameters: + - topologyName + """ + pass + + def getClusterInfo(self): + pass + + def getSupervisorWorkers(self, host): + """ + Parameters: + - host + """ + pass + + def getTopologyInfo(self, id): + """ + Parameters: + - id + """ + pass + + def getTopologyInfoByName(self, topologyName): + """ + Parameters: + - topologyName + """ + pass + + def getTopology(self, id): + """ + Parameters: + - id + """ + pass + + def getUserTopology(self, id): + """ + Parameters: + - id + """ + pass + + def workerUploadMetric(self, uploadMetrics): + """ + Parameters: + - uploadMetrics + """ + pass + + def getTopologyMetric(self, topologyName): + """ + Parameters: + - topologyName + """ + pass + + def getNettyMetric(self, topologyName, pos): + """ + Parameters: + - topologyName + - pos + """ + pass + + def getServerNettyMetric(self, topologyName, serverName): + """ + Parameters: + - topologyName + - serverName + """ + pass + + def getVersion(self): + pass + + def updateConf(self, name, conf): + """ + Parameters: + - name + - conf + """ + pass + + +class Client(Iface): + def __init__(self, iprot, oprot=None): + self._iprot = self._oprot = iprot + if oprot is not None: + self._oprot = oprot + self._seqid = 0 + + def submitTopology(self, name, uploadedJarLocation, jsonConf, topology): + """ + Parameters: + - name + - uploadedJarLocation + - jsonConf + - topology + """ + self.send_submitTopology(name, uploadedJarLocation, jsonConf, topology) + self.recv_submitTopology() + + def send_submitTopology(self, name, uploadedJarLocation, jsonConf, topology): + self._oprot.writeMessageBegin('submitTopology', TMessageType.CALL, self._seqid) + args = submitTopology_args() + args.name = name + args.uploadedJarLocation = uploadedJarLocation + args.jsonConf = jsonConf + args.topology = topology + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_submitTopology(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = submitTopology_result() + result.read(iprot) + iprot.readMessageEnd() + if result.e is not None: + raise result.e + if result.ite is not None: + raise result.ite + if result.tae is not None: + raise result.tae + return + + def submitTopologyWithOpts(self, name, uploadedJarLocation, jsonConf, topology, options): + """ + Parameters: + - name + - uploadedJarLocation + - jsonConf + - topology + - options + """ + self.send_submitTopologyWithOpts(name, uploadedJarLocation, jsonConf, topology, options) + self.recv_submitTopologyWithOpts() + + def send_submitTopologyWithOpts(self, name, uploadedJarLocation, jsonConf, topology, options): + self._oprot.writeMessageBegin('submitTopologyWithOpts', TMessageType.CALL, self._seqid) + args = submitTopologyWithOpts_args() + args.name = name + args.uploadedJarLocation = uploadedJarLocation + args.jsonConf = jsonConf + args.topology = topology + args.options = options + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_submitTopologyWithOpts(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = submitTopologyWithOpts_result() + result.read(iprot) + iprot.readMessageEnd() + if result.e is not None: + raise result.e + if result.ite is not None: + raise result.ite + if result.tae is not None: + raise result.tae + return + + def killTopology(self, name): + """ + Parameters: + - name + """ + self.send_killTopology(name) + self.recv_killTopology() + + def send_killTopology(self, name): + self._oprot.writeMessageBegin('killTopology', TMessageType.CALL, self._seqid) + args = killTopology_args() + args.name = name + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_killTopology(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = killTopology_result() + result.read(iprot) + iprot.readMessageEnd() + if result.e is not None: + raise result.e + return + + def killTopologyWithOpts(self, name, options): + """ + Parameters: + - name + - options + """ + self.send_killTopologyWithOpts(name, options) + self.recv_killTopologyWithOpts() + + def send_killTopologyWithOpts(self, name, options): + self._oprot.writeMessageBegin('killTopologyWithOpts', TMessageType.CALL, self._seqid) + args = killTopologyWithOpts_args() + args.name = name + args.options = options + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_killTopologyWithOpts(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = killTopologyWithOpts_result() + result.read(iprot) + iprot.readMessageEnd() + if result.e is not None: + raise result.e + return + + def activate(self, name): + """ + Parameters: + - name + """ + self.send_activate(name) + self.recv_activate() + + def send_activate(self, name): + self._oprot.writeMessageBegin('activate', TMessageType.CALL, self._seqid) + args = activate_args() + args.name = name + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_activate(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = activate_result() + result.read(iprot) + iprot.readMessageEnd() + if result.e is not None: + raise result.e + return + + def deactivate(self, name): + """ + Parameters: + - name + """ + self.send_deactivate(name) + self.recv_deactivate() + + def send_deactivate(self, name): + self._oprot.writeMessageBegin('deactivate', TMessageType.CALL, self._seqid) + args = deactivate_args() + args.name = name + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_deactivate(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = deactivate_result() + result.read(iprot) + iprot.readMessageEnd() + if result.e is not None: + raise result.e + return + + def rebalance(self, name, options): + """ + Parameters: + - name + - options + """ + self.send_rebalance(name, options) + self.recv_rebalance() + + def send_rebalance(self, name, options): + self._oprot.writeMessageBegin('rebalance', TMessageType.CALL, self._seqid) + args = rebalance_args() + args.name = name + args.options = options + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_rebalance(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = rebalance_result() + result.read(iprot) + iprot.readMessageEnd() + if result.e is not None: + raise result.e + if result.ite is not None: + raise result.ite + return + + def metricMonitor(self, name, options): + """ + Parameters: + - name + - options + """ + self.send_metricMonitor(name, options) + self.recv_metricMonitor() + + def send_metricMonitor(self, name, options): + self._oprot.writeMessageBegin('metricMonitor', TMessageType.CALL, self._seqid) + args = metricMonitor_args() + args.name = name + args.options = options + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_metricMonitor(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = metricMonitor_result() + result.read(iprot) + iprot.readMessageEnd() + if result.e is not None: + raise result.e + return + + def restart(self, name, jsonConf): + """ + Parameters: + - name + - jsonConf + """ + self.send_restart(name, jsonConf) + self.recv_restart() + + def send_restart(self, name, jsonConf): + self._oprot.writeMessageBegin('restart', TMessageType.CALL, self._seqid) + args = restart_args() + args.name = name + args.jsonConf = jsonConf + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_restart(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = restart_result() + result.read(iprot) + iprot.readMessageEnd() + if result.e is not None: + raise result.e + if result.ite is not None: + raise result.ite + if result.tae is not None: + raise result.tae + return + + def beginLibUpload(self, libName): + """ + Parameters: + - libName + """ + self.send_beginLibUpload(libName) + self.recv_beginLibUpload() + + def send_beginLibUpload(self, libName): + self._oprot.writeMessageBegin('beginLibUpload', TMessageType.CALL, self._seqid) + args = beginLibUpload_args() + args.libName = libName + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_beginLibUpload(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = beginLibUpload_result() + result.read(iprot) + iprot.readMessageEnd() + return + + def beginFileUpload(self): + self.send_beginFileUpload() + return self.recv_beginFileUpload() + + def send_beginFileUpload(self): + self._oprot.writeMessageBegin('beginFileUpload', TMessageType.CALL, self._seqid) + args = beginFileUpload_args() + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_beginFileUpload(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = beginFileUpload_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + raise TApplicationException(TApplicationException.MISSING_RESULT, "beginFileUpload failed: unknown result"); + + def uploadChunk(self, location, chunk): + """ + Parameters: + - location + - chunk + """ + self.send_uploadChunk(location, chunk) + self.recv_uploadChunk() + + def send_uploadChunk(self, location, chunk): + self._oprot.writeMessageBegin('uploadChunk', TMessageType.CALL, self._seqid) + args = uploadChunk_args() + args.location = location + args.chunk = chunk + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_uploadChunk(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = uploadChunk_result() + result.read(iprot) + iprot.readMessageEnd() + return + + def finishFileUpload(self, location): + """ + Parameters: + - location + """ + self.send_finishFileUpload(location) + self.recv_finishFileUpload() + + def send_finishFileUpload(self, location): + self._oprot.writeMessageBegin('finishFileUpload', TMessageType.CALL, self._seqid) + args = finishFileUpload_args() + args.location = location + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_finishFileUpload(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = finishFileUpload_result() + result.read(iprot) + iprot.readMessageEnd() + return + + def beginFileDownload(self, file): + """ + Parameters: + - file + """ + self.send_beginFileDownload(file) + return self.recv_beginFileDownload() + + def send_beginFileDownload(self, file): + self._oprot.writeMessageBegin('beginFileDownload', TMessageType.CALL, self._seqid) + args = beginFileDownload_args() + args.file = file + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_beginFileDownload(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = beginFileDownload_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + raise TApplicationException(TApplicationException.MISSING_RESULT, "beginFileDownload failed: unknown result"); + + def downloadChunk(self, id): + """ + Parameters: + - id + """ + self.send_downloadChunk(id) + return self.recv_downloadChunk() + + def send_downloadChunk(self, id): + self._oprot.writeMessageBegin('downloadChunk', TMessageType.CALL, self._seqid) + args = downloadChunk_args() + args.id = id + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_downloadChunk(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = downloadChunk_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + raise TApplicationException(TApplicationException.MISSING_RESULT, "downloadChunk failed: unknown result"); + + def finishFileDownload(self, id): + """ + Parameters: + - id + """ + self.send_finishFileDownload(id) + self.recv_finishFileDownload() + + def send_finishFileDownload(self, id): + self._oprot.writeMessageBegin('finishFileDownload', TMessageType.CALL, self._seqid) + args = finishFileDownload_args() + args.id = id + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_finishFileDownload(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = finishFileDownload_result() + result.read(iprot) + iprot.readMessageEnd() + return + + def getNimbusConf(self): + self.send_getNimbusConf() + return self.recv_getNimbusConf() + + def send_getNimbusConf(self): + self._oprot.writeMessageBegin('getNimbusConf', TMessageType.CALL, self._seqid) + args = getNimbusConf_args() + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_getNimbusConf(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = getNimbusConf_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + raise TApplicationException(TApplicationException.MISSING_RESULT, "getNimbusConf failed: unknown result"); + + def getTopologyConf(self, id): + """ + Parameters: + - id + """ + self.send_getTopologyConf(id) + return self.recv_getTopologyConf() + + def send_getTopologyConf(self, id): + self._oprot.writeMessageBegin('getTopologyConf', TMessageType.CALL, self._seqid) + args = getTopologyConf_args() + args.id = id + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_getTopologyConf(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = getTopologyConf_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + if result.e is not None: + raise result.e + raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyConf failed: unknown result"); + + def getTopologyId(self, topologyName): + """ + Parameters: + - topologyName + """ + self.send_getTopologyId(topologyName) + return self.recv_getTopologyId() + + def send_getTopologyId(self, topologyName): + self._oprot.writeMessageBegin('getTopologyId', TMessageType.CALL, self._seqid) + args = getTopologyId_args() + args.topologyName = topologyName + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_getTopologyId(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = getTopologyId_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + if result.e is not None: + raise result.e + raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyId failed: unknown result"); + + def getClusterInfo(self): + self.send_getClusterInfo() + return self.recv_getClusterInfo() + + def send_getClusterInfo(self): + self._oprot.writeMessageBegin('getClusterInfo', TMessageType.CALL, self._seqid) + args = getClusterInfo_args() + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_getClusterInfo(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = getClusterInfo_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + raise TApplicationException(TApplicationException.MISSING_RESULT, "getClusterInfo failed: unknown result"); + + def getSupervisorWorkers(self, host): + """ + Parameters: + - host + """ + self.send_getSupervisorWorkers(host) + return self.recv_getSupervisorWorkers() + + def send_getSupervisorWorkers(self, host): + self._oprot.writeMessageBegin('getSupervisorWorkers', TMessageType.CALL, self._seqid) + args = getSupervisorWorkers_args() + args.host = host + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_getSupervisorWorkers(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = getSupervisorWorkers_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + if result.e is not None: + raise result.e + raise TApplicationException(TApplicationException.MISSING_RESULT, "getSupervisorWorkers failed: unknown result"); + + def getTopologyInfo(self, id): + """ + Parameters: + - id + """ + self.send_getTopologyInfo(id) + return self.recv_getTopologyInfo() + + def send_getTopologyInfo(self, id): + self._oprot.writeMessageBegin('getTopologyInfo', TMessageType.CALL, self._seqid) + args = getTopologyInfo_args() + args.id = id + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_getTopologyInfo(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = getTopologyInfo_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + if result.e is not None: + raise result.e + raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyInfo failed: unknown result"); + + def getTopologyInfoByName(self, topologyName): + """ + Parameters: + - topologyName + """ + self.send_getTopologyInfoByName(topologyName) + return self.recv_getTopologyInfoByName() + + def send_getTopologyInfoByName(self, topologyName): + self._oprot.writeMessageBegin('getTopologyInfoByName', TMessageType.CALL, self._seqid) + args = getTopologyInfoByName_args() + args.topologyName = topologyName + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_getTopologyInfoByName(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = getTopologyInfoByName_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + if result.e is not None: + raise result.e + raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyInfoByName failed: unknown result"); + + def getTopology(self, id): + """ + Parameters: + - id + """ + self.send_getTopology(id) + return self.recv_getTopology() + + def send_getTopology(self, id): + self._oprot.writeMessageBegin('getTopology', TMessageType.CALL, self._seqid) + args = getTopology_args() + args.id = id + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_getTopology(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = getTopology_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + if result.e is not None: + raise result.e + raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopology failed: unknown result"); + + def getUserTopology(self, id): + """ + Parameters: + - id + """ + self.send_getUserTopology(id) + return self.recv_getUserTopology() + + def send_getUserTopology(self, id): + self._oprot.writeMessageBegin('getUserTopology', TMessageType.CALL, self._seqid) + args = getUserTopology_args() + args.id = id + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_getUserTopology(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = getUserTopology_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + if result.e is not None: + raise result.e + raise TApplicationException(TApplicationException.MISSING_RESULT, "getUserTopology failed: unknown result"); + + def workerUploadMetric(self, uploadMetrics): + """ + Parameters: + - uploadMetrics + """ + self.send_workerUploadMetric(uploadMetrics) + self.recv_workerUploadMetric() + + def send_workerUploadMetric(self, uploadMetrics): + self._oprot.writeMessageBegin('workerUploadMetric', TMessageType.CALL, self._seqid) + args = workerUploadMetric_args() + args.uploadMetrics = uploadMetrics + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_workerUploadMetric(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = workerUploadMetric_result() + result.read(iprot) + iprot.readMessageEnd() + return + + def getTopologyMetric(self, topologyName): + """ + Parameters: + - topologyName + """ + self.send_getTopologyMetric(topologyName) + return self.recv_getTopologyMetric() + + def send_getTopologyMetric(self, topologyName): + self._oprot.writeMessageBegin('getTopologyMetric', TMessageType.CALL, self._seqid) + args = getTopologyMetric_args() + args.topologyName = topologyName + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_getTopologyMetric(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = getTopologyMetric_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyMetric failed: unknown result"); + + def getNettyMetric(self, topologyName, pos): + """ + Parameters: + - topologyName + - pos + """ + self.send_getNettyMetric(topologyName, pos) + return self.recv_getNettyMetric() + + def send_getNettyMetric(self, topologyName, pos): + self._oprot.writeMessageBegin('getNettyMetric', TMessageType.CALL, self._seqid) + args = getNettyMetric_args() + args.topologyName = topologyName + args.pos = pos + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_getNettyMetric(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = getNettyMetric_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + raise TApplicationException(TApplicationException.MISSING_RESULT, "getNettyMetric failed: unknown result"); + + def getServerNettyMetric(self, topologyName, serverName): + """ + Parameters: + - topologyName + - serverName + """ + self.send_getServerNettyMetric(topologyName, serverName) + return self.recv_getServerNettyMetric() + + def send_getServerNettyMetric(self, topologyName, serverName): + self._oprot.writeMessageBegin('getServerNettyMetric', TMessageType.CALL, self._seqid) + args = getServerNettyMetric_args() + args.topologyName = topologyName + args.serverName = serverName + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_getServerNettyMetric(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = getServerNettyMetric_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + raise TApplicationException(TApplicationException.MISSING_RESULT, "getServerNettyMetric failed: unknown result"); + + def getVersion(self): + self.send_getVersion() + return self.recv_getVersion() + + def send_getVersion(self): + self._oprot.writeMessageBegin('getVersion', TMessageType.CALL, self._seqid) + args = getVersion_args() + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_getVersion(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = getVersion_result() + result.read(iprot) + iprot.readMessageEnd() + if result.success is not None: + return result.success + raise TApplicationException(TApplicationException.MISSING_RESULT, "getVersion failed: unknown result"); + + def updateConf(self, name, conf): + """ + Parameters: + - name + - conf + """ + self.send_updateConf(name, conf) + self.recv_updateConf() + + def send_updateConf(self, name, conf): + self._oprot.writeMessageBegin('updateConf', TMessageType.CALL, self._seqid) + args = updateConf_args() + args.name = name + args.conf = conf + args.write(self._oprot) + self._oprot.writeMessageEnd() + self._oprot.trans.flush() + + def recv_updateConf(self): + iprot = self._iprot + (fname, mtype, rseqid) = iprot.readMessageBegin() + if mtype == TMessageType.EXCEPTION: + x = TApplicationException() + x.read(iprot) + iprot.readMessageEnd() + raise x + result = updateConf_result() + result.read(iprot) + iprot.readMessageEnd() + if result.e is not None: + raise result.e + if result.ite is not None: + raise result.ite + return + + +class Processor(Iface, TProcessor): + def __init__(self, handler): + self._handler = handler + self._processMap = {} + self._processMap["submitTopology"] = Processor.process_submitTopology + self._processMap["submitTopologyWithOpts"] = Processor.process_submitTopologyWithOpts + self._processMap["killTopology"] = Processor.process_killTopology + self._processMap["killTopologyWithOpts"] = Processor.process_killTopologyWithOpts + self._processMap["activate"] = Processor.process_activate + self._processMap["deactivate"] = Processor.process_deactivate + self._processMap["rebalance"] = Processor.process_rebalance + self._processMap["metricMonitor"] = Processor.process_metricMonitor + self._processMap["restart"] = Processor.process_restart + self._processMap["beginLibUpload"] = Processor.process_beginLibUpload + self._processMap["beginFileUpload"] = Processor.process_beginFileUpload + self._processMap["uploadChunk"] = Processor.process_uploadChunk + self._processMap["finishFileUpload"] = Processor.process_finishFileUpload + self._processMap["beginFileDownload"] = Processor.process_beginFileDownload + self._processMap["downloadChunk"] = Processor.process_downloadChunk + self._processMap["finishFileDownload"] = Processor.process_finishFileDownload + self._processMap["getNimbusConf"] = Processor.process_getNimbusConf + self._processMap["getTopologyConf"] = Processor.process_getTopologyConf + self._processMap["getTopologyId"] = Processor.process_getTopologyId + self._processMap["getClusterInfo"] = Processor.process_getClusterInfo + self._processMap["getSupervisorWorkers"] = Processor.process_getSupervisorWorkers + self._processMap["getTopologyInfo"] = Processor.process_getTopologyInfo + self._processMap["getTopologyInfoByName"] = Processor.process_getTopologyInfoByName + self._processMap["getTopology"] = Processor.process_getTopology + self._processMap["getUserTopology"] = Processor.process_getUserTopology + self._processMap["workerUploadMetric"] = Processor.process_workerUploadMetric + self._processMap["getTopologyMetric"] = Processor.process_getTopologyMetric + self._processMap["getNettyMetric"] = Processor.process_getNettyMetric + self._processMap["getServerNettyMetric"] = Processor.process_getServerNettyMetric + self._processMap["getVersion"] = Processor.process_getVersion + self._processMap["updateConf"] = Processor.process_updateConf + + def process(self, iprot, oprot): + (name, type, seqid) = iprot.readMessageBegin() + if name not in self._processMap: + iprot.skip(TType.STRUCT) + iprot.readMessageEnd() + x = TApplicationException(TApplicationException.UNKNOWN_METHOD, 'Unknown function %s' % (name)) + oprot.writeMessageBegin(name, TMessageType.EXCEPTION, seqid) + x.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + return + else: + self._processMap[name](self, seqid, iprot, oprot) + return True + + def process_submitTopology(self, seqid, iprot, oprot): + args = submitTopology_args() + args.read(iprot) + iprot.readMessageEnd() + result = submitTopology_result() + try: + self._handler.submitTopology(args.name, args.uploadedJarLocation, args.jsonConf, args.topology) + except AlreadyAliveException, e: + result.e = e + except InvalidTopologyException, ite: + result.ite = ite + except TopologyAssignException, tae: + result.tae = tae + oprot.writeMessageBegin("submitTopology", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_submitTopologyWithOpts(self, seqid, iprot, oprot): + args = submitTopologyWithOpts_args() + args.read(iprot) + iprot.readMessageEnd() + result = submitTopologyWithOpts_result() + try: + self._handler.submitTopologyWithOpts(args.name, args.uploadedJarLocation, args.jsonConf, args.topology, args.options) + except AlreadyAliveException, e: + result.e = e + except InvalidTopologyException, ite: + result.ite = ite + except TopologyAssignException, tae: + result.tae = tae + oprot.writeMessageBegin("submitTopologyWithOpts", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_killTopology(self, seqid, iprot, oprot): + args = killTopology_args() + args.read(iprot) + iprot.readMessageEnd() + result = killTopology_result() + try: + self._handler.killTopology(args.name) + except NotAliveException, e: + result.e = e + oprot.writeMessageBegin("killTopology", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_killTopologyWithOpts(self, seqid, iprot, oprot): + args = killTopologyWithOpts_args() + args.read(iprot) + iprot.readMessageEnd() + result = killTopologyWithOpts_result() + try: + self._handler.killTopologyWithOpts(args.name, args.options) + except NotAliveException, e: + result.e = e + oprot.writeMessageBegin("killTopologyWithOpts", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_activate(self, seqid, iprot, oprot): + args = activate_args() + args.read(iprot) + iprot.readMessageEnd() + result = activate_result() + try: + self._handler.activate(args.name) + except NotAliveException, e: + result.e = e + oprot.writeMessageBegin("activate", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_deactivate(self, seqid, iprot, oprot): + args = deactivate_args() + args.read(iprot) + iprot.readMessageEnd() + result = deactivate_result() + try: + self._handler.deactivate(args.name) + except NotAliveException, e: + result.e = e + oprot.writeMessageBegin("deactivate", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_rebalance(self, seqid, iprot, oprot): + args = rebalance_args() + args.read(iprot) + iprot.readMessageEnd() + result = rebalance_result() + try: + self._handler.rebalance(args.name, args.options) + except NotAliveException, e: + result.e = e + except InvalidTopologyException, ite: + result.ite = ite + oprot.writeMessageBegin("rebalance", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_metricMonitor(self, seqid, iprot, oprot): + args = metricMonitor_args() + args.read(iprot) + iprot.readMessageEnd() + result = metricMonitor_result() + try: + self._handler.metricMonitor(args.name, args.options) + except NotAliveException, e: + result.e = e + oprot.writeMessageBegin("metricMonitor", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_restart(self, seqid, iprot, oprot): + args = restart_args() + args.read(iprot) + iprot.readMessageEnd() + result = restart_result() + try: + self._handler.restart(args.name, args.jsonConf) + except NotAliveException, e: + result.e = e + except InvalidTopologyException, ite: + result.ite = ite + except TopologyAssignException, tae: + result.tae = tae + oprot.writeMessageBegin("restart", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_beginLibUpload(self, seqid, iprot, oprot): + args = beginLibUpload_args() + args.read(iprot) + iprot.readMessageEnd() + result = beginLibUpload_result() + self._handler.beginLibUpload(args.libName) + oprot.writeMessageBegin("beginLibUpload", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_beginFileUpload(self, seqid, iprot, oprot): + args = beginFileUpload_args() + args.read(iprot) + iprot.readMessageEnd() + result = beginFileUpload_result() + result.success = self._handler.beginFileUpload() + oprot.writeMessageBegin("beginFileUpload", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_uploadChunk(self, seqid, iprot, oprot): + args = uploadChunk_args() + args.read(iprot) + iprot.readMessageEnd() + result = uploadChunk_result() + self._handler.uploadChunk(args.location, args.chunk) + oprot.writeMessageBegin("uploadChunk", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_finishFileUpload(self, seqid, iprot, oprot): + args = finishFileUpload_args() + args.read(iprot) + iprot.readMessageEnd() + result = finishFileUpload_result() + self._handler.finishFileUpload(args.location) + oprot.writeMessageBegin("finishFileUpload", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_beginFileDownload(self, seqid, iprot, oprot): + args = beginFileDownload_args() + args.read(iprot) + iprot.readMessageEnd() + result = beginFileDownload_result() + result.success = self._handler.beginFileDownload(args.file) + oprot.writeMessageBegin("beginFileDownload", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_downloadChunk(self, seqid, iprot, oprot): + args = downloadChunk_args() + args.read(iprot) + iprot.readMessageEnd() + result = downloadChunk_result() + result.success = self._handler.downloadChunk(args.id) + oprot.writeMessageBegin("downloadChunk", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_finishFileDownload(self, seqid, iprot, oprot): + args = finishFileDownload_args() + args.read(iprot) + iprot.readMessageEnd() + result = finishFileDownload_result() + self._handler.finishFileDownload(args.id) + oprot.writeMessageBegin("finishFileDownload", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_getNimbusConf(self, seqid, iprot, oprot): + args = getNimbusConf_args() + args.read(iprot) + iprot.readMessageEnd() + result = getNimbusConf_result() + result.success = self._handler.getNimbusConf() + oprot.writeMessageBegin("getNimbusConf", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_getTopologyConf(self, seqid, iprot, oprot): + args = getTopologyConf_args() + args.read(iprot) + iprot.readMessageEnd() + result = getTopologyConf_result() + try: + result.success = self._handler.getTopologyConf(args.id) + except NotAliveException, e: + result.e = e + oprot.writeMessageBegin("getTopologyConf", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_getTopologyId(self, seqid, iprot, oprot): + args = getTopologyId_args() + args.read(iprot) + iprot.readMessageEnd() + result = getTopologyId_result() + try: + result.success = self._handler.getTopologyId(args.topologyName) + except NotAliveException, e: + result.e = e + oprot.writeMessageBegin("getTopologyId", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_getClusterInfo(self, seqid, iprot, oprot): + args = getClusterInfo_args() + args.read(iprot) + iprot.readMessageEnd() + result = getClusterInfo_result() + result.success = self._handler.getClusterInfo() + oprot.writeMessageBegin("getClusterInfo", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_getSupervisorWorkers(self, seqid, iprot, oprot): + args = getSupervisorWorkers_args() + args.read(iprot) + iprot.readMessageEnd() + result = getSupervisorWorkers_result() + try: + result.success = self._handler.getSupervisorWorkers(args.host) + except NotAliveException, e: + result.e = e + oprot.writeMessageBegin("getSupervisorWorkers", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_getTopologyInfo(self, seqid, iprot, oprot): + args = getTopologyInfo_args() + args.read(iprot) + iprot.readMessageEnd() + result = getTopologyInfo_result() + try: + result.success = self._handler.getTopologyInfo(args.id) + except NotAliveException, e: + result.e = e + oprot.writeMessageBegin("getTopologyInfo", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_getTopologyInfoByName(self, seqid, iprot, oprot): + args = getTopologyInfoByName_args() + args.read(iprot) + iprot.readMessageEnd() + result = getTopologyInfoByName_result() + try: + result.success = self._handler.getTopologyInfoByName(args.topologyName) + except NotAliveException, e: + result.e = e + oprot.writeMessageBegin("getTopologyInfoByName", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_getTopology(self, seqid, iprot, oprot): + args = getTopology_args() + args.read(iprot) + iprot.readMessageEnd() + result = getTopology_result() + try: + result.success = self._handler.getTopology(args.id) + except NotAliveException, e: + result.e = e + oprot.writeMessageBegin("getTopology", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_getUserTopology(self, seqid, iprot, oprot): + args = getUserTopology_args() + args.read(iprot) + iprot.readMessageEnd() + result = getUserTopology_result() + try: + result.success = self._handler.getUserTopology(args.id) + except NotAliveException, e: + result.e = e + oprot.writeMessageBegin("getUserTopology", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_workerUploadMetric(self, seqid, iprot, oprot): + args = workerUploadMetric_args() + args.read(iprot) + iprot.readMessageEnd() + result = workerUploadMetric_result() + self._handler.workerUploadMetric(args.uploadMetrics) + oprot.writeMessageBegin("workerUploadMetric", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_getTopologyMetric(self, seqid, iprot, oprot): + args = getTopologyMetric_args() + args.read(iprot) + iprot.readMessageEnd() + result = getTopologyMetric_result() + result.success = self._handler.getTopologyMetric(args.topologyName) + oprot.writeMessageBegin("getTopologyMetric", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_getNettyMetric(self, seqid, iprot, oprot): + args = getNettyMetric_args() + args.read(iprot) + iprot.readMessageEnd() + result = getNettyMetric_result() + result.success = self._handler.getNettyMetric(args.topologyName, args.pos) + oprot.writeMessageBegin("getNettyMetric", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_getServerNettyMetric(self, seqid, iprot, oprot): + args = getServerNettyMetric_args() + args.read(iprot) + iprot.readMessageEnd() + result = getServerNettyMetric_result() + result.success = self._handler.getServerNettyMetric(args.topologyName, args.serverName) + oprot.writeMessageBegin("getServerNettyMetric", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_getVersion(self, seqid, iprot, oprot): + args = getVersion_args() + args.read(iprot) + iprot.readMessageEnd() + result = getVersion_result() + result.success = self._handler.getVersion() + oprot.writeMessageBegin("getVersion", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + def process_updateConf(self, seqid, iprot, oprot): + args = updateConf_args() + args.read(iprot) + iprot.readMessageEnd() + result = updateConf_result() + try: + self._handler.updateConf(args.name, args.conf) + except NotAliveException, e: + result.e = e + except InvalidTopologyException, ite: + result.ite = ite + oprot.writeMessageBegin("updateConf", TMessageType.REPLY, seqid) + result.write(oprot) + oprot.writeMessageEnd() + oprot.trans.flush() + + +# HELPER FUNCTIONS AND STRUCTURES + +class submitTopology_args: + """ + Attributes: + - name + - uploadedJarLocation + - jsonConf + - topology + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'name', None, None, ), # 1 + (2, TType.STRING, 'uploadedJarLocation', None, None, ), # 2 + (3, TType.STRING, 'jsonConf', None, None, ), # 3 + (4, TType.STRUCT, 'topology', (StormTopology, StormTopology.thrift_spec), None, ), # 4 + ) + + def __init__(self, name=None, uploadedJarLocation=None, jsonConf=None, topology=None,): + self.name = name + self.uploadedJarLocation = uploadedJarLocation + self.jsonConf = jsonConf + self.topology = topology + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.name = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.uploadedJarLocation = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.jsonConf = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.topology = StormTopology() + self.topology.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('submitTopology_args') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name.encode('utf-8')) + oprot.writeFieldEnd() + if self.uploadedJarLocation is not None: + oprot.writeFieldBegin('uploadedJarLocation', TType.STRING, 2) + oprot.writeString(self.uploadedJarLocation.encode('utf-8')) + oprot.writeFieldEnd() + if self.jsonConf is not None: + oprot.writeFieldBegin('jsonConf', TType.STRING, 3) + oprot.writeString(self.jsonConf.encode('utf-8')) + oprot.writeFieldEnd() + if self.topology is not None: + oprot.writeFieldBegin('topology', TType.STRUCT, 4) + self.topology.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.name) + value = (value * 31) ^ hash(self.uploadedJarLocation) + value = (value * 31) ^ hash(self.jsonConf) + value = (value * 31) ^ hash(self.topology) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class submitTopology_result: + """ + Attributes: + - e + - ite + - tae + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'e', (AlreadyAliveException, AlreadyAliveException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'ite', (InvalidTopologyException, InvalidTopologyException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'tae', (TopologyAssignException, TopologyAssignException.thrift_spec), None, ), # 3 + ) + + def __init__(self, e=None, ite=None, tae=None,): + self.e = e + self.ite = ite + self.tae = tae + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.e = AlreadyAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.ite = InvalidTopologyException() + self.ite.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.tae = TopologyAssignException() + self.tae.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('submitTopology_result') + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + if self.ite is not None: + oprot.writeFieldBegin('ite', TType.STRUCT, 2) + self.ite.write(oprot) + oprot.writeFieldEnd() + if self.tae is not None: + oprot.writeFieldBegin('tae', TType.STRUCT, 3) + self.tae.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.e) + value = (value * 31) ^ hash(self.ite) + value = (value * 31) ^ hash(self.tae) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class submitTopologyWithOpts_args: + """ + Attributes: + - name + - uploadedJarLocation + - jsonConf + - topology + - options + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'name', None, None, ), # 1 + (2, TType.STRING, 'uploadedJarLocation', None, None, ), # 2 + (3, TType.STRING, 'jsonConf', None, None, ), # 3 + (4, TType.STRUCT, 'topology', (StormTopology, StormTopology.thrift_spec), None, ), # 4 + (5, TType.STRUCT, 'options', (SubmitOptions, SubmitOptions.thrift_spec), None, ), # 5 + ) + + def __init__(self, name=None, uploadedJarLocation=None, jsonConf=None, topology=None, options=None,): + self.name = name + self.uploadedJarLocation = uploadedJarLocation + self.jsonConf = jsonConf + self.topology = topology + self.options = options + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.name = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.uploadedJarLocation = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.jsonConf = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.topology = StormTopology() + self.topology.read(iprot) + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRUCT: + self.options = SubmitOptions() + self.options.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('submitTopologyWithOpts_args') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name.encode('utf-8')) + oprot.writeFieldEnd() + if self.uploadedJarLocation is not None: + oprot.writeFieldBegin('uploadedJarLocation', TType.STRING, 2) + oprot.writeString(self.uploadedJarLocation.encode('utf-8')) + oprot.writeFieldEnd() + if self.jsonConf is not None: + oprot.writeFieldBegin('jsonConf', TType.STRING, 3) + oprot.writeString(self.jsonConf.encode('utf-8')) + oprot.writeFieldEnd() + if self.topology is not None: + oprot.writeFieldBegin('topology', TType.STRUCT, 4) + self.topology.write(oprot) + oprot.writeFieldEnd() + if self.options is not None: + oprot.writeFieldBegin('options', TType.STRUCT, 5) + self.options.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.name) + value = (value * 31) ^ hash(self.uploadedJarLocation) + value = (value * 31) ^ hash(self.jsonConf) + value = (value * 31) ^ hash(self.topology) + value = (value * 31) ^ hash(self.options) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class submitTopologyWithOpts_result: + """ + Attributes: + - e + - ite + - tae + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'e', (AlreadyAliveException, AlreadyAliveException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'ite', (InvalidTopologyException, InvalidTopologyException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'tae', (TopologyAssignException, TopologyAssignException.thrift_spec), None, ), # 3 + ) + + def __init__(self, e=None, ite=None, tae=None,): + self.e = e + self.ite = ite + self.tae = tae + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.e = AlreadyAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.ite = InvalidTopologyException() + self.ite.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.tae = TopologyAssignException() + self.tae.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('submitTopologyWithOpts_result') + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + if self.ite is not None: + oprot.writeFieldBegin('ite', TType.STRUCT, 2) + self.ite.write(oprot) + oprot.writeFieldEnd() + if self.tae is not None: + oprot.writeFieldBegin('tae', TType.STRUCT, 3) + self.tae.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.e) + value = (value * 31) ^ hash(self.ite) + value = (value * 31) ^ hash(self.tae) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class killTopology_args: + """ + Attributes: + - name + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'name', None, None, ), # 1 + ) + + def __init__(self, name=None,): + self.name = name + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.name = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('killTopology_args') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.name) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class killTopology_result: + """ + Attributes: + - e + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + ) + + def __init__(self, e=None,): + self.e = e + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.e = NotAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('killTopology_result') + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.e) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class killTopologyWithOpts_args: + """ + Attributes: + - name + - options + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'name', None, None, ), # 1 + (2, TType.STRUCT, 'options', (KillOptions, KillOptions.thrift_spec), None, ), # 2 + ) + + def __init__(self, name=None, options=None,): + self.name = name + self.options = options + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.name = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.options = KillOptions() + self.options.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('killTopologyWithOpts_args') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name.encode('utf-8')) + oprot.writeFieldEnd() + if self.options is not None: + oprot.writeFieldBegin('options', TType.STRUCT, 2) + self.options.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.name) + value = (value * 31) ^ hash(self.options) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class killTopologyWithOpts_result: + """ + Attributes: + - e + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + ) + + def __init__(self, e=None,): + self.e = e + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.e = NotAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('killTopologyWithOpts_result') + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.e) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class activate_args: + """ + Attributes: + - name + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'name', None, None, ), # 1 + ) + + def __init__(self, name=None,): + self.name = name + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.name = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('activate_args') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.name) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class activate_result: + """ + Attributes: + - e + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + ) + + def __init__(self, e=None,): + self.e = e + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.e = NotAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('activate_result') + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.e) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class deactivate_args: + """ + Attributes: + - name + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'name', None, None, ), # 1 + ) + + def __init__(self, name=None,): + self.name = name + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.name = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('deactivate_args') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.name) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class deactivate_result: + """ + Attributes: + - e + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + ) + + def __init__(self, e=None,): + self.e = e + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.e = NotAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('deactivate_result') + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.e) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class rebalance_args: + """ + Attributes: + - name + - options + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'name', None, None, ), # 1 + (2, TType.STRUCT, 'options', (RebalanceOptions, RebalanceOptions.thrift_spec), None, ), # 2 + ) + + def __init__(self, name=None, options=None,): + self.name = name + self.options = options + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.name = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.options = RebalanceOptions() + self.options.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('rebalance_args') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name.encode('utf-8')) + oprot.writeFieldEnd() + if self.options is not None: + oprot.writeFieldBegin('options', TType.STRUCT, 2) + self.options.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.name) + value = (value * 31) ^ hash(self.options) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class rebalance_result: + """ + Attributes: + - e + - ite + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'ite', (InvalidTopologyException, InvalidTopologyException.thrift_spec), None, ), # 2 + ) + + def __init__(self, e=None, ite=None,): + self.e = e + self.ite = ite + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.e = NotAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.ite = InvalidTopologyException() + self.ite.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('rebalance_result') + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + if self.ite is not None: + oprot.writeFieldBegin('ite', TType.STRUCT, 2) + self.ite.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.e) + value = (value * 31) ^ hash(self.ite) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class metricMonitor_args: + """ + Attributes: + - name + - options + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'name', None, None, ), # 1 + (2, TType.STRUCT, 'options', (MonitorOptions, MonitorOptions.thrift_spec), None, ), # 2 + ) + + def __init__(self, name=None, options=None,): + self.name = name + self.options = options + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.name = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.options = MonitorOptions() + self.options.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('metricMonitor_args') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name.encode('utf-8')) + oprot.writeFieldEnd() + if self.options is not None: + oprot.writeFieldBegin('options', TType.STRUCT, 2) + self.options.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.name) + value = (value * 31) ^ hash(self.options) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class metricMonitor_result: + """ + Attributes: + - e + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + ) + + def __init__(self, e=None,): + self.e = e + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.e = NotAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('metricMonitor_result') + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.e) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class restart_args: + """ + Attributes: + - name + - jsonConf + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'name', None, None, ), # 1 + (2, TType.STRING, 'jsonConf', None, None, ), # 2 + ) + + def __init__(self, name=None, jsonConf=None,): + self.name = name + self.jsonConf = jsonConf + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.name = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.jsonConf = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('restart_args') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name.encode('utf-8')) + oprot.writeFieldEnd() + if self.jsonConf is not None: + oprot.writeFieldBegin('jsonConf', TType.STRING, 2) + oprot.writeString(self.jsonConf.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.name) + value = (value * 31) ^ hash(self.jsonConf) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class restart_result: + """ + Attributes: + - e + - ite + - tae + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'ite', (InvalidTopologyException, InvalidTopologyException.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'tae', (TopologyAssignException, TopologyAssignException.thrift_spec), None, ), # 3 + ) + + def __init__(self, e=None, ite=None, tae=None,): + self.e = e + self.ite = ite + self.tae = tae + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.e = NotAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.ite = InvalidTopologyException() + self.ite.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.tae = TopologyAssignException() + self.tae.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('restart_result') + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + if self.ite is not None: + oprot.writeFieldBegin('ite', TType.STRUCT, 2) + self.ite.write(oprot) + oprot.writeFieldEnd() + if self.tae is not None: + oprot.writeFieldBegin('tae', TType.STRUCT, 3) + self.tae.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.e) + value = (value * 31) ^ hash(self.ite) + value = (value * 31) ^ hash(self.tae) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class beginLibUpload_args: + """ + Attributes: + - libName + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'libName', None, None, ), # 1 + ) + + def __init__(self, libName=None,): + self.libName = libName + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.libName = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('beginLibUpload_args') + if self.libName is not None: + oprot.writeFieldBegin('libName', TType.STRING, 1) + oprot.writeString(self.libName.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.libName) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class beginLibUpload_result: + + thrift_spec = ( + ) + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('beginLibUpload_result') + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class beginFileUpload_args: + + thrift_spec = ( + ) + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('beginFileUpload_args') + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class beginFileUpload_result: + """ + Attributes: + - success + """ + + thrift_spec = ( + (0, TType.STRING, 'success', None, None, ), # 0 + ) + + def __init__(self, success=None,): + self.success = success + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRING: + self.success = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('beginFileUpload_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRING, 0) + oprot.writeString(self.success.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class uploadChunk_args: + """ + Attributes: + - location + - chunk + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'location', None, None, ), # 1 + (2, TType.STRING, 'chunk', None, None, ), # 2 + ) + + def __init__(self, location=None, chunk=None,): + self.location = location + self.chunk = chunk + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.location = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.chunk = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('uploadChunk_args') + if self.location is not None: + oprot.writeFieldBegin('location', TType.STRING, 1) + oprot.writeString(self.location.encode('utf-8')) + oprot.writeFieldEnd() + if self.chunk is not None: + oprot.writeFieldBegin('chunk', TType.STRING, 2) + oprot.writeString(self.chunk) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.location) + value = (value * 31) ^ hash(self.chunk) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class uploadChunk_result: + + thrift_spec = ( + ) + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('uploadChunk_result') + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class finishFileUpload_args: + """ + Attributes: + - location + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'location', None, None, ), # 1 + ) + + def __init__(self, location=None,): + self.location = location + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.location = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('finishFileUpload_args') + if self.location is not None: + oprot.writeFieldBegin('location', TType.STRING, 1) + oprot.writeString(self.location.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.location) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class finishFileUpload_result: + + thrift_spec = ( + ) + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('finishFileUpload_result') + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class beginFileDownload_args: + """ + Attributes: + - file + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'file', None, None, ), # 1 + ) + + def __init__(self, file=None,): + self.file = file + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.file = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('beginFileDownload_args') + if self.file is not None: + oprot.writeFieldBegin('file', TType.STRING, 1) + oprot.writeString(self.file.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.file) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class beginFileDownload_result: + """ + Attributes: + - success + """ + + thrift_spec = ( + (0, TType.STRING, 'success', None, None, ), # 0 + ) + + def __init__(self, success=None,): + self.success = success + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRING: + self.success = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('beginFileDownload_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRING, 0) + oprot.writeString(self.success.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class downloadChunk_args: + """ + Attributes: + - id + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'id', None, None, ), # 1 + ) + + def __init__(self, id=None,): + self.id = id + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('downloadChunk_args') + if self.id is not None: + oprot.writeFieldBegin('id', TType.STRING, 1) + oprot.writeString(self.id.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.id) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class downloadChunk_result: + """ + Attributes: + - success + """ + + thrift_spec = ( + (0, TType.STRING, 'success', None, None, ), # 0 + ) + + def __init__(self, success=None,): + self.success = success + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRING: + self.success = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('downloadChunk_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRING, 0) + oprot.writeString(self.success) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class finishFileDownload_args: + """ + Attributes: + - id + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'id', None, None, ), # 1 + ) + + def __init__(self, id=None,): + self.id = id + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('finishFileDownload_args') + if self.id is not None: + oprot.writeFieldBegin('id', TType.STRING, 1) + oprot.writeString(self.id.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.id) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class finishFileDownload_result: + + thrift_spec = ( + ) + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('finishFileDownload_result') + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getNimbusConf_args: + + thrift_spec = ( + ) + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getNimbusConf_args') + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getNimbusConf_result: + """ + Attributes: + - success + """ + + thrift_spec = ( + (0, TType.STRING, 'success', None, None, ), # 0 + ) + + def __init__(self, success=None,): + self.success = success + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRING: + self.success = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getNimbusConf_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRING, 0) + oprot.writeString(self.success.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getTopologyConf_args: + """ + Attributes: + - id + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'id', None, None, ), # 1 + ) + + def __init__(self, id=None,): + self.id = id + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getTopologyConf_args') + if self.id is not None: + oprot.writeFieldBegin('id', TType.STRING, 1) + oprot.writeString(self.id.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.id) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getTopologyConf_result: + """ + Attributes: + - success + - e + """ + + thrift_spec = ( + (0, TType.STRING, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + ) + + def __init__(self, success=None, e=None,): + self.success = success + self.e = e + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRING: + self.success = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.e = NotAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getTopologyConf_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRING, 0) + oprot.writeString(self.success.encode('utf-8')) + oprot.writeFieldEnd() + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + value = (value * 31) ^ hash(self.e) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getTopologyId_args: + """ + Attributes: + - topologyName + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'topologyName', None, None, ), # 1 + ) + + def __init__(self, topologyName=None,): + self.topologyName = topologyName + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.topologyName = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getTopologyId_args') + if self.topologyName is not None: + oprot.writeFieldBegin('topologyName', TType.STRING, 1) + oprot.writeString(self.topologyName.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.topologyName) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getTopologyId_result: + """ + Attributes: + - success + - e + """ + + thrift_spec = ( + (0, TType.STRING, 'success', None, None, ), # 0 + (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + ) + + def __init__(self, success=None, e=None,): + self.success = success + self.e = e + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRING: + self.success = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.e = NotAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getTopologyId_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRING, 0) + oprot.writeString(self.success.encode('utf-8')) + oprot.writeFieldEnd() + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + value = (value * 31) ^ hash(self.e) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getClusterInfo_args: + + thrift_spec = ( + ) + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getClusterInfo_args') + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getClusterInfo_result: + """ + Attributes: + - success + """ + + thrift_spec = ( + (0, TType.STRUCT, 'success', (ClusterSummary, ClusterSummary.thrift_spec), None, ), # 0 + ) + + def __init__(self, success=None,): + self.success = success + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRUCT: + self.success = ClusterSummary() + self.success.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getClusterInfo_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getSupervisorWorkers_args: + """ + Attributes: + - host + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'host', None, None, ), # 1 + ) + + def __init__(self, host=None,): + self.host = host + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.host = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getSupervisorWorkers_args') + if self.host is not None: + oprot.writeFieldBegin('host', TType.STRING, 1) + oprot.writeString(self.host.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.host) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getSupervisorWorkers_result: + """ + Attributes: + - success + - e + """ + + thrift_spec = ( + (0, TType.STRUCT, 'success', (SupervisorWorkers, SupervisorWorkers.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + ) + + def __init__(self, success=None, e=None,): + self.success = success + self.e = e + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRUCT: + self.success = SupervisorWorkers() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.e = NotAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getSupervisorWorkers_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + value = (value * 31) ^ hash(self.e) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getTopologyInfo_args: + """ + Attributes: + - id + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'id', None, None, ), # 1 + ) + + def __init__(self, id=None,): + self.id = id + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getTopologyInfo_args') + if self.id is not None: + oprot.writeFieldBegin('id', TType.STRING, 1) + oprot.writeString(self.id.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.id) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getTopologyInfo_result: + """ + Attributes: + - success + - e + """ + + thrift_spec = ( + (0, TType.STRUCT, 'success', (TopologyInfo, TopologyInfo.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + ) + + def __init__(self, success=None, e=None,): + self.success = success + self.e = e + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRUCT: + self.success = TopologyInfo() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.e = NotAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getTopologyInfo_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + value = (value * 31) ^ hash(self.e) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getTopologyInfoByName_args: + """ + Attributes: + - topologyName + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'topologyName', None, None, ), # 1 + ) + + def __init__(self, topologyName=None,): + self.topologyName = topologyName + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.topologyName = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getTopologyInfoByName_args') + if self.topologyName is not None: + oprot.writeFieldBegin('topologyName', TType.STRING, 1) + oprot.writeString(self.topologyName.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.topologyName) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getTopologyInfoByName_result: + """ + Attributes: + - success + - e + """ + + thrift_spec = ( + (0, TType.STRUCT, 'success', (TopologyInfo, TopologyInfo.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + ) + + def __init__(self, success=None, e=None,): + self.success = success + self.e = e + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRUCT: + self.success = TopologyInfo() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.e = NotAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getTopologyInfoByName_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + value = (value * 31) ^ hash(self.e) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getTopology_args: + """ + Attributes: + - id + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'id', None, None, ), # 1 + ) + + def __init__(self, id=None,): + self.id = id + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getTopology_args') + if self.id is not None: + oprot.writeFieldBegin('id', TType.STRING, 1) + oprot.writeString(self.id.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.id) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getTopology_result: + """ + Attributes: + - success + - e + """ + + thrift_spec = ( + (0, TType.STRUCT, 'success', (StormTopology, StormTopology.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + ) + + def __init__(self, success=None, e=None,): + self.success = success + self.e = e + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRUCT: + self.success = StormTopology() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.e = NotAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getTopology_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + value = (value * 31) ^ hash(self.e) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getUserTopology_args: + """ + Attributes: + - id + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'id', None, None, ), # 1 + ) + + def __init__(self, id=None,): + self.id = id + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getUserTopology_args') + if self.id is not None: + oprot.writeFieldBegin('id', TType.STRING, 1) + oprot.writeString(self.id.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.id) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getUserTopology_result: + """ + Attributes: + - success + - e + """ + + thrift_spec = ( + (0, TType.STRUCT, 'success', (StormTopology, StormTopology.thrift_spec), None, ), # 0 + (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + ) + + def __init__(self, success=None, e=None,): + self.success = success + self.e = e + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRUCT: + self.success = StormTopology() + self.success.read(iprot) + else: + iprot.skip(ftype) + elif fid == 1: + if ftype == TType.STRUCT: + self.e = NotAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getUserTopology_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + value = (value * 31) ^ hash(self.e) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class workerUploadMetric_args: + """ + Attributes: + - uploadMetrics + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'uploadMetrics', (WorkerUploadMetrics, WorkerUploadMetrics.thrift_spec), None, ), # 1 + ) + + def __init__(self, uploadMetrics=None,): + self.uploadMetrics = uploadMetrics + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.uploadMetrics = WorkerUploadMetrics() + self.uploadMetrics.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('workerUploadMetric_args') + if self.uploadMetrics is not None: + oprot.writeFieldBegin('uploadMetrics', TType.STRUCT, 1) + self.uploadMetrics.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.uploadMetrics) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class workerUploadMetric_result: + + thrift_spec = ( + ) + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('workerUploadMetric_result') + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getTopologyMetric_args: + """ + Attributes: + - topologyName + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'topologyName', None, None, ), # 1 + ) + + def __init__(self, topologyName=None,): + self.topologyName = topologyName + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.topologyName = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getTopologyMetric_args') + if self.topologyName is not None: + oprot.writeFieldBegin('topologyName', TType.STRING, 1) + oprot.writeString(self.topologyName.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.topologyName) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getTopologyMetric_result: + """ + Attributes: + - success + """ + + thrift_spec = ( + (0, TType.STRUCT, 'success', (TopologyMetric, TopologyMetric.thrift_spec), None, ), # 0 + ) + + def __init__(self, success=None,): + self.success = success + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRUCT: + self.success = TopologyMetric() + self.success.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getTopologyMetric_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getNettyMetric_args: + """ + Attributes: + - topologyName + - pos + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'topologyName', None, None, ), # 1 + (2, TType.I32, 'pos', None, None, ), # 2 + ) + + def __init__(self, topologyName=None, pos=None,): + self.topologyName = topologyName + self.pos = pos + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.topologyName = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.I32: + self.pos = iprot.readI32(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getNettyMetric_args') + if self.topologyName is not None: + oprot.writeFieldBegin('topologyName', TType.STRING, 1) + oprot.writeString(self.topologyName.encode('utf-8')) + oprot.writeFieldEnd() + if self.pos is not None: + oprot.writeFieldBegin('pos', TType.I32, 2) + oprot.writeI32(self.pos) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.topologyName) + value = (value * 31) ^ hash(self.pos) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getNettyMetric_result: + """ + Attributes: + - success + """ + + thrift_spec = ( + (0, TType.STRUCT, 'success', (NettyMetric, NettyMetric.thrift_spec), None, ), # 0 + ) + + def __init__(self, success=None,): + self.success = success + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRUCT: + self.success = NettyMetric() + self.success.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getNettyMetric_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getServerNettyMetric_args: + """ + Attributes: + - topologyName + - serverName + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'topologyName', None, None, ), # 1 + (2, TType.STRING, 'serverName', None, None, ), # 2 + ) + + def __init__(self, topologyName=None, serverName=None,): + self.topologyName = topologyName + self.serverName = serverName + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.topologyName = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.serverName = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getServerNettyMetric_args') + if self.topologyName is not None: + oprot.writeFieldBegin('topologyName', TType.STRING, 1) + oprot.writeString(self.topologyName.encode('utf-8')) + oprot.writeFieldEnd() + if self.serverName is not None: + oprot.writeFieldBegin('serverName', TType.STRING, 2) + oprot.writeString(self.serverName.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.topologyName) + value = (value * 31) ^ hash(self.serverName) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getServerNettyMetric_result: + """ + Attributes: + - success + """ + + thrift_spec = ( + (0, TType.STRUCT, 'success', (NettyMetric, NettyMetric.thrift_spec), None, ), # 0 + ) + + def __init__(self, success=None,): + self.success = success + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRUCT: + self.success = NettyMetric() + self.success.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getServerNettyMetric_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRUCT, 0) + self.success.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getVersion_args: + + thrift_spec = ( + ) + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getVersion_args') + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class getVersion_result: + """ + Attributes: + - success + """ + + thrift_spec = ( + (0, TType.STRING, 'success', None, None, ), # 0 + ) + + def __init__(self, success=None,): + self.success = success + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 0: + if ftype == TType.STRING: + self.success = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('getVersion_result') + if self.success is not None: + oprot.writeFieldBegin('success', TType.STRING, 0) + oprot.writeString(self.success.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.success) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class updateConf_args: + """ + Attributes: + - name + - conf + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'name', None, None, ), # 1 + (2, TType.STRING, 'conf', None, None, ), # 2 + ) + + def __init__(self, name=None, conf=None,): + self.name = name + self.conf = conf + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.name = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.conf = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('updateConf_args') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name.encode('utf-8')) + oprot.writeFieldEnd() + if self.conf is not None: + oprot.writeFieldBegin('conf', TType.STRING, 2) + oprot.writeString(self.conf.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.name) + value = (value * 31) ^ hash(self.conf) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class updateConf_result: + """ + Attributes: + - e + - ite + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'ite', (InvalidTopologyException, InvalidTopologyException.thrift_spec), None, ), # 2 + ) + + def __init__(self, e=None, ite=None,): + self.e = e + self.ite = ite + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.e = NotAliveException() + self.e.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.ite = InvalidTopologyException() + self.ite.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('updateConf_result') + if self.e is not None: + oprot.writeFieldBegin('e', TType.STRUCT, 1) + self.e.write(oprot) + oprot.writeFieldEnd() + if self.ite is not None: + oprot.writeFieldBegin('ite', TType.STRUCT, 2) + self.ite.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.e) + value = (value * 31) ^ hash(self.ite) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) diff --git a/jstorm-client/src/main/py/storm/__init__.py b/jstorm-core/src/main/py/storm/__init__.py old mode 100644 new mode 100755 similarity index 100% rename from jstorm-client/src/main/py/storm/__init__.py rename to jstorm-core/src/main/py/storm/__init__.py diff --git a/jstorm-core/src/main/py/storm/constants.py b/jstorm-core/src/main/py/storm/constants.py new file mode 100755 index 000000000..deb19428f --- /dev/null +++ b/jstorm-core/src/main/py/storm/constants.py @@ -0,0 +1,11 @@ +# +# Autogenerated by Thrift Compiler (0.9.2) +# +# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING +# +# options string: py:utf8strings +# + +from thrift.Thrift import TType, TMessageType, TException, TApplicationException +from ttypes import * + diff --git a/jstorm-core/src/main/py/storm/ttypes.py b/jstorm-core/src/main/py/storm/ttypes.py new file mode 100644 index 000000000..dc05493dd --- /dev/null +++ b/jstorm-core/src/main/py/storm/ttypes.py @@ -0,0 +1,4456 @@ +# +# Autogenerated by Thrift Compiler (0.9.2) +# +# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING +# +# options string: py:utf8strings +# + +from thrift.Thrift import TType, TMessageType, TException, TApplicationException + +from thrift.transport import TTransport +from thrift.protocol import TBinaryProtocol, TProtocol +try: + from thrift.protocol import fastbinary +except: + fastbinary = None + + +class TopologyInitialStatus: + ACTIVE = 1 + INACTIVE = 2 + + _VALUES_TO_NAMES = { + 1: "ACTIVE", + 2: "INACTIVE", + } + + _NAMES_TO_VALUES = { + "ACTIVE": 1, + "INACTIVE": 2, + } + + +class JavaObjectArg: + """ + Attributes: + - int_arg + - long_arg + - string_arg + - bool_arg + - binary_arg + - double_arg + """ + + thrift_spec = ( + None, # 0 + (1, TType.I32, 'int_arg', None, None, ), # 1 + (2, TType.I64, 'long_arg', None, None, ), # 2 + (3, TType.STRING, 'string_arg', None, None, ), # 3 + (4, TType.BOOL, 'bool_arg', None, None, ), # 4 + (5, TType.STRING, 'binary_arg', None, None, ), # 5 + (6, TType.DOUBLE, 'double_arg', None, None, ), # 6 + ) + + def __init__(self, int_arg=None, long_arg=None, string_arg=None, bool_arg=None, binary_arg=None, double_arg=None,): + self.int_arg = int_arg + self.long_arg = long_arg + self.string_arg = string_arg + self.bool_arg = bool_arg + self.binary_arg = binary_arg + self.double_arg = double_arg + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.I32: + self.int_arg = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.I64: + self.long_arg = iprot.readI64(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.string_arg = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.BOOL: + self.bool_arg = iprot.readBool(); + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRING: + self.binary_arg = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.DOUBLE: + self.double_arg = iprot.readDouble(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('JavaObjectArg') + if self.int_arg is not None: + oprot.writeFieldBegin('int_arg', TType.I32, 1) + oprot.writeI32(self.int_arg) + oprot.writeFieldEnd() + if self.long_arg is not None: + oprot.writeFieldBegin('long_arg', TType.I64, 2) + oprot.writeI64(self.long_arg) + oprot.writeFieldEnd() + if self.string_arg is not None: + oprot.writeFieldBegin('string_arg', TType.STRING, 3) + oprot.writeString(self.string_arg.encode('utf-8')) + oprot.writeFieldEnd() + if self.bool_arg is not None: + oprot.writeFieldBegin('bool_arg', TType.BOOL, 4) + oprot.writeBool(self.bool_arg) + oprot.writeFieldEnd() + if self.binary_arg is not None: + oprot.writeFieldBegin('binary_arg', TType.STRING, 5) + oprot.writeString(self.binary_arg) + oprot.writeFieldEnd() + if self.double_arg is not None: + oprot.writeFieldBegin('double_arg', TType.DOUBLE, 6) + oprot.writeDouble(self.double_arg) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.int_arg) + value = (value * 31) ^ hash(self.long_arg) + value = (value * 31) ^ hash(self.string_arg) + value = (value * 31) ^ hash(self.bool_arg) + value = (value * 31) ^ hash(self.binary_arg) + value = (value * 31) ^ hash(self.double_arg) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class JavaObject: + """ + Attributes: + - full_class_name + - args_list + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'full_class_name', None, None, ), # 1 + (2, TType.LIST, 'args_list', (TType.STRUCT,(JavaObjectArg, JavaObjectArg.thrift_spec)), None, ), # 2 + ) + + def __init__(self, full_class_name=None, args_list=None,): + self.full_class_name = full_class_name + self.args_list = args_list + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.full_class_name = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.LIST: + self.args_list = [] + (_etype3, _size0) = iprot.readListBegin() + for _i4 in xrange(_size0): + _elem5 = JavaObjectArg() + _elem5.read(iprot) + self.args_list.append(_elem5) + iprot.readListEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('JavaObject') + if self.full_class_name is not None: + oprot.writeFieldBegin('full_class_name', TType.STRING, 1) + oprot.writeString(self.full_class_name.encode('utf-8')) + oprot.writeFieldEnd() + if self.args_list is not None: + oprot.writeFieldBegin('args_list', TType.LIST, 2) + oprot.writeListBegin(TType.STRUCT, len(self.args_list)) + for iter6 in self.args_list: + iter6.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.full_class_name is None: + raise TProtocol.TProtocolException(message='Required field full_class_name is unset!') + if self.args_list is None: + raise TProtocol.TProtocolException(message='Required field args_list is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.full_class_name) + value = (value * 31) ^ hash(self.args_list) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class NullStruct: + + thrift_spec = ( + ) + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('NullStruct') + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class GlobalStreamId: + """ + Attributes: + - componentId + - streamId + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'componentId', None, None, ), # 1 + (2, TType.STRING, 'streamId', None, None, ), # 2 + ) + + def __init__(self, componentId=None, streamId=None,): + self.componentId = componentId + self.streamId = streamId + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.componentId = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.streamId = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('GlobalStreamId') + if self.componentId is not None: + oprot.writeFieldBegin('componentId', TType.STRING, 1) + oprot.writeString(self.componentId.encode('utf-8')) + oprot.writeFieldEnd() + if self.streamId is not None: + oprot.writeFieldBegin('streamId', TType.STRING, 2) + oprot.writeString(self.streamId.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.componentId is None: + raise TProtocol.TProtocolException(message='Required field componentId is unset!') + if self.streamId is None: + raise TProtocol.TProtocolException(message='Required field streamId is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.componentId) + value = (value * 31) ^ hash(self.streamId) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class Grouping: + """ + Attributes: + - fields + - shuffle + - all + - none + - direct + - custom_object + - custom_serialized + - local_or_shuffle + - localFirst + """ + + thrift_spec = ( + None, # 0 + (1, TType.LIST, 'fields', (TType.STRING,None), None, ), # 1 + (2, TType.STRUCT, 'shuffle', (NullStruct, NullStruct.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'all', (NullStruct, NullStruct.thrift_spec), None, ), # 3 + (4, TType.STRUCT, 'none', (NullStruct, NullStruct.thrift_spec), None, ), # 4 + (5, TType.STRUCT, 'direct', (NullStruct, NullStruct.thrift_spec), None, ), # 5 + (6, TType.STRUCT, 'custom_object', (JavaObject, JavaObject.thrift_spec), None, ), # 6 + (7, TType.STRING, 'custom_serialized', None, None, ), # 7 + (8, TType.STRUCT, 'local_or_shuffle', (NullStruct, NullStruct.thrift_spec), None, ), # 8 + (9, TType.STRUCT, 'localFirst', (NullStruct, NullStruct.thrift_spec), None, ), # 9 + ) + + def __init__(self, fields=None, shuffle=None, all=None, none=None, direct=None, custom_object=None, custom_serialized=None, local_or_shuffle=None, localFirst=None,): + self.fields = fields + self.shuffle = shuffle + self.all = all + self.none = none + self.direct = direct + self.custom_object = custom_object + self.custom_serialized = custom_serialized + self.local_or_shuffle = local_or_shuffle + self.localFirst = localFirst + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.LIST: + self.fields = [] + (_etype10, _size7) = iprot.readListBegin() + for _i11 in xrange(_size7): + _elem12 = iprot.readString().decode('utf-8') + self.fields.append(_elem12) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.shuffle = NullStruct() + self.shuffle.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.all = NullStruct() + self.all.read(iprot) + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.none = NullStruct() + self.none.read(iprot) + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRUCT: + self.direct = NullStruct() + self.direct.read(iprot) + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.STRUCT: + self.custom_object = JavaObject() + self.custom_object.read(iprot) + else: + iprot.skip(ftype) + elif fid == 7: + if ftype == TType.STRING: + self.custom_serialized = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 8: + if ftype == TType.STRUCT: + self.local_or_shuffle = NullStruct() + self.local_or_shuffle.read(iprot) + else: + iprot.skip(ftype) + elif fid == 9: + if ftype == TType.STRUCT: + self.localFirst = NullStruct() + self.localFirst.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('Grouping') + if self.fields is not None: + oprot.writeFieldBegin('fields', TType.LIST, 1) + oprot.writeListBegin(TType.STRING, len(self.fields)) + for iter13 in self.fields: + oprot.writeString(iter13.encode('utf-8')) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.shuffle is not None: + oprot.writeFieldBegin('shuffle', TType.STRUCT, 2) + self.shuffle.write(oprot) + oprot.writeFieldEnd() + if self.all is not None: + oprot.writeFieldBegin('all', TType.STRUCT, 3) + self.all.write(oprot) + oprot.writeFieldEnd() + if self.none is not None: + oprot.writeFieldBegin('none', TType.STRUCT, 4) + self.none.write(oprot) + oprot.writeFieldEnd() + if self.direct is not None: + oprot.writeFieldBegin('direct', TType.STRUCT, 5) + self.direct.write(oprot) + oprot.writeFieldEnd() + if self.custom_object is not None: + oprot.writeFieldBegin('custom_object', TType.STRUCT, 6) + self.custom_object.write(oprot) + oprot.writeFieldEnd() + if self.custom_serialized is not None: + oprot.writeFieldBegin('custom_serialized', TType.STRING, 7) + oprot.writeString(self.custom_serialized) + oprot.writeFieldEnd() + if self.local_or_shuffle is not None: + oprot.writeFieldBegin('local_or_shuffle', TType.STRUCT, 8) + self.local_or_shuffle.write(oprot) + oprot.writeFieldEnd() + if self.localFirst is not None: + oprot.writeFieldBegin('localFirst', TType.STRUCT, 9) + self.localFirst.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.fields) + value = (value * 31) ^ hash(self.shuffle) + value = (value * 31) ^ hash(self.all) + value = (value * 31) ^ hash(self.none) + value = (value * 31) ^ hash(self.direct) + value = (value * 31) ^ hash(self.custom_object) + value = (value * 31) ^ hash(self.custom_serialized) + value = (value * 31) ^ hash(self.local_or_shuffle) + value = (value * 31) ^ hash(self.localFirst) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class StreamInfo: + """ + Attributes: + - output_fields + - direct + """ + + thrift_spec = ( + None, # 0 + (1, TType.LIST, 'output_fields', (TType.STRING,None), None, ), # 1 + (2, TType.BOOL, 'direct', None, None, ), # 2 + ) + + def __init__(self, output_fields=None, direct=None,): + self.output_fields = output_fields + self.direct = direct + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.LIST: + self.output_fields = [] + (_etype17, _size14) = iprot.readListBegin() + for _i18 in xrange(_size14): + _elem19 = iprot.readString().decode('utf-8') + self.output_fields.append(_elem19) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.BOOL: + self.direct = iprot.readBool(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('StreamInfo') + if self.output_fields is not None: + oprot.writeFieldBegin('output_fields', TType.LIST, 1) + oprot.writeListBegin(TType.STRING, len(self.output_fields)) + for iter20 in self.output_fields: + oprot.writeString(iter20.encode('utf-8')) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.direct is not None: + oprot.writeFieldBegin('direct', TType.BOOL, 2) + oprot.writeBool(self.direct) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.output_fields is None: + raise TProtocol.TProtocolException(message='Required field output_fields is unset!') + if self.direct is None: + raise TProtocol.TProtocolException(message='Required field direct is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.output_fields) + value = (value * 31) ^ hash(self.direct) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class ShellComponent: + """ + Attributes: + - execution_command + - script + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'execution_command', None, None, ), # 1 + (2, TType.STRING, 'script', None, None, ), # 2 + ) + + def __init__(self, execution_command=None, script=None,): + self.execution_command = execution_command + self.script = script + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.execution_command = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.script = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('ShellComponent') + if self.execution_command is not None: + oprot.writeFieldBegin('execution_command', TType.STRING, 1) + oprot.writeString(self.execution_command.encode('utf-8')) + oprot.writeFieldEnd() + if self.script is not None: + oprot.writeFieldBegin('script', TType.STRING, 2) + oprot.writeString(self.script.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.execution_command) + value = (value * 31) ^ hash(self.script) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class ComponentObject: + """ + Attributes: + - serialized_java + - shell + - java_object + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'serialized_java', None, None, ), # 1 + (2, TType.STRUCT, 'shell', (ShellComponent, ShellComponent.thrift_spec), None, ), # 2 + (3, TType.STRUCT, 'java_object', (JavaObject, JavaObject.thrift_spec), None, ), # 3 + ) + + def __init__(self, serialized_java=None, shell=None, java_object=None,): + self.serialized_java = serialized_java + self.shell = shell + self.java_object = java_object + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.serialized_java = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.shell = ShellComponent() + self.shell.read(iprot) + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRUCT: + self.java_object = JavaObject() + self.java_object.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('ComponentObject') + if self.serialized_java is not None: + oprot.writeFieldBegin('serialized_java', TType.STRING, 1) + oprot.writeString(self.serialized_java) + oprot.writeFieldEnd() + if self.shell is not None: + oprot.writeFieldBegin('shell', TType.STRUCT, 2) + self.shell.write(oprot) + oprot.writeFieldEnd() + if self.java_object is not None: + oprot.writeFieldBegin('java_object', TType.STRUCT, 3) + self.java_object.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.serialized_java) + value = (value * 31) ^ hash(self.shell) + value = (value * 31) ^ hash(self.java_object) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class ComponentCommon: + """ + Attributes: + - inputs + - streams + - parallelism_hint + - json_conf + """ + + thrift_spec = ( + None, # 0 + (1, TType.MAP, 'inputs', (TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.STRUCT,(Grouping, Grouping.thrift_spec)), None, ), # 1 + (2, TType.MAP, 'streams', (TType.STRING,None,TType.STRUCT,(StreamInfo, StreamInfo.thrift_spec)), None, ), # 2 + (3, TType.I32, 'parallelism_hint', None, None, ), # 3 + (4, TType.STRING, 'json_conf', None, None, ), # 4 + ) + + def __init__(self, inputs=None, streams=None, parallelism_hint=None, json_conf=None,): + self.inputs = inputs + self.streams = streams + self.parallelism_hint = parallelism_hint + self.json_conf = json_conf + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.MAP: + self.inputs = {} + (_ktype22, _vtype23, _size21 ) = iprot.readMapBegin() + for _i25 in xrange(_size21): + _key26 = GlobalStreamId() + _key26.read(iprot) + _val27 = Grouping() + _val27.read(iprot) + self.inputs[_key26] = _val27 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.MAP: + self.streams = {} + (_ktype29, _vtype30, _size28 ) = iprot.readMapBegin() + for _i32 in xrange(_size28): + _key33 = iprot.readString().decode('utf-8') + _val34 = StreamInfo() + _val34.read(iprot) + self.streams[_key33] = _val34 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.I32: + self.parallelism_hint = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRING: + self.json_conf = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('ComponentCommon') + if self.inputs is not None: + oprot.writeFieldBegin('inputs', TType.MAP, 1) + oprot.writeMapBegin(TType.STRUCT, TType.STRUCT, len(self.inputs)) + for kiter35,viter36 in self.inputs.items(): + kiter35.write(oprot) + viter36.write(oprot) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.streams is not None: + oprot.writeFieldBegin('streams', TType.MAP, 2) + oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.streams)) + for kiter37,viter38 in self.streams.items(): + oprot.writeString(kiter37.encode('utf-8')) + viter38.write(oprot) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.parallelism_hint is not None: + oprot.writeFieldBegin('parallelism_hint', TType.I32, 3) + oprot.writeI32(self.parallelism_hint) + oprot.writeFieldEnd() + if self.json_conf is not None: + oprot.writeFieldBegin('json_conf', TType.STRING, 4) + oprot.writeString(self.json_conf.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.inputs is None: + raise TProtocol.TProtocolException(message='Required field inputs is unset!') + if self.streams is None: + raise TProtocol.TProtocolException(message='Required field streams is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.inputs) + value = (value * 31) ^ hash(self.streams) + value = (value * 31) ^ hash(self.parallelism_hint) + value = (value * 31) ^ hash(self.json_conf) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class SpoutSpec: + """ + Attributes: + - spout_object + - common + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'spout_object', (ComponentObject, ComponentObject.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'common', (ComponentCommon, ComponentCommon.thrift_spec), None, ), # 2 + ) + + def __init__(self, spout_object=None, common=None,): + self.spout_object = spout_object + self.common = common + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.spout_object = ComponentObject() + self.spout_object.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.common = ComponentCommon() + self.common.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('SpoutSpec') + if self.spout_object is not None: + oprot.writeFieldBegin('spout_object', TType.STRUCT, 1) + self.spout_object.write(oprot) + oprot.writeFieldEnd() + if self.common is not None: + oprot.writeFieldBegin('common', TType.STRUCT, 2) + self.common.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.spout_object is None: + raise TProtocol.TProtocolException(message='Required field spout_object is unset!') + if self.common is None: + raise TProtocol.TProtocolException(message='Required field common is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.spout_object) + value = (value * 31) ^ hash(self.common) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class Bolt: + """ + Attributes: + - bolt_object + - common + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'bolt_object', (ComponentObject, ComponentObject.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'common', (ComponentCommon, ComponentCommon.thrift_spec), None, ), # 2 + ) + + def __init__(self, bolt_object=None, common=None,): + self.bolt_object = bolt_object + self.common = common + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.bolt_object = ComponentObject() + self.bolt_object.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.common = ComponentCommon() + self.common.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('Bolt') + if self.bolt_object is not None: + oprot.writeFieldBegin('bolt_object', TType.STRUCT, 1) + self.bolt_object.write(oprot) + oprot.writeFieldEnd() + if self.common is not None: + oprot.writeFieldBegin('common', TType.STRUCT, 2) + self.common.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.bolt_object is None: + raise TProtocol.TProtocolException(message='Required field bolt_object is unset!') + if self.common is None: + raise TProtocol.TProtocolException(message='Required field common is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.bolt_object) + value = (value * 31) ^ hash(self.common) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class StateSpoutSpec: + """ + Attributes: + - state_spout_object + - common + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'state_spout_object', (ComponentObject, ComponentObject.thrift_spec), None, ), # 1 + (2, TType.STRUCT, 'common', (ComponentCommon, ComponentCommon.thrift_spec), None, ), # 2 + ) + + def __init__(self, state_spout_object=None, common=None,): + self.state_spout_object = state_spout_object + self.common = common + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.state_spout_object = ComponentObject() + self.state_spout_object.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRUCT: + self.common = ComponentCommon() + self.common.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('StateSpoutSpec') + if self.state_spout_object is not None: + oprot.writeFieldBegin('state_spout_object', TType.STRUCT, 1) + self.state_spout_object.write(oprot) + oprot.writeFieldEnd() + if self.common is not None: + oprot.writeFieldBegin('common', TType.STRUCT, 2) + self.common.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.state_spout_object is None: + raise TProtocol.TProtocolException(message='Required field state_spout_object is unset!') + if self.common is None: + raise TProtocol.TProtocolException(message='Required field common is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.state_spout_object) + value = (value * 31) ^ hash(self.common) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class StormTopology: + """ + Attributes: + - spouts + - bolts + - state_spouts + """ + + thrift_spec = ( + None, # 0 + (1, TType.MAP, 'spouts', (TType.STRING,None,TType.STRUCT,(SpoutSpec, SpoutSpec.thrift_spec)), None, ), # 1 + (2, TType.MAP, 'bolts', (TType.STRING,None,TType.STRUCT,(Bolt, Bolt.thrift_spec)), None, ), # 2 + (3, TType.MAP, 'state_spouts', (TType.STRING,None,TType.STRUCT,(StateSpoutSpec, StateSpoutSpec.thrift_spec)), None, ), # 3 + ) + + def __init__(self, spouts=None, bolts=None, state_spouts=None,): + self.spouts = spouts + self.bolts = bolts + self.state_spouts = state_spouts + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.MAP: + self.spouts = {} + (_ktype40, _vtype41, _size39 ) = iprot.readMapBegin() + for _i43 in xrange(_size39): + _key44 = iprot.readString().decode('utf-8') + _val45 = SpoutSpec() + _val45.read(iprot) + self.spouts[_key44] = _val45 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.MAP: + self.bolts = {} + (_ktype47, _vtype48, _size46 ) = iprot.readMapBegin() + for _i50 in xrange(_size46): + _key51 = iprot.readString().decode('utf-8') + _val52 = Bolt() + _val52.read(iprot) + self.bolts[_key51] = _val52 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.MAP: + self.state_spouts = {} + (_ktype54, _vtype55, _size53 ) = iprot.readMapBegin() + for _i57 in xrange(_size53): + _key58 = iprot.readString().decode('utf-8') + _val59 = StateSpoutSpec() + _val59.read(iprot) + self.state_spouts[_key58] = _val59 + iprot.readMapEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('StormTopology') + if self.spouts is not None: + oprot.writeFieldBegin('spouts', TType.MAP, 1) + oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.spouts)) + for kiter60,viter61 in self.spouts.items(): + oprot.writeString(kiter60.encode('utf-8')) + viter61.write(oprot) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.bolts is not None: + oprot.writeFieldBegin('bolts', TType.MAP, 2) + oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.bolts)) + for kiter62,viter63 in self.bolts.items(): + oprot.writeString(kiter62.encode('utf-8')) + viter63.write(oprot) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.state_spouts is not None: + oprot.writeFieldBegin('state_spouts', TType.MAP, 3) + oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.state_spouts)) + for kiter64,viter65 in self.state_spouts.items(): + oprot.writeString(kiter64.encode('utf-8')) + viter65.write(oprot) + oprot.writeMapEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.spouts is None: + raise TProtocol.TProtocolException(message='Required field spouts is unset!') + if self.bolts is None: + raise TProtocol.TProtocolException(message='Required field bolts is unset!') + if self.state_spouts is None: + raise TProtocol.TProtocolException(message='Required field state_spouts is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.spouts) + value = (value * 31) ^ hash(self.bolts) + value = (value * 31) ^ hash(self.state_spouts) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class TopologySummary: + """ + Attributes: + - id + - name + - status + - uptime_secs + - num_tasks + - num_workers + - error_info + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'id', None, None, ), # 1 + (2, TType.STRING, 'name', None, None, ), # 2 + (3, TType.STRING, 'status', None, None, ), # 3 + (4, TType.I32, 'uptime_secs', None, None, ), # 4 + (5, TType.I32, 'num_tasks', None, None, ), # 5 + (6, TType.I32, 'num_workers', None, None, ), # 6 + (7, TType.STRING, 'error_info', None, None, ), # 7 + ) + + def __init__(self, id=None, name=None, status=None, uptime_secs=None, num_tasks=None, num_workers=None, error_info=None,): + self.id = id + self.name = name + self.status = status + self.uptime_secs = uptime_secs + self.num_tasks = num_tasks + self.num_workers = num_workers + self.error_info = error_info + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.name = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.status = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.I32: + self.uptime_secs = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.I32: + self.num_tasks = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.I32: + self.num_workers = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 7: + if ftype == TType.STRING: + self.error_info = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('TopologySummary') + if self.id is not None: + oprot.writeFieldBegin('id', TType.STRING, 1) + oprot.writeString(self.id.encode('utf-8')) + oprot.writeFieldEnd() + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 2) + oprot.writeString(self.name.encode('utf-8')) + oprot.writeFieldEnd() + if self.status is not None: + oprot.writeFieldBegin('status', TType.STRING, 3) + oprot.writeString(self.status.encode('utf-8')) + oprot.writeFieldEnd() + if self.uptime_secs is not None: + oprot.writeFieldBegin('uptime_secs', TType.I32, 4) + oprot.writeI32(self.uptime_secs) + oprot.writeFieldEnd() + if self.num_tasks is not None: + oprot.writeFieldBegin('num_tasks', TType.I32, 5) + oprot.writeI32(self.num_tasks) + oprot.writeFieldEnd() + if self.num_workers is not None: + oprot.writeFieldBegin('num_workers', TType.I32, 6) + oprot.writeI32(self.num_workers) + oprot.writeFieldEnd() + if self.error_info is not None: + oprot.writeFieldBegin('error_info', TType.STRING, 7) + oprot.writeString(self.error_info.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.id is None: + raise TProtocol.TProtocolException(message='Required field id is unset!') + if self.name is None: + raise TProtocol.TProtocolException(message='Required field name is unset!') + if self.status is None: + raise TProtocol.TProtocolException(message='Required field status is unset!') + if self.uptime_secs is None: + raise TProtocol.TProtocolException(message='Required field uptime_secs is unset!') + if self.num_tasks is None: + raise TProtocol.TProtocolException(message='Required field num_tasks is unset!') + if self.num_workers is None: + raise TProtocol.TProtocolException(message='Required field num_workers is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.id) + value = (value * 31) ^ hash(self.name) + value = (value * 31) ^ hash(self.status) + value = (value * 31) ^ hash(self.uptime_secs) + value = (value * 31) ^ hash(self.num_tasks) + value = (value * 31) ^ hash(self.num_workers) + value = (value * 31) ^ hash(self.error_info) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class SupervisorSummary: + """ + Attributes: + - host + - supervisor_id + - uptime_secs + - num_workers + - num_used_workers + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'host', None, None, ), # 1 + (2, TType.STRING, 'supervisor_id', None, None, ), # 2 + (3, TType.I32, 'uptime_secs', None, None, ), # 3 + (4, TType.I32, 'num_workers', None, None, ), # 4 + (5, TType.I32, 'num_used_workers', None, None, ), # 5 + ) + + def __init__(self, host=None, supervisor_id=None, uptime_secs=None, num_workers=None, num_used_workers=None,): + self.host = host + self.supervisor_id = supervisor_id + self.uptime_secs = uptime_secs + self.num_workers = num_workers + self.num_used_workers = num_used_workers + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.host = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.supervisor_id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.I32: + self.uptime_secs = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.I32: + self.num_workers = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.I32: + self.num_used_workers = iprot.readI32(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('SupervisorSummary') + if self.host is not None: + oprot.writeFieldBegin('host', TType.STRING, 1) + oprot.writeString(self.host.encode('utf-8')) + oprot.writeFieldEnd() + if self.supervisor_id is not None: + oprot.writeFieldBegin('supervisor_id', TType.STRING, 2) + oprot.writeString(self.supervisor_id.encode('utf-8')) + oprot.writeFieldEnd() + if self.uptime_secs is not None: + oprot.writeFieldBegin('uptime_secs', TType.I32, 3) + oprot.writeI32(self.uptime_secs) + oprot.writeFieldEnd() + if self.num_workers is not None: + oprot.writeFieldBegin('num_workers', TType.I32, 4) + oprot.writeI32(self.num_workers) + oprot.writeFieldEnd() + if self.num_used_workers is not None: + oprot.writeFieldBegin('num_used_workers', TType.I32, 5) + oprot.writeI32(self.num_used_workers) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.host is None: + raise TProtocol.TProtocolException(message='Required field host is unset!') + if self.supervisor_id is None: + raise TProtocol.TProtocolException(message='Required field supervisor_id is unset!') + if self.uptime_secs is None: + raise TProtocol.TProtocolException(message='Required field uptime_secs is unset!') + if self.num_workers is None: + raise TProtocol.TProtocolException(message='Required field num_workers is unset!') + if self.num_used_workers is None: + raise TProtocol.TProtocolException(message='Required field num_used_workers is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.host) + value = (value * 31) ^ hash(self.supervisor_id) + value = (value * 31) ^ hash(self.uptime_secs) + value = (value * 31) ^ hash(self.num_workers) + value = (value * 31) ^ hash(self.num_used_workers) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class NimbusStat: + """ + Attributes: + - host + - uptime_secs + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'host', None, None, ), # 1 + (2, TType.STRING, 'uptime_secs', None, None, ), # 2 + ) + + def __init__(self, host=None, uptime_secs=None,): + self.host = host + self.uptime_secs = uptime_secs + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.host = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.uptime_secs = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('NimbusStat') + if self.host is not None: + oprot.writeFieldBegin('host', TType.STRING, 1) + oprot.writeString(self.host.encode('utf-8')) + oprot.writeFieldEnd() + if self.uptime_secs is not None: + oprot.writeFieldBegin('uptime_secs', TType.STRING, 2) + oprot.writeString(self.uptime_secs.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.host is None: + raise TProtocol.TProtocolException(message='Required field host is unset!') + if self.uptime_secs is None: + raise TProtocol.TProtocolException(message='Required field uptime_secs is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.host) + value = (value * 31) ^ hash(self.uptime_secs) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class NimbusSummary: + """ + Attributes: + - nimbus_master + - nimbus_slaves + - supervisor_num + - total_port_num + - used_port_num + - free_port_num + - version + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'nimbus_master', (NimbusStat, NimbusStat.thrift_spec), None, ), # 1 + (2, TType.LIST, 'nimbus_slaves', (TType.STRUCT,(NimbusStat, NimbusStat.thrift_spec)), None, ), # 2 + (3, TType.I32, 'supervisor_num', None, None, ), # 3 + (4, TType.I32, 'total_port_num', None, None, ), # 4 + (5, TType.I32, 'used_port_num', None, None, ), # 5 + (6, TType.I32, 'free_port_num', None, None, ), # 6 + (7, TType.STRING, 'version', None, None, ), # 7 + ) + + def __init__(self, nimbus_master=None, nimbus_slaves=None, supervisor_num=None, total_port_num=None, used_port_num=None, free_port_num=None, version=None,): + self.nimbus_master = nimbus_master + self.nimbus_slaves = nimbus_slaves + self.supervisor_num = supervisor_num + self.total_port_num = total_port_num + self.used_port_num = used_port_num + self.free_port_num = free_port_num + self.version = version + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.nimbus_master = NimbusStat() + self.nimbus_master.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.LIST: + self.nimbus_slaves = [] + (_etype69, _size66) = iprot.readListBegin() + for _i70 in xrange(_size66): + _elem71 = NimbusStat() + _elem71.read(iprot) + self.nimbus_slaves.append(_elem71) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.I32: + self.supervisor_num = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.I32: + self.total_port_num = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.I32: + self.used_port_num = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.I32: + self.free_port_num = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 7: + if ftype == TType.STRING: + self.version = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('NimbusSummary') + if self.nimbus_master is not None: + oprot.writeFieldBegin('nimbus_master', TType.STRUCT, 1) + self.nimbus_master.write(oprot) + oprot.writeFieldEnd() + if self.nimbus_slaves is not None: + oprot.writeFieldBegin('nimbus_slaves', TType.LIST, 2) + oprot.writeListBegin(TType.STRUCT, len(self.nimbus_slaves)) + for iter72 in self.nimbus_slaves: + iter72.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.supervisor_num is not None: + oprot.writeFieldBegin('supervisor_num', TType.I32, 3) + oprot.writeI32(self.supervisor_num) + oprot.writeFieldEnd() + if self.total_port_num is not None: + oprot.writeFieldBegin('total_port_num', TType.I32, 4) + oprot.writeI32(self.total_port_num) + oprot.writeFieldEnd() + if self.used_port_num is not None: + oprot.writeFieldBegin('used_port_num', TType.I32, 5) + oprot.writeI32(self.used_port_num) + oprot.writeFieldEnd() + if self.free_port_num is not None: + oprot.writeFieldBegin('free_port_num', TType.I32, 6) + oprot.writeI32(self.free_port_num) + oprot.writeFieldEnd() + if self.version is not None: + oprot.writeFieldBegin('version', TType.STRING, 7) + oprot.writeString(self.version.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.nimbus_master is None: + raise TProtocol.TProtocolException(message='Required field nimbus_master is unset!') + if self.nimbus_slaves is None: + raise TProtocol.TProtocolException(message='Required field nimbus_slaves is unset!') + if self.supervisor_num is None: + raise TProtocol.TProtocolException(message='Required field supervisor_num is unset!') + if self.total_port_num is None: + raise TProtocol.TProtocolException(message='Required field total_port_num is unset!') + if self.used_port_num is None: + raise TProtocol.TProtocolException(message='Required field used_port_num is unset!') + if self.free_port_num is None: + raise TProtocol.TProtocolException(message='Required field free_port_num is unset!') + if self.version is None: + raise TProtocol.TProtocolException(message='Required field version is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.nimbus_master) + value = (value * 31) ^ hash(self.nimbus_slaves) + value = (value * 31) ^ hash(self.supervisor_num) + value = (value * 31) ^ hash(self.total_port_num) + value = (value * 31) ^ hash(self.used_port_num) + value = (value * 31) ^ hash(self.free_port_num) + value = (value * 31) ^ hash(self.version) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class ClusterSummary: + """ + Attributes: + - nimbus + - supervisors + - topologies + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'nimbus', (NimbusSummary, NimbusSummary.thrift_spec), None, ), # 1 + (2, TType.LIST, 'supervisors', (TType.STRUCT,(SupervisorSummary, SupervisorSummary.thrift_spec)), None, ), # 2 + (3, TType.LIST, 'topologies', (TType.STRUCT,(TopologySummary, TopologySummary.thrift_spec)), None, ), # 3 + ) + + def __init__(self, nimbus=None, supervisors=None, topologies=None,): + self.nimbus = nimbus + self.supervisors = supervisors + self.topologies = topologies + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.nimbus = NimbusSummary() + self.nimbus.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.LIST: + self.supervisors = [] + (_etype76, _size73) = iprot.readListBegin() + for _i77 in xrange(_size73): + _elem78 = SupervisorSummary() + _elem78.read(iprot) + self.supervisors.append(_elem78) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.LIST: + self.topologies = [] + (_etype82, _size79) = iprot.readListBegin() + for _i83 in xrange(_size79): + _elem84 = TopologySummary() + _elem84.read(iprot) + self.topologies.append(_elem84) + iprot.readListEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('ClusterSummary') + if self.nimbus is not None: + oprot.writeFieldBegin('nimbus', TType.STRUCT, 1) + self.nimbus.write(oprot) + oprot.writeFieldEnd() + if self.supervisors is not None: + oprot.writeFieldBegin('supervisors', TType.LIST, 2) + oprot.writeListBegin(TType.STRUCT, len(self.supervisors)) + for iter85 in self.supervisors: + iter85.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.topologies is not None: + oprot.writeFieldBegin('topologies', TType.LIST, 3) + oprot.writeListBegin(TType.STRUCT, len(self.topologies)) + for iter86 in self.topologies: + iter86.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.nimbus is None: + raise TProtocol.TProtocolException(message='Required field nimbus is unset!') + if self.supervisors is None: + raise TProtocol.TProtocolException(message='Required field supervisors is unset!') + if self.topologies is None: + raise TProtocol.TProtocolException(message='Required field topologies is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.nimbus) + value = (value * 31) ^ hash(self.supervisors) + value = (value * 31) ^ hash(self.topologies) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class TaskComponent: + """ + Attributes: + - taskId + - component + """ + + thrift_spec = ( + None, # 0 + (1, TType.I32, 'taskId', None, None, ), # 1 + (2, TType.STRING, 'component', None, None, ), # 2 + ) + + def __init__(self, taskId=None, component=None,): + self.taskId = taskId + self.component = component + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.I32: + self.taskId = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.component = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('TaskComponent') + if self.taskId is not None: + oprot.writeFieldBegin('taskId', TType.I32, 1) + oprot.writeI32(self.taskId) + oprot.writeFieldEnd() + if self.component is not None: + oprot.writeFieldBegin('component', TType.STRING, 2) + oprot.writeString(self.component.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.taskId is None: + raise TProtocol.TProtocolException(message='Required field taskId is unset!') + if self.component is None: + raise TProtocol.TProtocolException(message='Required field component is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.taskId) + value = (value * 31) ^ hash(self.component) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class WorkerSummary: + """ + Attributes: + - port + - uptime + - topology + - tasks + """ + + thrift_spec = ( + None, # 0 + (1, TType.I32, 'port', None, None, ), # 1 + (2, TType.I32, 'uptime', None, None, ), # 2 + (3, TType.STRING, 'topology', None, None, ), # 3 + (4, TType.LIST, 'tasks', (TType.STRUCT,(TaskComponent, TaskComponent.thrift_spec)), None, ), # 4 + ) + + def __init__(self, port=None, uptime=None, topology=None, tasks=None,): + self.port = port + self.uptime = uptime + self.topology = topology + self.tasks = tasks + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.I32: + self.port = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.I32: + self.uptime = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.topology = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.LIST: + self.tasks = [] + (_etype90, _size87) = iprot.readListBegin() + for _i91 in xrange(_size87): + _elem92 = TaskComponent() + _elem92.read(iprot) + self.tasks.append(_elem92) + iprot.readListEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('WorkerSummary') + if self.port is not None: + oprot.writeFieldBegin('port', TType.I32, 1) + oprot.writeI32(self.port) + oprot.writeFieldEnd() + if self.uptime is not None: + oprot.writeFieldBegin('uptime', TType.I32, 2) + oprot.writeI32(self.uptime) + oprot.writeFieldEnd() + if self.topology is not None: + oprot.writeFieldBegin('topology', TType.STRING, 3) + oprot.writeString(self.topology.encode('utf-8')) + oprot.writeFieldEnd() + if self.tasks is not None: + oprot.writeFieldBegin('tasks', TType.LIST, 4) + oprot.writeListBegin(TType.STRUCT, len(self.tasks)) + for iter93 in self.tasks: + iter93.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.port is None: + raise TProtocol.TProtocolException(message='Required field port is unset!') + if self.uptime is None: + raise TProtocol.TProtocolException(message='Required field uptime is unset!') + if self.topology is None: + raise TProtocol.TProtocolException(message='Required field topology is unset!') + if self.tasks is None: + raise TProtocol.TProtocolException(message='Required field tasks is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.port) + value = (value * 31) ^ hash(self.uptime) + value = (value * 31) ^ hash(self.topology) + value = (value * 31) ^ hash(self.tasks) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class MetricWindow: + """ + Attributes: + - metricWindow + """ + + thrift_spec = ( + None, # 0 + (1, TType.MAP, 'metricWindow', (TType.I32,None,TType.DOUBLE,None), None, ), # 1 + ) + + def __init__(self, metricWindow=None,): + self.metricWindow = metricWindow + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.MAP: + self.metricWindow = {} + (_ktype95, _vtype96, _size94 ) = iprot.readMapBegin() + for _i98 in xrange(_size94): + _key99 = iprot.readI32(); + _val100 = iprot.readDouble(); + self.metricWindow[_key99] = _val100 + iprot.readMapEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('MetricWindow') + if self.metricWindow is not None: + oprot.writeFieldBegin('metricWindow', TType.MAP, 1) + oprot.writeMapBegin(TType.I32, TType.DOUBLE, len(self.metricWindow)) + for kiter101,viter102 in self.metricWindow.items(): + oprot.writeI32(kiter101) + oprot.writeDouble(viter102) + oprot.writeMapEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.metricWindow is None: + raise TProtocol.TProtocolException(message='Required field metricWindow is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.metricWindow) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class MetricInfo: + """ + Attributes: + - baseMetric + - inputMetric + - outputMetric + """ + + thrift_spec = ( + None, # 0 + (1, TType.MAP, 'baseMetric', (TType.STRING,None,TType.STRUCT,(MetricWindow, MetricWindow.thrift_spec)), None, ), # 1 + (2, TType.MAP, 'inputMetric', (TType.STRING,None,TType.MAP,(TType.STRING,None,TType.STRUCT,(MetricWindow, MetricWindow.thrift_spec))), None, ), # 2 + (3, TType.MAP, 'outputMetric', (TType.STRING,None,TType.MAP,(TType.STRING,None,TType.STRUCT,(MetricWindow, MetricWindow.thrift_spec))), None, ), # 3 + ) + + def __init__(self, baseMetric=None, inputMetric=None, outputMetric=None,): + self.baseMetric = baseMetric + self.inputMetric = inputMetric + self.outputMetric = outputMetric + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.MAP: + self.baseMetric = {} + (_ktype104, _vtype105, _size103 ) = iprot.readMapBegin() + for _i107 in xrange(_size103): + _key108 = iprot.readString().decode('utf-8') + _val109 = MetricWindow() + _val109.read(iprot) + self.baseMetric[_key108] = _val109 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.MAP: + self.inputMetric = {} + (_ktype111, _vtype112, _size110 ) = iprot.readMapBegin() + for _i114 in xrange(_size110): + _key115 = iprot.readString().decode('utf-8') + _val116 = {} + (_ktype118, _vtype119, _size117 ) = iprot.readMapBegin() + for _i121 in xrange(_size117): + _key122 = iprot.readString().decode('utf-8') + _val123 = MetricWindow() + _val123.read(iprot) + _val116[_key122] = _val123 + iprot.readMapEnd() + self.inputMetric[_key115] = _val116 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.MAP: + self.outputMetric = {} + (_ktype125, _vtype126, _size124 ) = iprot.readMapBegin() + for _i128 in xrange(_size124): + _key129 = iprot.readString().decode('utf-8') + _val130 = {} + (_ktype132, _vtype133, _size131 ) = iprot.readMapBegin() + for _i135 in xrange(_size131): + _key136 = iprot.readString().decode('utf-8') + _val137 = MetricWindow() + _val137.read(iprot) + _val130[_key136] = _val137 + iprot.readMapEnd() + self.outputMetric[_key129] = _val130 + iprot.readMapEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('MetricInfo') + if self.baseMetric is not None: + oprot.writeFieldBegin('baseMetric', TType.MAP, 1) + oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.baseMetric)) + for kiter138,viter139 in self.baseMetric.items(): + oprot.writeString(kiter138.encode('utf-8')) + viter139.write(oprot) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.inputMetric is not None: + oprot.writeFieldBegin('inputMetric', TType.MAP, 2) + oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.inputMetric)) + for kiter140,viter141 in self.inputMetric.items(): + oprot.writeString(kiter140.encode('utf-8')) + oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(viter141)) + for kiter142,viter143 in viter141.items(): + oprot.writeString(kiter142.encode('utf-8')) + viter143.write(oprot) + oprot.writeMapEnd() + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.outputMetric is not None: + oprot.writeFieldBegin('outputMetric', TType.MAP, 3) + oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.outputMetric)) + for kiter144,viter145 in self.outputMetric.items(): + oprot.writeString(kiter144.encode('utf-8')) + oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(viter145)) + for kiter146,viter147 in viter145.items(): + oprot.writeString(kiter146.encode('utf-8')) + viter147.write(oprot) + oprot.writeMapEnd() + oprot.writeMapEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.baseMetric is None: + raise TProtocol.TProtocolException(message='Required field baseMetric is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.baseMetric) + value = (value * 31) ^ hash(self.inputMetric) + value = (value * 31) ^ hash(self.outputMetric) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class TopologyMetric: + """ + Attributes: + - topologyMetric + - componentMetric + - workerMetric + - taskMetric + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'topologyMetric', (MetricInfo, MetricInfo.thrift_spec), None, ), # 1 + (2, TType.MAP, 'componentMetric', (TType.STRING,None,TType.STRUCT,(MetricInfo, MetricInfo.thrift_spec)), None, ), # 2 + (3, TType.MAP, 'workerMetric', (TType.STRING,None,TType.STRUCT,(MetricInfo, MetricInfo.thrift_spec)), None, ), # 3 + (4, TType.MAP, 'taskMetric', (TType.I32,None,TType.STRUCT,(MetricInfo, MetricInfo.thrift_spec)), None, ), # 4 + ) + + def __init__(self, topologyMetric=None, componentMetric=None, workerMetric=None, taskMetric=None,): + self.topologyMetric = topologyMetric + self.componentMetric = componentMetric + self.workerMetric = workerMetric + self.taskMetric = taskMetric + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.topologyMetric = MetricInfo() + self.topologyMetric.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.MAP: + self.componentMetric = {} + (_ktype149, _vtype150, _size148 ) = iprot.readMapBegin() + for _i152 in xrange(_size148): + _key153 = iprot.readString().decode('utf-8') + _val154 = MetricInfo() + _val154.read(iprot) + self.componentMetric[_key153] = _val154 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.MAP: + self.workerMetric = {} + (_ktype156, _vtype157, _size155 ) = iprot.readMapBegin() + for _i159 in xrange(_size155): + _key160 = iprot.readString().decode('utf-8') + _val161 = MetricInfo() + _val161.read(iprot) + self.workerMetric[_key160] = _val161 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.MAP: + self.taskMetric = {} + (_ktype163, _vtype164, _size162 ) = iprot.readMapBegin() + for _i166 in xrange(_size162): + _key167 = iprot.readI32(); + _val168 = MetricInfo() + _val168.read(iprot) + self.taskMetric[_key167] = _val168 + iprot.readMapEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('TopologyMetric') + if self.topologyMetric is not None: + oprot.writeFieldBegin('topologyMetric', TType.STRUCT, 1) + self.topologyMetric.write(oprot) + oprot.writeFieldEnd() + if self.componentMetric is not None: + oprot.writeFieldBegin('componentMetric', TType.MAP, 2) + oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.componentMetric)) + for kiter169,viter170 in self.componentMetric.items(): + oprot.writeString(kiter169.encode('utf-8')) + viter170.write(oprot) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.workerMetric is not None: + oprot.writeFieldBegin('workerMetric', TType.MAP, 3) + oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.workerMetric)) + for kiter171,viter172 in self.workerMetric.items(): + oprot.writeString(kiter171.encode('utf-8')) + viter172.write(oprot) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.taskMetric is not None: + oprot.writeFieldBegin('taskMetric', TType.MAP, 4) + oprot.writeMapBegin(TType.I32, TType.STRUCT, len(self.taskMetric)) + for kiter173,viter174 in self.taskMetric.items(): + oprot.writeI32(kiter173) + viter174.write(oprot) + oprot.writeMapEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.topologyMetric is None: + raise TProtocol.TProtocolException(message='Required field topologyMetric is unset!') + if self.componentMetric is None: + raise TProtocol.TProtocolException(message='Required field componentMetric is unset!') + if self.workerMetric is None: + raise TProtocol.TProtocolException(message='Required field workerMetric is unset!') + if self.taskMetric is None: + raise TProtocol.TProtocolException(message='Required field taskMetric is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.topologyMetric) + value = (value * 31) ^ hash(self.componentMetric) + value = (value * 31) ^ hash(self.workerMetric) + value = (value * 31) ^ hash(self.taskMetric) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class SupervisorWorkers: + """ + Attributes: + - supervisor + - workers + - workerMetric + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'supervisor', (SupervisorSummary, SupervisorSummary.thrift_spec), None, ), # 1 + (2, TType.LIST, 'workers', (TType.STRUCT,(WorkerSummary, WorkerSummary.thrift_spec)), None, ), # 2 + (3, TType.MAP, 'workerMetric', (TType.STRING,None,TType.STRUCT,(MetricInfo, MetricInfo.thrift_spec)), None, ), # 3 + ) + + def __init__(self, supervisor=None, workers=None, workerMetric=None,): + self.supervisor = supervisor + self.workers = workers + self.workerMetric = workerMetric + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.supervisor = SupervisorSummary() + self.supervisor.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.LIST: + self.workers = [] + (_etype178, _size175) = iprot.readListBegin() + for _i179 in xrange(_size175): + _elem180 = WorkerSummary() + _elem180.read(iprot) + self.workers.append(_elem180) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.MAP: + self.workerMetric = {} + (_ktype182, _vtype183, _size181 ) = iprot.readMapBegin() + for _i185 in xrange(_size181): + _key186 = iprot.readString().decode('utf-8') + _val187 = MetricInfo() + _val187.read(iprot) + self.workerMetric[_key186] = _val187 + iprot.readMapEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('SupervisorWorkers') + if self.supervisor is not None: + oprot.writeFieldBegin('supervisor', TType.STRUCT, 1) + self.supervisor.write(oprot) + oprot.writeFieldEnd() + if self.workers is not None: + oprot.writeFieldBegin('workers', TType.LIST, 2) + oprot.writeListBegin(TType.STRUCT, len(self.workers)) + for iter188 in self.workers: + iter188.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.workerMetric is not None: + oprot.writeFieldBegin('workerMetric', TType.MAP, 3) + oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.workerMetric)) + for kiter189,viter190 in self.workerMetric.items(): + oprot.writeString(kiter189.encode('utf-8')) + viter190.write(oprot) + oprot.writeMapEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.supervisor is None: + raise TProtocol.TProtocolException(message='Required field supervisor is unset!') + if self.workers is None: + raise TProtocol.TProtocolException(message='Required field workers is unset!') + if self.workerMetric is None: + raise TProtocol.TProtocolException(message='Required field workerMetric is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.supervisor) + value = (value * 31) ^ hash(self.workers) + value = (value * 31) ^ hash(self.workerMetric) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class ErrorInfo: + """ + Attributes: + - error + - error_time_secs + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'error', None, None, ), # 1 + (2, TType.I32, 'error_time_secs', None, None, ), # 2 + ) + + def __init__(self, error=None, error_time_secs=None,): + self.error = error + self.error_time_secs = error_time_secs + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.error = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.I32: + self.error_time_secs = iprot.readI32(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('ErrorInfo') + if self.error is not None: + oprot.writeFieldBegin('error', TType.STRING, 1) + oprot.writeString(self.error.encode('utf-8')) + oprot.writeFieldEnd() + if self.error_time_secs is not None: + oprot.writeFieldBegin('error_time_secs', TType.I32, 2) + oprot.writeI32(self.error_time_secs) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.error is None: + raise TProtocol.TProtocolException(message='Required field error is unset!') + if self.error_time_secs is None: + raise TProtocol.TProtocolException(message='Required field error_time_secs is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.error) + value = (value * 31) ^ hash(self.error_time_secs) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class ComponentSummary: + """ + Attributes: + - name + - parallel + - type + - task_ids + - errors + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'name', None, None, ), # 1 + (2, TType.I32, 'parallel', None, None, ), # 2 + (3, TType.STRING, 'type', None, None, ), # 3 + (4, TType.LIST, 'task_ids', (TType.I32,None), None, ), # 4 + (5, TType.LIST, 'errors', (TType.STRUCT,(ErrorInfo, ErrorInfo.thrift_spec)), None, ), # 5 + ) + + def __init__(self, name=None, parallel=None, type=None, task_ids=None, errors=None,): + self.name = name + self.parallel = parallel + self.type = type + self.task_ids = task_ids + self.errors = errors + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.name = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.I32: + self.parallel = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.type = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.LIST: + self.task_ids = [] + (_etype194, _size191) = iprot.readListBegin() + for _i195 in xrange(_size191): + _elem196 = iprot.readI32(); + self.task_ids.append(_elem196) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.LIST: + self.errors = [] + (_etype200, _size197) = iprot.readListBegin() + for _i201 in xrange(_size197): + _elem202 = ErrorInfo() + _elem202.read(iprot) + self.errors.append(_elem202) + iprot.readListEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('ComponentSummary') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name.encode('utf-8')) + oprot.writeFieldEnd() + if self.parallel is not None: + oprot.writeFieldBegin('parallel', TType.I32, 2) + oprot.writeI32(self.parallel) + oprot.writeFieldEnd() + if self.type is not None: + oprot.writeFieldBegin('type', TType.STRING, 3) + oprot.writeString(self.type.encode('utf-8')) + oprot.writeFieldEnd() + if self.task_ids is not None: + oprot.writeFieldBegin('task_ids', TType.LIST, 4) + oprot.writeListBegin(TType.I32, len(self.task_ids)) + for iter203 in self.task_ids: + oprot.writeI32(iter203) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.errors is not None: + oprot.writeFieldBegin('errors', TType.LIST, 5) + oprot.writeListBegin(TType.STRUCT, len(self.errors)) + for iter204 in self.errors: + iter204.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.name is None: + raise TProtocol.TProtocolException(message='Required field name is unset!') + if self.parallel is None: + raise TProtocol.TProtocolException(message='Required field parallel is unset!') + if self.type is None: + raise TProtocol.TProtocolException(message='Required field type is unset!') + if self.task_ids is None: + raise TProtocol.TProtocolException(message='Required field task_ids is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.name) + value = (value * 31) ^ hash(self.parallel) + value = (value * 31) ^ hash(self.type) + value = (value * 31) ^ hash(self.task_ids) + value = (value * 31) ^ hash(self.errors) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class TaskSummary: + """ + Attributes: + - task_id + - uptime + - status + - host + - port + - errors + """ + + thrift_spec = ( + None, # 0 + (1, TType.I32, 'task_id', None, None, ), # 1 + (2, TType.I32, 'uptime', None, None, ), # 2 + (3, TType.STRING, 'status', None, None, ), # 3 + (4, TType.STRING, 'host', None, None, ), # 4 + (5, TType.I32, 'port', None, None, ), # 5 + (6, TType.LIST, 'errors', (TType.STRUCT,(ErrorInfo, ErrorInfo.thrift_spec)), None, ), # 6 + ) + + def __init__(self, task_id=None, uptime=None, status=None, host=None, port=None, errors=None,): + self.task_id = task_id + self.uptime = uptime + self.status = status + self.host = host + self.port = port + self.errors = errors + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.I32: + self.task_id = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.I32: + self.uptime = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.status = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRING: + self.host = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.I32: + self.port = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.LIST: + self.errors = [] + (_etype208, _size205) = iprot.readListBegin() + for _i209 in xrange(_size205): + _elem210 = ErrorInfo() + _elem210.read(iprot) + self.errors.append(_elem210) + iprot.readListEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('TaskSummary') + if self.task_id is not None: + oprot.writeFieldBegin('task_id', TType.I32, 1) + oprot.writeI32(self.task_id) + oprot.writeFieldEnd() + if self.uptime is not None: + oprot.writeFieldBegin('uptime', TType.I32, 2) + oprot.writeI32(self.uptime) + oprot.writeFieldEnd() + if self.status is not None: + oprot.writeFieldBegin('status', TType.STRING, 3) + oprot.writeString(self.status.encode('utf-8')) + oprot.writeFieldEnd() + if self.host is not None: + oprot.writeFieldBegin('host', TType.STRING, 4) + oprot.writeString(self.host.encode('utf-8')) + oprot.writeFieldEnd() + if self.port is not None: + oprot.writeFieldBegin('port', TType.I32, 5) + oprot.writeI32(self.port) + oprot.writeFieldEnd() + if self.errors is not None: + oprot.writeFieldBegin('errors', TType.LIST, 6) + oprot.writeListBegin(TType.STRUCT, len(self.errors)) + for iter211 in self.errors: + iter211.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.task_id is None: + raise TProtocol.TProtocolException(message='Required field task_id is unset!') + if self.uptime is None: + raise TProtocol.TProtocolException(message='Required field uptime is unset!') + if self.status is None: + raise TProtocol.TProtocolException(message='Required field status is unset!') + if self.host is None: + raise TProtocol.TProtocolException(message='Required field host is unset!') + if self.port is None: + raise TProtocol.TProtocolException(message='Required field port is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.task_id) + value = (value * 31) ^ hash(self.uptime) + value = (value * 31) ^ hash(self.status) + value = (value * 31) ^ hash(self.host) + value = (value * 31) ^ hash(self.port) + value = (value * 31) ^ hash(self.errors) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class TopologyInfo: + """ + Attributes: + - topology + - components + - tasks + - metrics + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRUCT, 'topology', (TopologySummary, TopologySummary.thrift_spec), None, ), # 1 + (2, TType.LIST, 'components', (TType.STRUCT,(ComponentSummary, ComponentSummary.thrift_spec)), None, ), # 2 + (3, TType.LIST, 'tasks', (TType.STRUCT,(TaskSummary, TaskSummary.thrift_spec)), None, ), # 3 + (4, TType.STRUCT, 'metrics', (TopologyMetric, TopologyMetric.thrift_spec), None, ), # 4 + ) + + def __init__(self, topology=None, components=None, tasks=None, metrics=None,): + self.topology = topology + self.components = components + self.tasks = tasks + self.metrics = metrics + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRUCT: + self.topology = TopologySummary() + self.topology.read(iprot) + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.LIST: + self.components = [] + (_etype215, _size212) = iprot.readListBegin() + for _i216 in xrange(_size212): + _elem217 = ComponentSummary() + _elem217.read(iprot) + self.components.append(_elem217) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.LIST: + self.tasks = [] + (_etype221, _size218) = iprot.readListBegin() + for _i222 in xrange(_size218): + _elem223 = TaskSummary() + _elem223.read(iprot) + self.tasks.append(_elem223) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.metrics = TopologyMetric() + self.metrics.read(iprot) + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('TopologyInfo') + if self.topology is not None: + oprot.writeFieldBegin('topology', TType.STRUCT, 1) + self.topology.write(oprot) + oprot.writeFieldEnd() + if self.components is not None: + oprot.writeFieldBegin('components', TType.LIST, 2) + oprot.writeListBegin(TType.STRUCT, len(self.components)) + for iter224 in self.components: + iter224.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.tasks is not None: + oprot.writeFieldBegin('tasks', TType.LIST, 3) + oprot.writeListBegin(TType.STRUCT, len(self.tasks)) + for iter225 in self.tasks: + iter225.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.metrics is not None: + oprot.writeFieldBegin('metrics', TType.STRUCT, 4) + self.metrics.write(oprot) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.topology is None: + raise TProtocol.TProtocolException(message='Required field topology is unset!') + if self.components is None: + raise TProtocol.TProtocolException(message='Required field components is unset!') + if self.tasks is None: + raise TProtocol.TProtocolException(message='Required field tasks is unset!') + if self.metrics is None: + raise TProtocol.TProtocolException(message='Required field metrics is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.topology) + value = (value * 31) ^ hash(self.components) + value = (value * 31) ^ hash(self.tasks) + value = (value * 31) ^ hash(self.metrics) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class TopologyAssignException(TException): + """ + Attributes: + - msg + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'msg', None, None, ), # 1 + ) + + def __init__(self, msg=None,): + self.msg = msg + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.msg = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('TopologyAssignException') + if self.msg is not None: + oprot.writeFieldBegin('msg', TType.STRING, 1) + oprot.writeString(self.msg.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.msg is None: + raise TProtocol.TProtocolException(message='Required field msg is unset!') + return + + + def __str__(self): + return repr(self) + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.msg) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class AlreadyAliveException(TException): + """ + Attributes: + - msg + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'msg', None, None, ), # 1 + ) + + def __init__(self, msg=None,): + self.msg = msg + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.msg = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('AlreadyAliveException') + if self.msg is not None: + oprot.writeFieldBegin('msg', TType.STRING, 1) + oprot.writeString(self.msg.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.msg is None: + raise TProtocol.TProtocolException(message='Required field msg is unset!') + return + + + def __str__(self): + return repr(self) + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.msg) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class NotAliveException(TException): + """ + Attributes: + - msg + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'msg', None, None, ), # 1 + ) + + def __init__(self, msg=None,): + self.msg = msg + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.msg = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('NotAliveException') + if self.msg is not None: + oprot.writeFieldBegin('msg', TType.STRING, 1) + oprot.writeString(self.msg.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.msg is None: + raise TProtocol.TProtocolException(message='Required field msg is unset!') + return + + + def __str__(self): + return repr(self) + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.msg) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class InvalidTopologyException(TException): + """ + Attributes: + - msg + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'msg', None, None, ), # 1 + ) + + def __init__(self, msg=None,): + self.msg = msg + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.msg = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('InvalidTopologyException') + if self.msg is not None: + oprot.writeFieldBegin('msg', TType.STRING, 1) + oprot.writeString(self.msg.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.msg is None: + raise TProtocol.TProtocolException(message='Required field msg is unset!') + return + + + def __str__(self): + return repr(self) + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.msg) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class AuthorizationException(TException): + """ + Attributes: + - msg + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'msg', None, None, ), # 1 + ) + + def __init__(self, msg=None,): + self.msg = msg + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.msg = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('AuthorizationException') + if self.msg is not None: + oprot.writeFieldBegin('msg', TType.STRING, 1) + oprot.writeString(self.msg.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.msg is None: + raise TProtocol.TProtocolException(message='Required field msg is unset!') + return + + + def __str__(self): + return repr(self) + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.msg) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class KillOptions: + """ + Attributes: + - wait_secs + """ + + thrift_spec = ( + None, # 0 + (1, TType.I32, 'wait_secs', None, None, ), # 1 + ) + + def __init__(self, wait_secs=None,): + self.wait_secs = wait_secs + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.I32: + self.wait_secs = iprot.readI32(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('KillOptions') + if self.wait_secs is not None: + oprot.writeFieldBegin('wait_secs', TType.I32, 1) + oprot.writeI32(self.wait_secs) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.wait_secs) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class RebalanceOptions: + """ + Attributes: + - wait_secs + - reassign + - conf + """ + + thrift_spec = ( + None, # 0 + (1, TType.I32, 'wait_secs', None, None, ), # 1 + (2, TType.BOOL, 'reassign', None, None, ), # 2 + (3, TType.STRING, 'conf', None, None, ), # 3 + ) + + def __init__(self, wait_secs=None, reassign=None, conf=None,): + self.wait_secs = wait_secs + self.reassign = reassign + self.conf = conf + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.I32: + self.wait_secs = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.BOOL: + self.reassign = iprot.readBool(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.STRING: + self.conf = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('RebalanceOptions') + if self.wait_secs is not None: + oprot.writeFieldBegin('wait_secs', TType.I32, 1) + oprot.writeI32(self.wait_secs) + oprot.writeFieldEnd() + if self.reassign is not None: + oprot.writeFieldBegin('reassign', TType.BOOL, 2) + oprot.writeBool(self.reassign) + oprot.writeFieldEnd() + if self.conf is not None: + oprot.writeFieldBegin('conf', TType.STRING, 3) + oprot.writeString(self.conf.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.wait_secs) + value = (value * 31) ^ hash(self.reassign) + value = (value * 31) ^ hash(self.conf) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class SubmitOptions: + """ + Attributes: + - initial_status + """ + + thrift_spec = ( + None, # 0 + (1, TType.I32, 'initial_status', None, None, ), # 1 + ) + + def __init__(self, initial_status=None,): + self.initial_status = initial_status + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.I32: + self.initial_status = iprot.readI32(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('SubmitOptions') + if self.initial_status is not None: + oprot.writeFieldBegin('initial_status', TType.I32, 1) + oprot.writeI32(self.initial_status) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.initial_status is None: + raise TProtocol.TProtocolException(message='Required field initial_status is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.initial_status) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class MonitorOptions: + """ + Attributes: + - isEnable + """ + + thrift_spec = ( + None, # 0 + (1, TType.BOOL, 'isEnable', None, None, ), # 1 + ) + + def __init__(self, isEnable=None,): + self.isEnable = isEnable + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.BOOL: + self.isEnable = iprot.readBool(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('MonitorOptions') + if self.isEnable is not None: + oprot.writeFieldBegin('isEnable', TType.BOOL, 1) + oprot.writeBool(self.isEnable) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.isEnable) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class Credentials: + """ + Attributes: + - creds + """ + + thrift_spec = ( + None, # 0 + (1, TType.MAP, 'creds', (TType.STRING,None,TType.STRING,None), None, ), # 1 + ) + + def __init__(self, creds=None,): + self.creds = creds + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.MAP: + self.creds = {} + (_ktype227, _vtype228, _size226 ) = iprot.readMapBegin() + for _i230 in xrange(_size226): + _key231 = iprot.readString().decode('utf-8') + _val232 = iprot.readString().decode('utf-8') + self.creds[_key231] = _val232 + iprot.readMapEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('Credentials') + if self.creds is not None: + oprot.writeFieldBegin('creds', TType.MAP, 1) + oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.creds)) + for kiter233,viter234 in self.creds.items(): + oprot.writeString(kiter233.encode('utf-8')) + oprot.writeString(viter234.encode('utf-8')) + oprot.writeMapEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.creds is None: + raise TProtocol.TProtocolException(message='Required field creds is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.creds) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class ThriftSerializedObject: + """ + Attributes: + - name + - bits + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'name', None, None, ), # 1 + (2, TType.STRING, 'bits', None, None, ), # 2 + ) + + def __init__(self, name=None, bits=None,): + self.name = name + self.bits = bits + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.name = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.bits = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('ThriftSerializedObject') + if self.name is not None: + oprot.writeFieldBegin('name', TType.STRING, 1) + oprot.writeString(self.name.encode('utf-8')) + oprot.writeFieldEnd() + if self.bits is not None: + oprot.writeFieldBegin('bits', TType.STRING, 2) + oprot.writeString(self.bits) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.name is None: + raise TProtocol.TProtocolException(message='Required field name is unset!') + if self.bits is None: + raise TProtocol.TProtocolException(message='Required field bits is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.name) + value = (value * 31) ^ hash(self.bits) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class LocalStateData: + """ + Attributes: + - serialized_parts + """ + + thrift_spec = ( + None, # 0 + (1, TType.MAP, 'serialized_parts', (TType.STRING,None,TType.STRUCT,(ThriftSerializedObject, ThriftSerializedObject.thrift_spec)), None, ), # 1 + ) + + def __init__(self, serialized_parts=None,): + self.serialized_parts = serialized_parts + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.MAP: + self.serialized_parts = {} + (_ktype236, _vtype237, _size235 ) = iprot.readMapBegin() + for _i239 in xrange(_size235): + _key240 = iprot.readString().decode('utf-8') + _val241 = ThriftSerializedObject() + _val241.read(iprot) + self.serialized_parts[_key240] = _val241 + iprot.readMapEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('LocalStateData') + if self.serialized_parts is not None: + oprot.writeFieldBegin('serialized_parts', TType.MAP, 1) + oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.serialized_parts)) + for kiter242,viter243 in self.serialized_parts.items(): + oprot.writeString(kiter242.encode('utf-8')) + viter243.write(oprot) + oprot.writeMapEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.serialized_parts is None: + raise TProtocol.TProtocolException(message='Required field serialized_parts is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.serialized_parts) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class NettyMetric: + """ + Attributes: + - connections + - connectionNum + """ + + thrift_spec = ( + None, # 0 + (1, TType.MAP, 'connections', (TType.STRING,None,TType.STRUCT,(MetricInfo, MetricInfo.thrift_spec)), None, ), # 1 + (2, TType.I32, 'connectionNum', None, None, ), # 2 + ) + + def __init__(self, connections=None, connectionNum=None,): + self.connections = connections + self.connectionNum = connectionNum + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.MAP: + self.connections = {} + (_ktype245, _vtype246, _size244 ) = iprot.readMapBegin() + for _i248 in xrange(_size244): + _key249 = iprot.readString().decode('utf-8') + _val250 = MetricInfo() + _val250.read(iprot) + self.connections[_key249] = _val250 + iprot.readMapEnd() + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.I32: + self.connectionNum = iprot.readI32(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('NettyMetric') + if self.connections is not None: + oprot.writeFieldBegin('connections', TType.MAP, 1) + oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.connections)) + for kiter251,viter252 in self.connections.items(): + oprot.writeString(kiter251.encode('utf-8')) + viter252.write(oprot) + oprot.writeMapEnd() + oprot.writeFieldEnd() + if self.connectionNum is not None: + oprot.writeFieldBegin('connectionNum', TType.I32, 2) + oprot.writeI32(self.connectionNum) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.connections is None: + raise TProtocol.TProtocolException(message='Required field connections is unset!') + if self.connectionNum is None: + raise TProtocol.TProtocolException(message='Required field connectionNum is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.connections) + value = (value * 31) ^ hash(self.connectionNum) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class WorkerUploadMetrics: + """ + Attributes: + - topology_id + - supervisor_id + - port + - workerMetric + - nettyMetric + - taskMetric + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'topology_id', None, None, ), # 1 + (2, TType.STRING, 'supervisor_id', None, None, ), # 2 + (3, TType.I32, 'port', None, None, ), # 3 + (4, TType.STRUCT, 'workerMetric', (MetricInfo, MetricInfo.thrift_spec), None, ), # 4 + (5, TType.STRUCT, 'nettyMetric', (NettyMetric, NettyMetric.thrift_spec), None, ), # 5 + (6, TType.MAP, 'taskMetric', (TType.I32,None,TType.STRUCT,(MetricInfo, MetricInfo.thrift_spec)), None, ), # 6 + ) + + def __init__(self, topology_id=None, supervisor_id=None, port=None, workerMetric=None, nettyMetric=None, taskMetric=None,): + self.topology_id = topology_id + self.supervisor_id = supervisor_id + self.port = port + self.workerMetric = workerMetric + self.nettyMetric = nettyMetric + self.taskMetric = taskMetric + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.topology_id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.supervisor_id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.I32: + self.port = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.STRUCT: + self.workerMetric = MetricInfo() + self.workerMetric.read(iprot) + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRUCT: + self.nettyMetric = NettyMetric() + self.nettyMetric.read(iprot) + else: + iprot.skip(ftype) + elif fid == 6: + if ftype == TType.MAP: + self.taskMetric = {} + (_ktype254, _vtype255, _size253 ) = iprot.readMapBegin() + for _i257 in xrange(_size253): + _key258 = iprot.readI32(); + _val259 = MetricInfo() + _val259.read(iprot) + self.taskMetric[_key258] = _val259 + iprot.readMapEnd() + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('WorkerUploadMetrics') + if self.topology_id is not None: + oprot.writeFieldBegin('topology_id', TType.STRING, 1) + oprot.writeString(self.topology_id.encode('utf-8')) + oprot.writeFieldEnd() + if self.supervisor_id is not None: + oprot.writeFieldBegin('supervisor_id', TType.STRING, 2) + oprot.writeString(self.supervisor_id.encode('utf-8')) + oprot.writeFieldEnd() + if self.port is not None: + oprot.writeFieldBegin('port', TType.I32, 3) + oprot.writeI32(self.port) + oprot.writeFieldEnd() + if self.workerMetric is not None: + oprot.writeFieldBegin('workerMetric', TType.STRUCT, 4) + self.workerMetric.write(oprot) + oprot.writeFieldEnd() + if self.nettyMetric is not None: + oprot.writeFieldBegin('nettyMetric', TType.STRUCT, 5) + self.nettyMetric.write(oprot) + oprot.writeFieldEnd() + if self.taskMetric is not None: + oprot.writeFieldBegin('taskMetric', TType.MAP, 6) + oprot.writeMapBegin(TType.I32, TType.STRUCT, len(self.taskMetric)) + for kiter260,viter261 in self.taskMetric.items(): + oprot.writeI32(kiter260) + viter261.write(oprot) + oprot.writeMapEnd() + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.topology_id is None: + raise TProtocol.TProtocolException(message='Required field topology_id is unset!') + if self.supervisor_id is None: + raise TProtocol.TProtocolException(message='Required field supervisor_id is unset!') + if self.port is None: + raise TProtocol.TProtocolException(message='Required field port is unset!') + if self.workerMetric is None: + raise TProtocol.TProtocolException(message='Required field workerMetric is unset!') + if self.nettyMetric is None: + raise TProtocol.TProtocolException(message='Required field nettyMetric is unset!') + if self.taskMetric is None: + raise TProtocol.TProtocolException(message='Required field taskMetric is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.topology_id) + value = (value * 31) ^ hash(self.supervisor_id) + value = (value * 31) ^ hash(self.port) + value = (value * 31) ^ hash(self.workerMetric) + value = (value * 31) ^ hash(self.nettyMetric) + value = (value * 31) ^ hash(self.taskMetric) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class DRPCRequest: + """ + Attributes: + - func_args + - request_id + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'func_args', None, None, ), # 1 + (2, TType.STRING, 'request_id', None, None, ), # 2 + ) + + def __init__(self, func_args=None, request_id=None,): + self.func_args = func_args + self.request_id = request_id + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.func_args = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.request_id = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('DRPCRequest') + if self.func_args is not None: + oprot.writeFieldBegin('func_args', TType.STRING, 1) + oprot.writeString(self.func_args.encode('utf-8')) + oprot.writeFieldEnd() + if self.request_id is not None: + oprot.writeFieldBegin('request_id', TType.STRING, 2) + oprot.writeString(self.request_id.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.func_args is None: + raise TProtocol.TProtocolException(message='Required field func_args is unset!') + if self.request_id is None: + raise TProtocol.TProtocolException(message='Required field request_id is unset!') + return + + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.func_args) + value = (value * 31) ^ hash(self.request_id) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + +class DRPCExecutionException(TException): + """ + Attributes: + - msg + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'msg', None, None, ), # 1 + ) + + def __init__(self, msg=None,): + self.msg = msg + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.msg = iprot.readString().decode('utf-8') + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('DRPCExecutionException') + if self.msg is not None: + oprot.writeFieldBegin('msg', TType.STRING, 1) + oprot.writeString(self.msg.encode('utf-8')) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.msg is None: + raise TProtocol.TProtocolException(message='Required field msg is unset!') + return + + + def __str__(self): + return repr(self) + + def __hash__(self): + value = 17 + value = (value * 31) ^ hash(self.msg) + return value + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) diff --git a/jstorm-core/src/main/resources/defaults.yaml b/jstorm-core/src/main/resources/defaults.yaml new file mode 100644 index 000000000..3eecf97f6 --- /dev/null +++ b/jstorm-core/src/main/resources/defaults.yaml @@ -0,0 +1,342 @@ +# 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. + + +########### These all have default values as shown +########### Additional configuration goes into storm.yaml + +java.library.path: "/usr/local/lib:/opt/local/lib:/usr/lib" + +### storm.* configs are general configurations +# the local dir is where jars are kept +storm.local.dir: "jstorm-local" +storm.zookeeper.servers: + - "localhost" +storm.zookeeper.port: 2181 +storm.zookeeper.root: "/jstorm" +storm.zookeeper.session.timeout: 20000 +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.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: 10485760 +nimbus.childopts: " -Xms4g -Xmx4g -Xmn1536m -XX:PermSize=256m -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: 240 +nimbus.supervisor.timeout.secs: 180 +nimbus.monitor.freq.secs: 10 +nimbus.cleanup.inbox.freq.secs: 600 +nimbus.inbox.jar.expiration.secs: 3600 +nimbus.task.launch.secs: 240 +nimbus.reassign: true +nimbus.file.copy.expiration.secs: 120 +nimbus.topology.validator: "backtype.storm.nimbus.DefaultTopologyValidator" +nimbus.classpath: "" +nimbus.use.ip: true +nimbus.credential.renewers.freq.secs: 600 +## Two type cache +## "com.alibaba.jstorm.cache.TimeoutMemCache" is suitable for small cluster +## "com.alibaba.jstorm.cache.TimeoutMemCache" can only run under linux/mac, it is suitable for huge cluster +## if it is null, it will detected by environment +nimbus.cache.class: null +## if this is true, nimbus db cache will be reset when start nimbus +nimbus.cache.reset: true +cache.timeout.list: null + +### ui.* configs are for the master +ui.port: 8080 +ui.childopts: " -Xms1g -Xmx1g -Xmn256m -XX:PermSize=96m -XX:+UseConcMarkSweepGC -XX:+UseCMSInitiatingOccupancyOnly -XX:CMSInitiatingOccupancyFraction=70 -XX:CMSFullGCsBeforeCompaction=5 -XX:+HeapDumpOnOutOfMemoryError -XX:+UseCMSCompactAtFullCollection -XX:CMSMaxAbortablePrecleanTime=5000 " +ui.actions.enabled: true +ui.filter: null +ui.filter.params: null +ui.users: null +ui.header.buffer.bytes: 4096 +ui.http.creds.plugin: backtype.storm.security.auth.DefaultHttpCredentialsPlugin + +logviewer.port: 8000 +logviewer.childopts: "-Xmx128m" +logviewer.cleanup.age.mins: 10080 +logviewer.appender.name: "A1" + +logs.users: null +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 +transactional.zookeeper.port: null +# +# +# ##### These may optionally be filled in: +# +## Map of tokens to a serialization class. tokens less than 32 are reserved by storm. +## Tokens are written on the wire to identify the field. +# topology.serializations: +# - "org.mycompany.MyObjectSerialization" +# - "org.mycompany.MyOtherObjectSerialization" +## Locations of the drpc servers +drpc.servers: + - "localhost" + +### 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 + +# 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 +supervisor.worker.timeout.secs: 120 +#how many seconds to sleep for before shutting down threads on worker +supervisor.worker.shutdown.sleep.secs: 1 +#how frequently the supervisor checks on the status of the processes it's monitoring and restarts if necessary +supervisor.monitor.frequency.secs: 10 +#how frequently the supervisor heartbeats to the cluster state (for nimbus) +supervisor.heartbeat.frequency.secs: 60 +supervisor.enable: true +#if set null, it will be get by system +supervisor.hostname: null +# use ip +supervisor.use.ip: true +supervisor.supervisors: [] +supervisor.supervisors.commands: [] + +### 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=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.metric.report.frequency.secs: 60 +worker.classpath: "" +worker.redirect.output: true +# if worker.redirect.output.file is null, then it will be $LOG.out +# please use absolute path +worker.redirect.output.file: null + + +# control how many worker receiver threads we need per worker +topology.worker.receiver.thread.count: 1 + +# when supervisor is shutdown, automatically shutdown worker +worker.stop.without.supervisor: false +worker.memory.size: 2147483648 + +task.heartbeat.frequency.secs: 10 +task.refresh.poll.secs: 10 +# how long task do cleanup +task.cleanup.timeout.sec: 10 +task.credentials.poll.secs: 30 + +zmq.threads: 1 +zmq.linger.millis: 5000 +zmq.hwm: 0 + +# Netty thread num: 0 means no limit +storm.messaging.netty.server_worker_threads: 1 +storm.messaging.netty.client_worker_threads: 1 +storm.messaging.netty.buffer_size: 5242880 #5MB buffer +storm.messaging.netty.max_retries: 30 +storm.messaging.netty.max_wait_ms: 1000 +storm.messaging.netty.min_wait_ms: 100 +storm.messaging.netty.disruptor: true +# If async and batch is used in netty transfer, netty will batch message +storm.messaging.netty.transfer.async.batch: true +# If the Netty messaging layer is busy(netty internal buffer not writable), the Netty client will try to batch message as more as possible up to the size of storm.messaging.netty.transfer.batch.size bytes, otherwise it will try to flush message as soon as possible to reduce latency. +storm.messaging.netty.transfer.batch.size: 262144 +# Sets the backlog value to specify when the channel binds to a local address +storm.messaging.netty.socket.backlog: 500 +# We check with this interval that whether the Netty channel is writable and try to write pending messages if it is. +storm.messaging.netty.flush.check.interval.ms: 10 +# when netty connection is broken, +# when buffer size is more than storm.messaging.netty.buffer.threshold +# it will slow down the netty sending speed +storm.messaging.netty.buffer.threshold: 8388608 +storm.messaging.netty.max.pending: 4 +## send message with sync or async mode +storm.messaging.netty.sync.mode: false +## when netty is in sync mode and client channel is unavailable, +## it will block sending until channel is ready +storm.messaging.netty.async.block: true + +# By default, the Netty SASL authentication is set to false. Users can override and set it true for a specific topology. +storm.messaging.netty.authentication: false + +# default number of seconds group mapping service will cache user group +storm.group.mapping.service.cache.duration.secs: 120 + +### topology.* configs are for specific executing storms +topology.enable.message.timeouts: true +topology.debug: false +topology.optimize: true +topology.workers: 1 +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 +# to change the component's parallelism when executing restart command +#topology.spout.parallelism: +# { "spoutName" : Num} +#topology.bolt.parallelism: +# { "BoltName_1" : Num, "BoltName_2" : Num } +topology.max.spout.pending: null +topology.state.synchronization.timeout.secs: 60 +topology.stats.sample.rate: 0.05 +topology.builtin.metrics.bucket.size.secs: 60 +topology.fall.back.on.java.serialization: true +topology.worker.childopts: null +topology.executor.receive.buffer.size: 1024 #batched +topology.executor.send.buffer.size: 1024 #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: 2048 # batched +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.TimeoutBlockingWaitStrategy" +topology.disruptor.wait.timeout: 10 # ms +topology.spout.wait.strategy: "backtype.storm.spout.SleepSpoutWaitStrategy" +topology.sleep.spout.wait.strategy.time.ms: 1 +topology.error.throttle.interval.secs: 10 +topology.max.error.report.per.interval: 5 +topology.kryo.factory: "backtype.storm.serialization.DefaultKryoFactory" +topology.tuple.serializer: "backtype.storm.serialization.types.ListDelegateSerializer" +topology.trident.batch.emit.interval.millis: 500 + +# jstorm metrics monitor configuration +topology.performance.metrics: true +topology.alimonitor.metrics.post: false +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 + +topology.testing.always.try.serialize: false +topology.classpath: null +topology.environment: null +topology.bolts.outgoing.overflow.buffer.enable: false +topology.enable.netty.metrics: null + + +# enable supervisor use cgroup to make resource isolation +# Before enable it, you should make sure: +# 1. Linux version (>= 2.6.18) +# 2. Have installed cgroup (check the file's existence:/proc/cgroups) +# 3. You should start your supervisor on root +# You can get more about cgroup: +# http://t.cn/8s7nexU +# +# For cgroup root dir, the full path is "/cgroup/cpu + root_dir". +# The /cgroup/cpu part is fixed. The root dir is configurable, +# which should be consistent with the part configured in /etc/cgconfig.conf +supervisor.enable.cgroup: false +supervisor.cgroup.rootdir: "jstorm" +worker.cpu.core.upper.limit: 3 + +dev.zookeeper.path: "/tmp/dev-storm-zookeeper" + +#if this configuration has been set, +# the spout or bolt will log all received tuples +# topology.debug just for logging all sent tuples +topology.debug.recv.tuple: false + +#Usually, spout finish preparation before bolt, +#so spout need wait several seconds so that bolt finish preparation +# the default setting is 30 seconds +spout.delay.run: 30 + +#Force spout use single thread +spout.single.thread: false + +#When spout pending number is full, spout nextTuple will do sleep operation +spout.pending.full.sleep: true + +# container setting means jstorm is running under other system, such as hadoop-yarn/Ali-Apsara +# For example, in Ali-Apsara, Fuxi start c++ container process, +# the C++ container fork process to start nimbus or supervisor +container.heartbeat.timeout.seconds: 240 +container.heartbeat.frequence: 10 + + +# enable java sandbox or not +java.sandbox.enable: false. + +# logview port +nimbus.deamon.logview.port: 7621 +supervisor.deamon.logview.port: 7622 + +# logview encoding +supervisor.deamon.logview.encoding: "utf-8" + +# one log page size +log.page.size: 32768 + +### when disruptor queue is full, sleep until queue isn't full +### the default disruptor will query disruptor until queue isn't full +### this will cost much cpu +disruptor.use.sleep: true diff --git a/jstorm-core/src/main/resources/logback-test.xml b/jstorm-core/src/main/resources/logback-test.xml new file mode 100755 index 000000000..4ac2b1dbb --- /dev/null +++ b/jstorm-core/src/main/resources/logback-test.xml @@ -0,0 +1,17 @@ + + + + + + %-4r [%t] %-5p %c - %m%n + + + + + + + + + + + \ No newline at end of file diff --git a/jstorm-core/src/main/resources/version b/jstorm-core/src/main/resources/version new file mode 100755 index 000000000..9226a700f --- /dev/null +++ b/jstorm-core/src/main/resources/version @@ -0,0 +1 @@ +2.0.4-SNAPSHOT diff --git a/jstorm-core/src/test/java/com/alibaba/jstorm/cache/RocksDBTest.java b/jstorm-core/src/test/java/com/alibaba/jstorm/cache/RocksDBTest.java new file mode 100755 index 000000000..99ac92e3b --- /dev/null +++ b/jstorm-core/src/test/java/com/alibaba/jstorm/cache/RocksDBTest.java @@ -0,0 +1,466 @@ +/** + * 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.cache; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; + +import junit.framework.Assert; + +import org.rocksdb.ColumnFamilyDescriptor; +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.DBOptions; +import org.rocksdb.RocksDB; +import org.rocksdb.RocksDBException; +import org.rocksdb.TtlDB; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.OSInfo; +import com.alibaba.jstorm.utils.RunCounter; + +public class RocksDBTest { + private static Logger LOG = LoggerFactory.getLogger(RocksDBTest.class); + + public static String rootDir = "rocksdb_test"; + //ATTENTIME + // DUE to windows can't load RocksDB jni libray, + // So these junit tests can't run under window + // + + + + RocksTTLDBCache cache = new RocksTTLDBCache(); + { + Map conf = new HashMap(); +// TemporaryFolder cacheRoot = new TemporaryFolder() + + conf.put(RocksDBCache.ROCKSDB_ROOT_DIR, rootDir); + conf.put(RocksDBCache.ROCKSDB_RESET, true); + List list = new ArrayList(){{ + add(60); + add(120); + }}; + conf.put(RocksDBCache.TAG_TIMEOUT_LIST, list); + + try { + cache.init(conf); + } catch (Exception e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + } + + public void testSimple() { + List > dataList = new ArrayList >(); + for (int i = 0; i < 4; i++) { + dataList.add(new HashMap()); + } + + for (int i = 0; i < 10; i++) { + String key = "key" + i; + for (int j = 0; j < 4; j++) { + Map data = dataList.get(j); + String value = "value--" + i + "--" + j; + + data.put(key, value); + } + } + + Map data = dataList.get(0); + for (Entry entry : data.entrySet()) { + cache.put(entry.getKey(), entry.getValue()); + } + + for (Entry entry : data.entrySet()) { + String rawValue = (String)entry.getValue(); + String fetchValue = (String)cache.get(entry.getKey()); + + Assert.assertEquals(rawValue, fetchValue); + } + + for (Entry entry : data.entrySet()) { + cache.remove(entry.getKey()); + } + + + for (Entry entry : data.entrySet()) { + String fetchValue = (String)cache.get(entry.getKey()); + + Assert.assertEquals(fetchValue, null); + } + + + Map data1 = dataList.get(1); + cache.putBatch(data1); + + + Map getBatch = new HashMap(); + + for (Entry entry : data1.entrySet()) { + getBatch.put(entry.getKey(), null); + } + Map emptyBatch = new HashMap(); + emptyBatch.putAll(getBatch); + cache.getBatch(getBatch); + + for (String key : getBatch.keySet()) { + String rawValue = (String)data1.get(key); + String fetchValue = (String)getBatch.get(key); + + Assert.assertEquals(rawValue, fetchValue); + } + + cache.removeBatch(data1.keySet()); + + getBatch.clear(); + getBatch.putAll(emptyBatch); + cache.getBatch(getBatch); + + for (String key : getBatch.keySet()) { + String rawValue = null; + String fetchValue = (String)getBatch.get(key); + + Assert.assertEquals(rawValue, fetchValue); + } + + + Map data2 = dataList.get(2); + for (Entry entry : data2.entrySet()) { + cache.put(entry.getKey(), entry.getValue(), 60); + } + + for (Entry entry : data2.entrySet()) { + String rawValue = (String)entry.getValue(); + String fetchValue = (String)cache.get(entry.getKey()); + + Assert.assertEquals(rawValue, fetchValue); + } + + for (Entry entry : data2.entrySet()) { + cache.remove(entry.getKey()); + } + + + for (Entry entry : data2.entrySet()) { + String fetchValue = (String)cache.get(entry.getKey()); + + Assert.assertEquals(fetchValue, null); + } + + Map data3 = dataList.get(3); + cache.putBatch(data3, 60); + + + for (Entry entry : data3.entrySet()) { + getBatch.put(entry.getKey(), null); + } + + cache.getBatch(getBatch); + + for (String key : getBatch.keySet()) { + String rawValue = (String)data3.get(key); + String fetchValue = (String)getBatch.get(key); + + Assert.assertEquals(rawValue, fetchValue); + } + + cache.removeBatch(data3.keySet()); + + getBatch.clear(); + getBatch.putAll(emptyBatch); + cache.getBatch(getBatch); + + for (String key : getBatch.keySet()) { + String rawValue = null; + String fetchValue = (String)getBatch.get(key); + + Assert.assertEquals(rawValue, fetchValue); + } + + for (Entry entry : data.entrySet()) { + cache.put(entry.getKey(), entry.getValue()); + } + cache.putBatch(data3, 60); + + for (Entry entry : data3.entrySet()) { + String rawValue = (String)entry.getValue(); + String fetchValue = (String)cache.get(entry.getKey()); + + Assert.assertEquals(rawValue, fetchValue); + } + + cache.removeBatch(data3.keySet()); + + for (Entry entry : data3.entrySet()) { + String rawValue = null; + String fetchValue = (String)cache.get(entry.getKey()); + + Assert.assertEquals(rawValue, fetchValue); + } + + // double delete + cache.removeBatch(data3.keySet()); + } + + public void visitorAccross() throws RocksDBException, InterruptedException { + DBOptions dbOptions = null; + TtlDB ttlDB = null; + List cfNames = new ArrayList(); + List columnFamilyHandleList = new ArrayList(); + cfNames.add(new ColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY)); + cfNames.add(new ColumnFamilyDescriptor("new_cf".getBytes())); + + List ttlValues = new ArrayList(); + // new column family with 1 second ttl + ttlValues.add(1); + + // Default column family with infinite lifetime + ttlValues.add(0); + + try { + System.out.println("Begin to open db"); + dbOptions = new DBOptions().setCreateMissingColumnFamilies(true).setCreateIfMissing(true); + ttlDB = TtlDB.open(dbOptions, rootDir, cfNames, columnFamilyHandleList, ttlValues, false); + + System.out.println("Successfully open db " + rootDir); + + List keys = new ArrayList(); + keys.add("key"); + ttlDB.put("key".getBytes(), "key".getBytes()); + for (int i = 0; i < 2; i++) { + String key = "key" + i; + keys.add(key); + + ttlDB.put(columnFamilyHandleList.get(i), key.getBytes(), key.getBytes()); + } + + try { + byte[] value = ttlDB.get("others".getBytes()); + if (value != null) { + System.out.println("Raw get :" + new String(value)); + }else { + System.out.println("No value of other"); + } + }catch(Exception e) { + System.out.println("Occur exception other"); + } + + for (String key : keys) { + try { + byte[] value = ttlDB.get(key.getBytes()); + if (value != null) { + System.out.println("Raw get :" + new String(value)); + }else { + System.out.println("No value of " + key); + } + }catch(Exception e) { + System.out.println("Occur exception " + key + ", Raw"); + } + + for (int i = 0; i < 2; i++) { + try { + byte[] value = ttlDB.get(columnFamilyHandleList.get(i), key.getBytes()); + if (value != null) { + System.out.println("handler index" + i + " get :" + new String(value)); + }else { + System.out.println("No value of index" + i + " get :" + key); + } + }catch(Exception e) { + System.out.println("Occur exception " + key + ", handler index:" + i); + } + + } + } + + + + } finally { + for (ColumnFamilyHandle columnFamilyHandle : columnFamilyHandleList) { + columnFamilyHandle.dispose(); + } + if (ttlDB != null) { + ttlDB.close(); + } + if (dbOptions != null) { + dbOptions.dispose(); + } + } + } + + public void ttlDbOpenWithColumnFamilies() throws RocksDBException, InterruptedException { + DBOptions dbOptions = null; + TtlDB ttlDB = null; + List cfNames = new ArrayList(); + List columnFamilyHandleList = new ArrayList(); + cfNames.add(new ColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY)); + cfNames.add(new ColumnFamilyDescriptor("new_cf".getBytes())); + + List ttlValues = new ArrayList(); + // Default column family with infinite lifetime + ttlValues.add(0); + // new column family with 1 second ttl + ttlValues.add(1); + + try { + System.out.println("Begin to open db"); + dbOptions = new DBOptions().setCreateMissingColumnFamilies(true).setCreateIfMissing(true); + ttlDB = TtlDB.open(dbOptions, rootDir, cfNames, columnFamilyHandleList, ttlValues, false); + + System.out.println("Successfully open db " + rootDir); + + ttlDB.put("key".getBytes(), "value".getBytes()); + assertThat(ttlDB.get("key".getBytes())).isEqualTo("value".getBytes()); + ttlDB.put(columnFamilyHandleList.get(1), "key".getBytes(), "value".getBytes()); + assertThat(ttlDB.get(columnFamilyHandleList.get(1), "key".getBytes())).isEqualTo("value".getBytes()); + TimeUnit.SECONDS.sleep(2); + + ttlDB.compactRange(); + ttlDB.compactRange(columnFamilyHandleList.get(1)); + + assertThat(ttlDB.get("key".getBytes())).isNotNull(); + assertThat(ttlDB.get(columnFamilyHandleList.get(1), "key".getBytes())).isNull(); + + } finally { + for (ColumnFamilyHandle columnFamilyHandle : columnFamilyHandleList) { + columnFamilyHandle.dispose(); + } + if (ttlDB != null) { + ttlDB.close(); + } + if (dbOptions != null) { + dbOptions.dispose(); + } + } + } + + public void performanceTest() { + final LinkedBlockingDeque getQueue = new LinkedBlockingDeque(); + final LinkedBlockingDeque rmQueue = new LinkedBlockingDeque(); + + final int TIMES = 10000000; + + Thread putThread = new Thread(new Runnable() { + RunCounter runCounter = new RunCounter("Put"); + @Override + public void run() { + // TODO Auto-generated method stub + for (int i = 0; i < TIMES; i++) { + String key = "Key" + i; + + cache.put(key, key); + + runCounter.count(1l); + + getQueue.offer(key); + } + + getQueue.offer(null); + + System.out.println("Shutdown "); + } + }); + + Thread getThread = new Thread(new Runnable() { + RunCounter runCounter = new RunCounter("Get"); + @Override + public void run() { + // TODO Auto-generated method stub + while(true) { + String key = null; + try { + key = getQueue.take(); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + if (key == null){ + break; + } + + cache.get(key); + + runCounter.count(1l); + + rmQueue.offer(key); + } + System.out.println("Shutdown get"); + } + }); + + Thread rmThread = new Thread(new Runnable() { + RunCounter runCounter = new RunCounter("Rm"); + @Override + public void run() { + // TODO Auto-generated method stub + while(true) { + String key = null; + try { + key = rmQueue.take(); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + if (key == null){ + break; + } + + cache.remove(key); + + runCounter.count(1l); + + + } + System.out.println("Shutdown rm"); + } + }); + + System.out.println("Start performance test"); + putThread.start(); + getThread.start(); + rmThread.start(); + + JStormUtils.sleepMs(1000000); + } + + public static void main(String[] args) throws Exception{ + if (OSInfo.isLinux() || OSInfo.isMac()) { + RocksDB.loadLibrary(); + + RocksDBTest instance = new RocksDBTest(); + + //instance.testSimple(); + //instance.ttlDbOpenWithColumnFamilies(); + //instance.visitorAccross(); + + instance.performanceTest(); + } + + + } + + +} diff --git a/jstorm-core/src/test/java/com/alibaba/jstorm/message/context/ContextTest.java b/jstorm-core/src/test/java/com/alibaba/jstorm/message/context/ContextTest.java new file mode 100755 index 000000000..2b567a83e --- /dev/null +++ b/jstorm-core/src/test/java/com/alibaba/jstorm/message/context/ContextTest.java @@ -0,0 +1,60 @@ +/** + * 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.message.context; + +import java.lang.reflect.Constructor; +import java.util.HashMap; +import java.util.Map; + +import junit.framework.Assert; + +import org.junit.Test; + +import com.google.common.collect.Maps; + +import backtype.storm.Config; +import backtype.storm.messaging.IContext; +import backtype.storm.messaging.TransportFactory; + +public class ContextTest { + + // @Test + // public void test_zmq() throws Exception { + // String klassName = "com.alibaba.jstorm.message.zeroMq.MQContext"; + // Class klass = Class.forName(klassName); + // Constructor constructor = klass.getDeclaredConstructor(); + // constructor.setAccessible(true); + // IContext context = (IContext) constructor.newInstance(); + // Assert.assertNotNull(context); + // } + + @Test + public void test_netty() { + Map storm_conf = Maps.newHashMap(); + storm_conf.put(Config.STORM_MESSAGING_TRANSPORT, + "com.alibaba.jstorm.message.netty.NettyContext"); + storm_conf.put(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE, 1024); + storm_conf.put(Config.STORM_MESSAGING_NETTY_MAX_RETRIES, 10); + storm_conf.put(Config.STORM_MESSAGING_NETTY_MIN_SLEEP_MS, 1000); + storm_conf.put(Config.STORM_MESSAGING_NETTY_MAX_SLEEP_MS, 5000); + storm_conf.put(Config.STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS, 1); + storm_conf.put(Config.STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS, 1); + IContext context = TransportFactory.makeContext(storm_conf); + Assert.assertNotNull(context); + } +} diff --git a/jstorm-core/src/test/java/com/alibaba/jstorm/message/netty/NettyUnitTest.java b/jstorm-core/src/test/java/com/alibaba/jstorm/message/netty/NettyUnitTest.java new file mode 100644 index 000000000..923465e54 --- /dev/null +++ b/jstorm-core/src/test/java/com/alibaba/jstorm/message/netty/NettyUnitTest.java @@ -0,0 +1,794 @@ +/** + * 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.message.netty; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +import junit.framework.Assert; + +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.messaging.IConnection; +import backtype.storm.messaging.IContext; +import backtype.storm.messaging.TaskMessage; +import backtype.storm.messaging.TransportFactory; +import backtype.storm.utils.DisruptorQueue; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.utils.JStormUtils; +import com.lmax.disruptor.WaitStrategy; +import com.lmax.disruptor.dsl.ProducerType; + +public class NettyUnitTest { + + private static final Logger LOG = LoggerFactory + .getLogger(NettyUnitTest.class); + + private static int port = 6700; + private static int task = 1; + private static Lock lock = new ReentrantLock(); + private static Condition clientClose = lock.newCondition(); + private static Condition contextClose = lock.newCondition(); + + private static Map storm_conf = new HashMap(); + private static IContext context = null; + + static { + storm_conf = Utils.readDefaultConfig(); + ConfigExtension.setLocalWorkerPort(storm_conf, port); + boolean syncMode = false; + if (syncMode) { + DisruptorQueue.setLimited(true); + ConfigExtension.setNettyMaxSendPending(storm_conf, 1); + ConfigExtension.setNettySyncMode(storm_conf, true); + } else { + ConfigExtension.setNettySyncMode(storm_conf, false); + ConfigExtension.setNettyASyncBlock(storm_conf, false); + } + + // Check whether context can be reused or not + context = TransportFactory.makeContext(storm_conf); + } + + private IConnection initNettyServer() { + return initNettyServer(port); + } + + private IConnection initNettyServer(int port) { + ConcurrentHashMap deserializeQueues = new ConcurrentHashMap(); + IConnection server = context.bind(null, port, deserializeQueues); + + WaitStrategy waitStrategy = + (WaitStrategy) JStormUtils.createDisruptorWaitStrategy(storm_conf); + DisruptorQueue recvQueue = + DisruptorQueue.mkInstance("NettyUnitTest", ProducerType.SINGLE, + 1024, waitStrategy); + server.registerQueue(task, recvQueue); + + return server; + } + + @Test + public void test_small_message() { + System.out.println("!!!!!!!!Start test_small_message !!!!!!!!!!!"); + String req_msg = "Aloha is the most Hawaiian word."; + + IConnection server = null; + IConnection client = null; + + server = initNettyServer(); + + client = context.connect(null, "localhost", port); + + List list = new ArrayList(); + TaskMessage message = new TaskMessage(task, req_msg.getBytes()); + list.add(message); + + client.send(message); + + byte[] recv = (byte[]) server.recv(task, 0); + Assert.assertEquals(req_msg, new String(recv)); + + System.out.println("!!!!!!!!!!!!!!!!!!Test one time!!!!!!!!!!!!!!!!!"); + + server.close(); + client.close(); + + System.out.println("!!!!!!!!!!!!End test_small_message!!!!!!!!!!!!!"); + } + + public String setupLargMsg() { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < Short.MAX_VALUE * 64; i++) { + sb.append("Aloha is the most Hawaiian word."); + } + + return sb.toString(); + } + + @Test + public void test_large_msg() { + System.out.println("!!!!!!!!!!start large message test!!!!!!!!"); + String req_msg = setupLargMsg(); + System.out.println("!!!!Finish batch data, size:" + req_msg.length() + + "!!!!"); + + IConnection server = null; + IConnection client = null; + + server = initNettyServer(); + + client = context.connect(null, "localhost", port); + + List list = new ArrayList(); + TaskMessage message = new TaskMessage(task, req_msg.getBytes()); + list.add(message); + + LOG.info("Client send data"); + client.send(message); + + byte[] recv = (byte[]) server.recv(task, 0); + Assert.assertEquals(req_msg, new String(recv)); + + client.close(); + server.close(); + System.out.println("!!!!!!!!!!End larget message test!!!!!!!!"); + } + + @Test + public void test_server_delay() throws InterruptedException { + System.out.println("!!!!!!!!!!Start delay message test!!!!!!!!"); + String req_msg = setupLargMsg(); + + IConnection server = null; + IConnection client = null; + + server = initNettyServer(); + + client = context.connect(null, "localhost", port); + + List list = new ArrayList(); + TaskMessage message = new TaskMessage(task, req_msg.getBytes()); + list.add(message); + + LOG.info("Client send data"); + client.send(message); + Thread.sleep(1000); + + byte[] recv = (byte[]) server.recv(task, 0); + Assert.assertEquals(req_msg, new String(recv)); + + server.close(); + client.close(); + System.out.println("!!!!!!!!!!End delay message test!!!!!!!!"); + } + + @Test + public void test_first_client() throws InterruptedException { + System.out.println("!!!!!!!!Start test_first_client !!!!!!!!!!!"); + final String req_msg = setupLargMsg(); + + final IContext context = TransportFactory.makeContext(storm_conf); + + new Thread(new Runnable() { + + @Override + public void run() { + + lock.lock(); + IConnection client = context.connect(null, "localhost", port); + + List list = new ArrayList(); + TaskMessage message = new TaskMessage(task, req_msg.getBytes()); + list.add(message); + + client.send(message); + System.out.println("!!Client has sent data"); + JStormUtils.sleepMs(1000); + + try { + clientClose.await(); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + client.close(); + contextClose.signal(); + lock.unlock(); + + } + }).start(); + + IConnection server = null; + + JStormUtils.sleepMs(1000); + System.out.println("!!server begin start!!!!!"); + + server = initNettyServer(); + JStormUtils.sleepMs(5000); + + System.out.println("Begin to receive message"); + byte[] recv = (byte[]) server.recv(task, 1); + Assert.assertEquals(req_msg, new String(recv)); + + System.out.println("Finished to receive message"); + + lock.lock(); + clientClose.signal(); + server.close(); + contextClose.await(); + context.term(); + lock.unlock(); + + System.out.println("!!!!!!!!!!!!End test_first_client!!!!!!!!!!!!!"); + } + + @Test + public void test_msg_buffer_timeout() throws InterruptedException { + System.out.println("!!!!!!!!Start test_msg_buffer_timeout !!!!!!!!!!!"); + final String req_msg = setupLargMsg(); + + ConfigExtension.setNettyPendingBufferTimeout(storm_conf, 10*1000l); + final IContext context = TransportFactory.makeContext(storm_conf); + + new Thread(new Runnable() { + + @Override + public void run() { + + lock.lock(); + IConnection client = context.connect(null, "localhost", port); + + List list = new ArrayList(); + TaskMessage message = new TaskMessage(task, req_msg.getBytes()); + list.add(message); + + client.send(message); + System.out.println("!!Client has sent data"); + JStormUtils.sleepMs(1000); + + try { + clientClose.await(); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + client.close(); + contextClose.signal(); + lock.unlock(); + + } + }).start(); + + IConnection server = null; + + JStormUtils.sleepMs(11000); + System.out.println("!!server begin start!!!!!"); + + server = initNettyServer(); + JStormUtils.sleepMs(5000); + + System.out.println("Begin to receive message"); + byte[] recv = (byte[]) server.recv(task, 1); + Assert.assertEquals(null, recv); + + System.out.println("Pending message was timouout:" + (recv == null)); + + lock.lock(); + clientClose.signal(); + server.close(); + contextClose.await(); + context.term(); + lock.unlock(); + + System.out.println("!!!!!!!!!!!!End test_msg_buffer_timeout!!!!!!!!!!!!!"); + } + + @Test + public void test_batch() throws InterruptedException { + System.out.println("!!!!!!!!!!Start batch message test!!!!!!!!"); + final int base = 100000; + + final IContext context = TransportFactory.makeContext(storm_conf); + final IConnection server = initNettyServer(); + + new Thread(new Runnable() { + + public void send() { + final IConnection client = + context.connect(null, "localhost", port); + + List list = new ArrayList(); + + for (int i = 1; i < Short.MAX_VALUE; i++) { + + String req_msg = String.valueOf(i + base); + + TaskMessage message = + new TaskMessage(task, req_msg.getBytes()); + list.add(message); + + } + + client.send(list); + + System.out.println("Finish Send "); + JStormUtils.sleepMs(1000); + + try { + clientClose.await(); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + client.close(); + contextClose.signal(); + + } + + @Override + public void run() { + lock.lock(); + try { + send(); + } finally { + lock.unlock(); + } + } + }).start(); + + for (int i = 1; i < Short.MAX_VALUE; i++) { + byte[] message = (byte[]) server.recv(task, 0); + + Assert.assertEquals(String.valueOf(i + base), + new String(message)); + + if (i % 1000 == 0) { + //System.out.println("Receive " + new String(message)); + } + } + + System.out.println("Finish Receive "); + + lock.lock(); + clientClose.signal(); + server.close(); + contextClose.await(); + context.term(); + lock.unlock(); + System.out.println("!!!!!!!!!!End batch message test!!!!!!!!"); + } + + @Test + public void test_slow_receive() throws InterruptedException { + System.out + .println("!!!!!!!!!!Start test_slow_receive message test!!!!!!!!"); + final int base = 100000; + + final IContext context = TransportFactory.makeContext(storm_conf); + final IConnection server = initNettyServer(); + + new Thread(new Runnable() { + + @Override + public void run() { + lock.lock(); + + IConnection client = null; + + client = context.connect(null, "localhost", port); + + List list = new ArrayList(); + + for (int i = 1; i < Short.MAX_VALUE; i++) { + + String req_msg = String.valueOf(i + base); + + TaskMessage message = + new TaskMessage(task, req_msg.getBytes()); + list.add(message); + + if (i % 1000 == 0) { + System.out.println("send " + i); + client.send(list); + list = new ArrayList(); + } + + } + + client.send(list); + + System.out.println("Finish Send "); + JStormUtils.sleepMs(1000); + + try { + clientClose.await(); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + client.close(); + contextClose.signal(); + lock.unlock(); + } + }).start(); + + for (int i = 1; i < Short.MAX_VALUE; i++) { + byte[] message = (byte[]) server.recv(task, 0); + JStormUtils.sleepMs(1); + + Assert.assertEquals(String.valueOf(i + base), + new String(message)); + + if (i % 1000 == 0) { + //System.out.println("Receive " + new String(message)); + } + } + + System.out.println("Finish Receive "); + + lock.lock(); + clientClose.signal(); + server.close(); + contextClose.await(); + context.term(); + lock.unlock(); + System.out + .println("!!!!!!!!!!End test_slow_receive message test!!!!!!!!"); + } + + @Test + public void test_slow_receive_big() throws InterruptedException { + System.out + .println("!!!!!!!!!!Start test_slow_receive_big message test!!!!!!!!"); + final int base = 100; + final String req_msg = setupLargMsg(); + + final IContext context = TransportFactory.makeContext(storm_conf); + final IConnection server = initNettyServer(); + + new Thread(new Runnable() { + + @Override + public void run() { + final IConnection client = + context.connect(null, "localhost", port); + + lock.lock(); + for (int i = 1; i < base; i++) { + + TaskMessage message = + new TaskMessage(task, req_msg.getBytes()); + System.out.println("send " + i); + client.send(message); + + } + + System.out.println("Finish Send "); + JStormUtils.sleepMs(1000); + + try { + clientClose.await(); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + client.close(); + contextClose.signal(); + lock.unlock(); + + } + }).start(); + + for (int i = 1; i < base; i++) { + byte[] message = (byte[]) server.recv(task, 0); + JStormUtils.sleepMs(100); + + Assert.assertEquals(req_msg, new String(message)); + System.out.println("receive msg-" + i); + + } + + System.out.println("Finish Receive "); + + lock.lock(); + clientClose.signal(); + server.close(); + contextClose.await(); + context.term(); + lock.unlock(); + System.out + .println("!!!!!!!!!!End test_slow_receive_big message test!!!!!!!!"); + } + + @Test + public void test_client_reboot() throws InterruptedException { + System.out.println("!!!!!!!!!!Start client reboot test!!!!!!!!"); + final String req_msg = setupLargMsg(); + + final IContext context = TransportFactory.makeContext(storm_conf); + + new Thread(new Runnable() { + + @Override + public void run() { + + IConnection client = context.connect(null, "localhost", port); + + lock.lock(); + + List list = new ArrayList(); + TaskMessage message = new TaskMessage(task, req_msg.getBytes()); + list.add(message); + + client.send(message); + + System.out.println("Send first"); + + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + } + client.close(); + + IConnection client2 = context.connect(null, "localhost", port); + System.out.println("!!!!!!! restart client !!!!!!!!!!"); + + client2.send(message); + System.out.println("Send second"); + JStormUtils.sleepMs(1000); + + try { + clientClose.await(); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + client2.close(); + contextClose.signal(); + lock.unlock(); + } + }).start(); + + IConnection server = initNettyServer(); + + byte[] recv = (byte[]) server.recv(task, 0); + System.out.println("Sever receive first"); + Assert.assertEquals(req_msg, new String(recv)); + + Thread.sleep(1000); + + byte[] recv2 = (byte[]) server.recv(task, 0); + System.out.println("Sever receive second"); + Assert.assertEquals(req_msg, new String(recv2)); + + lock.lock(); + clientClose.signal(); + server.close(); + contextClose.await(); + context.term(); + lock.unlock(); + System.out.println("!!!!!!!!!!End client reboot test!!!!!!!!"); + } + + @Test + public void test_server_reboot() throws InterruptedException { + System.out.println("!!!!!!!!!!Start server reboot test!!!!!!!!"); + final String req_msg = setupLargMsg(); + + final IContext context = TransportFactory.makeContext(storm_conf); + IConnection server = null; + + new Thread(new Runnable() { + + @Override + public void run() { + final IConnection client = + context.connect(null, "localhost", port); + + lock.lock(); + + List list = new ArrayList(); + TaskMessage message = new TaskMessage(task, req_msg.getBytes()); + list.add(message); + + client.send(message); + + System.out.println("Send first"); + + JStormUtils.sleepMs(10000); + + System.out.println("Begin to Send second"); + client.send(message); + System.out.println("Send second"); + + JStormUtils.sleepMs(15000); + client.send(message); + System.out.println("Send third time"); + + try { + clientClose.await(); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + client.close(); + contextClose.signal(); + lock.unlock(); + + } + }).start(); + + server = initNettyServer(); + + byte[] recv = (byte[]) server.recv(task, 0); + System.out.println("Receive first"); + Assert.assertEquals(req_msg, new String(recv)); + + server.close(); + + System.out.println("!!shutdow server and sleep 30s, please wait!!"); + Thread.sleep(30000); + + IConnection server2 = server = initNettyServer(); + System.out.println("!!!!!!!!!!!!!!!!!!!! restart server !!!!!!!!!!!"); + + byte[] recv2 = (byte[]) server2.recv(task, 0); + Assert.assertEquals(req_msg, new String(recv2)); + + lock.lock(); + clientClose.signal(); + server2.close(); + contextClose.await(); + context.term(); + lock.unlock(); + System.out.println("!!!!!!!!!!End server reboot test!!!!!!!!"); + } + + /** + * Due to there is only one client to one server in one jvm It can't do this + * test + * + * @throws InterruptedException + */ + public void test_multiple_client() throws InterruptedException { + System.out.println("!!!!!!!!Start test_multiple_client !!!!!!!!!!!"); + final String req_msg = setupLargMsg(); + + final int clientNum = 3; + final AtomicLong received = new AtomicLong(clientNum); + + for (int i = 0; i < clientNum; i++) { + + new Thread(new Runnable() { + + @Override + public void run() { + + IConnection client = + context.connect(null, "localhost", port); + + List list = new ArrayList(); + TaskMessage message = + new TaskMessage(task, req_msg.getBytes()); + list.add(message); + + client.send(message); + System.out.println("!!Client has sent data"); + + while (received.get() != 0) { + JStormUtils.sleepMs(1000); + } + + client.close(); + + } + }).start(); + } + + IConnection server = null; + + JStormUtils.sleepMs(1000); + System.out.println("!!server begin start!!!!!"); + + server = initNettyServer(); + + for (int i = 0; i < clientNum; i++) { + byte[] recv = (byte[]) server.recv(task, 0); + Assert.assertEquals(req_msg, new String(recv)); + received.decrementAndGet(); + } + + server.close(); + + System.out.println("!!!!!!!!!!!!End test_multiple_client!!!!!!!!!!!!!"); + } + + @Test + public void test_multiple_server() throws InterruptedException { + System.out.println("!!!!!!!!Start test_multiple_server !!!!!!!!!!!"); + final String req_msg = setupLargMsg(); + + final int clientNum = 3; + final AtomicLong received = new AtomicLong(clientNum); + + for (int i = 0; i < clientNum; i++) { + final int realPort = port + i; + + new Thread(new Runnable() { + + @Override + public void run() { + + IConnection server = null; + + JStormUtils.sleepMs(1000); + System.out.println("!!server begin start!!!!!"); + + server = initNettyServer(realPort); + + byte[] recv = (byte[]) server.recv(task, 0); + Assert.assertEquals(req_msg, new String(recv)); + received.decrementAndGet(); + System.out.println("!!server received !!!!!" + realPort); + + server.close(); + + } + }).start(); + } + + List list = new ArrayList(); + TaskMessage message = new TaskMessage(task, req_msg.getBytes()); + list.add(message); + + List clients = new ArrayList(); + + for (int i = 0; i < clientNum; i++) { + final int realPort = port + i; + + IConnection client = context.connect(null, "localhost", realPort); + clients.add(client); + + client.send(message); + System.out.println("!!Client has sent data to " + realPort); + } + + while (received.get() != 0) { + JStormUtils.sleepMs(1000); + } + + for (int i = 0; i < clientNum; i++) { + clients.get(i).close(); + } + + System.out.println("!!!!!!!!!!!!End test_multiple_server!!!!!!!!!!!!!"); + } +} diff --git a/jstorm-core/src/test/java/com/alibaba/jstorm/message/zmq/ZmqUnitTest.java b/jstorm-core/src/test/java/com/alibaba/jstorm/message/zmq/ZmqUnitTest.java new file mode 100755 index 000000000..16ecb73b9 --- /dev/null +++ b/jstorm-core/src/test/java/com/alibaba/jstorm/message/zmq/ZmqUnitTest.java @@ -0,0 +1,317 @@ +/** + * 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.message.zmq; + +import java.util.HashMap; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; + +public class ZmqUnitTest { + + private static final Logger LOG = LoggerFactory + .getLogger(ZmqUnitTest.class); + + private static int port = 6700; + private static int task = 1; + + private static String context_class_name = + "com.alibaba.jstorm.message.zeroMq.MQContext"; + + private static Map storm_conf = new HashMap(); + static { + storm_conf.put(Config.STORM_MESSAGING_TRANSPORT, context_class_name); + } + + /** + * This is only can be test under linux + */ + + // @Test + // public void test_basic() { + // System.out + // .println("!!!!!!!!!!!!!!!!!Start basic test!!!!!!!!!!!!!!!!!"); + // String req_msg = "Aloha is the most Hawaiian word."; + // + // IContext context = TransportFactory.makeContext(storm_conf); + // IConnection server = null; + // IConnection client = null; + // + // server = context.bind(null, port); + // + // WaitStrategy waitStrategy = (WaitStrategy) Utils + // .newInstance((String) storm_conf + // .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); + // DisruptorQueue recvQueue = DisruptorQueue.mkInstance( + // new SingleThreadedClaimStrategy(1024), waitStrategy); + // server.registerQueue(recvQueue); + // + // client = context.connect(null, "localhost", port); + // + // List list = new ArrayList(); + // TaskMessage message = new TaskMessage(task, req_msg.getBytes()); + // list.add(message); + // + // client.send(list); + // + // TaskMessage recv = server.recv(0); + // Assert.assertEquals(req_msg, new String(recv.message())); + // + // System.out.println("!!!!!!!!!!!!!!!!!!Test one time!!!!!!!!!!!!!!!!!"); + // + // server.close(); + // client.close(); + // context.term(); + // + // System.out + // .println("!!!!!!!!!!!!!!!!!!!!End basic test!!!!!!!!!!!!!!!!!!!"); + // } + // + // public String setupLargMsg() { + // StringBuilder sb = new StringBuilder(); + // for (int i = 0; i < Short.MAX_VALUE * 10; i++) { + // sb.append("Aloha is the most Hawaiian word.").append(i); + // } + // + // return sb.toString(); + // } + // + // @Test + // public void test_large_msg() { + // System.out.println("!!!!!!!!!!start larget message test!!!!!!!!"); + // String req_msg = setupLargMsg(); + // System.out.println("!!!!Finish batch data, size:" + req_msg.length() + // + "!!!!"); + // + // IContext context = TransportFactory.makeContext(storm_conf); + // IConnection server = null; + // IConnection client = null; + // + // server = context.bind(null, port); + // + // WaitStrategy waitStrategy = (WaitStrategy) Utils + // .newInstance((String) storm_conf + // .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); + // DisruptorQueue recvQueue = DisruptorQueue.mkInstance( + // new SingleThreadedClaimStrategy(1024), waitStrategy); + // server.registerQueue(recvQueue); + // + // client = context.connect(null, "localhost", port); + // + // List list = new ArrayList(); + // TaskMessage message = new TaskMessage(task, req_msg.getBytes()); + // list.add(message); + // + // client.send(list); + // + // TaskMessage recv = server.recv(0); + // Assert.assertEquals(req_msg, new String(recv.message())); + // + // server.close(); + // client.close(); + // context.term(); + // System.out.println("!!!!!!!!!!End larget message test!!!!!!!!"); + // } + // + // @Test + // public void test_server_delay() throws InterruptedException { + // System.out.println("!!!!!!!!!!Start delay message test!!!!!!!!"); + // String req_msg = setupLargMsg(); + // + // IContext context = TransportFactory.makeContext(storm_conf); + // IConnection server = null; + // IConnection client = null; + // + // server = context.bind(null, port); + // + // WaitStrategy waitStrategy = (WaitStrategy) Utils + // .newInstance((String) storm_conf + // .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); + // DisruptorQueue recvQueue = DisruptorQueue.mkInstance( + // new SingleThreadedClaimStrategy(1024), waitStrategy); + // server.registerQueue(recvQueue); + // + // client = context.connect(null, "localhost", port); + // + // List list = new ArrayList(); + // TaskMessage message = new TaskMessage(task, req_msg.getBytes()); + // list.add(message); + // + // client.send(list); + // Thread.sleep(1000); + // + // TaskMessage recv = server.recv(0); + // Assert.assertEquals(req_msg, new String(recv.message())); + // + // server.close(); + // client.close(); + // context.term(); + // System.out.println("!!!!!!!!!!End delay message test!!!!!!!!"); + // } + // + // @Test + // public void test_batch() { + // System.out.println("!!!!!!!!!!Start batch message test!!!!!!!!"); + // + // IContext context = TransportFactory.makeContext(storm_conf); + // final IConnection server = context.bind(null, port); + // IConnection client = null; + // + // WaitStrategy waitStrategy = (WaitStrategy) Utils + // .newInstance((String) storm_conf + // .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); + // DisruptorQueue recvQueue = DisruptorQueue.mkInstance( + // new SingleThreadedClaimStrategy(1024), waitStrategy); + // server.registerQueue(recvQueue); + // + // client = context.connect(null, "localhost", port); + // + // final int base = 100000; + // + // List list = new ArrayList(); + // + // client.send(list); + // for (int i = 1; i < Short.MAX_VALUE; i++) { + // + // String req_msg = String.valueOf(i + base); + // + // TaskMessage message = new TaskMessage(task, req_msg.getBytes()); + // list.add(message); + // + // } + // + // client.send(list); + // + // System.out.println("Finish Send "); + // + // for (int i = 1; i < Short.MAX_VALUE; i++) { + // TaskMessage message = server.recv(0); + // + // Assert.assertEquals(String.valueOf(i + base), + // new String(message.message())); + // + // if (i % 1000 == 0) { + // System.out.println("Receive " + i); + // } + // } + // + // System.out.println("Finish Receive "); + // + // client.close(); + // server.close(); + // context.term(); + // System.out.println("!!!!!!!!!!End batch message test!!!!!!!!"); + // } + // + // @Test + // public void test_client_reboot() throws InterruptedException { + // System.out.println("!!!!!!!!!!Start client reboot test!!!!!!!!"); + // String req_msg = setupLargMsg(); + // + // IContext context = TransportFactory.makeContext(storm_conf); + // IConnection server = null; + // IConnection client = null; + // + // server = context.bind(null, port); + // + // WaitStrategy waitStrategy = (WaitStrategy) Utils + // .newInstance((String) storm_conf + // .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); + // DisruptorQueue recvQueue = DisruptorQueue.mkInstance( + // new SingleThreadedClaimStrategy(1024), waitStrategy); + // server.registerQueue(recvQueue); + // + // client = context.connect(null, "localhost", port); + // + // List list = new ArrayList(); + // TaskMessage message = new TaskMessage(task, req_msg.getBytes()); + // list.add(message); + // + // client.send(list); + // + // TaskMessage recv = server.recv(0); + // Assert.assertEquals(req_msg, new String(recv.message())); + // + // client.close(); + // IConnection client2 = context.connect(null, "localhost", port); + // System.out.println("!!!!!!! restart client !!!!!!!!!!"); + // + // client2.send(list); + // Thread.sleep(1000); + // + // TaskMessage recv2 = server.recv(0); + // Assert.assertEquals(req_msg, new String(recv2.message())); + // + // client2.close(); + // server.close(); + // context.term(); + // System.out.println("!!!!!!!!!!End client reboot test!!!!!!!!"); + // } + // + // @Test + // public void test_server_reboot() throws InterruptedException { + // System.out.println("!!!!!!!!!!Start server reboot test!!!!!!!!"); + // String req_msg = setupLargMsg(); + // + // IContext context = TransportFactory.makeContext(storm_conf); + // IConnection server = null; + // IConnection client = null; + // + // server = context.bind(null, port); + // + // WaitStrategy waitStrategy = (WaitStrategy) Utils + // .newInstance((String) storm_conf + // .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); + // DisruptorQueue recvQueue = DisruptorQueue.mkInstance( + // new SingleThreadedClaimStrategy(1024), waitStrategy); + // server.registerQueue(recvQueue); + // + // client = context.connect(null, "localhost", port); + // + // List list = new ArrayList(); + // TaskMessage message = new TaskMessage(task, req_msg.getBytes()); + // list.add(message); + // + // client.send(list); + // + // TaskMessage recv = server.recv(0); + // Assert.assertEquals(req_msg, new String(recv.message())); + // + // server.close(); + // + // client.send(list); + // System.out.println("!!!!!!!! shutdow server and sleep 30s !!!!!"); + // Thread.sleep(30000); + // + // IConnection server2 = context.bind(null, port); + // server2.registerQueue(recvQueue); + // System.out.println("!!!!!!!!!!!!!!!!!!!! restart server !!!!!!!!!!!"); + // + // TaskMessage recv2 = server2.recv(0); + // Assert.assertEquals(req_msg, new String(recv2.message())); + // + // client.close(); + // server2.close(); + // context.term(); + // System.out.println("!!!!!!!!!!End server reboot test!!!!!!!!"); + // } + +} diff --git a/jstorm-core/src/test/java/com/alibaba/jstorm/metric/MetricTest.java b/jstorm-core/src/test/java/com/alibaba/jstorm/metric/MetricTest.java new file mode 100755 index 000000000..fee55409e --- /dev/null +++ b/jstorm-core/src/test/java/com/alibaba/jstorm/metric/MetricTest.java @@ -0,0 +1,380 @@ +/** + * 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.metric; + +import java.util.Date; +import java.util.Map; +import java.util.Map.Entry; + +import junit.framework.Assert; + +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.generated.MetricInfo; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.common.metric.Counter; +import com.alibaba.jstorm.common.metric.Histogram; +import com.alibaba.jstorm.common.metric.LongCounter; +import com.alibaba.jstorm.common.metric.Meter; +import com.alibaba.jstorm.common.metric.MetricRegistry; +import com.alibaba.jstorm.common.metric.window.Metric; +import com.alibaba.jstorm.utils.JStormUtils; + +public class MetricTest { + private static final Logger LOG = LoggerFactory.getLogger(MetricTest.class); + static int[] windows = new int[4]; + + static { + windows[0] = 60; + for (int i = 1; i < windows.length; i++) { + windows[i] = windows[i - 1] * 2; + } + System.out.println("Start Test " + new Date()); + } + static int interval = windows[0]/20; + + public void check_value(Map map, Long[] result) { + int i = 0; + for (Integer key : map.keySet()) { + StringBuilder sb = new StringBuilder(); + sb.append("Time window ").append(key).append(" should be "); + sb.append(result[i]).append(", but it is ").append(map.get(key)); + Assert.assertTrue(sb.toString(), map.get(key).equals(result[i])); + } + + } + + + public Output testTimes(Input input) { + int firstRunSeconds = input.firstRunSeconds; + int interruptSeconds = input.interruptSeconds; + int endRunSeconds = input.endRunSeconds; + + System.out.println("Start test " + new Date() + ", first:" + firstRunSeconds + ", interrupt:" + interruptSeconds + ", endRun:" + endRunSeconds); + final long intervalTimes = 1000000l; + + long firstCounter = 0l; + long start = System.currentTimeMillis(); + long now = System.currentTimeMillis(); + long firstStop = start + firstRunSeconds * 1000; + Metric metric = input.metric; + while(now < firstStop) { + for (long l = 0; l < intervalTimes; l++) { + metric.update(1.0d); + } + firstCounter += intervalTimes; + now = System.currentTimeMillis(); + } + + try { + Thread.sleep(interruptSeconds * 1000); + }catch(Exception e) { + e.printStackTrace(); + } + + now = System.currentTimeMillis(); + long endStop = now + endRunSeconds * 1000; + long secondCounter = 0l; + while(now < endStop) { + for (long l = 0; l < intervalTimes; l++) { + metric.update(1.0d); + } + secondCounter += intervalTimes; + now = System.currentTimeMillis(); + } + + long end = System.currentTimeMillis(); + long totalCounter = firstCounter + secondCounter; + Output output = new Output(); + + output.output = metric.getSnapshot(); + output.firstCounter = firstCounter; + output.secondCounter = secondCounter; + System.out.println("update " + totalCounter + " cost " + (end - start) / 1000 + ", firstCounter:" + firstCounter + ", endCounter:" + secondCounter); + System.out.println("End test " + new Date()); + return output; + + } + + public void testUpdates(Input input) { + int firstRunSeconds = input.firstRunSeconds; + int interruptSeconds = input.interruptSeconds; + int endRunSeconds = input.endRunSeconds; + + System.out.println("Start test " + new Date() + ", first:" + firstRunSeconds + ", interrupt:" + interruptSeconds + ", endRun:" + endRunSeconds); + final long intervalTimes = 1000000l; + + long firstCounter = 0l; + long start = System.currentTimeMillis(); + long now = System.currentTimeMillis(); + long firstStop = start + firstRunSeconds * 1000; + Metric metric = input.metric; + while(now < firstStop) { + for (long l = 0; l < intervalTimes; l++) { + metric.update(1.0d); + } + firstCounter += intervalTimes; + now = System.currentTimeMillis(); + } + + try { + Thread.sleep(interruptSeconds * 1000); + }catch(Exception e) { + e.printStackTrace(); + } + + now = System.currentTimeMillis(); + long endStop = now + endRunSeconds * 1000; + long secondCounter = 0l; + while(now < endStop) { + for (long l = 0; l < intervalTimes; l++) { + metric.update(1.0d); + } + secondCounter += intervalTimes; + now = System.currentTimeMillis(); + } + + long end = System.currentTimeMillis(); + long totalCounter = firstCounter + secondCounter; + Output output = new Output(); + + + System.out.println("update " + totalCounter + " cost " + (end - start) / 1000 + ", firstCounter:" + firstCounter + ", endCounter:" + secondCounter); + System.out.println("End test " + new Date()); + return ; + + } + + public Metric getMetric(int type) { + Metric metric = null; + if (type == 0) { + Counter counter = new Counter(0.0d); + metric = counter; + }else if (type == 1) { + metric = new Meter(); + }else if (type == 2) { + metric = new Histogram(); + }else { + return null; + } + metric.setWindowSeconds(windows); + metric.init(); + return metric; + + + } + + public Metric getJStormMetric(int type) { + Metric metric = null; + if (type == 0) { + metric = JStormMetrics.registerWorkerCounter("Counter"); + }else if (type == 1) { + metric = JStormMetrics.registerWorkerMeter("Meter"); + }else if (type == 2) { + metric = JStormMetrics.registerWorkerHistogram("Historgram"); + }else { + return null; + } + + metric.setWindowSeconds(windows); + metric.init(); + return metric; + + + } + + public Metric getJStormMetric(int type, String name) { + Metric metric = null; + if (type == 0) { + metric = JStormMetrics.registerWorkerCounter("Counter", name); + }else if (type == 1) { + metric = JStormMetrics.registerWorkerMeter("Meter", name); + }else if (type == 2) { + metric = JStormMetrics.registerWorkerHistogram("Historgram", name); + }else { + return null; + } + + + return metric; + + + } + + public void unregisterJStormMetric(int type) { + if (type == 0) { + JStormMetrics.unregisterWorkerMetric("Counter"); + }else if (type == 1) { + JStormMetrics.unregisterWorkerMetric("Meter"); + }else if (type == 2) { + JStormMetrics.unregisterWorkerMetric("Historgram"); + }else { + return ; + } + return ; + + + } + + + public void test_speed() { + for (int m = 0; m < 2; m++) { + Input input = new Input(); + input.metric = getMetric(m); + input.firstRunSeconds = 20; + input.interruptSeconds = 0; + input.endRunSeconds = 0; + Output result = testTimes(input); + System.out.println(input.metric.getClass().getSimpleName() + ":" + + result.output); + + } + } + + // this test will cost too much time to run + //@Test + public void test_single_thread() { + LOG.info("Begin to test"); + //int[] timeWindow = {0, interval/2, interval, interval + 1, windows[0]/2, windows[0], windows[0] + 1}; + int[] timeWindow = {0, windows[0]}; + + for (int m = 2; m >= 0; m--) { + for (int i = 0; i < timeWindow.length; i++) { + // for (int j = 0; j < timeWindow.length; j++) { + int j = 0; + for (int l = 0; l < timeWindow.length; l++) { + Input input = new Input(); + input.metric = getJStormMetric(m); + input.firstRunSeconds = timeWindow[i]; + input.interruptSeconds = timeWindow[j]; + input.endRunSeconds = timeWindow[l]; + Output result = testTimes(input); + System.out.println(input.metric.getClass() + .getSimpleName() + ":" + result.output); + unregisterJStormMetric(m); + } + + // } + } + } + + } + + public void test_one_case() { + int[] timeWindow = {0, windows[0]}; + int m = 1, i = 1, j = 0, l = 1; + Input input = new Input(); + input.metric = getJStormMetric(m); + input.metric.setWindowSeconds(windows); + input.firstRunSeconds = timeWindow[i]; + input.interruptSeconds = timeWindow[j]; + input.endRunSeconds = timeWindow[l]; + Output result = testTimes(input); + System.out.println(input.metric.getClass() + .getSimpleName() + ":" + result.output); + unregisterJStormMetric(m); + } + + public void alltest() { + test_one_case(); + //test_single_thread(); + //testMultipleThread(); + } + + public MetricInfo computWorkerMetrics(MetricRegistry workerMetrics) { + MetricInfo workerMetricInfo = MetricThrift.mkMetricInfo(); + Map workerMetricMap = workerMetrics.getMetrics(); + for (Entry entry : workerMetricMap.entrySet()) { + String name = entry.getKey(); + Map snapshot = entry.getValue().getSnapshot(); + + MetricThrift.insert(workerMetricInfo, name, snapshot); + } + + System.out.println(Utils.toPrettyJsonString(workerMetricInfo)); + return workerMetricInfo; + } + + public void testMultipleThread() { + LOG.info("Begin to test"); + //int[] timeWindow = {0, interval/2, interval, interval + 1, windows[0]/2, windows[0], windows[0] + 1}; + int[] timeWindow = {0, windows[0]}; + + //for (int m = 2; m >= 0; m--) { + for (int i = 0; i < timeWindow.length; i++) { + for (int j = 0; j < timeWindow.length; j++) { + //int j = 0; + for (int l = 0; l < timeWindow.length; l++) { + + final Input input = new Input(); + input.metric = getJStormMetric(1, String.valueOf(i) + ":" + String.valueOf(j) + ":" + String.valueOf(l)); + input.firstRunSeconds = timeWindow[i]; + input.interruptSeconds = timeWindow[j]; + input.endRunSeconds = timeWindow[l]; + + Thread thread = new Thread(new Runnable(){ + + @Override + public void run() { + // TODO Auto-generated method stub + testUpdates(input); + } + + } + ); + + thread.start(); + + + } + + //} + } + } + + for (int i = 0; i < 4 * windows[0]/60 ; i ++) { + MetricInfo metricInfo = computWorkerMetrics(JStormMetrics.workerMetrics); + JStormUtils.sleepMs(60 * 1000); + } + + JStormUtils.haltProcess(0); + + } + + public static class Input { + public Metric metric; + public int firstRunSeconds; + public int interruptSeconds; + public int endRunSeconds; + } + + public static class Output { + public Object output; + public long firstCounter; + public long secondCounter; + } + + public static void main(String[] args) { + MetricTest test = new MetricTest(); + test.alltest(); + + } +} diff --git a/jstorm-core/src/test/java/com/alibaba/jstorm/superivosr/deamon/HttpserverTest.java b/jstorm-core/src/test/java/com/alibaba/jstorm/superivosr/deamon/HttpserverTest.java new file mode 100755 index 000000000..e10ad937c --- /dev/null +++ b/jstorm-core/src/test/java/com/alibaba/jstorm/superivosr/deamon/HttpserverTest.java @@ -0,0 +1,43 @@ +/** + * 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.superivosr.deamon; + +import java.util.HashMap; + +import org.junit.Test; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.daemon.supervisor.Httpserver; +import com.google.common.collect.Maps; + +public class HttpserverTest { + + @Test + public void test_httpserver() { + int port = + ConfigExtension.getSupervisorDeamonHttpserverPort(Maps + .newHashMap()); + Httpserver httpserver = + new Httpserver(port, new HashMap()); + httpserver.start(); + System.out.println("start...."); + + httpserver.shutdown(); + } + +} diff --git a/jstorm-core/src/test/java/com/alibaba/jstorm/topology/SingleJoinBolt.java b/jstorm-core/src/test/java/com/alibaba/jstorm/topology/SingleJoinBolt.java new file mode 100755 index 000000000..d62f40809 --- /dev/null +++ b/jstorm-core/src/test/java/com/alibaba/jstorm/topology/SingleJoinBolt.java @@ -0,0 +1,132 @@ +/** + * 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-core/src/test/java/com/alibaba/jstorm/topology/SingleJoinTest.java b/jstorm-core/src/test/java/com/alibaba/jstorm/topology/SingleJoinTest.java new file mode 100755 index 000000000..9458d31d5 --- /dev/null +++ b/jstorm-core/src/test/java/com/alibaba/jstorm/topology/SingleJoinTest.java @@ -0,0 +1,80 @@ +/** + * 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.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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 = LoggerFactory.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-core/src/test/java/com/alibaba/jstorm/topology/TransactionalWordsTest.java b/jstorm-core/src/test/java/com/alibaba/jstorm/topology/TransactionalWordsTest.java new file mode 100755 index 000000000..0bec51c6f --- /dev/null +++ b/jstorm-core/src/test/java/com/alibaba/jstorm/topology/TransactionalWordsTest.java @@ -0,0 +1,280 @@ +/** + * 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.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 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 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-core/src/test/java/com/alibaba/jstorm/util/queue/DisruptorTest.java b/jstorm-core/src/test/java/com/alibaba/jstorm/util/queue/DisruptorTest.java new file mode 100755 index 000000000..10bd87886 --- /dev/null +++ b/jstorm-core/src/test/java/com/alibaba/jstorm/util/queue/DisruptorTest.java @@ -0,0 +1,566 @@ +/** + * 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.util.queue; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; + +import junit.framework.Assert; + +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.utils.DisruptorQueue; + +import com.lmax.disruptor.BlockingWaitStrategy; +import com.lmax.disruptor.EventHandler; +import com.lmax.disruptor.InsufficientCapacityException; +import com.lmax.disruptor.dsl.ProducerType; + +public class DisruptorTest { + + static { + DisruptorQueue.setUseSleep(true); + DisruptorQueue.setLimited(true); + } + + private int count = 100000000; + private int buffer_size = 8 * 1024; + + private Logger logger = LoggerFactory.getLogger(DisruptorTest.class); + + @Test + public void testMultipleConsume() { + final DisruptorQueue disruptorQueue = + createQueue("test", ProducerType.MULTI, 1024); + + // new Thread(new Runnable() { + // + // @Override + // public void run() { + // System.out.println("Begin to produce item"); + // JStormUtils.sleepMs(1000); + // + // for (int i = 0; i < 1000000; i++) { + // disruptorQueue.publish(Integer.valueOf(i)); + // } + // + // System.out.println("Finish produce item"); + // } + // }).start(); + // + // + // new Thread(new Runnable() { + // + // @Override + // public void run() { + // while(true) { + // disruptorQueue.consumeBatchWhenAvailable(new EventHandler() { + // + // @Override + // public void onEvent(Object event, long sequence, + // boolean endOfBatch) throws Exception { + // + // System.out.println("Consumer 1:" + (Integer)event); + // } + // + // }); + // } + // + // } + // }).start(); + // + // new Thread(new Runnable() { + // + // @Override + // public void run() { + // while(true) { + // disruptorQueue.consumeBatchWhenAvailable(new EventHandler() { + // + // @Override + // public void onEvent(Object event, long sequence, + // boolean endOfBatch) throws Exception { + // + // System.out.println("Consumer 2:" + (Integer)event); + // } + // + // }); + // } + // + // } + // }).start(); + // + // JStormUtils.sleepMs(100000); + } + + private final static int TIMEOUT = 5; // MS + private final static int PRODUCER_NUM = 4; + + @Test + public void testLaterStartConsumer() throws InterruptedException { + System.out + .println("!!!!!!!!!!!!!!!Begin testLaterStartConsumer!!!!!!!!!!"); + final AtomicBoolean messageConsumed = new AtomicBoolean(false); + + // Set queue length to 1, so that the RingBuffer can be easily full + // to trigger consumer blocking + DisruptorQueue queue = + createQueue("consumerHang", ProducerType.MULTI, 2); + push(queue, 1); + Runnable producer = new Producer(queue); + Runnable consumer = new Consumer(queue, new EventHandler() { + long count = 0; + + @Override + public void onEvent(Object obj, long sequence, boolean endOfBatch) + throws Exception { + + messageConsumed.set(true); + System.out.println("Consume " + count++); + } + }); + + run(producer, 0, 0, consumer, 50); + Assert.assertTrue( + "disruptor message is never consumed due to consumer thread hangs", + messageConsumed.get()); + + System.out + .println("!!!!!!!!!!!!!!!!End testLaterStartConsumer!!!!!!!!!!"); + } + + @Test + public void testBeforeStartConsumer() throws InterruptedException { + System.out + .println("!!!!!!!!!!!!Begin testBeforeStartConsumer!!!!!!!!!"); + final AtomicBoolean messageConsumed = new AtomicBoolean(false); + + // Set queue length to 1, so that the RingBuffer can be easily full + // to trigger consumer blocking + DisruptorQueue queue = + createQueue("consumerHang", ProducerType.MULTI, 2); + queue.consumerStarted(); + push(queue, 1); + Runnable producer = new Producer(queue); + Runnable consumer = new Consumer(queue, new EventHandler() { + long count = 0; + + @Override + public void onEvent(Object obj, long sequence, boolean endOfBatch) + throws Exception { + + messageConsumed.set(true); + System.out.println("Consume " + count++); + } + }); + + run(producer, 0, 0, consumer, 50); + Assert.assertTrue( + "disruptor message is never consumed due to consumer thread hangs", + messageConsumed.get()); + + System.out + .println("!!!!!!!!!!!!!End testBeforeStartConsumer!!!!!!!!!!"); + } + + @Test + public void testSingleProducer() throws InterruptedException { + System.out + .println("!!!!!!!!!!!!!!Begin testSingleProducer!!!!!!!!!!!!!!"); + final AtomicBoolean messageConsumed = new AtomicBoolean(false); + + // Set queue length to 1, so that the RingBuffer can be easily full + // to trigger consumer blocking + DisruptorQueue queue = + createQueue("consumerHang", ProducerType.SINGLE, 1); + push(queue, 1); + Runnable producer = new Producer(queue); + Runnable consumer = new Consumer(queue, new EventHandler() { + long count = 0; + + @Override + public void onEvent(Object obj, long sequence, boolean endOfBatch) + throws Exception { + + messageConsumed.set(true); + System.out.println("Consume " + count++); + } + }); + + run(producer, 0, 0, consumer, 50); + Assert.assertTrue( + "disruptor message is never consumed due to consumer thread hangs", + messageConsumed.get()); + + System.out + .println("!!!!!!!!!!!!!!End testSingleProducer!!!!!!!!!!!!!!"); + } + + public static AtomicLong produceNum = new AtomicLong(0); + public static AtomicLong consumerNum = new AtomicLong(0); + + public static EventHandlerTest handler = new EventHandlerTest(); + + public static void resetNum() { + produceNum.set(0); + consumerNum.set(0); + handler.reset(); + + } + + @Test + public void testMessageDisorder() throws InterruptedException { + + System.out + .println("!!!!!!!!!!!!!!!!Begin testMessageDisorder!!!!!!!!!!"); + // Set queue length to bigger enough + DisruptorQueue queue = + createQueue("messageOrder", ProducerType.MULTI, 128); + + queue.publish("1"); + + Runnable producer = new Producer(queue); + + final Object[] result = new Object[1]; + Runnable consumer = new Consumer(queue, new EventHandler() { + private boolean head = true; + private Map lastIdMap = new HashMap(); + + @Override + public void onEvent(Object obj, long sequence, boolean endOfBatch) + throws Exception { + consumerNum.incrementAndGet(); + if (head) { + head = false; + result[0] = obj; + } else { + String event = (String) obj; + String[] item = event.split("@"); + Long current = Long.valueOf(item[1]); + Long last = lastIdMap.get(item[0]); + if (last != null) { + if (current <= last) { + String msg = + "Consume disorder of " + item[0] + + ", current" + current + ",last:" + + last; + System.err + .println("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!"); + System.err.println(msg); + System.err + .println("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!"); + Assert.fail(msg); + } + } + + lastIdMap.put(item[0], current); + + } + } + }); + + run(producer, PRODUCER_NUM, 1000, consumer, 30000); + Assert.assertEquals( + "We expect to receive first published message first, but received " + + result[0], "1", result[0]); + produceNum.incrementAndGet(); + Assert.assertEquals("produce: " + produceNum.get() + ", consume:" + + consumerNum.get(), produceNum.get(), consumerNum.get()); + System.out.println("!!!!!!!!!!!!!!End testMessageDisorder!!!!!!!!!!!!"); + } + + @Test + public void testPull() { + // @@@ TODO + } + + @Test + public void testTake() { + // @@@ TODO + } + + public void push(DisruptorQueue queue, int num) { + for (int i = 0; i < num; i++) { + String msg = + String.valueOf(Thread.currentThread().getId()) + "@" + i; + try { + queue.publish(msg, false); + } catch (InsufficientCapacityException e) { + e.printStackTrace(); + } + produceNum.incrementAndGet(); + System.out.println(Thread.currentThread().getId() + + " Publish one :" + i); + } + } + + @Test + public void testConsumeBatchWhenAvailable() { + System.out + .println("!!!!!!!!!!!!!!!Begin testConsumeBatchWhenAvailable!!!!!!!!!!!!"); + + resetNum(); + + // Set queue length to bigger enough + DisruptorQueue queue = + createQueue("messageOrder", ProducerType.MULTI, 128); + + push(queue, 128); + + queue.consumeBatchWhenAvailable(handler); + + Assert.assertEquals("produce: " + produceNum.get() + ", consume:" + + consumerNum.get(), produceNum.get(), consumerNum.get()); + System.out + .println("!!!!!! finish testConsumeBatchWhenAvailable test 1"); + resetNum(); + + queue.consumerStarted(); + + push(queue, 128); + + queue.consumeBatchWhenAvailable(handler); + + Assert.assertEquals("produce: " + produceNum.get() + ", consume:" + + consumerNum.get(), produceNum.get(), consumerNum.get()); + System.out + .println("!!!!!! finish testConsumeBatchWhenAvailable test 2"); + + System.out + .println("!!!!!!!!!!!!!!!Finsh testConsumeBatchWhenAvailable for MULTI!!!!!!!!!!!!"); + + resetNum(); + // Set queue length to bigger enough + DisruptorQueue queue2 = + createQueue("messageOrder", ProducerType.SINGLE, 128); + + push(queue2, 128); + + queue2.consumeBatchWhenAvailable(handler); + + Assert.assertEquals("produce: " + produceNum.get() + ", consume:" + + consumerNum.get(), produceNum.get(), consumerNum.get()); + System.out + .println("!!!!!! finish testConsumeBatchWhenAvailable test 3"); + resetNum(); + + queue2.consumerStarted(); + + push(queue2, 128); + + queue2.consumeBatchWhenAvailable(handler); + + Assert.assertEquals("produce: " + produceNum.get() + ", consume:" + + consumerNum.get(), produceNum.get(), consumerNum.get()); + System.out + .println("!!!!!! finish testConsumeBatchWhenAvailable test 4"); + + System.out + .println("!!!!!!!!!!!!!!!Finsh testConsumeBatchWhenAvailable for single !!!!!!!!!!!!"); + System.out + .println("!!!!!!!!!!!!!End testConsumeBatchWhenAvailable!!!!!!!!!!!"); + } + + @Test + public void testTryConsume() { + System.out.println("!!!!!!!!!!!!Begin testTryConsume!!!!!!!!!!!!!!!!"); + + resetNum(); + // Set queue length to bigger enough + DisruptorQueue queue = + createQueue("messageOrder", ProducerType.MULTI, 128); + + push(queue, 128); + + queue.consumeBatch(handler); + + Assert.assertEquals("produce: " + produceNum.get() + ", consume:" + + consumerNum.get(), produceNum.get(), consumerNum.get()); + System.out.println("!!!!!! finish testTryConsume test 1"); + resetNum(); + + queue.consumerStarted(); + + push(queue, 128); + + queue.consumeBatch(handler); + + Assert.assertEquals("produce: " + produceNum.get() + ", consume:" + + consumerNum.get(), produceNum.get(), consumerNum.get()); + System.out.println("!!!!!! finish testTryConsume test 2"); + + resetNum(); + // Set queue length to bigger enough + DisruptorQueue queue2 = + createQueue("messageOrder", ProducerType.SINGLE, 128); + + push(queue2, 128); + + queue2.consumeBatch(handler); + + Assert.assertEquals("produce: " + produceNum.get() + ", consume:" + + consumerNum.get(), produceNum.get(), consumerNum.get()); + System.out.println("!!!!!! finish testTryConsume test 3"); + resetNum(); + + queue2.consumerStarted(); + + push(queue2, 128); + + queue2.consumeBatch(handler); + + Assert.assertEquals("produce: " + produceNum.get() + ", consume:" + + consumerNum.get(), produceNum.get(), consumerNum.get()); + System.out.println("!!!!!! finish testTryConsume test 4"); + + System.out.println("!!!!!!!!!!!!!!!!!End testTryConsume!!!!!!!!!!!!!!"); + } + + private void run(Runnable producer, int producerNum, long produceMs, + Runnable consumer, long waitMs) { + try { + + resetNum(); + + Thread[] producerThreads = new Thread[producerNum]; + for (int i = 0; i < producerNum; i++) { + producerThreads[i] = new Thread(producer); + producerThreads[i].start(); + } + + Thread consumerThread = new Thread(consumer); + consumerThread.start(); + System.out.println("Please wait seconds" + produceMs / 1000); + + Thread.sleep(produceMs); + + for (int i = 0; i < producerNum; i++) { + producerThreads[i].interrupt(); + producerThreads[i].stop(); + producerThreads[i].join(TIMEOUT); + } + + Thread.sleep(waitMs); + System.out.println("Please wait seconds" + waitMs / 1000); + + consumerThread.interrupt(); + consumerThread.stop(); + consumerThread.join(TIMEOUT); + } catch (Throwable e) { + e.printStackTrace(); + } + } + + private class Producer implements Runnable { + private String msg; + private DisruptorQueue queue; + + Producer(DisruptorQueue queue) { + this.queue = queue; + } + + @Override + public void run() { + long count = 0; + try { + while (true) { + + String msg = + String.valueOf(Thread.currentThread().getId()) + + "@" + count; + queue.publish(msg, false); + produceNum.incrementAndGet(); + System.out.println(msg); + count++; + } + } catch (InsufficientCapacityException e) { + System.out.println(Thread.currentThread().getId() + + " quit, insufficientCapacityException " + count); + return; + } catch (Exception e) { + System.out.println(Thread.currentThread().getId() + + " quit, Exception " + count); + return; + } + } + } + + private class Consumer implements Runnable { + private EventHandler handler; + private DisruptorQueue queue; + + Consumer(DisruptorQueue queue, EventHandler handler) { + this.handler = handler; + this.queue = queue; + } + + @Override + public void run() { + queue.consumerStarted(); + try { + while (true) { + queue.consumeBatchWhenAvailable(handler); + } + } catch (Exception e) { + // break + } + } + } + + static class EventHandlerTest implements EventHandler { + private Map lastIdMap = new HashMap(); + + public void reset() { + lastIdMap.clear(); + } + + @Override + public void onEvent(Object obj, long sequence, boolean endOfBatch) + throws Exception { + + String event = (String) obj; + String[] item = event.split("@"); + Long current = Long.valueOf(item[1]); + Long last = lastIdMap.get(item[0]); + if (last != null) { + if (current <= last) { + String msg = + "Consume disorder of " + item[0] + ", current" + + current + ",last:" + last; + System.err.println("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!"); + System.err.println(msg + "," + event); + System.err.println("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!"); + Assert.fail(msg); + } + } + + lastIdMap.put(item[0], current); + consumerNum.incrementAndGet(); + } + }; + + private static DisruptorQueue createQueue(String name, ProducerType type, + int queueSize) { + + return DisruptorQueue.mkInstance(name, type, queueSize, + new BlockingWaitStrategy()); + } +} diff --git a/jstorm-core/src/test/java/com/alibaba/jstorm/utils/JStormUtilsTest.java b/jstorm-core/src/test/java/com/alibaba/jstorm/utils/JStormUtilsTest.java new file mode 100755 index 000000000..d690c35f5 --- /dev/null +++ b/jstorm-core/src/test/java/com/alibaba/jstorm/utils/JStormUtilsTest.java @@ -0,0 +1,171 @@ +/** + * 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.utils; + +import java.io.BufferedOutputStream; +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class JStormUtilsTest { + private static final Logger LOG = LoggerFactory + .getLogger(JStormUtilsTest.class); + + private static final int RESTART_TIMES = 1000; + + private static String cmd = ""; + + private static String pidDir = "/tmp/pids"; + + private static int killedTimes = 0; + private static int forkTimes = 0; + + public static void testRestartProcess() { + final int intervalSec = Integer.valueOf("10"); + new Thread(new Runnable() { + + @Override + public void run() { + LOG.info("Begin to start fork thread"); + Map environment = new HashMap(); + + // TODO Auto-generated method stub + while (forkTimes < RESTART_TIMES) { + + try { + JStormUtils.launch_process(cmd + " " + forkTimes, + environment, true); + } catch (IOException e) { + // TODO Auto-generated catch block + LOG.error("Failed to fork process " + cmd + forkTimes, + e); + continue; + } + + LOG.info("Successfully launch " + forkTimes); + + JStormUtils.sleepMs(1000); + + forkTimes++; + + } + LOG.info("Successfully shutdown fok thread"); + } + + }).start(); + + new Thread(new Runnable() { + + @Override + public void run() { + LOG.info("Begin to start killing thread"); + + File file = new File(pidDir); + // TODO Auto-generated method stub + while (killedTimes < RESTART_TIMES) { + File[] pids = file.listFiles(); + if (pids == null) { + JStormUtils.sleepMs(100); + continue; + } + + for (File pidFile : pids) { + String pid = pidFile.getName(); + + JStormUtils.ensure_process_killed(Integer.valueOf(pid)); + + killedTimes++; + + pidFile.delete(); + } + + JStormUtils.sleepMs(100); + + } + LOG.info("Successfully shutdown killing thread"); + } + + }).start(); + + while (killedTimes < RESTART_TIMES) { + JStormUtils.sleepMs(100); + } + } + + public static void fillData() { + Map map = new HashMap(); + + for (long l = 0; l < 1000000; l++) { + map.put(l, String.valueOf(l)); + } + } + + public static void testJar(String id) { + try { + + PathUtils.local_mkdirs(pidDir); + } catch (IOException e) { + LOG.error("Failed to rmr " + pidDir, e); + } + + fillData(); + LOG.info("Finish load data"); + + String pid = JStormUtils.process_pid(); + + String pidFile = pidDir + File.separator + pid; + + try { + PathUtils.touch(pidFile); + } catch (IOException e) { + // TODO Auto-generated catch block + LOG.error("Failed to touch " + pidFile, e); + } + try { + + DataOutputStream raf = + new DataOutputStream(new BufferedOutputStream( + new FileOutputStream(new File(pidFile), true))); + + raf.writeBytes(pid); + } catch (Exception e) { + LOG.error("", e); + } + + while (true) { + JStormUtils.sleepMs(1000); + LOG.info(id + " is living"); + } + + } + + public static void main(String[] args) { + if (args.length == 0) { + testRestartProcess(); + } else { + testJar(args[0]); + } + } + +} diff --git a/jstorm-core/src/test/java/com/alibaba/jstorm/utils/TestThrift.java b/jstorm-core/src/test/java/com/alibaba/jstorm/utils/TestThrift.java new file mode 100755 index 000000000..f1afc71fa --- /dev/null +++ b/jstorm-core/src/test/java/com/alibaba/jstorm/utils/TestThrift.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.utils; + +import org.junit.Test; + +import backtype.storm.generated.MetricInfo; + +public class TestThrift { + @Test + public void testEmpty() { + MetricInfo metricInfo = new MetricInfo(); + + System.out.println("base:" + metricInfo.get_baseMetric()); + } +} diff --git a/jstorm-server/bin/.project b/jstorm-server/bin/.project deleted file mode 100644 index 28832e111..000000000 --- a/jstorm-server/bin/.project +++ /dev/null @@ -1,29 +0,0 @@ - - - jstorm-server - - - - - - ccw.builder - - - - - org.eclipse.jdt.core.javabuilder - - - - - org.eclipse.m2e.core.maven2Builder - - - - - - org.eclipse.jdt.core.javanature - org.eclipse.m2e.core.maven2Nature - ccw.nature - - diff --git a/jstorm-server/bin/.settings/org.eclipse.jdt.core.prefs b/jstorm-server/bin/.settings/org.eclipse.jdt.core.prefs deleted file mode 100644 index 60105c1b9..000000000 --- a/jstorm-server/bin/.settings/org.eclipse.jdt.core.prefs +++ /dev/null @@ -1,5 +0,0 @@ -eclipse.preferences.version=1 -org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.6 -org.eclipse.jdt.core.compiler.compliance=1.6 -org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning -org.eclipse.jdt.core.compiler.source=1.6 diff --git a/jstorm-server/bin/.settings/org.eclipse.m2e.core.prefs b/jstorm-server/bin/.settings/org.eclipse.m2e.core.prefs deleted file mode 100644 index f897a7f1c..000000000 --- a/jstorm-server/bin/.settings/org.eclipse.m2e.core.prefs +++ /dev/null @@ -1,4 +0,0 @@ -activeProfiles= -eclipse.preferences.version=1 -resolveWorkspaceProjects=true -version=1 diff --git a/jstorm-server/bin/start.sh b/jstorm-server/bin/start.sh deleted file mode 100644 index db456e6e4..000000000 --- a/jstorm-server/bin/start.sh +++ /dev/null @@ -1,75 +0,0 @@ -#!/bin/sh - -if [ -e ~/.bashrc ] -then - source ~/.bashrc -fi - -if [ -e ~/.bash_profile ] -then - source ~/.bash_profile -fi - -if [ "x$JAVA_HOME" != "x" ] -then - echo "JAVA_HOME has been set " -else - export JAVA_HOME=/opt/taobao/java -fi -echo "JAVA_HOME =" $JAVA_HOME - -if [ "x$JSTORM_HOME" != "x" ] -then - echo "JSTORM_HOME has been set " -else - export JSTORM_HOME=/home/admin/jstorm -fi -echo "JSTORM_HOME =" $JSTORM_HOME - -if [ "x$JSTORM_CONF_DIR" != "x" ] -then - echo "JSTORM_CONF_DIR has been set " -else - export JSTORM_CONF_DIR=$JSTORM_HOME/conf -fi -echo "JSTORM_CONF_DIR =" $JSTORM_CONF_DIR - - - -export PATH=$JAVA_HOME/bin:$JSTORM_HOME/bin:$PATH - - -which java - -if [ $? -eq 0 ] -then - echo "Find java" -else - echo "No java, please install java firstly !!!" - exit 1 -fi - -function startJStorm() -{ - PROCESS=$1 - echo "start $PROCESS" - cd $JSTORM_HOME/bin; nohup $JSTORM_HOME/bin/jstorm $PROCESS >/dev/null 2>&1 & - sleep 4 - rm -rf nohup - ps -ef|grep $2 -} - - - -HOSTNAME=`hostname -i` -NIMBUS_HOST=`grep "nimbus.host:" $JSTORM_CONF_DIR/storm.yaml | grep -w $HOSTNAME` - -if [ "X${NIMBUS_HOST}" != "X" ] -then - startJStorm "nimbus" "NimbusServer" - startJStorm "supervisor" "Supervisor" -else - startJStorm "supervisor" "Supervisor" -fi - -echo "Successfully start jstorm daemon...." diff --git a/jstorm-server/conf/aloha_logback.xml b/jstorm-server/conf/aloha_logback.xml deleted file mode 100644 index adb550a0a..000000000 --- a/jstorm-server/conf/aloha_logback.xml +++ /dev/null @@ -1,14 +0,0 @@ - - - - - - %-4r [%t] %-5p %c - %m%n - - - - - - - - \ No newline at end of file diff --git a/jstorm-server/pom.xml b/jstorm-server/pom.xml deleted file mode 100644 index 713f7408c..000000000 --- a/jstorm-server/pom.xml +++ /dev/null @@ -1,87 +0,0 @@ - - - - - com.alibaba.jstorm - jstorm-all - 0.9.6.3 - .. - - - 4.0.0 - com.alibaba.jstorm - jstorm-server - 0.9.6.3 - jar - ${project.artifactId}-${project.version} - jstorm server modules - - - UTF-8 - - - - - - org.apache.maven.plugins - maven-surefire-plugin - - pertest - -Xms1024m -Xmx3072m - - - - maven-compiler-plugin - 2.3.2 - - 1.6 - 1.6 - - - - org.apache.maven.plugins - maven-source-plugin - - - attach-sources - - jar - - - - - - - - - - com.alibaba.jstorm - jstorm-client - ${project.version} - provided - - - - com.alibaba.jstorm - jstorm-client-extension - ${project.version} - provided - - - junit - junit - 4.10 - test - - - - - - - \ No newline at end of file diff --git a/jstorm-server/src/main/java/backtype/storm/LocalCluster.java b/jstorm-server/src/main/java/backtype/storm/LocalCluster.java deleted file mode 100644 index 303bc11bc..000000000 --- a/jstorm-server/src/main/java/backtype/storm/LocalCluster.java +++ /dev/null @@ -1,238 +0,0 @@ -package backtype.storm; - -import java.util.Enumeration; -import java.util.Map; - -import org.apache.log4j.BasicConfigurator; -import org.apache.log4j.Level; -import org.apache.log4j.Logger; -import org.apache.thrift7.TException; - -import backtype.storm.generated.ClusterSummary; -import backtype.storm.generated.KillOptions; -import backtype.storm.generated.NotAliveException; -import backtype.storm.generated.RebalanceOptions; -import backtype.storm.generated.StormTopology; -import backtype.storm.generated.SubmitOptions; -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() { - 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"); - } - - @Override - public void submitTopology(String topologyName, Map conf, - StormTopology topology) { - submitTopologyWithOpts(topologyName, conf, topology, null); - } - - @Override - public void submitTopologyWithOpts(String topologyName, Map conf, - StormTopology topology, SubmitOptions submitOpts){ - // 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, - Utils.to_json(conf), topology); - }else { - state.getNimbus().submitTopologyWithOpts(topologyName, null, - Utils.to_json(conf), topology, submitOpts); - } - - } catch (Exception e) { - // TODO Auto-generated catch block - LOG.error("Failed to submit topology " + topologyName, e); - throw new RuntimeException(e); - } - } - - @Override - public void killTopology(String topologyName) { - // TODO Auto-generated method stub - try { - // 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); - } - } - - @Override - public void killTopologyWithOpts(String name, KillOptions options) - throws NotAliveException { - // TODO Auto-generated method stub - try { - state.getNimbus().killTopologyWithOpts(name, options); - } catch (TException e) { - // TODO Auto-generated catch block - LOG.error("fail to kill Topology " + name, e); - throw new RuntimeException(e); - } - } - - @Override - public void activate(String topologyName) { - // TODO Auto-generated method stub - try { - state.getNimbus().activate(topologyName); - } catch (Exception e) { - // TODO Auto-generated catch block - LOG.error("fail to activate " + topologyName, e); - throw new RuntimeException(e); - } - } - - @Override - public void deactivate(String topologyName) { - // TODO Auto-generated method stub - try { - state.getNimbus().deactivate(topologyName); - } catch (Exception e) { - // TODO Auto-generated catch block - LOG.error("fail to deactivate " + topologyName, e); - throw new RuntimeException(e); - } - } - - @Override - public void rebalance(String name, RebalanceOptions options){ - // TODO Auto-generated method stub - try { - state.getNimbus().rebalance(name, options); - } catch (Exception e) { - // TODO Auto-generated catch block - LOG.error("fail to rebalance " + name, e); - throw new RuntimeException(e); - } - } - - @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(); - } - - @Override - public String getTopologyConf(String id) { - // TODO Auto-generated method stub - try { - return state.getNimbus().getTopologyConf(id); - } catch (Exception e) { - // TODO Auto-generated catch block - LOG.error("fail to get topology Conf of topologId: " + id, e); - } - return null; - } - - @Override - public StormTopology getTopology(String id) { - // TODO Auto-generated method stub - try { - return state.getNimbus().getTopology(id); - } catch (NotAliveException e) { - // TODO Auto-generated catch block - LOG.error("fail to get topology of topologId: " + id, e); - } catch (TException e) { - // TODO Auto-generated catch block - LOG.error("fail to get topology of topologId: " + id, e); - } - return null; - } - - @Override - public ClusterSummary getClusterInfo() { - // TODO Auto-generated method stub - try { - return state.getNimbus().getClusterInfo(); - } catch (TException e) { - // TODO Auto-generated catch block - LOG.error("fail to get cluster info", e); - } - return null; - } - - @Override - public TopologyInfo getTopologyInfo(String id) { - // TODO Auto-generated method stub - try { - return state.getNimbus().getTopologyInfo(id); - } catch (NotAliveException e) { - // TODO Auto-generated catch block - LOG.error("fail to get topology info of topologyId: " + id, e); - } catch (TException e) { - // TODO Auto-generated catch block - LOG.error("fail to get topology info of topologyId: " + id, e); - } - return null; - } - - /*** - * You should use getLocalClusterMap() to instead.This function will always - * return null - * */ - @Deprecated - @Override - public Map getState() { - // TODO Auto-generated method stub - return null; - } - - public LocalClusterMap getLocalClusterMap() { - return state; - } - - public static void main(String[] args) throws Exception { - LocalCluster localCluster = null; - try { - localCluster = new LocalCluster(); - } finally { - if(localCluster != null) { - localCluster.shutdown(); - } - } - } - -} diff --git a/jstorm-server/src/main/java/backtype/storm/LocalClusterMap.java b/jstorm-server/src/main/java/backtype/storm/LocalClusterMap.java deleted file mode 100644 index 7c29cdbda..000000000 --- a/jstorm-server/src/main/java/backtype/storm/LocalClusterMap.java +++ /dev/null @@ -1,111 +0,0 @@ -package backtype.storm; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; - -import org.apache.log4j.Logger; - -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.daemon.nimbus.NimbusServer; -import com.alibaba.jstorm.daemon.nimbus.ServiceHandler; -import com.alibaba.jstorm.daemon.supervisor.SupervisorManger; -import com.alibaba.jstorm.utils.PathUtils; -import com.alibaba.jstorm.zk.Factory; - -public class LocalClusterMap { - - public static Logger LOG = Logger.getLogger(LocalClusterMap.class); - - private NimbusServer nimbusServer; - - private ServiceHandler nimbus; - - private Factory zookeeper; - - private Map conf; - - private List tmpDir; - - private SupervisorManger supervisor; - - public ServiceHandler getNimbus() { - return nimbus; - } - - public void setNimbus(ServiceHandler nimbus) { - this.nimbus = nimbus; - } - - public Factory getZookeeper() { - return zookeeper; - } - - public void setZookeeper(Factory zookeeper) { - this.zookeeper = zookeeper; - } - - public Map getConf() { - return conf; - } - - public void setConf(Map conf) { - this.conf = conf; - } - - public NimbusServer getNimbusServer() { - return nimbusServer; - } - - public void setNimbusServer(NimbusServer nimbusServer) { - this.nimbusServer = nimbusServer; - } - - public SupervisorManger getSupervisor() { - return supervisor; - } - - public void setSupervisor(SupervisorManger supervisor) { - this.supervisor = supervisor; - } - - public List getTmpDir() { - return tmpDir; - } - - public void setTmpDir(List tmpDir) { - this.tmpDir = tmpDir; - } - - public void clean() { - - if (supervisor != null) { - supervisor.ShutdownAllWorkers(); - supervisor.shutdown(); - } - - if (nimbusServer != null) { - nimbusServer.cleanup(); - } - - if (zookeeper != null) - zookeeper.shutdown(); - - // it will hava a problem: - // java.io.IOException: Unable to delete file: - // {TmpPath}\{UUID}\version-2\log.1 - if (tmpDir != null) { - for (String dir : tmpDir) { - try { - PathUtils.rmr(dir); - } catch (IOException e) { - // TODO Auto-generated catch block - LOG.error("Fail to delete " + dir); - } - } - } - } - -} diff --git a/jstorm-server/src/main/java/backtype/storm/LocalDRPC.java b/jstorm-server/src/main/java/backtype/storm/LocalDRPC.java deleted file mode 100644 index 023524fdf..000000000 --- a/jstorm-server/src/main/java/backtype/storm/LocalDRPC.java +++ /dev/null @@ -1,87 +0,0 @@ -package backtype.storm; - - -import org.apache.log4j.Logger; -import org.apache.thrift7.TException; - -import backtype.storm.ILocalDRPC; -import backtype.storm.generated.DRPCExecutionException; -import backtype.storm.generated.DRPCRequest; -import backtype.storm.utils.ServiceRegistry; - -import com.alibaba.jstorm.drpc.Drpc; - -public class LocalDRPC implements ILocalDRPC { - private static final Logger LOG = Logger.getLogger(LocalDRPC.class); - - private Drpc handler = new Drpc(); - private Thread thread; - - private final String serviceId; - - public LocalDRPC() { - - - thread = new Thread(new Runnable() { - - @Override - public void run() { - LOG.info("Begin to init local Drpc"); - try { - handler.init(); - } catch (Exception e) { - LOG.info("Failed to start local drpc"); - System.exit(-1); - } - LOG.info("Successfully start local drpc"); - } - }); - thread.start(); - - serviceId = ServiceRegistry.registerService(handler); - } - - @Override - public String execute(String functionName, String funcArgs) - { - // TODO Auto-generated method stub - try { - return handler.execute(functionName, funcArgs); - } catch (Exception e) { - LOG.error("", e); - throw new RuntimeException(e); - } - } - - @Override - public void result(String id, String result) throws TException { - // TODO Auto-generated method stub - handler.result(id, result); - } - - @Override - public DRPCRequest fetchRequest(String functionName) throws TException { - // TODO Auto-generated method stub - return handler.fetchRequest(functionName); - } - - @Override - public void failRequest(String id) throws TException { - // TODO Auto-generated method stub - handler.failRequest(id); - } - - @Override - public void shutdown() { - // TODO Auto-generated method stub - ServiceRegistry.unregisterService(this.serviceId); - this.handler.shutdown(); - } - - @Override - public String getServiceId() { - // TODO Auto-generated method stub - return serviceId; - } - -} diff --git a/jstorm-server/src/main/java/backtype/storm/LocalUtils.java b/jstorm-server/src/main/java/backtype/storm/LocalUtils.java deleted file mode 100644 index 30b6daa12..000000000 --- a/jstorm-server/src/main/java/backtype/storm/LocalUtils.java +++ /dev/null @@ -1,114 +0,0 @@ -package backtype.storm; - -import java.io.File; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; -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; -import com.alibaba.jstorm.message.netty.NettyContext; -import com.alibaba.jstorm.zk.Factory; -import com.alibaba.jstorm.zk.Zookeeper; - -public class LocalUtils { - - public static Logger LOG = Logger.getLogger(LocalUtils.class); - - public static LocalClusterMap prepareLocalCluster() { - LocalClusterMap state = new LocalClusterMap(); - try { - List tmpDirs = new ArrayList(); - - String zkDir = getTmpDir(); - tmpDirs.add(zkDir); - Factory zookeeper = startLocalZookeeper(zkDir); - Map conf = getLocalConf(zookeeper.getZooKeeperServer() - .getClientPort()); - - String nimbusDir = getTmpDir(); - tmpDirs.add(nimbusDir); - Map nimbusConf = deepCopyMap(conf); - nimbusConf.put(Config.STORM_LOCAL_DIR, nimbusDir); - NimbusServer instance = new NimbusServer(); - - Map supervisorConf = deepCopyMap(conf); - String supervisorDir = getTmpDir(); - tmpDirs.add(supervisorDir); - supervisorConf.put(Config.STORM_LOCAL_DIR, supervisorDir); - Supervisor supervisor = new Supervisor(); - IContext context = getLocalContext(supervisorConf); - - state.setNimbusServer(instance); - state.setNimbus(instance.launcherLocalServer(nimbusConf, - new DefaultInimbus())); - state.setZookeeper(zookeeper); - state.setConf(conf); - state.setTmpDir(tmpDirs); - state.setSupervisor(supervisor - .mkSupervisor(supervisorConf, context)); - return state; - } catch (Exception e) { - LOG.error("prepare cluster error!", e); - state.clean(); - - } - return null; - } - - private static Factory startLocalZookeeper(String tmpDir) { - for (int i = 2000; i < 65535; i++) { - try { - return Zookeeper.mkInprocessZookeeper(tmpDir, i); - } catch (Exception e) { - LOG.error("fail to launch zookeeper at port: " + i, e); - } - } - throw new RuntimeException( - "No port is available to launch an inprocess zookeeper."); - } - - private static String getTmpDir() { - return System.getProperty("java.io.tmpdir") + File.separator - + UUID.randomUUID(); - } - - private static Map getLocalConf(int port) { - List zkServers = new ArrayList(1); - zkServers.add("localhost"); - Map conf = Utils.readStormConfig(); - conf.put(Config.STORM_CLUSTER_MODE, "local"); - conf.put(Config.STORM_ZOOKEEPER_SERVERS, zkServers); - conf.put(Config.STORM_ZOOKEEPER_PORT, port); - conf.put(Config.TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS, true); - 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; - } - - private static IContext getLocalContext(Map conf) { - if (!(Boolean) conf.get(Config.STORM_LOCAL_MODE_ZMQ)) { - IContext result = new NettyContext(); - result.prepare(conf); - return result; - } - return null; - } - - private static Map deepCopyMap(Map map) { - return new HashMap(map); - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/ActiveTransitionCallback.java b/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/ActiveTransitionCallback.java deleted file mode 100644 index 269f1ae70..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/ActiveTransitionCallback.java +++ /dev/null @@ -1,19 +0,0 @@ -package com.alibaba.jstorm.callback.impl; - -import com.alibaba.jstorm.callback.BaseCallback; -import com.alibaba.jstorm.cluster.StormStatus; -import com.alibaba.jstorm.daemon.nimbus.StatusType; - -/** - * Set the topology status as Active - * - */ -public class ActiveTransitionCallback extends BaseCallback { - - @Override - public Object execute(T... args) { - - return new StormStatus(StatusType.active); - } - -} 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 deleted file mode 100644 index ed82071e4..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/DelayStatusTransitionCallback.java +++ /dev/null @@ -1,96 +0,0 @@ -package com.alibaba.jstorm.callback.impl; - -import java.io.IOException; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; -import clojure.lang.IFn.OLD; - -import com.alibaba.jstorm.callback.BaseCallback; -import com.alibaba.jstorm.cluster.StormConfig; -import com.alibaba.jstorm.cluster.StormStatus; -import com.alibaba.jstorm.daemon.nimbus.NimbusData; -import com.alibaba.jstorm.daemon.nimbus.StatusType; -import com.alibaba.jstorm.schedule.DelayEventRunnable; -import com.alibaba.jstorm.utils.JStormUtils; - -/** - * - * - * The action when nimbus receive kill command 1. set the topology status as - * target 2. wait 2 * Timeout seconds later, do removing topology from ZK - * - * @author Longda - */ -public class DelayStatusTransitionCallback extends BaseCallback { - - private static Logger LOG = Logger.getLogger(DelayStatusTransitionCallback.class); - - public static final int DEFAULT_DELAY_SECONDS = 30; - - protected NimbusData data; - protected String topologyid; - protected StormStatus oldStatus; - protected StatusType newType; - protected StatusType nextAction; - - - public DelayStatusTransitionCallback(NimbusData data, - String topologyid, - StormStatus oldStatus, - StatusType newType, - StatusType nextAction) { - this.data = data; - this.topologyid = topologyid; - this.oldStatus = oldStatus; - this.newType = newType; - this.nextAction = nextAction; - } - - public int getDelaySeconds(Object[] args) { - if (oldStatus != null && oldStatus.getDelaySecs() > 0) { - return oldStatus.getDelaySecs(); - } - - Integer delaySecs = DelayStatusTransitionCallback.DEFAULT_DELAY_SECONDS; - if (args == null || args.length == 0 || args[0] == null) { - Map map = null; - try { - - map = StormConfig.read_nimbus_topology_conf(data.getConf(), - topologyid); - delaySecs = JStormUtils.parseInt( - map.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS), - DEFAULT_DELAY_SECONDS); - } catch (Exception e) { - LOG.info("Failed to get topology configuration " + topologyid); - } - - } else { - delaySecs = JStormUtils.parseInt(args[0]); - } - - if (delaySecs == null || delaySecs < 0) { - delaySecs = DelayStatusTransitionCallback.DEFAULT_DELAY_SECONDS; - } - - return delaySecs; - } - - @Override - public Object execute(T... args) { - int delaySecs = getDelaySeconds(args); - LOG.info("Delaying event " + newType + " for " - + delaySecs + " secs for " + topologyid); - - data.getScheduExec().schedule( - new DelayEventRunnable(data, topologyid, nextAction), - delaySecs, TimeUnit.SECONDS); - - return new StormStatus(delaySecs, newType); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/DoRebalanceTransitionCallback.java b/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/DoRebalanceTransitionCallback.java deleted file mode 100644 index bbe40d283..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/DoRebalanceTransitionCallback.java +++ /dev/null @@ -1,54 +0,0 @@ -package com.alibaba.jstorm.callback.impl; - -import org.apache.log4j.Logger; - -import com.alibaba.jstorm.callback.BaseCallback; -import com.alibaba.jstorm.cluster.StormStatus; -import com.alibaba.jstorm.daemon.nimbus.NimbusData; -import com.alibaba.jstorm.daemon.nimbus.TopologyAssign; -import com.alibaba.jstorm.daemon.nimbus.TopologyAssignEvent; - -/** - * Do real rebalance action. - * - * After nimbus receive one rebalance command, it will do as following: 1. set - * topology status as rebalancing 2. delay 2 * timeout seconds 3. do this - * callback - * - * @author Xin.Li/Longda - * - */ -public class DoRebalanceTransitionCallback extends BaseCallback { - - private static Logger LOG = Logger - .getLogger(DoRebalanceTransitionCallback.class); - - private String topologyid; - private StormStatus oldStatus; - - public DoRebalanceTransitionCallback(NimbusData data, String topologyid, - StormStatus status) { - // this.data = data; - this.topologyid = topologyid; - this.oldStatus = status; - } - - @Override - public Object execute(T... args) { - try { - TopologyAssignEvent event = new TopologyAssignEvent(); - - event.setTopologyId(topologyid); - event.setScratch(true); - event.setOldStatus(oldStatus); - - TopologyAssign.push(event); - - } catch (Exception e) { - LOG.error("do-rebalance error!", e); - } - // FIXME Why oldStatus? - return oldStatus.getOldStatus(); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/InactiveTransitionCallback.java b/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/InactiveTransitionCallback.java deleted file mode 100644 index c2defc308..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/InactiveTransitionCallback.java +++ /dev/null @@ -1,21 +0,0 @@ -package com.alibaba.jstorm.callback.impl; - -import com.alibaba.jstorm.callback.BaseCallback; -import com.alibaba.jstorm.cluster.StormStatus; -import com.alibaba.jstorm.daemon.nimbus.StatusType; - -/** - * - * set Topology status as inactive - * - * Here just return inactive status Later, it will set inactive status to ZK - */ -public class InactiveTransitionCallback extends BaseCallback { - - @Override - public Object execute(T... args) { - - return new StormStatus(StatusType.inactive); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/KillTransitionCallback.java b/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/KillTransitionCallback.java deleted file mode 100644 index 40a98d1a2..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/KillTransitionCallback.java +++ /dev/null @@ -1,21 +0,0 @@ -package com.alibaba.jstorm.callback.impl; - -import com.alibaba.jstorm.daemon.nimbus.NimbusData; -import com.alibaba.jstorm.daemon.nimbus.StatusType; - -/** - * The action when nimbus receive killed command. - * - * 1. change current topology status as killed 2. one TIMEOUT seconds later, do - * remove action, which remove topology from ZK - * - * @author Longda - * - */ -public class KillTransitionCallback extends DelayStatusTransitionCallback { - - public KillTransitionCallback(NimbusData data, String topologyid) { - super(data, topologyid, null, StatusType.killed, StatusType.remove); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/ReassignTransitionCallback.java b/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/ReassignTransitionCallback.java deleted file mode 100644 index afb620c53..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/ReassignTransitionCallback.java +++ /dev/null @@ -1,49 +0,0 @@ -package com.alibaba.jstorm.callback.impl; - -import com.alibaba.jstorm.callback.BaseCallback; -import com.alibaba.jstorm.cluster.StormStatus; -import com.alibaba.jstorm.daemon.nimbus.NimbusData; -import com.alibaba.jstorm.daemon.nimbus.TopologyAssign; -import com.alibaba.jstorm.daemon.nimbus.TopologyAssignEvent; - -/** - * 1. every Config.NIMBUS_MONITOR_FREQ_SECS will call MonitorRunnable 2. - * MonitorRunnable will call NimbusData.transition 3. NimbusData.transition will - * this callback - * - * - */ -public class ReassignTransitionCallback extends BaseCallback { - - private NimbusData data; - private String topologyid; - private StormStatus oldStatus; - - public ReassignTransitionCallback(NimbusData data, String topologyid) { - this.data = data; - this.topologyid = topologyid; - this.oldStatus = null; - } - - public ReassignTransitionCallback(NimbusData data, String topologyid, - StormStatus oldStatus) { - this.data = data; - this.topologyid = topologyid; - this.oldStatus = oldStatus; - } - - @Override - public Object execute(T... args) { - - // default is true - TopologyAssignEvent assignEvent = new TopologyAssignEvent(); - assignEvent.setTopologyId(topologyid); - assignEvent.setScratch(false); - assignEvent.setOldStatus(oldStatus); - - TopologyAssign.push(assignEvent); - - return null; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/RebalanceTransitionCallback.java b/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/RebalanceTransitionCallback.java deleted file mode 100644 index d5d3303ac..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/RebalanceTransitionCallback.java +++ /dev/null @@ -1,27 +0,0 @@ -package com.alibaba.jstorm.callback.impl; - -import com.alibaba.jstorm.cluster.StormStatus; -import com.alibaba.jstorm.daemon.nimbus.NimbusData; -import com.alibaba.jstorm.daemon.nimbus.StatusType; - -/** - * The action when nimbus receive rebalance command. Rebalance command is only - * valid when current status is active - * - * 1. change current topology status as rebalancing 2. do_rebalance action after - * 2 * TIMEOUT seconds - * - * @author Lixin/Longda - * - */ -public class RebalanceTransitionCallback extends DelayStatusTransitionCallback { - - - public RebalanceTransitionCallback(NimbusData data, String topologyid, - StormStatus status) { - super(data, topologyid, status, StatusType.rebalancing, StatusType.do_rebalance); - } - - - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/RemoveTransitionCallback.java b/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/RemoveTransitionCallback.java deleted file mode 100644 index 3b64fc92d..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/callback/impl/RemoveTransitionCallback.java +++ /dev/null @@ -1,51 +0,0 @@ -package com.alibaba.jstorm.callback.impl; - -import org.apache.log4j.Logger; - -import com.alibaba.jstorm.callback.BaseCallback; -import com.alibaba.jstorm.cluster.StormBase; -import com.alibaba.jstorm.daemon.nimbus.NimbusData; - -/** - * Remove topology /ZK-DIR/topology data - * - * remove this ZK node will trigger watch on this topology - * - * And Monitor thread every 10 seconds will clean these disappear topology - * - */ -public class RemoveTransitionCallback extends BaseCallback { - - private static Logger LOG = Logger - .getLogger(RemoveTransitionCallback.class); - - private NimbusData data; - private String topologyid; - - public RemoveTransitionCallback(NimbusData data, String topologyid) { - this.data = data; - this.topologyid = topologyid; - } - - @Override - public Object execute(T... args) { - LOG.info("Begin to remove topology: " + topologyid); - try { - - StormBase stormBase = data.getStormClusterState().storm_base( - topologyid, null); - if (stormBase == null) { - LOG.info("Topology " + topologyid + " has been removed "); - return null; - } - data.getStormClusterState().remove_storm(topologyid); - LOG.info("Successfully removed ZK items topology: " + topologyid); - - } catch (Exception e) { - // TODO Auto-generated catch block - LOG.warn("Failed to remove StormBase " + topologyid + " from ZK", e); - } - return null; - } - -} 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 deleted file mode 100644 index 53e2cf815..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/Cluster.java +++ /dev/null @@ -1,339 +0,0 @@ -package com.alibaba.jstorm.cluster; - -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; - -import org.apache.log4j.Logger; - -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; -import com.alibaba.jstorm.task.Assignment; -import com.alibaba.jstorm.task.TaskInfo; - -/** - * storm operation ZK - * - * @author yannian/zhiyuan.ls - * - */ -public class Cluster { - - //TODO Need Migrate constants to ZkConstant - - private static Logger LOG = Logger.getLogger(Cluster.class); - - public static final String ZK_SEPERATOR = "/"; - - public static final String ASSIGNMENTS_ROOT = "assignments"; - public static final String ASSIGNMENTS_BAK = "assignments_bak"; - public static final String TASKS_ROOT = "tasks"; - public static final String CODE_ROOT = "code"; - public static final String STORMS_ROOT = "topology"; - public static final String SUPERVISORS_ROOT = "supervisors"; - public static final String TASKBEATS_ROOT = "taskbeats"; - public static final String TASKERRORS_ROOT = "taskerrors"; - public static final String MASTER_ROOT = "nimbus_master"; - public static final String MONITOR_ROOT = "monitor"; - - public static final String STATUS_DIR = "status"; - public static final String TASK_DIR = "task"; - public static final String WORKER_DIR = "worker"; - public static final String USER_DIR = "user"; - - public static final String LAST_ERROR = "last_error"; - - public static final String ASSIGNMENTS_SUBTREE; - public static final String TASKS_SUBTREE; - public static final String STORMS_SUBTREE; - public static final String SUPERVISORS_SUBTREE; - public static final String TASKBEATS_SUBTREE; - public static final String TASKERRORS_SUBTREE; - public static final String MASTER_SUBTREE; - public static final String MONITOR_SUBTREE; - - static { - ASSIGNMENTS_SUBTREE = ZK_SEPERATOR + ASSIGNMENTS_ROOT; - TASKS_SUBTREE = ZK_SEPERATOR + TASKS_ROOT; - STORMS_SUBTREE = ZK_SEPERATOR + STORMS_ROOT; - SUPERVISORS_SUBTREE = ZK_SEPERATOR + SUPERVISORS_ROOT; - TASKBEATS_SUBTREE = ZK_SEPERATOR + TASKBEATS_ROOT; - TASKERRORS_SUBTREE = ZK_SEPERATOR + TASKERRORS_ROOT; - MASTER_SUBTREE = ZK_SEPERATOR + MASTER_ROOT; - MONITOR_SUBTREE = ZK_SEPERATOR + MONITOR_ROOT; - } - - public static String supervisor_path(String id) { - return SUPERVISORS_SUBTREE + ZK_SEPERATOR + id; - } - - public static String assignment_path(String id) { - return ASSIGNMENTS_SUBTREE + ZK_SEPERATOR + id; - } - - public static String storm_path(String id) { - return STORMS_SUBTREE + ZK_SEPERATOR + id; - } - - public static String storm_task_root(String topology_id) { - return TASKS_SUBTREE + ZK_SEPERATOR + topology_id; - } - - public static String task_path(String topology_id, int task_id) { - return storm_task_root(topology_id) + ZK_SEPERATOR + task_id; - } - - public static String taskbeat_storm_root(String topology_id) { - return TASKBEATS_SUBTREE + ZK_SEPERATOR + topology_id; - } - - public static String taskbeat_path(String topology_id, int task_id) { - return taskbeat_storm_root(topology_id) + ZK_SEPERATOR + task_id; - } - - public static String taskerror_storm_root(String topology_id) { - return TASKERRORS_SUBTREE + ZK_SEPERATOR + topology_id; - } - - public static String lasterror_path(String topology_id) { - return taskerror_storm_root(topology_id) + ZK_SEPERATOR + LAST_ERROR; - } - - public static String taskerror_path(String topology_id, int task_id) { - return taskerror_storm_root(topology_id) + ZK_SEPERATOR + task_id; - } - - public static String monitor_path(String topology_id) { - return MONITOR_SUBTREE + ZK_SEPERATOR + topology_id; - } - - public static String monitor_status_path(String topology_id) { - return monitor_path(topology_id) + ZK_SEPERATOR + STATUS_DIR; - } - - public static String monitor_taskdir_path(String topology_id) { - return monitor_path(topology_id) + ZK_SEPERATOR + TASK_DIR; - } - - public static String monitor_workerdir_path(String topology_id) { - return monitor_path(topology_id) + ZK_SEPERATOR + WORKER_DIR; - } - - public static String monitor_userdir_path(String topology_id) { - return monitor_path(topology_id) + ZK_SEPERATOR + USER_DIR; - } - - public static String monitor_task_path(String topology_id, String task_id) { - return monitor_taskdir_path(topology_id) + ZK_SEPERATOR + task_id; - } - - public static String monitor_worker_path(String topology_id, String worker_id) { - return monitor_workerdir_path(topology_id) + ZK_SEPERATOR + worker_id; - } - - public static String monitor_user_path(String topology_id, String worker_id) { - return monitor_userdir_path(topology_id) + ZK_SEPERATOR + worker_id; - } - - public static Object maybe_deserialize(byte[] data) { - if (data == null) { - return null; - } - return Utils.deserialize(data, null); - } - - @SuppressWarnings("rawtypes") - public static StormClusterState mk_storm_cluster_state( - Map cluster_state_spec) throws Exception { - return new StormZkClusterState(cluster_state_spec); - } - - public static StormClusterState mk_storm_cluster_state( - ClusterState cluster_state_spec) throws Exception { - return new StormZkClusterState(cluster_state_spec); - } - - /** - * return Map - * - * @param zkCluster - * @param topology_id - * @return - * @throws Exception - */ - public static HashMap topology_task_info( - StormClusterState zkCluster, String topology_id) throws Exception { - HashMap rtn = new HashMap(); - - List taks_ids = zkCluster.task_ids(topology_id); - - for (Integer task : taks_ids) { - TaskInfo info = zkCluster.task_info(topology_id, task); - if (info == null) { - LOG.error("Failed to get TaskInfo of " + topology_id - + ",taskid:" + task); - continue; - } - String componentId = info.getComponentId(); - rtn.put(task, componentId); - } - - return rtn; - } - - /** - * return Map - * - * @param zkCluster - * @param topology_id - * @return - * @throws Exception - */ - public static HashMap topology_task_compType( - StormClusterState zkCluster, String topology_id) throws Exception { - HashMap rtn = new HashMap(); - - List taks_ids = zkCluster.task_ids(topology_id); - - for (Integer task : taks_ids) { - TaskInfo info = zkCluster.task_info(topology_id, task); - if (info == null) { - LOG.error("Failed to get TaskInfo of " + topology_id - + ",taskid:" + task); - continue; - } - String componentType = info.getComponentType(); - rtn.put(task, componentType); - } - - return rtn; - } - - /** - * if one topology's name equal the input storm_name, then return the - * topology id, otherwise return null - * - * @param zkCluster - * @param storm_name - * @return - * @throws Exception - */ - public static String get_topology_id(StormClusterState zkCluster, - String storm_name) throws Exception { - List active_storms = zkCluster.active_storms(); - String rtn = null; - if (active_storms != null) { - for (String topology_id : active_storms) { - if (topology_id.indexOf(storm_name) < 0) { - continue; - } - - StormBase base = zkCluster.storm_base(topology_id, null); - if (base != null && storm_name.equals(base.getStormName())) { - rtn = topology_id; - break; - } - } - } - return rtn; - } - - /** - * get all topology's StormBase - * - * @param zkCluster - * @return - * @throws Exception - */ - public static HashMap topology_bases( - StormClusterState zkCluster) throws Exception { - return get_topology_id(zkCluster); - } - - public static HashMap get_topology_id( - StormClusterState zkCluster) throws Exception { - HashMap rtn = new HashMap(); - List active_storms = zkCluster.active_storms(); - if (active_storms != null) { - for (String topology_id : active_storms) { - StormBase base = zkCluster.storm_base(topology_id, null); - if (base != null) { - rtn.put(topology_id, base); - } - } - } - return rtn; - } - - /** - * get all SupervisorInfo of storm cluster - * - * @param stormClusterState - * @param callback - * @return Map String: supervisorId SupervisorInfo: - * [time-secs hostname worker-ports uptime-secs] - * @throws Exception - */ - public static Map allSupervisorInfo( - StormClusterState stormClusterState, RunnableCallback callback) - throws Exception { - - Map rtn = new TreeMap(); - // get /ZK/supervisors - List supervisorIds = stormClusterState.supervisors(callback); - if (supervisorIds != null) { - for (Iterator iter = supervisorIds.iterator(); iter - .hasNext();) { - - String supervisorId = iter.next(); - // get /supervisors/supervisorid - SupervisorInfo supervisorInfo = stormClusterState - .supervisor_info(supervisorId); - if (supervisorInfo == null) { - LOG.warn("Failed to get SupervisorInfo of " + supervisorId); - } else { - - rtn.put(supervisorId, supervisorInfo); - } - } - } else { - LOG.info("No alive supervisor"); - } - - return rtn; - } - - public static Map get_all_assignment( - StormClusterState stormClusterState, RunnableCallback callback) - throws Exception { - Map ret = new HashMap(); - - // get /assignments {topology_id} - List assignments = stormClusterState.assignments(callback); - if (assignments == null) { - LOG.debug("No assignment of ZK"); - return ret; - } - - for (String topology_id : assignments) { - - Assignment assignment = stormClusterState.assignment_info( - topology_id, callback); - - if (assignment == null) { - LOG.error("Failed to get Assignment of " + topology_id - + " from ZK"); - continue; - } - - ret.put(topology_id, assignment); - } - - return ret; - } - -} 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 deleted file mode 100644 index 3ab090399..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/Common.java +++ /dev/null @@ -1,691 +0,0 @@ -package com.alibaba.jstorm.cluster; - -import java.net.URLClassLoader; -import java.security.InvalidParameterException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -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; -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.generated.Grouping; -import backtype.storm.generated.InvalidTopologyException; -import backtype.storm.generated.JavaObject; -import backtype.storm.generated.ShellComponent; -import backtype.storm.generated.SpoutSpec; -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; -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Fields; -import backtype.storm.utils.ThriftTopologyUtils; -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.daemon.worker.WorkerData; -import com.alibaba.jstorm.schedule.default_assign.DefaultTopologyAssignContext; -import com.alibaba.jstorm.task.acker.Acker; -import com.alibaba.jstorm.task.group.MkGrouper; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.Thrift; -import com.alibaba.jstorm.utils.TimeUtils; -import com.google.common.collect.Maps; - -/** - * Base utility function - * - * 1. base topology validation 2. add streams/inputs - * - * @author yannian/Longda - * - */ -public class Common { - public static final String ACKER_COMPONENT_ID = Acker.ACKER_COMPONENT_ID; - public static final String ACKER_INIT_STREAM_ID = Acker.ACKER_INIT_STREAM_ID; - public static final String ACKER_ACK_STREAM_ID = Acker.ACKER_ACK_STREAM_ID; - public static final String ACKER_FAIL_STREAM_ID = Acker.ACKER_FAIL_STREAM_ID; - - public static final String SYSTEM_STREAM_ID = "__system"; - - public static final String LS_WORKER_HEARTBEAT = "worker-heartbeat"; - public static final String LS_ID = "supervisor-id"; - public static final String LS_LOCAL_ASSIGNMENTS = "local-assignments"; - public static final String LS_APPROVED_WORKERS = "approved-workers"; - - public static final String compErrorInfo = "ID can only contains a-z, A-Z, 0-9, '-', '_', '.', '$', and should not start with \"__\"."; - public static final String nameErrorInfo = "Name can only contains a-z, A-Z, 0-9, '-', '_', '.'"; - - public static boolean system_id(String id) { - return Utils.isSystemId(id); - } - - private static void validate_component(Object obj) - throws InvalidTopologyException { - - if (obj instanceof StateSpoutSpec) { - StateSpoutSpec spec = (StateSpoutSpec) obj; - for (String id : spec.get_common().get_streams().keySet()) { - if (system_id(id) || !charComponentValidate(id)) { - throw new InvalidTopologyException(id - + " is not a valid component id. " + compErrorInfo); - } - } - - } else if (obj instanceof SpoutSpec) { - SpoutSpec spec = (SpoutSpec) obj; - for (String id : spec.get_common().get_streams().keySet()) { - if (system_id(id) || !charComponentValidate(id)) { - throw new InvalidTopologyException(id - + " is not a valid component id. " + compErrorInfo); - } - } - } else if (obj instanceof Bolt) { - Bolt spec = (Bolt) obj; - for (String id : spec.get_common().get_streams().keySet()) { - if (system_id(id) || !charComponentValidate(id)) { - throw new InvalidTopologyException(id - + " is not a valid component id. " + compErrorInfo); - } - } - } else { - throw new InvalidTopologyException("Unknow type component"); - } - - } - - public static String TopologyNameToId(String topologyName, int counter) { - return topologyName + "-" + counter + "-" + TimeUtils.current_time_secs(); - } - - /** - * Convert topologyId to topologyName. TopologyId = topoloygName-counter-timeStamp - * @param topologyId - * @return - */ - public static String TopologyIdToName(String topologyId) throws InvalidTopologyException { - String ret = null; - int index = topologyId.lastIndexOf('-'); - if (index != -1 && index > 2) { - index = topologyId.lastIndexOf('-', index - 1); - if (index != -1 && index > 0) - ret = topologyId.substring(0, index); - else - throw new InvalidTopologyException(topologyId + " is not a valid topologyId"); - } else - throw new InvalidTopologyException(topologyId + " is not a valid topologyId"); - return ret; - } - - /** - * Validation of topology name chars. Only alpha char, number, '-', '_', '.' are valid. - * @return - */ - public static boolean charValidate(String name) { - return name.matches("[a-zA-Z0-9-_.]+"); - } - - /** - * Validation of topology component chars. Only alpha char, number, '-', '_', '.', '$' are valid. - * @return - */ - public static boolean charComponentValidate(String name) { - return name.matches("[a-zA-Z0-9-_/.$]+"); - } - - /** - * Check Whether ID of Bolt or spout is system_id - * - * @param topology - * @throws InvalidTopologyException - */ - @SuppressWarnings("unchecked") - public static void validate_ids(StormTopology topology, String topologyId) - throws InvalidTopologyException { - String topologyName = TopologyIdToName(topologyId); - if (!charValidate(topologyName)) { - throw new InvalidTopologyException(topologyName + - " is not a valid topology name. " + nameErrorInfo); - } - - List list = new ArrayList(); - - for (StormTopology._Fields field : Thrift.STORM_TOPOLOGY_FIELDS) { - Object value = topology.getFieldValue(field); - if (value != null) { - Map obj_map = (Map) value; - - Set commids = obj_map.keySet(); - - for (String id : commids) { - if (system_id(id) || !charComponentValidate(id)) { - throw new InvalidTopologyException(id - + " is not a valid component id. " + compErrorInfo); - } - } - - for (Object obj : obj_map.values()) { - validate_component(obj); - } - - list.addAll(commids); - } - } - - List offending = JStormUtils.getRepeat(list); - if (offending.isEmpty() == false) { - throw new InvalidTopologyException("Duplicate component ids: " - + offending); - } - - } - - private static void validate_component_inputs(Object obj) - throws InvalidTopologyException { - if (obj instanceof StateSpoutSpec) { - StateSpoutSpec spec = (StateSpoutSpec) obj; - if (!spec.get_common().get_inputs().isEmpty()) { - throw new InvalidTopologyException( - "May not declare inputs for a spout"); - } - - } - - if (obj instanceof SpoutSpec) { - SpoutSpec spec = (SpoutSpec) obj; - if (!spec.get_common().get_inputs().isEmpty()) { - throw new InvalidTopologyException( - "May not declare inputs for a spout"); - } - } - } - - /** - * Validate the topology 1. component id name is valid or not 2. check some - * spout's input is empty or not - * - * @param topology - * @throws InvalidTopologyException - */ - public static void validate_basic(StormTopology topology, - Map totalStormConf, String topologyid) - throws InvalidTopologyException { - validate_ids(topology, topologyid); - - for (StormTopology._Fields field : Thrift.SPOUT_FIELDS) { - Object value = topology.getFieldValue(field); - if (value != null) { - Map obj_map = (Map) value; - for (Object obj : obj_map.values()) { - validate_component_inputs(obj); - } - } - - } - - Integer workerNum = JStormUtils.parseInt(totalStormConf - .get(Config.TOPOLOGY_WORKERS)); - if (workerNum == null || workerNum <= 0) { - String errMsg = "There are no Config.TOPOLOGY_WORKERS in configuration of " - + topologyid; - throw new InvalidParameterException(errMsg); - } - - Integer ackerNum = JStormUtils.parseInt(totalStormConf - .get(Config.TOPOLOGY_ACKER_EXECUTORS)); - if (ackerNum != null && ackerNum < 0) { - String errMsg = "Invalide Config.TOPOLOGY_ACKERS in configuration of " - + topologyid; - throw new InvalidParameterException(errMsg); - } - - } - - /** - * Generate acker's input Map - * - * for spout for bolt - * - * - * - * @param topology - * @return - */ - public static Map acker_inputs( - StormTopology topology) { - Map spout_inputs = new HashMap(); - Map spout_ids = topology.get_spouts(); - for (Entry spout : spout_ids.entrySet()) { - String id = spout.getKey(); - - GlobalStreamId stream = new GlobalStreamId(id, ACKER_INIT_STREAM_ID); - - Grouping group = Thrift.mkFieldsGrouping(JStormUtils.mk_list("id")); - - spout_inputs.put(stream, group); - } - - Map bolt_ids = topology.get_bolts(); - Map bolt_inputs = new HashMap(); - for (Entry bolt : bolt_ids.entrySet()) { - String id = bolt.getKey(); - - GlobalStreamId streamAck = new GlobalStreamId(id, - ACKER_ACK_STREAM_ID); - Grouping groupAck = Thrift.mkFieldsGrouping(JStormUtils - .mk_list("id")); - - GlobalStreamId streamFail = new GlobalStreamId(id, - ACKER_FAIL_STREAM_ID); - Grouping groupFail = Thrift.mkFieldsGrouping(JStormUtils - .mk_list("id")); - - bolt_inputs.put(streamAck, groupAck); - bolt_inputs.put(streamFail, groupFail); - } - - Map allInputs = new HashMap(); - allInputs.putAll(bolt_inputs); - allInputs.putAll(spout_inputs); - return allInputs; - } - - /** - * Add acker bolt to topology - * - * @param num_tasks - * @param 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(); - fields.add("id"); - - outputs.put(ACKER_ACK_STREAM_ID, Thrift.directOutputFields(fields)); - outputs.put(ACKER_FAIL_STREAM_ID, Thrift.directOutputFields(fields)); - - IBolt ackerbolt = new Acker(); - - // generate inputs - Map inputs = acker_inputs(ret); - - // generate acker which will be stored in topology - Bolt acker_bolt = Thrift.mkBolt(inputs, ackerbolt, outputs, ackerNum); - - // add every bolt two output stream - // ACKER_ACK_STREAM_ID/ACKER_FAIL_STREAM_ID - for (Entry e : ret.get_bolts().entrySet()) { - - Bolt bolt = e.getValue(); - - ComponentCommon common = bolt.get_common(); - - List ackList = JStormUtils.mk_list("id", "ack-val"); - - common.put_to_streams(ACKER_ACK_STREAM_ID, - Thrift.outputFields(ackList)); - - List failList = JStormUtils.mk_list("id"); - common.put_to_streams(ACKER_FAIL_STREAM_ID, - Thrift.outputFields(failList)); - - bolt.set_common(common); - } - - // add every spout output stream ACKER_INIT_STREAM_ID - // add every spout two intput source - // ((ACKER_COMPONENT_ID, ACKER_ACK_STREAM_ID), directGrouping) - // ((ACKER_COMPONENT_ID, ACKER_FAIL_STREAM_ID), directGrouping) - for (Entry kv : ret.get_spouts().entrySet()) { - SpoutSpec bolt = kv.getValue(); - ComponentCommon common = bolt.get_common(); - List initList = JStormUtils.mk_list("id", "init-val", - "spout-task"); - common.put_to_streams(ACKER_INIT_STREAM_ID, - Thrift.outputFields(initList)); - - GlobalStreamId ack_ack = new GlobalStreamId(ACKER_COMPONENT_ID, - ACKER_ACK_STREAM_ID); - common.put_to_inputs(ack_ack, Thrift.mkDirectGrouping()); - - GlobalStreamId ack_fail = new GlobalStreamId(ACKER_COMPONENT_ID, - ACKER_FAIL_STREAM_ID); - common.put_to_inputs(ack_fail, Thrift.mkDirectGrouping()); - } - - ret.put_to_bolts(ACKER_COMPONENT_ID, acker_bolt); - } - - public static List all_components(StormTopology topology) { - List rtn = new ArrayList(); - for (StormTopology._Fields field : Thrift.STORM_TOPOLOGY_FIELDS) { - Object fields = topology.getFieldValue(field); - if (fields != null) { - rtn.addAll(((Map) fields).values()); - } - } - return rtn; - } - - private static List sysEventFields = JStormUtils.mk_list("event"); - - private static void add_component_system_streams(Object obj) { - ComponentCommon common = null; - if (obj instanceof StateSpoutSpec) { - StateSpoutSpec spec = (StateSpoutSpec) obj; - common = spec.get_common(); - } - - if (obj instanceof SpoutSpec) { - SpoutSpec spec = (SpoutSpec) obj; - common = spec.get_common(); - } - - if (obj instanceof Bolt) { - Bolt spec = (Bolt) obj; - common = spec.get_common(); - } - - if (common != null) { - StreamInfo sinfo = Thrift.outputFields(sysEventFields); - common.put_to_streams(SYSTEM_STREAM_ID, sinfo); - } - } - - /** - * Add every bolt or spout one output stream - * - * @param topology - */ - public static void add_system_streams(StormTopology topology) { - for (Object obj : all_components(topology)) { - add_component_system_streams(obj); - } - } - - 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(); - - add_acker(storm_conf, ret); - - add_metrics_component(ret); - - add_system_components(ret); - - return ret; - } - - /** - * get component configuration - * - * @param storm_conf - * @param topology_context - * @param component_id - * @return - */ - @SuppressWarnings("unchecked") - public static Map component_conf(Map storm_conf, - TopologyContext topology_context, String component_id) { - List to_remove = StormConfig.All_CONFIGS(); - to_remove.remove(Config.TOPOLOGY_DEBUG); - to_remove.remove(Config.TOPOLOGY_MAX_SPOUT_PENDING); - to_remove.remove(Config.TOPOLOGY_MAX_TASK_PARALLELISM); - to_remove.remove(Config.TOPOLOGY_TRANSACTIONAL_ID); - - Map componentConf = new HashMap(); - - String jconf = topology_context.getComponentCommon(component_id) - .get_json_conf(); - if (jconf != null) { - componentConf = (Map) JStormUtils.from_json(jconf); - } - - /** - * @@@ Don't know why need remove system configuration from component - * conf? // - */ - // for (Object p : to_remove) { - // componentConf.remove(p); - // } - - Map ret = new HashMap(); - ret.putAll(storm_conf); - ret.putAll(componentConf); - - return ret; - } - - /** - * get object of component_id - * - * @param topology - * @param component_id - * @return - */ - public static Object get_task_object(StormTopology topology, - String component_id, URLClassLoader loader) { - Map spouts = topology.get_spouts(); - Map bolts = topology.get_bolts(); - Map state_spouts = topology.get_state_spouts(); - - ComponentObject obj = null; - if (spouts.containsKey(component_id)) { - obj = spouts.get(component_id).get_spout_object(); - } else if (bolts.containsKey(component_id)) { - obj = bolts.get(component_id).get_bolt_object(); - } else if (state_spouts.containsKey(component_id)) { - obj = state_spouts.get(component_id).get_state_spout_object(); - } - - if (obj == null) { - throw new RuntimeException("Could not find " + component_id - + " in " + topology.toString()); - } - - Object componentObject = Utils.getSetComponentObject(obj, loader); - - Object rtn = null; - - if (componentObject instanceof JavaObject) { - rtn = Thrift.instantiateJavaObject((JavaObject) componentObject); - } else if (componentObject instanceof ShellComponent) { - if (spouts.containsKey(component_id)) { - rtn = new ShellSpout((ShellComponent) componentObject); - } else { - rtn = new ShellBolt((ShellComponent) componentObject); - } - } else { - rtn = componentObject; - } - return rtn; - - } - - /** - * get current task's output > - * - * @param topology_context - * @return - */ - public static Map> outbound_components( - TopologyContext topology_context, WorkerData workerData) { - Map> rr = new HashMap>(); - - // > - Map> output_groupings = topology_context - .getThisTargets(); - - for (Entry> entry : output_groupings - .entrySet()) { - - String stream_id = entry.getKey(); - Map component_grouping = entry.getValue(); - - Fields out_fields = topology_context.getThisOutputFields(stream_id); - - Map componentGrouper = new HashMap(); - - for (Entry cg : component_grouping.entrySet()) { - - String component = cg.getKey(); - Grouping tgrouping = cg.getValue(); - - List outTasks = topology_context - .getComponentTasks(component); - // ATTENTION: If topology set one component parallelism as 0 - // so we don't need send tuple to it - if (outTasks.size() > 0) { - MkGrouper grouper = new MkGrouper(topology_context, - out_fields, tgrouping, outTasks, stream_id, - workerData); - componentGrouper.put(component, grouper); - } - } - if (componentGrouper.size() > 0) { - rr.put(stream_id, componentGrouper); - } - } - return rr; - } - - /** - * get the component's configuration - * - * @param topology_context - * @param task_id - * @return component's configurations - */ - public static Map getComponentMap(DefaultTopologyAssignContext context, - Integer task) { - String componentName = context.getTaskToComponent().get(task); - ComponentCommon componentCommon = ThriftTopologyUtils - .getComponentCommon(context.getSysTopology(), componentName); - - Map componentMap = (Map) JStormUtils.from_json(componentCommon - .get_json_conf()); - if (componentMap == null) { - componentMap = Maps.newHashMap(); - } - return componentMap; - } - - /** - * get all bolts' inputs and spouts' outputs > - * > - * - * @param topology_context - * @return all bolts' inputs and spouts' outputs - */ - public static Map> buildSpoutOutoputAndBoltInputMap( - DefaultTopologyAssignContext context) { - Set bolts = context.getRawTopology().get_bolts().keySet(); - Set spouts = context.getRawTopology().get_spouts().keySet(); - Map> relationship = new HashMap>(); - for (Entry entry : context.getRawTopology().get_bolts() - .entrySet()) { - Map inputs = entry.getValue() - .get_common().get_inputs(); - Set input = new HashSet(); - relationship.put(entry.getKey(), input); - for (Entry inEntry : inputs.entrySet()) { - String component = inEntry.getKey().get_componentId(); - input.add(component); - if (!bolts.contains(component)) { - // spout - Set spoutOutput = relationship.get(component); - if (spoutOutput == null) { - spoutOutput = new HashSet(); - relationship.put(component, spoutOutput); - } - spoutOutput.add(entry.getKey()); - } - } - } - for (String spout : spouts) { - if (relationship.get(spout) == null) - relationship.put(spout, new HashSet()); - } - for (String bolt : bolts) { - if (relationship.get(bolt) == null) - relationship.put(bolt, new HashSet()); - } - return relationship; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/DaemonCommon.java b/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/DaemonCommon.java deleted file mode 100644 index 0f70a7c13..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/DaemonCommon.java +++ /dev/null @@ -1,5 +0,0 @@ -package com.alibaba.jstorm.cluster; - -public interface DaemonCommon { - public boolean waiting(); -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormBase.java b/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormBase.java deleted file mode 100644 index 05bac1672..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormBase.java +++ /dev/null @@ -1,114 +0,0 @@ -package com.alibaba.jstorm.cluster; - -import java.io.Serializable; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; - -import com.alibaba.jstorm.daemon.nimbus.StatusType; - -/** - * Topology stored in ZK - */ - -public class StormBase implements Serializable { - - private static final long serialVersionUID = -3013095336395395213L; - private String stormName; - private int lanchTimeSecs; - private StormStatus status; - private String group; - - public StormBase(String stormName, int lanchTimeSecs, StormStatus status, - String group) { - this.stormName = stormName; - this.lanchTimeSecs = lanchTimeSecs; - this.status = status; - this.setGroup(group); - } - - public String getStormName() { - return stormName; - } - - public void setStormName(String stormName) { - this.stormName = stormName; - } - - public int getLanchTimeSecs() { - return lanchTimeSecs; - } - - public void setLanchTimeSecs(int lanchTimeSecs) { - this.lanchTimeSecs = lanchTimeSecs; - } - - public StormStatus getStatus() { - return status; - } - - public void setStatus(StormStatus status) { - this.status = status; - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + ((group == null) ? 0 : group.hashCode()); - result = prime * result + lanchTimeSecs; - result = prime * result + ((status == null) ? 0 : status.hashCode()); - result = prime * result - + ((stormName == null) ? 0 : stormName.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - StormBase other = (StormBase) obj; - if (group == null) { - if (other.group != null) - return false; - } else if (!group.equals(other.group)) - return false; - if (lanchTimeSecs != other.lanchTimeSecs) - return false; - if (status == null) { - if (other.status != null) - return false; - } else if (!status.equals(other.status)) - return false; - if (stormName == null) { - if (other.stormName != null) - return false; - } else if (!stormName.equals(other.stormName)) - return false; - return true; - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this, - ToStringStyle.SHORT_PREFIX_STYLE); - } - - public String getStatusString() { - StatusType t = status.getStatusType(); - return t.getStatus().toUpperCase(); - } - - public String getGroup() { - return group; - } - - public void setGroup(String group) { - this.group = group; - } - -} 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 deleted file mode 100644 index 8bdcdc218..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormClusterState.java +++ /dev/null @@ -1,160 +0,0 @@ -package com.alibaba.jstorm.cluster; - -import java.util.List; -import java.util.Map; - -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.UserDefMetricData; -import com.alibaba.jstorm.task.Assignment; -import com.alibaba.jstorm.task.AssignmentBak; -import com.alibaba.jstorm.task.TaskInfo; -import com.alibaba.jstorm.task.TaskMetricInfo; -import com.alibaba.jstorm.task.error.TaskError; -import com.alibaba.jstorm.task.heartbeat.TaskHeartbeat; - -/** - * all storm in zk operation interface - */ -public interface StormClusterState { - public List assignments(RunnableCallback callback) throws Exception; - - public Assignment assignment_info(String topology_id, - RunnableCallback callback) throws Exception; - - public void set_assignment(String topology_id, Assignment info) - throws Exception; - - public AssignmentBak assignment_bak(String topologyName) throws Exception; - - public void backup_assignment(String topology_id, AssignmentBak info) - throws Exception; - - public List active_storms() throws Exception; - - public StormBase storm_base(String topology_id, RunnableCallback callback) - throws Exception; - - public void activate_storm(String topology_id, StormBase storm_base) - throws Exception; - - public void update_storm(String topology_id, StormStatus new_elems) - throws Exception; - - 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; - - public void set_task(String topology_id, int task_id, TaskInfo info) - throws Exception; - - public TaskInfo task_info(String topology_id, int task_id) throws Exception; - - public List task_storms() throws Exception; - - public void setup_heartbeats(String topology_id) throws Exception; - - public void teardown_heartbeats(String topology_id) throws Exception; - - public List heartbeat_storms() throws Exception; - - public List heartbeat_tasks(String topology_id) throws Exception; - - public TaskHeartbeat task_heartbeat(String topology_id, int task_id) - throws Exception; - - public Map task_heartbeat(String topologyId) - throws Exception; - - public void task_heartbeat(String topology_id, int task_id, - TaskHeartbeat info) throws Exception; - - public void remove_task_heartbeat(String topology_id, int task_id) - throws Exception; - - public List supervisors(RunnableCallback callback) throws Exception; - - public SupervisorInfo supervisor_info(String supervisor_id) - throws Exception; - - public void supervisor_heartbeat(String supervisor_id, SupervisorInfo info) - throws Exception; - - public void teardown_task_errors(String topology_id) throws Exception; - - public List task_error_storms() throws Exception; - - public void report_task_error(String topology_id, int task_id, - Throwable error) throws Exception; - - public void report_task_error(String topology_id, int task_id, - String error) throws Exception; - - public Map topo_lastErr_time(String topologyId) throws Exception; - - public void remove_lastErr_time(String topologyId) throws Exception; - - public List task_errors(String topology_id, int task_id) - throws Exception; - - public boolean try_to_be_leader(String path, String host, RunnableCallback callback) throws Exception; - - public String get_leader_host() throws Exception; - - public void update_follower_hb(String host, int time) throws Exception; - - public boolean leader_existed() throws Exception; - - public void register_nimbus_host(String host) throws Exception; - - public void unregister_nimbus_host(String host) throws Exception; - - public void disconnect() throws Exception; - - public void set_storm_monitor(String topologyId, StormMonitor metricsMonitor) throws Exception; - - public StormMonitor get_storm_monitor(String topologyId) throws Exception; - - public UserDefMetricData get_userDef_metric(String topologyId,String workerId) throws Exception; - - public Map task_info_list(String topologyId) throws Exception; - - public void update_task_metric(String topologyId, String taskId, TaskMetricInfo metricInfo) throws Exception; - - public void update_worker_metric(String topologyId, String workerId, WorkerMetricInfo metricInfo) throws Exception; - - public List get_task_metric_list(String topologyId) throws Exception; - - public List get_metric_taskIds(String topologyId) throws Exception; - - public void remove_metric_task(String topologyId, String taskId) throws Exception; - - public List get_worker_metric_list(String topologyId) throws Exception; - - public List get_metric_workerIds(String topologyId) throws Exception; - - public void remove_metric_worker(String topologyId, String workerId) throws Exception; - - public List get_metric_users(String topologyId) throws Exception; - - public void remove_metric_user(String topologyId, String workerId) throws Exception; - - public void update_userDef_metric(String topologyId, String workerId, UserDefMetricData metricInfo) throws Exception; - - 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 deleted file mode 100644 index 094f68e9a..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormConfig.java +++ /dev/null @@ -1,476 +0,0 @@ -package com.alibaba.jstorm.cluster; - -import java.io.File; -import java.io.IOException; -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.commons.io.FileUtils; -import org.apache.log4j.Logger; - -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"; - public final static String WORKER_DATA_SUBDIR = "worker_shared_data"; - - public static final String FILE_SEPERATEOR = File.separator; - - public static String clojureConfigName(String name) { - return name.toUpperCase().replace("_", "-"); - } - - public static Map read_storm_config() { - return Utils.readStormConfig(); - } - - public static Map read_yaml_config(String name) { - return Utils.findAndReadConfigFile(name, true); - } - - public static Map read_default_config() { - return Utils.readDefaultConfig(); - } - - public static List All_CONFIGS() { - List rtn = new ArrayList(); - Config config = new Config(); - Class ConfigClass = config.getClass(); - Field[] fields = ConfigClass.getFields(); - for (int i = 0; i < fields.length; i++) { - try { - Object obj = fields[i].get(null); - rtn.add(obj); - } catch (IllegalArgumentException e) { - LOG.error(e.getMessage(), e); - } catch (IllegalAccessException e) { - LOG.error(e.getMessage(), e); - } - } - return rtn; - } - - public static HashMap getClassFields(Class cls) - throws IllegalArgumentException, IllegalAccessException { - java.lang.reflect.Field[] list = cls.getDeclaredFields(); - HashMap rtn = new HashMap(); - for (java.lang.reflect.Field f : list) { - String name = f.getName(); - rtn.put(name, f.get(null).toString()); - - } - return rtn; - } - - public static String cluster_mode(Map conf) { - String mode = (String) conf.get(Config.STORM_CLUSTER_MODE); - return mode; - - } - - public static boolean local_mode(Map conf) { - String mode = (String) conf.get(Config.STORM_CLUSTER_MODE); - if (mode != null) { - if (mode.equals("local")) { - return true; - } - - if (mode.equals("distributed")) { - return false; - } - } - throw new IllegalArgumentException("Illegal cluster mode in conf:" - + mode); - - } - - /** - * validate whether the mode is distributed - * - * @param conf - */ - public static void validate_distributed_mode(Map conf) { - if (StormConfig.local_mode(conf)) { - throw new IllegalArgumentException( - "Cannot start server in local mode!"); - } - - } - - public static void validate_local_mode(Map conf) { - if (!StormConfig.local_mode(conf)) { - throw new IllegalArgumentException( - "Cannot start server in distributed mode!"); - } - - } - - public static String worker_root(Map conf) throws IOException { - String ret = String.valueOf(conf.get(Config.STORM_LOCAL_DIR)) - + FILE_SEPERATEOR + "workers"; - FileUtils.forceMkdir(new File(ret)); - return ret; - } - - public static String worker_root(Map conf, String id) throws IOException { - String ret = worker_root(conf) + FILE_SEPERATEOR + id; - FileUtils.forceMkdir(new File(ret)); - return ret; - } - - public static String worker_pids_root(Map conf, String id) - throws IOException { - String ret = worker_root(conf, id) + FILE_SEPERATEOR + "pids"; - FileUtils.forceMkdir(new File(ret)); - return ret; - } - - public static String worker_pid_path(Map conf, String id, String pid) - throws IOException { - String ret = worker_pids_root(conf, id) + FILE_SEPERATEOR + pid; - return ret; - } - - public static String worker_heartbeats_root(Map conf, String id) - throws IOException { - String ret = worker_root(conf, id) + FILE_SEPERATEOR + "heartbeats"; - FileUtils.forceMkdir(new File(ret)); - return ret; - } - - public static String default_worker_shared_dir(Map conf) throws IOException { - String ret = String.valueOf(conf.get(Config.STORM_LOCAL_DIR)) - + FILE_SEPERATEOR + WORKER_DATA_SUBDIR; - - FileUtils.forceMkdir(new File(ret)); - return ret; - } - - private static String supervisor_local_dir(Map conf) throws IOException { - String ret = String.valueOf(conf.get(Config.STORM_LOCAL_DIR)) - + FILE_SEPERATEOR + "supervisor"; - FileUtils.forceMkdir(new File(ret)); - return ret; - } - - public static String supervisor_stormdist_root(Map conf) throws IOException { - String ret = stormdist_path(supervisor_local_dir(conf)); - FileUtils.forceMkdir(new File(ret)); - return ret; - } - - public static String supervisor_stormdist_root(Map conf, String topologyId) - throws IOException { - return supervisor_stormdist_root(conf) + FILE_SEPERATEOR + topologyId; - } - - /** - * Return supervisor's pid dir - * - * @param conf - * @return - * @throws IOException - */ - public static String supervisorPids(Map conf)throws IOException { - String ret = supervisor_local_dir(conf) + FILE_SEPERATEOR + "pids"; - try { - FileUtils.forceMkdir(new File(ret)); - } catch (IOException e) { - LOG.error("Failed to create dir " + ret, e); - throw e; - } - return ret; - } - - - /** - * Return nimbus's heartbeat dir for apsara - * - * @param conf - * @return - * @throws IOException - */ - public static String supervisorHearbeatForContainer(Map conf)throws IOException { - String ret = supervisor_local_dir(conf) + FILE_SEPERATEOR + "supervisor.heartbeat"; - try { - FileUtils.forceMkdir(new File(ret)); - } catch (IOException e) { - LOG.error("Failed to create dir " + ret, e); - throw e; - } - return ret; - } - - public static String stormjar_path(String stormroot) { - return stormroot + FILE_SEPERATEOR + "stormjar.jar"; - } - - public static String stormcode_path(String stormroot) { - return stormroot + FILE_SEPERATEOR + "stormcode.ser"; - } - - public static String stormconf_path(String stormroot) { - return stormroot + FILE_SEPERATEOR + "stormconf.ser"; - } - - public static String stormlib_path(String stormroot, String libname) { - return stormroot + FILE_SEPERATEOR + "lib" + FILE_SEPERATEOR + libname; - } - - public static String stormlib_path(String stormroot) { - return stormroot + FILE_SEPERATEOR + "lib"; - } - - public static String stormdist_path(String stormroot) { - return stormroot + FILE_SEPERATEOR + "stormdist"; - } - - public static String supervisor_storm_resources_path(String stormroot) { - return stormroot + FILE_SEPERATEOR + RESOURCES_SUBDIR; - } - - public static String stormtmp_path(String stormroot) { - return stormroot + FILE_SEPERATEOR + "tmp"; - } - - public static LocalState worker_state(Map conf, String id) - throws IOException { - String path = worker_heartbeats_root(conf, id); - - LocalState rtn = new LocalState(path); - return rtn; - - } - - public static String masterLocalDir(Map conf) throws IOException { - String ret = String.valueOf(conf.get(Config.STORM_LOCAL_DIR)) - + FILE_SEPERATEOR + "nimbus"; - try { - FileUtils.forceMkdir(new File(ret)); - } catch (IOException e) { - LOG.error("Failed to create dir " + ret, e); - throw e; - } - return ret; - } - - public static String masterStormdistRoot(Map conf) throws IOException { - String ret = stormdist_path(masterLocalDir(conf)); - FileUtils.forceMkdir(new File(ret)); - return ret; - } - - public static String masterStormdistRoot(Map conf, String topologyId) - throws IOException { - return masterStormdistRoot(conf) + FILE_SEPERATEOR + topologyId; - } - - public static String masterStormTmpRoot(Map conf) throws IOException { - String ret = stormtmp_path(masterLocalDir(conf)); - FileUtils.forceMkdir(new File(ret)); - return ret; - } - - public static String masterStormTmpRoot(Map conf, String topologyId) - throws IOException { - return masterStormTmpRoot(conf) + FILE_SEPERATEOR + topologyId; - } - - public static String masterInbox(Map conf) throws IOException { - String ret = masterLocalDir(conf) + FILE_SEPERATEOR + "inbox"; - try { - FileUtils.forceMkdir(new File(ret)); - } catch (IOException e) { - LOG.error("Failed to create dir " + ret, e); - throw e; - } - return ret; - } - - public static String masterInimbus(Map conf) throws IOException { - String ret = masterLocalDir(conf) + FILE_SEPERATEOR + "ininumbus"; - try { - FileUtils.forceMkdir(new File(ret)); - } catch (IOException e) { - LOG.error("Failed to create dir " + ret, e); - throw e; - } - return ret; - } - - /** - * Return nimbus's pid dir - * - * @param conf - * @return - * @throws IOException - */ - public static String masterPids(Map conf)throws IOException { - String ret = masterLocalDir(conf) + FILE_SEPERATEOR + "pids"; - try { - FileUtils.forceMkdir(new File(ret)); - } catch (IOException e) { - LOG.error("Failed to create dir " + ret, e); - throw e; - } - return ret; - } - - - /** - * Return nimbus's heartbeat dir for apsara - * - * @param conf - * @return - * @throws IOException - */ - public static String masterHearbeatForContainer(Map conf)throws IOException { - String ret = masterLocalDir(conf) + FILE_SEPERATEOR + "nimbus.heartbeat"; - try { - FileUtils.forceMkdir(new File(ret)); - } catch (IOException e) { - LOG.error("Failed to create dir " + ret, e); - throw e; - } - return ret; - } - - public static String supervisorTmpDir(Map conf) throws IOException { - String ret = null; - try { - ret = supervisor_local_dir(conf) + FILE_SEPERATEOR + "tmp"; - FileUtils.forceMkdir(new File(ret)); - } catch (IOException e) { - LOG.error("Failed to create dir " + ret, e); - throw e; - - } - - return ret; - } - - public static LocalState supervisorState(Map conf) throws IOException { - LocalState localState = null; - try { - String localstateDir = supervisor_local_dir(conf) + FILE_SEPERATEOR - + "localstate"; - FileUtils.forceMkdir(new File(localstateDir)); - localState = new LocalState(localstateDir); - } catch (IOException e) { - LOG.error("Failed to create supervisor LocalState", e); - throw e; - } - return localState; - } - - /** - * stormconf is mergered into clusterconf - * - * @param conf - * @param topologyId - * @return - * @throws IOException - */ - public static Map read_supervisor_topology_conf(Map conf, String topologyId) - throws IOException { - String topologyRoot = StormConfig.supervisor_stormdist_root(conf, - topologyId); - String confPath = StormConfig.stormconf_path(topologyRoot); - return (Map) readLocalObject(topologyId, confPath); - } - - public static StormTopology read_supervisor_topology_code(Map conf, - String topologyId) throws IOException { - String topologyRoot = StormConfig.supervisor_stormdist_root(conf, - topologyId); - String codePath = StormConfig.stormcode_path(topologyRoot); - return (StormTopology) readLocalObject(topologyId, codePath); - } - - @SuppressWarnings("rawtypes") - public static List get_supervisor_toplogy_list(Map conf) - throws IOException { - - // get the path: STORM-LOCAL-DIR/supervisor/stormdist/ - String path = StormConfig.supervisor_stormdist_root(conf); - - List topologyids = PathUtils.read_dir_contents(path); - - return topologyids; - } - - public static Map read_nimbus_topology_conf(Map conf, String topologyId) - throws IOException { - String topologyRoot = StormConfig.masterStormdistRoot(conf, topologyId); - return read_topology_conf(topologyRoot, topologyId); - } - - public static Map read_nimbusTmp_topology_conf(Map conf, String topologyId) - throws IOException { - String topologyRoot = StormConfig.masterStormTmpRoot(conf, topologyId); - return read_topology_conf(topologyRoot, topologyId); - } - - public static Map read_topology_conf(String topologyRoot, String topologyId) - throws IOException { - String readFile = StormConfig.stormconf_path(topologyRoot); - return (Map) readLocalObject(topologyId, readFile); - } - - public static StormTopology read_nimbus_topology_code(Map conf, - String topologyId) throws IOException { - String topologyRoot = StormConfig.masterStormdistRoot(conf, topologyId); - String codePath = StormConfig.stormcode_path(topologyRoot); - return (StormTopology) readLocalObject(topologyId, codePath); - } - - /** - * stormconf has mergered into clusterconf - * - * @param conf - * @param topologyId - * @return Map - * @throws IOException - */ - @SuppressWarnings("unchecked") - public static Object readLocalObject(String topologyId, String readFile) - throws IOException { - - String errMsg = "Failed to get topology configuration of " + topologyId - + " file:" + readFile; - - byte[] bconf = FileUtils.readFileToByteArray(new File(readFile)); - if (bconf == null) { - errMsg += ", due to failed to read"; - LOG.error(errMsg); - throw new IOException(errMsg); - } - - Object ret = null; - try { - ret = Utils.deserialize(bconf); - } catch (Exception e) { - errMsg += ", due to failed to serialized the data"; - LOG.error(errMsg); - throw new IOException(errMsg); - } - - return ret; - } - - public static Integer sampling_rate(Map conf) { - return (int) (1 / Double.parseDouble(String.valueOf(conf - .get(Config.TOPOLOGY_STATS_SAMPLE_RATE)))); - } - - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormMonitor.java b/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormMonitor.java deleted file mode 100644 index f73f37a73..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormMonitor.java +++ /dev/null @@ -1,33 +0,0 @@ -package com.alibaba.jstorm.cluster; - -import java.io.Serializable; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; - -/** - * Topology metrics monitor in ZK - */ - -public class StormMonitor implements Serializable { - private static final long serialVersionUID = -6023196346496305314L; - private boolean metricsMonitor; - - public StormMonitor(boolean metricsMonitor) { - this.metricsMonitor = metricsMonitor; - } - - public void setMetrics(boolean metrics) { - this.metricsMonitor = metrics; - } - - public boolean getMetrics() { - return this.metricsMonitor; - } - - @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/cluster/StormStatus.java b/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormStatus.java deleted file mode 100644 index f33e176dd..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormStatus.java +++ /dev/null @@ -1,106 +0,0 @@ -package com.alibaba.jstorm.cluster; - -import java.io.Serializable; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; - -import com.alibaba.jstorm.daemon.nimbus.StatusType; - -/** - * author: lixin/longda - * - * Dedicate Topology status - * - * Topology status: active/inactive/killed/rebalancing killTimeSecs: when status - * isn't killed, it is -1 and useless. when status is killed, do kill operation - * after killTimeSecs seconds when status is rebalancing, do rebalancing opation - * after delaySecs seconds restore oldStatus as current status - */ -public class StormStatus implements Serializable { - - private static final long serialVersionUID = -2276901070967900100L; - private StatusType type; - @Deprecated - private int killTimeSecs; - private int delaySecs; - private StormStatus oldStatus = null; - - public StormStatus(StatusType type) { - this(0, type); - } - - public StormStatus(int delaySecs, StatusType type) { - this(type, delaySecs, null); - } - - public StormStatus(StatusType type, int delaySecs, StormStatus oldStatus) { - this.type = type; - this.delaySecs = delaySecs; - this.killTimeSecs = delaySecs; - this.oldStatus = oldStatus; - } - - public StatusType getStatusType() { - return type; - } - - public void setStatusType(StatusType type) { - this.type = type; - } - - @Deprecated - public Integer getKillTimeSecs() { - return killTimeSecs; - } - - @Deprecated - public void setKillTimeSecs(int killTimeSecs) { - this.killTimeSecs = killTimeSecs; - } - - public Integer getDelaySecs() { - return delaySecs; - } - - public void setDelaySecs(int delaySecs) { - this.delaySecs = delaySecs; - } - - public StormStatus getOldStatus() { - return oldStatus; - } - - public void setOldStatus(StormStatus oldStatus) { - this.oldStatus = oldStatus; - } - - @Override - public boolean equals(Object base) { - if ((base instanceof StormStatus) == false) { - return false; - } - - StormStatus check = (StormStatus) base; - if (check.getStatusType().equals(getStatusType()) - && check.getKillTimeSecs() == getKillTimeSecs() - && check.getDelaySecs().equals(getDelaySecs())) { - return true; - } - return false; - } - - @Override - public int hashCode() { - return this.getStatusType().hashCode() - + this.getKillTimeSecs().hashCode() - + this.getDelaySecs().hashCode(); - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this, - ToStringStyle.SHORT_PREFIX_STYLE); - } - -} 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 deleted file mode 100644 index 3fec09546..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/cluster/StormZkClusterState.java +++ /dev/null @@ -1,823 +0,0 @@ -package com.alibaba.jstorm.cluster; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.HashMap; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicReference; - -import org.apache.log4j.Logger; -import org.apache.zookeeper.KeeperException.NodeExistsException; -import org.apache.zookeeper.Watcher.Event.EventType; - -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.callback.ClusterStateCallback; -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; -import com.alibaba.jstorm.task.Assignment; -import com.alibaba.jstorm.task.AssignmentBak; -import com.alibaba.jstorm.task.TaskInfo; -import com.alibaba.jstorm.task.TaskMetricInfo; -import com.alibaba.jstorm.task.error.TaskError; -import com.alibaba.jstorm.task.heartbeat.TaskHeartbeat; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.PathUtils; -import com.alibaba.jstorm.utils.TimeUtils; -import com.alibaba.jstorm.zk.ZkConstant; -import com.alibaba.jstorm.zk.ZkTool; -import com.alibaba.jstorm.daemon.worker.WorkerMetricInfo; -import com.alibaba.jstorm.metric.UserDefMetric; -import com.alibaba.jstorm.metric.UserDefMetricData; - -public class StormZkClusterState implements StormClusterState { - private static Logger LOG = Logger.getLogger(StormZkClusterState.class); - - private ClusterState cluster_state; - - private ConcurrentHashMap assignment_info_callback; - private AtomicReference supervisors_callback; - private AtomicReference assignments_callback; - private ConcurrentHashMap storm_base_callback; - private AtomicReference master_callback; - - private UUID state_id; - - private boolean solo; - - public StormZkClusterState(Object cluster_state_spec) throws Exception { - - if (cluster_state_spec instanceof ClusterState) { - solo = false; - cluster_state = (ClusterState) cluster_state_spec; - } else { - - solo = true; - cluster_state = new DistributedClusterState( - (Map) cluster_state_spec); - } - - assignment_info_callback = new ConcurrentHashMap(); - supervisors_callback = new AtomicReference(null); - assignments_callback = new AtomicReference(null); - storm_base_callback = new ConcurrentHashMap(); - master_callback = new AtomicReference(null); - - state_id = cluster_state.register(new ClusterStateCallback() { - - public Object execute(T... args) { - if (args == null) { - LOG.warn("Input args is null"); - return null; - } else if (args.length < 2) { - LOG.warn("Input args is invalid, args length:" - + args.length); - return null; - } - - EventType zkEventTypes = (EventType) args[0]; - String path = (String) args[1]; - - List toks = PathUtils.tokenize_path(path); - int size = toks.size(); - if (size >= 1) { - String params = null; - String root = toks.get(0); - RunnableCallback fn = null; - if (root.equals(Cluster.ASSIGNMENTS_ROOT)) { - if (size == 1) { - // set null and get the old value - fn = assignments_callback.getAndSet(null); - } else { - params = toks.get(1); - fn = assignment_info_callback.remove(params); - } - - } else if (root.equals(Cluster.SUPERVISORS_ROOT)) { - fn = supervisors_callback.getAndSet(null); - } else if (root.equals(Cluster.STORMS_ROOT) && size > 1) { - params = toks.get(1); - fn = storm_base_callback.remove(params); - } else if (root.equals(Cluster.MASTER_ROOT)) { - fn = master_callback.getAndSet(null); - } else { - LOG.error("Unknown callback for subtree " + path); - } - - if (fn != null) { - // FIXME How to set the args - // fn.setArgs(params, zkEventTypes, path); - fn.run(); - } - } - - return null; - } - - }); - - String[] pathlist = JStormUtils.mk_arr(Cluster.ASSIGNMENTS_SUBTREE, - Cluster.TASKS_SUBTREE, Cluster.STORMS_SUBTREE, - Cluster.SUPERVISORS_SUBTREE, Cluster.TASKBEATS_SUBTREE, - Cluster.TASKERRORS_SUBTREE, Cluster.MONITOR_SUBTREE); - for (String path : pathlist) { - cluster_state.mkdirs(path); - } - - } - - @Override - public Assignment assignment_info(String topologyId, - RunnableCallback callback) throws Exception { - if (callback != null) { - assignment_info_callback.put(topologyId, callback); - } - - String assgnmentPath = Cluster.assignment_path(topologyId); - - byte[] znodeData = cluster_state.get_data(assgnmentPath, - callback != null); - - Object data = Cluster.maybe_deserialize(znodeData); - - if (data == null) { - return null; - } - return (Assignment) data; - } - - @Override - public List assignments(RunnableCallback callback) throws Exception { - if (callback != null) { - assignments_callback.set(callback); - } - return cluster_state.get_children(Cluster.ASSIGNMENTS_SUBTREE, - callback != null); - } - - @Override - public void set_assignment(String topologyId, Assignment info) - throws Exception { - cluster_state.set_data(Cluster.assignment_path(topologyId), - Utils.serialize(info)); - } - - @Override - public AssignmentBak assignment_bak(String topologyName) throws Exception { - String assgnmentBakPath = ZkTool.assignment_bak_path(topologyName); - - byte[] znodeData = cluster_state.get_data(assgnmentBakPath, false); - - Object data = Cluster.maybe_deserialize(znodeData); - - if (data == null) { - return null; - } - return (AssignmentBak) data; - } - - @Override - public void backup_assignment(String topologyName, AssignmentBak info) - throws Exception { - cluster_state.set_data(ZkTool.assignment_bak_path(topologyName), - Utils.serialize(info)); - } - - @Override - public void activate_storm(String topologyId, StormBase stormBase) - throws Exception { - String stormPath = Cluster.storm_path(topologyId); - - byte[] stormBaseData = Utils.serialize(stormBase); - - cluster_state.set_data(stormPath, stormBaseData); - } - - @Override - public List active_storms() throws Exception { - return cluster_state.get_children(Cluster.STORMS_SUBTREE, false); - } - - @Override - 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 { - return cluster_state.get_children(Cluster.TASKBEATS_SUBTREE, false); - } - - @Override - public List heartbeat_tasks(String topologyId) throws Exception { - String taskbeatPath = Cluster.taskbeat_storm_root(topologyId); - - return cluster_state.get_children(taskbeatPath, false); - } - - @Override - public void remove_storm(String topologyId) throws Exception { - cluster_state.delete_node(Cluster.assignment_path(topologyId)); - // wait 10 seconds, so supervisor will kill worker smoothly - JStormUtils.sleepMs(10000); - cluster_state.delete_node(Cluster.storm_task_root(topologyId)); - 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 { - cluster_state.delete_node(Cluster.storm_path(topologyId)); - } - - @Override - public void remove_task_heartbeat(String topologyId, int taskId) - throws Exception { - String taskbeatPath = Cluster.taskbeat_path(topologyId, taskId); - - cluster_state.delete_node(taskbeatPath); - } - - @Override - public void report_task_error(String topologyId, int taskId, Throwable error) - throws Exception { - report_task_error(topologyId, taskId, new String(JStormUtils.getErrorInfo(error))); - } - - public void report_task_error(String topologyId, int taskId, String error) - throws Exception { - boolean found = false; - String path = Cluster.taskerror_path(topologyId, taskId); - cluster_state.mkdirs(path); - - List children = new ArrayList(); - - String timeStamp = String.valueOf(TimeUtils.current_time_secs()); - String timestampPath = path + Cluster.ZK_SEPERATOR + timeStamp; - - for (String str : cluster_state.get_children(path, false)) { - children.add(Integer.parseInt(str)); - - String errorPath = path + "/" + str; - byte[] data = cluster_state.get_data(errorPath, false); - if (data == null) continue; - String errorInfo = new String(data); - if (errorInfo.equals(error)) { - cluster_state.delete_node(errorPath); - cluster_state.set_data(timestampPath, error.getBytes()); - found = true; - break; - } - } - - if (found == false) { - Collections.sort(children); - - while (children.size() >= 10) { - cluster_state.delete_node(path + Cluster.ZK_SEPERATOR - + children.remove(0)); - } - - cluster_state.set_data(timestampPath, error.getBytes()); - } - - setLastErrInfo(topologyId, error, timeStamp); - } - - private static final String TASK_IS_DEAD = "is dead on"; // Full string is "task-id is dead on hostname:port" - - private void setLastErrInfo(String topologyId, String error, String timeStamp) throws Exception { - // Set error information in task error topology patch - // Last Error information format in ZK: map - // report_duration means only the errors will presented in web ui if the - // error happens within this duration. - // Currently, the duration for "queue full" error is 180sec(3min) while - // the duration for other errors is 1800sec(30min). - String lastErrTopoPath = Cluster.lasterror_path(topologyId); - Map lastErrInfo = null; - try { - lastErrInfo = (Map) - (Cluster.maybe_deserialize(cluster_state.get_data(lastErrTopoPath, false))); - } catch (Exception e) { - LOG.error("Failed to get last error time. Remove the corrupt node for " + topologyId, e); - remove_lastErr_time(topologyId); - lastErrInfo = null; - } - if (lastErrInfo == null) - lastErrInfo = new HashMap(); - - // The error time is used to indicate how long the error info is present in UI - if (error.indexOf(TaskMetricInfo.QEUEU_IS_FULL) != -1) - lastErrInfo.put(JStormUtils.MIN_1*3, timeStamp); - else if (error.indexOf(TASK_IS_DEAD) != -1) - lastErrInfo.put(JStormUtils.DAY_1*3, timeStamp); - else - lastErrInfo.put(JStormUtils.MIN_30, timeStamp); - - cluster_state.set_data(lastErrTopoPath, Utils.serialize(lastErrInfo)); - } - - @Override - public void set_task(String topologyId, int taskId, TaskInfo info) - throws Exception { - String taskPath = Cluster.task_path(topologyId, taskId); - - byte[] taskData = Utils.serialize(info); - - cluster_state.set_data(taskPath, taskData); - } - - @Override - public void setup_heartbeats(String topologyId) throws Exception { - String taskbeatPath = Cluster.taskbeat_storm_root(topologyId); - - cluster_state.mkdirs(taskbeatPath); - } - - @Override - public StormBase storm_base(String topologyId, RunnableCallback callback) - throws Exception { - if (callback != null) { - storm_base_callback.put(topologyId, callback); - } - Object data = Cluster.maybe_deserialize(cluster_state.get_data( - Cluster.storm_path(topologyId), callback != null)); - if (data == null) { - return null; - } - return (StormBase) data; - } - - @Override - public void supervisor_heartbeat(String supervisorId, SupervisorInfo info) - throws Exception { - - String supervisorPath = Cluster.supervisor_path(supervisorId); - - byte[] infoData = Utils.serialize(info); - - cluster_state.set_ephemeral_node(supervisorPath, infoData); - } - - @Override - public SupervisorInfo supervisor_info(String supervisorId) throws Exception { - String supervisorPath = Cluster.supervisor_path(supervisorId); - - byte[] znodeData = cluster_state.get_data(supervisorPath, false); - - Object data = Cluster.maybe_deserialize(znodeData); - if (data == null) { - return null; - } - return (SupervisorInfo) data; - - } - - @Override - public List supervisors(RunnableCallback callback) throws Exception { - if (callback != null) { - supervisors_callback.set(callback); - } - return cluster_state.get_children(Cluster.SUPERVISORS_SUBTREE, - callback != null); - } - - @Override - public Map topo_lastErr_time(String topologyId) throws Exception { - String path = Cluster.lasterror_path(topologyId); - Map lastErrTime; - lastErrTime = (Map) (Cluster.maybe_deserialize( - cluster_state.get_data(path, false))); - return lastErrTime; - } - - @Override - public void remove_lastErr_time(String topologyId) throws Exception { - String path = Cluster.lasterror_path(topologyId); - cluster_state.delete_node(path); - } - - @Override - 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) - throws Exception { - String path = Cluster.taskerror_path(topologyId, taskId); - cluster_state.mkdirs(path); - - List children = cluster_state.get_children(path, false); - List errors = new ArrayList(); - - for (String str : children) { - byte[] v = cluster_state.get_data(path + "/" + str, false); - if (v != null) { - TaskError error = new TaskError(new String(v), - Integer.parseInt(str)); - errors.add(error); - } - } - - Collections.sort(errors, new Comparator() { - - @Override - public int compare(TaskError o1, TaskError o2) { - if (o1.getTimSecs() > o2.getTimSecs()) { - return 1; - } - if (o1.getTimSecs() < o2.getTimSecs()) { - return -1; - } - return 0; - } - }); - - return errors; - - } - - @Override - public TaskHeartbeat task_heartbeat(String topologyId, int taskId) - throws Exception { - String taskbeatPath = Cluster.taskbeat_path(topologyId, taskId); - - byte[] znodeData = cluster_state.get_data(taskbeatPath, false); - - Object data = Cluster.maybe_deserialize(znodeData); - if (data == null) { - return null; - } - return (TaskHeartbeat) data; - } - - @Override - public Map task_heartbeat(String topologyId) - throws Exception { - Map ret = new HashMap(); - - String topoTbPath = Cluster.taskbeat_storm_root(topologyId); - List taskList = cluster_state.get_children(topoTbPath, false); - - for (String taskId : taskList) { - String taskbeatPath = Cluster.taskbeat_path(topologyId, Integer.parseInt(taskId)); - - byte[] znodeData = cluster_state.get_data(taskbeatPath, false); - - Object data = Cluster.maybe_deserialize(znodeData); - if (data == null) { - continue; - } - ret.put(taskId, (TaskHeartbeat)data); - } - - return ret; - } - - @Override - public void task_heartbeat(String topologyId, int taskId, TaskHeartbeat info) - throws Exception { - String taskPath = Cluster.taskbeat_path(topologyId, taskId); - - byte[] taskData = Utils.serialize(info); - - cluster_state.set_data(taskPath, taskData); - } - - @Override - public List task_ids(String stromId) throws Exception { - - String stormTaskPath = Cluster.storm_task_root(stromId); - - List list = cluster_state.get_children(stormTaskPath, false); - - List rtn = new ArrayList(); - for (String str : list) { - rtn.add(Integer.parseInt(str)); - } - return rtn; - } - - @Override - public TaskInfo task_info(String topologyId, int taskId) throws Exception { - - String taskPath = Cluster.task_path(topologyId, taskId); - - byte[] znodeData = cluster_state.get_data(taskPath, false); - - Object data = Cluster.maybe_deserialize(znodeData); - if (data == null) { - return null; - } - return (TaskInfo) data; - } - - @Override - public Map task_info_list(String topologyId) throws Exception { - Map taskInfoList = new HashMap(); - - List taskIds = task_ids(topologyId); - - for (Integer taskId : taskIds) { - TaskInfo taskInfo = task_info(topologyId, taskId); - taskInfoList.put(taskId, taskInfo); - } - - return taskInfoList; - } - - @Override - public List task_storms() throws Exception { - return cluster_state.get_children(Cluster.TASKS_SUBTREE, false); - } - - @Override - public void teardown_heartbeats(String topologyId) { - try { - String taskbeatPath = Cluster.taskbeat_storm_root(topologyId); - - cluster_state.delete_node(taskbeatPath); - } catch (Exception e) { - LOG.error("Could not teardown heartbeats for " + topologyId, e); - } - - } - - @Override - public void teardown_task_errors(String topologyId) { - try { - String taskerrPath = Cluster.taskerror_storm_root(topologyId); - cluster_state.delete_node(taskerrPath); - } catch (Exception e) { - LOG.error("Could not teardown errors for " + topologyId, e); - } - } - - @Override - public void update_storm(String topologyId, StormStatus newElems) - throws Exception { - /** - * FIXME, not sure where the old exist error or not The raw code - * (set-data cluster-state (storm-path storm-id) (-> (storm-base this - * storm-id nil) (merge new-elems) Utils/serialize))) - */ - - StormBase base = this.storm_base(topologyId, null); - - if (base != null) { - base.setStatus(newElems); - cluster_state.set_data(Cluster.storm_path(topologyId), - Utils.serialize(base)); - } - - } - - @Override - public String get_leader_host() throws Exception { - // TODO Auto-generated method stub - return new String(cluster_state.get_data(Cluster.MASTER_SUBTREE, false)); - } - - @Override - public boolean leader_existed() throws Exception { - // TODO Auto-generated method stub - return cluster_state.node_existed(Cluster.MASTER_SUBTREE, false); - } - - @Override - public void disconnect() { - cluster_state.unregister(state_id); - if (solo == true) { - cluster_state.close(); - } - } - - @Override - public void register_nimbus_host(String host) throws Exception { - // TODO Auto-generated method stub - cluster_state.set_ephemeral_node(ZkConstant.NIMBUS_SLAVE_SUBTREE - + Cluster.ZK_SEPERATOR + host, null); - } - - @Override - public void unregister_nimbus_host(String host) throws Exception { - cluster_state.delete_node(ZkConstant.NIMBUS_SLAVE_SUBTREE - + Cluster.ZK_SEPERATOR + host); - } - - @Override - public void update_follower_hb(String host, int time) throws Exception { - cluster_state.set_data(ZkConstant.NIMBUS_SLAVE_SUBTREE - + Cluster.ZK_SEPERATOR + host, - String.valueOf(time).getBytes("UTF-8")); - } - - @Override - public boolean try_to_be_leader(String path, String host, - RunnableCallback callback) throws Exception { - // TODO Auto-generated method stub - if (callback != null) - this.master_callback.set(callback); - try { - cluster_state.tryToBeLeader(path, host.getBytes()); - } catch (NodeExistsException e) { - cluster_state.node_existed(path, true); - LOG.info("leader is alive"); - return false; - } - return true; - } - - @Override - public void set_storm_monitor(String topologyId, StormMonitor metricsMonitor) throws Exception { - String monitorStatusPath = Cluster.monitor_status_path(topologyId); - cluster_state.set_data(monitorStatusPath, Utils.serialize(metricsMonitor)); - cluster_state.mkdirs(Cluster.monitor_taskdir_path(topologyId)); - cluster_state.mkdirs(Cluster.monitor_workerdir_path(topologyId)); - cluster_state.mkdirs(Cluster.monitor_userdir_path(topologyId)); - } - - @Override - public StormMonitor get_storm_monitor(String topologyId) throws Exception { - String monitorPath = Cluster.monitor_status_path(topologyId); - - byte[] metricsMonitorData = cluster_state.get_data(monitorPath, false); - Object metricsMonitor = Cluster.maybe_deserialize(metricsMonitorData); - - return (StormMonitor)metricsMonitor; - } - public UserDefMetricData get_userDef_metric(String topologyId,String workerId) throws Exception{ - String workerMetricPath = Cluster.monitor_user_path(topologyId, workerId); - byte[] userMetricsData=cluster_state.get_data(workerMetricPath, false); - Object userMetrics = Cluster.maybe_deserialize(userMetricsData); - return (UserDefMetricData)userMetrics; - } - - @Override - public void update_task_metric(String topologyId, String taskId, TaskMetricInfo metricInfo) throws Exception { - String taskMetricPath = Cluster.monitor_task_path(topologyId, taskId); - cluster_state.set_data(taskMetricPath, Utils.serialize(metricInfo)); - } - - @Override - public void update_worker_metric(String topologyId, String workerId, WorkerMetricInfo metricInfo) throws Exception { - String workerMetricPath = Cluster.monitor_worker_path(topologyId, workerId); - cluster_state.set_data(workerMetricPath, Utils.serialize(metricInfo)); - } - - @Override - public void update_userDef_metric(String topologyId, String workerId, UserDefMetricData metricInfo) throws Exception { - String userMetricPath = Cluster.monitor_user_path(topologyId, workerId); - cluster_state.set_data(userMetricPath, Utils.serialize(metricInfo)); - } - - @Override - public List get_task_metric_list(String topologyId) throws Exception { - List taskMetricList = new ArrayList(); - - String monitorTaskDirPath = Cluster.monitor_taskdir_path(topologyId); - List taskList = cluster_state.get_children(monitorTaskDirPath, false); - - for(String taskId : taskList) { - Object taskMetric = Cluster.maybe_deserialize( - cluster_state.get_data(Cluster.monitor_task_path(topologyId, taskId), false)); - if(taskMetric != null) { - taskMetricList.add((TaskMetricInfo)taskMetric); - } else { - LOG.warn("get_task_metric_list failed, topoId: " + topologyId + " taskId:" + taskId); - } - } - - return taskMetricList; - } - - @Override - public List get_metric_taskIds(String topologyId) throws Exception { - String monitorTaskDirPath = Cluster.monitor_taskdir_path(topologyId); - return cluster_state.get_children(monitorTaskDirPath, false); - } - - @Override - public void remove_metric_task(String topologyId, String taskId) throws Exception { - String monitorTaskPath = Cluster.monitor_task_path(topologyId, taskId); - cluster_state.delete_node(monitorTaskPath); - } - - @Override - public List get_worker_metric_list(String topologyId) throws Exception { - List workerMetricList = new ArrayList(); - - String monitorWorkerDirPath = Cluster.monitor_workerdir_path(topologyId); - List workerList = cluster_state.get_children(monitorWorkerDirPath, false); - - for(String workerId : workerList) { - byte[] byteArray = cluster_state.get_data(Cluster.monitor_worker_path(topologyId, workerId), false); - if(byteArray != null) { - WorkerMetricInfo workerMetric = (WorkerMetricInfo)Cluster.maybe_deserialize(byteArray); - if(workerMetric != null) { - workerMetricList.add(workerMetric); - } - } else { - LOG.warn("get_worker_metric_list failed, workerMetric is null, topoId: " + topologyId + " workerId:" + workerId); - } - } - - return workerMetricList; - } - - @Override - public List get_metric_workerIds(String topologyId) throws Exception { - String monitorWorkerDirPath = Cluster.monitor_workerdir_path(topologyId); - return cluster_state.get_children(monitorWorkerDirPath, false); - } - - @Override - public void remove_metric_worker(String topologyId, String workerId) throws Exception { - String monitorWorkerPath = Cluster.monitor_worker_path(topologyId, workerId); - cluster_state.delete_node(monitorWorkerPath); - } - - @Override - public List get_metric_users(String topologyId) throws Exception { - String monitorUserDirPath = Cluster.monitor_userdir_path(topologyId); - return cluster_state.get_children(monitorUserDirPath, false); - } - - @Override - public void remove_metric_user(String topologyId, String workerId) throws Exception { - String monitorUserPath = Cluster.monitor_user_path(topologyId, workerId); - cluster_state.delete_node(monitorUserPath); - } - - @Override - public TaskMetricInfo get_task_metric(String topologyId, int taskId) throws Exception { - TaskMetricInfo taskMetric = null; - - String monitorTaskPath = Cluster.monitor_task_path(topologyId, String.valueOf(taskId)); - taskMetric = (TaskMetricInfo)(Cluster.maybe_deserialize( - cluster_state.get_data(monitorTaskPath, false))); - - return taskMetric; - } - - @Override - public WorkerMetricInfo get_worker_metric(String topologyId, String workerId) throws Exception { - WorkerMetricInfo workerMetric = null; - - String monitorWorkerPath = Cluster.monitor_worker_path(topologyId, workerId); - workerMetric = (WorkerMetricInfo)(Cluster.maybe_deserialize( - cluster_state.get_data(monitorWorkerPath, false))); - - return workerMetric; - } -} \ No newline at end of file diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/CgroupCenter.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/CgroupCenter.java deleted file mode 100644 index f668e223c..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/CgroupCenter.java +++ /dev/null @@ -1,204 +0,0 @@ -package com.alibaba.jstorm.container; - -import java.io.BufferedReader; -import java.io.File; -import java.io.FileReader; -import java.io.IOException; -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 org.apache.log4j.Logger; - -import com.alibaba.jstorm.container.cgroup.CgroupCommon; - -public class CgroupCenter implements CgroupOperation { - - public static Logger LOG = Logger.getLogger(CgroupCenter.class); - - private static CgroupCenter instance; - - private CgroupCenter() { - - } - - /** - * Thread unsafe - * - * @return - */ - public synchronized static CgroupCenter getInstance() { - if (instance == null) - instance = new CgroupCenter(); - return CgroupUtils.enabled() ? instance : null; - } - - @Override - public List getHierarchies() { - // TODO Auto-generated method stub - Map hierarchies = new HashMap(); - FileReader reader = null; - BufferedReader br = null; - try { - reader = new FileReader(Constants.MOUNT_STATUS_FILE); - br = new BufferedReader(reader); - String str = null; - while ((str = br.readLine()) != null) { - String[] strSplit = str.split(" "); - if (!strSplit[2].equals("cgroup")) - continue; - String name = strSplit[0]; - String type = strSplit[3]; - String dir = strSplit[1]; - Hierarchy h = hierarchies.get(type); - h = new Hierarchy(name, CgroupUtils.analyse(type), dir); - hierarchies.put(type, h); - } - return new ArrayList(hierarchies.values()); - } catch (Exception e) { - LOG.error("Get hierarchies error", e); - } finally { - CgroupUtils.close(reader, br); - } - return null; - } - - @Override - public Set getSubSystems() { - // TODO Auto-generated method stub - Set subSystems = new HashSet(); - FileReader reader = null; - BufferedReader br = null; - try { - reader = new FileReader(Constants.CGROUP_STATUS_FILE); - br = new BufferedReader(reader); - String str = null; - while ((str = br.readLine()) != null) { - String[] split = str.split("\t"); - SubSystemType type = SubSystemType.getSubSystem(split[0]); - if (type == null) - continue; - subSystems.add(new SubSystem(type, Integer.valueOf(split[1]), - Integer.valueOf(split[2]), Integer.valueOf(split[3]) - .intValue() == 1 ? true : false)); - } - return subSystems; - } catch (Exception e) { - LOG.error("Get subSystems error ", e); - } finally { - CgroupUtils.close(reader, br); - } - return null; - } - - @Override - public boolean enabled(SubSystemType subsystem) { - // TODO Auto-generated method stub - Set subSystems = this.getSubSystems(); - for (SubSystem subSystem : subSystems) { - if (subSystem.getType() == subsystem) - return true; - } - return false; - } - - @Override - public Hierarchy busy(SubSystemType subsystem) { - List hierarchies = this.getHierarchies(); - for (Hierarchy hierarchy : hierarchies) { - for (SubSystemType type : hierarchy.getSubSystems()) { - if (type == subsystem) - return hierarchy; - } - } - return null; - } - - @Override - public Hierarchy mounted(Hierarchy hierarchy) { - // TODO Auto-generated method stub - List hierarchies = this.getHierarchies(); - if (CgroupUtils.dirExists(hierarchy.getDir())) { - for (Hierarchy h : hierarchies) { - if (h.equals(hierarchy)) - return h; - } - } - return null; - } - - @Override - public void mount(Hierarchy hierarchy) throws IOException { - // TODO Auto-generated method stub - if (this.mounted(hierarchy) != null) { - LOG.error(hierarchy.getDir() + " is mounted"); - return; - } - Set subsystems = hierarchy.getSubSystems(); - for (SubSystemType type : subsystems) { - if (this.busy(type) != null) { - LOG.error("subsystem: " + type.name() + " is busy"); - subsystems.remove(type); - } - } - if (subsystems.size() == 0) - return; - if (!CgroupUtils.dirExists(hierarchy.getDir())) - new File(hierarchy.getDir()).mkdirs(); - String subSystems = CgroupUtils.reAnalyse(subsystems); - SystemOperation.mount(subSystems, hierarchy.getDir(), "cgroup", - subSystems); - - } - - @Override - public void umount(Hierarchy hierarchy) throws IOException { - // TODO Auto-generated method stub - if (this.mounted(hierarchy) != null) { - hierarchy.getRootCgroups().delete(); - SystemOperation.umount(hierarchy.getDir()); - CgroupUtils.deleteDir(hierarchy.getDir()); - } - } - - @Override - public void create(CgroupCommon cgroup) throws SecurityException { - // TODO Auto-generated method stub - if (cgroup.isRoot()) { - LOG.error("You can't create rootCgroup in this function"); - return; - } - CgroupCommon parent = cgroup.getParent(); - while (parent != null) { - if (!CgroupUtils.dirExists(parent.getDir())) { - LOG.error(parent.getDir() + "is not existed"); - return; - } - parent = parent.getParent(); - } - Hierarchy h = cgroup.getHierarchy(); - if (mounted(h) == null) { - LOG.error(h.getDir() + " is not mounted"); - return; - } - if (CgroupUtils.dirExists(cgroup.getDir())) { - LOG.error(cgroup.getDir() + " is existed"); - return; - } - (new File(cgroup.getDir())).mkdir(); - } - - @Override - public void delete(CgroupCommon cgroup) throws IOException { - // TODO Auto-generated method stub - cgroup.delete(); - } - - public static void main(String args[]) { - System.out.println(CgroupCenter.getInstance().getHierarchies().get(0) - .getRootCgroups().getChildren().size()); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/CgroupOperation.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/CgroupOperation.java deleted file mode 100644 index af51ba274..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/CgroupOperation.java +++ /dev/null @@ -1,29 +0,0 @@ -package com.alibaba.jstorm.container; - -import java.io.IOException; -import java.util.List; -import java.util.Set; - -import com.alibaba.jstorm.container.cgroup.CgroupCommon; - -public interface CgroupOperation { - - public List getHierarchies(); - - public Set getSubSystems(); - - public boolean enabled(SubSystemType subsystem); - - public Hierarchy busy(SubSystemType subsystem); - - public Hierarchy mounted(Hierarchy hierarchy); - - public void mount(Hierarchy hierarchy) throws IOException; - - public void umount(Hierarchy hierarchy) throws IOException; - - public void create(CgroupCommon cgroup) throws SecurityException; - - public void delete(CgroupCommon cgroup) throws IOException; - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/CgroupUtils.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/CgroupUtils.java deleted file mode 100644 index 4348557c9..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/CgroupUtils.java +++ /dev/null @@ -1,158 +0,0 @@ -package com.alibaba.jstorm.container; - -import java.io.BufferedReader; -import java.io.BufferedWriter; -import java.io.File; -import java.io.FileReader; -import java.io.FileWriter; -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -import org.apache.log4j.Logger; - -public class CgroupUtils { - - public static final Logger LOG = Logger.getLogger(CgroupUtils.class); - - public static void deleteDir(String dir) { - try { - String cmd = "rmdir " + dir; - SystemOperation.exec(cmd); - } catch (IOException e) { - // TODO Auto-generated catch block - LOG.error("rm " + dir + " fail!", e); - } - } - - public static boolean fileExists(String dir) { - File file = new File(dir); - return file.exists(); - } - - public static boolean dirExists(String dir) { - File file = new File(dir); - return file.isDirectory(); - } - - public static Set analyse(String str) { - Set result = new HashSet(); - String[] subSystems = str.split(","); - for (String subSystem : subSystems) { - SubSystemType type = SubSystemType.getSubSystem(subSystem); - if (type != null) - result.add(type); - } - return result; - } - - public static String reAnalyse(Set subSystems) { - StringBuilder sb = new StringBuilder(); - if (subSystems.size() == 0) - return sb.toString(); - for (SubSystemType type : subSystems) { - sb.append(type.name()).append(","); - } - return sb.toString().substring(0, sb.length() - 1); - } - - public static boolean enabled() { - return CgroupUtils.fileExists(Constants.CGROUP_STATUS_FILE); - } - - public static List readFileByLine(String fileDir) - throws IOException { - List result = new ArrayList(); - FileReader fileReader = null; - BufferedReader reader = null; - try { - File file = new File(fileDir); - fileReader = new FileReader(file); - reader = new BufferedReader(fileReader); - String tempString = null; - while ((tempString = reader.readLine()) != null) { - result.add(tempString); - } - } finally { - CgroupUtils.close(fileReader, reader); - } - return result; - } - - public static void writeFileByLine(String fileDir, List strings) - throws IOException { - FileWriter writer = null; - BufferedWriter bw = null; - try { - File file = new File(fileDir); - if (!file.exists()) { - LOG.error(fileDir + " is no existed"); - return; - } - writer = new FileWriter(file, true); - bw = new BufferedWriter(writer); - for (String string : strings) { - bw.write(string); - bw.newLine(); - bw.flush(); - } - } finally { - CgroupUtils.close(writer, bw); - } - } - - public static void writeFileByLine(String fileDir, String string) - throws IOException { - FileWriter writer = null; - BufferedWriter bw = null; - try { - File file = new File(fileDir); - if (!file.exists()) { - LOG.error(fileDir + " is no existed"); - return; - } - writer = new FileWriter(file, true); - bw = new BufferedWriter(writer); - bw.write(string); - bw.newLine(); - bw.flush(); - - } finally { - CgroupUtils.close(writer, bw); - } - } - - public static void close(FileReader reader, BufferedReader br) { - try { - if (reader != null) - reader.close(); - if (br != null) - br.close(); - } catch (IOException e) { - // TODO Auto-generated catch block - - } - } - - public static void close(FileWriter writer, BufferedWriter bw) { - try { - if (writer != null) - writer.close(); - if (bw != null) - bw.close(); - } catch (IOException e) { - // TODO Auto-generated catch block - - } - } - - public static void sleep(long s) { - try { - Thread.sleep(s); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - } - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/Constants.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/Constants.java deleted file mode 100644 index dc73cc6d4..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/Constants.java +++ /dev/null @@ -1,13 +0,0 @@ -package com.alibaba.jstorm.container; - -public class Constants { - - public static final String CGROUP_STATUS_FILE = "/proc/cgroups"; - - public static final String MOUNT_STATUS_FILE = "/proc/mounts"; - - public static String getDir(String dir, String constant) { - return dir + constant; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/Hierarchy.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/Hierarchy.java deleted file mode 100644 index 8f8748dc2..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/Hierarchy.java +++ /dev/null @@ -1,92 +0,0 @@ -package com.alibaba.jstorm.container; - -import java.util.Set; - -import com.alibaba.jstorm.container.cgroup.CgroupCommon; - -public class Hierarchy { - - private final String name; - - private final Set subSystems; - - private final String type; - - private final String dir; - - private final CgroupCommon rootCgroups; - - public Hierarchy(String name, Set subSystems, String dir) { - this.name = name; - this.subSystems = subSystems; - this.dir = dir; - this.rootCgroups = new CgroupCommon(this, dir); - this.type = CgroupUtils.reAnalyse(subSystems); - } - - public Set getSubSystems() { - return subSystems; - } - - public String getType() { - return type; - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + ((dir == null) ? 0 : dir.hashCode()); - result = prime * result + ((name == null) ? 0 : name.hashCode()); - result = prime * result + ((type == null) ? 0 : type.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - Hierarchy other = (Hierarchy) obj; - if (dir == null) { - if (other.dir != null) - return false; - } else if (!dir.equals(other.dir)) - return false; - if (name == null) { - if (other.name != null) - return false; - } else if (!name.equals(other.name)) - return false; - if (type == null) { - if (other.type != null) - return false; - } else if (!type.equals(other.type)) - return false; - return true; - } - - public String getDir() { - return dir; - } - - public CgroupCommon getRootCgroups() { - return rootCgroups; - } - - public String getName() { - return name; - } - - public boolean subSystemMounted(SubSystemType subsystem) { - for (SubSystemType type : this.subSystems) { - if (type == subsystem) - return true; - } - return false; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/SubSystem.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/SubSystem.java deleted file mode 100644 index 0fa5d516b..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/SubSystem.java +++ /dev/null @@ -1,53 +0,0 @@ -package com.alibaba.jstorm.container; - -public class SubSystem { - - private SubSystemType type; - - private int hierarchyID; - - private int cgroupsNum; - - private boolean enable; - - public SubSystem(SubSystemType type, int hierarchyID, int cgroupNum, - boolean enable) { - this.type = type; - this.hierarchyID = hierarchyID; - this.cgroupsNum = cgroupNum; - this.enable = enable; - } - - public SubSystemType getType() { - return type; - } - - public void setType(SubSystemType type) { - this.type = type; - } - - public int getHierarchyID() { - return hierarchyID; - } - - public void setHierarchyID(int hierarchyID) { - this.hierarchyID = hierarchyID; - } - - public int getCgroupsNum() { - return cgroupsNum; - } - - public void setCgroupsNum(int cgroupsNum) { - this.cgroupsNum = cgroupsNum; - } - - public boolean isEnable() { - return enable; - } - - public void setEnable(boolean enable) { - this.enable = enable; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/SubSystemType.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/SubSystemType.java deleted file mode 100644 index 088ace82e..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/SubSystemType.java +++ /dev/null @@ -1,31 +0,0 @@ -package com.alibaba.jstorm.container; - -public enum SubSystemType { - - // net_cls,ns is not supposted in ubuntu - blkio, cpu, cpuacct, cpuset, devices, freezer, memory, perf_event, net_cls, net_prio; - - public static SubSystemType getSubSystem(String str) { - if (str.equals("blkio")) - return blkio; - else if (str.equals("cpu")) - return cpu; - else if (str.equals("cpuacct")) - return cpuacct; - else if (str.equals("cpuset")) - return cpuset; - else if (str.equals("devices")) - return devices; - else if (str.equals("freezer")) - return freezer; - else if (str.equals("memory")) - return memory; - else if (str.equals("perf_event")) - return perf_event; - else if (str.equals("net_cls")) - return net_cls; - else if (str.equals("net_prio")) - return net_prio; - return null; - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/SystemOperation.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/SystemOperation.java deleted file mode 100644 index 358040c68..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/SystemOperation.java +++ /dev/null @@ -1,58 +0,0 @@ -package com.alibaba.jstorm.container; - -import java.io.IOException; - -import org.apache.commons.io.IOUtils; -import org.apache.log4j.BasicConfigurator; -import org.apache.log4j.Logger; - -public class SystemOperation { - - public static final Logger LOG = Logger.getLogger(SystemOperation.class); - - public static boolean isRoot() throws IOException { - String result = SystemOperation.exec("echo $EUID").substring(0, 1); - return Integer.valueOf(result.substring(0, result.length())).intValue() == 0 ? true - : false; - }; - - public static void mount(String name, String target, String type, - String data) throws IOException { - StringBuilder sb = new StringBuilder(); - sb.append("mount -t ").append(type).append(" -o ").append(data) - .append(" ").append(name).append(" ").append(target); - SystemOperation.exec(sb.toString()); - } - - public static void umount(String name) throws IOException { - StringBuilder sb = new StringBuilder(); - sb.append("umount ").append(name); - SystemOperation.exec(sb.toString()); - } - - public static String exec(String cmd) throws IOException { - LOG.debug("Shell cmd: " + cmd); - Process process = new ProcessBuilder(new String[] { "/bin/bash", "-c", - cmd }).start(); - try { - process.waitFor(); - String output = IOUtils.toString(process.getInputStream()); - String errorOutput = IOUtils.toString(process.getErrorStream()); - LOG.debug("Shell Output: " + output); - if (errorOutput.length() != 0) { - LOG.error("Shell Error Output: " + errorOutput); - throw new IOException(errorOutput); - } - return output; - } catch (InterruptedException ie) { - throw new IOException(ie.toString()); - } - - } - - public static void main(String[] args) throws IOException { - BasicConfigurator.configure(); - SystemOperation.mount("test", "/cgroup/cpu", "cgroup", "cpu"); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/CgroupCommon.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/CgroupCommon.java deleted file mode 100644 index 9f631d766..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/CgroupCommon.java +++ /dev/null @@ -1,229 +0,0 @@ -package com.alibaba.jstorm.container.cgroup; - -import java.io.File; -import java.io.IOException; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import com.alibaba.jstorm.container.CgroupUtils; -import com.alibaba.jstorm.container.Constants; -import com.alibaba.jstorm.container.Hierarchy; -import com.alibaba.jstorm.container.SubSystemType; -import com.alibaba.jstorm.container.cgroup.core.CgroupCore; - -public class CgroupCommon implements CgroupCommonOperation { - - public static final String TASKS = "/tasks"; - public static final String NOTIFY_ON_RELEASE = "/notify_on_release"; - public static final String RELEASE_AGENT = "/release_agent"; - public static final String CGROUP_CLONE_CHILDREN = "/cgroup.clone_children"; - public static final String CGROUP_EVENT_CONTROL = "/cgroup.event_control"; - public static final String CGROUP_PROCS = "/cgroup.procs"; - - private final Hierarchy hierarchy; - - private final String name; - - private final String dir; - - private final CgroupCommon parent; - - private final Map cores; - - private final boolean isRoot; - - private final Set children = new HashSet(); - - public CgroupCommon(String name, Hierarchy hierarchy, CgroupCommon parent) { - this.name = parent.getName() + "/" + name; - this.hierarchy = hierarchy; - this.parent = parent; - this.dir = parent.getDir() + "/" + name; - this.init(); - cores = CgroupCoreFactory.getInstance(this.hierarchy.getSubSystems(), - this.dir); - this.isRoot = false; - } - - /** - * rootCgroup - */ - public CgroupCommon(Hierarchy hierarchy, String dir) { - this.name = ""; - this.hierarchy = hierarchy; - this.parent = null; - this.dir = dir; - this.init(); - cores = CgroupCoreFactory.getInstance(this.hierarchy.getSubSystems(), - this.dir); - this.isRoot = true; - } - - @Override - public void addTask(int taskId) throws IOException { - // TODO Auto-generated method stub - CgroupUtils.writeFileByLine(Constants.getDir(this.dir, TASKS), - String.valueOf(taskId)); - } - - @Override - public Set getTasks() throws IOException { - List stringTasks = CgroupUtils.readFileByLine(Constants.getDir( - this.dir, TASKS)); - Set tasks = new HashSet(); - for (String task : stringTasks) { - tasks.add(Integer.valueOf(task)); - } - return tasks; - } - - @Override - public void addProcs(int pid) throws IOException { - // TODO Auto-generated method stub - CgroupUtils.writeFileByLine(Constants.getDir(this.dir, CGROUP_PROCS), - String.valueOf(pid)); - } - - @Override - public Set getPids() throws IOException { - // TODO Auto-generated method stub - List stringPids = CgroupUtils.readFileByLine(Constants.getDir( - this.dir, CGROUP_PROCS)); - Set pids = new HashSet(); - for (String task : stringPids) { - pids.add(Integer.valueOf(task)); - } - return pids; - } - - @Override - public void setNotifyOnRelease(boolean flag) throws IOException { - // TODO Auto-generated method stub - CgroupUtils - .writeFileByLine(Constants.getDir(this.dir, NOTIFY_ON_RELEASE), - flag ? "1" : "0"); - } - - @Override - public boolean getNotifyOnRelease() throws IOException { - return CgroupUtils - .readFileByLine(Constants.getDir(this.dir, NOTIFY_ON_RELEASE)) - .get(0).equals("1") ? true : false; - } - - @Override - public void setReleaseAgent(String command) throws IOException { - // TODO Auto-generated method stub - if (!this.isRoot) - return; - CgroupUtils.writeFileByLine(Constants.getDir(this.dir, RELEASE_AGENT), - command); - } - - @Override - public String getReleaseAgent() throws IOException { - if (!this.isRoot) - return null; - return CgroupUtils.readFileByLine( - Constants.getDir(this.dir, RELEASE_AGENT)).get(0); - } - - @Override - public void setCgroupCloneChildren(boolean flag) throws IOException { - // TODO Auto-generated method stub - if (!this.cores.keySet().contains(SubSystemType.cpuset)) - return; - CgroupUtils.writeFileByLine(Constants.getDir(this.dir, - CGROUP_CLONE_CHILDREN), flag ? "1" : "0"); - } - - @Override - public boolean getCgroupCloneChildren() throws IOException { - return CgroupUtils - .readFileByLine( - Constants.getDir(this.dir, CGROUP_CLONE_CHILDREN)) - .get(0).equals("1") ? true : false; - } - - @Override - public void setEventControl(String eventFd, String controlFd, - String... args) throws IOException { - // TODO Auto-generated method stub - StringBuilder sb = new StringBuilder(); - sb.append(eventFd); - sb.append(' '); - sb.append(controlFd); - for (String arg : args) { - sb.append(' '); - sb.append(arg); - } - CgroupUtils - .writeFileByLine( - Constants.getDir(this.dir, CGROUP_EVENT_CONTROL), - sb.toString()); - } - - public Hierarchy getHierarchy() { - return hierarchy; - } - - public String getName() { - return name; - } - - public String getDir() { - return dir; - } - - public CgroupCommon getParent() { - return parent; - } - - public Set getChildren() { - return children; - } - - public boolean isRoot() { - return isRoot; - } - - public Map getCores() { - return cores; - } - - public void delete() throws IOException { - this.free(); - if (!this.isRoot) - this.parent.getChildren().remove(this); - } - - private void free() throws IOException { - for (CgroupCommon child : this.children) - child.free(); - if (this.isRoot) - return; - Set tasks = this.getTasks(); - if (tasks != null) { - for (Integer task : tasks) { - this.parent.addTask(task); - } - } - CgroupUtils.deleteDir(this.dir); - } - - private void init() { - File file = new File(this.dir); - File[] files = file.listFiles(); - if (files == null) - return; - for (File child : files) { - if (child.isDirectory()) { - this.children.add(new CgroupCommon(child.getName(), - this.hierarchy, this)); - } - } - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/CgroupCommonOperation.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/CgroupCommonOperation.java deleted file mode 100644 index c1adbdf5f..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/CgroupCommonOperation.java +++ /dev/null @@ -1,31 +0,0 @@ -package com.alibaba.jstorm.container.cgroup; - -import java.io.IOException; -import java.util.Set; - -public interface CgroupCommonOperation { - - public void addTask(int taskid) throws IOException; - - public Set getTasks() throws IOException; - - public void addProcs(int pid) throws IOException; - - public Set getPids() throws IOException; - - public void setNotifyOnRelease(boolean flag) throws IOException; - - public boolean getNotifyOnRelease() throws IOException; - - public void setReleaseAgent(String command) throws IOException; - - public String getReleaseAgent() throws IOException; - - public void setCgroupCloneChildren(boolean flag) throws IOException; - - public boolean getCgroupCloneChildren() throws IOException; - - public void setEventControl(String eventFd, String controlFd, - String... args) throws IOException; - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/CgroupCoreFactory.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/CgroupCoreFactory.java deleted file mode 100644 index ec8f11748..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/CgroupCoreFactory.java +++ /dev/null @@ -1,60 +0,0 @@ -package com.alibaba.jstorm.container.cgroup; - -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - -import com.alibaba.jstorm.container.SubSystemType; -import com.alibaba.jstorm.container.cgroup.core.BlkioCore; -import com.alibaba.jstorm.container.cgroup.core.CgroupCore; -import com.alibaba.jstorm.container.cgroup.core.CpuCore; -import com.alibaba.jstorm.container.cgroup.core.CpuacctCore; -import com.alibaba.jstorm.container.cgroup.core.CpusetCore; -import com.alibaba.jstorm.container.cgroup.core.DevicesCore; -import com.alibaba.jstorm.container.cgroup.core.FreezerCore; -import com.alibaba.jstorm.container.cgroup.core.MemoryCore; -import com.alibaba.jstorm.container.cgroup.core.NetClsCore; -import com.alibaba.jstorm.container.cgroup.core.NetPrioCore; - -public class CgroupCoreFactory { - - public static Map getInstance( - Set types, String dir) { - Map result = new HashMap(); - for (SubSystemType type : types) { - switch (type) { - case blkio: - result.put(SubSystemType.blkio, new BlkioCore(dir)); - break; - case cpuacct: - result.put(SubSystemType.cpuacct, new CpuacctCore(dir)); - break; - case cpuset: - result.put(SubSystemType.cpuset, new CpusetCore(dir)); - break; - case cpu: - result.put(SubSystemType.cpu, new CpuCore(dir)); - break; - case devices: - result.put(SubSystemType.devices, new DevicesCore(dir)); - break; - case freezer: - result.put(SubSystemType.freezer, new FreezerCore(dir)); - break; - case memory: - result.put(SubSystemType.memory, new MemoryCore(dir)); - break; - case net_cls: - result.put(SubSystemType.net_cls, new NetClsCore(dir)); - break; - case net_prio: - result.put(SubSystemType.net_prio, new NetPrioCore(dir)); - break; - default: - break; - } - } - return result; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/Device.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/Device.java deleted file mode 100644 index 3eddd0f90..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/Device.java +++ /dev/null @@ -1,53 +0,0 @@ -package com.alibaba.jstorm.container.cgroup; - -public class Device { - - public final int major; - public final int minor; - - public Device(int major, int minor) { - this.major = major; - this.minor = minor; - } - - public Device(String str) { - String[] strArgs = str.split(":"); - this.major = Integer.valueOf(strArgs[0]); - this.minor = Integer.valueOf(strArgs[1]); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append(major).append(":").append(minor); - return sb.toString(); - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + major; - result = prime * result + minor; - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - Device other = (Device) obj; - if (major != other.major) - return false; - if (minor != other.minor) - return false; - return true; - } - - - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/BlkioCore.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/BlkioCore.java deleted file mode 100644 index 985725f07..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/BlkioCore.java +++ /dev/null @@ -1,274 +0,0 @@ -package com.alibaba.jstorm.container.cgroup.core; - -import java.util.HashMap; -import java.util.List; -import java.io.IOException; -import java.util.Map; - -import com.alibaba.jstorm.container.CgroupUtils; -import com.alibaba.jstorm.container.Constants; -import com.alibaba.jstorm.container.SubSystemType; -import com.alibaba.jstorm.container.cgroup.Device; - -public class BlkioCore implements CgroupCore { - - public static final String BLKIO_WEIGHT = "/blkio.weight"; - public static final String BLKIO_WEIGHT_DEVICE = "/blkio.weight_device"; - public static final String BLKIO_RESET_STATS = "/blkio.reset_stats"; - - public static final String BLKIO_THROTTLE_READ_BPS_DEVICE = "/blkio.throttle.read_bps_device"; - public static final String BLKIO_THROTTLE_WRITE_BPS_DEVICE = "/blkio.throttle.write_bps_device"; - public static final String BLKIO_THROTTLE_READ_IOPS_DEVICE = "/blkio.throttle.read_iops_device"; - public static final String BLKIO_THROTTLE_WRITE_IOPS_DEVICE = "/blkio.throttle.write_iops_device"; - - public static final String BLKIO_THROTTLE_IO_SERVICED = "/blkio.throttle.io_serviced"; - public static final String BLKIO_THROTTLE_IO_SERVICE_BYTES = "/blkio.throttle.io_service_bytes"; - - public static final String BLKIO_TIME = "/blkio.time"; - public static final String BLKIO_SECTORS = "/blkio.sectors"; - public static final String BLKIO_IO_SERVICED = "/blkio.io_serviced"; - public static final String BLKIO_IO_SERVICE_BYTES = "/blkio.io_service_bytes"; - public static final String BLKIO_IO_SERVICE_TIME = "/blkio.io_service_time"; - public static final String BLKIO_IO_WAIT_TIME = "/blkio.io_wait_time"; - public static final String BLKIO_IO_MERGED = "/blkio.io_merged"; - public static final String BLKIO_IO_QUEUED = "/blkio.io_queued"; - - private final String dir; - - public BlkioCore(String dir) { - this.dir = dir; - } - - @Override - public SubSystemType getType() { - // TODO Auto-generated method stub - return SubSystemType.blkio; - } - - /* weight: 100-1000 */ - public void setBlkioWeight(int weight) throws IOException { - CgroupUtils.writeFileByLine(Constants.getDir(this.dir, BLKIO_WEIGHT), - String.valueOf(weight)); - } - - public int getBlkioWeight() throws IOException { - return Integer.valueOf( - CgroupUtils.readFileByLine( - Constants.getDir(this.dir, BLKIO_WEIGHT)).get(0)) - .intValue(); - } - - public void setBlkioWeightDevice(Device device, int weight) - throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, BLKIO_WEIGHT_DEVICE), - makeContext(device, weight)); - } - - public Map getBlkioWeightDevice() throws IOException { - List strings = CgroupUtils.readFileByLine(Constants.getDir( - this.dir, BLKIO_WEIGHT_DEVICE)); - Map result = new HashMap(); - for (String string : strings) { - String[] strArgs = string.split(" "); - Device device = new Device(strArgs[0]); - Integer weight = Integer.valueOf(strArgs[1]); - result.put(device, weight); - } - return result; - } - - public void setReadBps(Device device, long bps) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, BLKIO_THROTTLE_READ_BPS_DEVICE), - makeContext(device, bps)); - } - - public Map getReadBps() throws IOException { - List strings = CgroupUtils.readFileByLine(Constants.getDir( - this.dir, BLKIO_THROTTLE_READ_BPS_DEVICE)); - Map result = new HashMap(); - for (String string : strings) { - String[] strArgs = string.split(" "); - Device device = new Device(strArgs[0]); - Long bps = Long.valueOf(strArgs[1]); - result.put(device, bps); - } - return result; - } - - public void setWriteBps(Device device, long bps) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, BLKIO_THROTTLE_WRITE_BPS_DEVICE), - makeContext(device, bps)); - } - - public Map getWriteBps() throws IOException { - List strings = CgroupUtils.readFileByLine(Constants.getDir( - this.dir, BLKIO_THROTTLE_WRITE_BPS_DEVICE)); - Map result = new HashMap(); - for (String string : strings) { - String[] strArgs = string.split(" "); - Device device = new Device(strArgs[0]); - Long bps = Long.valueOf(strArgs[1]); - result.put(device, bps); - } - return result; - } - - public void setReadIOps(Device device, long iops) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, BLKIO_THROTTLE_READ_IOPS_DEVICE), - makeContext(device, iops)); - } - - public Map getReadIOps() throws IOException { - List strings = CgroupUtils.readFileByLine(Constants.getDir( - this.dir, BLKIO_THROTTLE_READ_IOPS_DEVICE)); - Map result = new HashMap(); - for (String string : strings) { - String[] strArgs = string.split(" "); - Device device = new Device(strArgs[0]); - Long iops = Long.valueOf(strArgs[1]); - result.put(device, iops); - } - return result; - } - - public void setWriteIOps(Device device, long iops) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, BLKIO_THROTTLE_WRITE_IOPS_DEVICE), - makeContext(device, iops)); - } - - public Map getWriteIOps() throws IOException { - List strings = CgroupUtils.readFileByLine(Constants.getDir( - this.dir, BLKIO_THROTTLE_WRITE_IOPS_DEVICE)); - Map result = new HashMap(); - for (String string : strings) { - String[] strArgs = string.split(" "); - Device device = new Device(strArgs[0]); - Long iops = Long.valueOf(strArgs[1]); - result.put(device, iops); - } - return result; - } - - public Map> getThrottleIOServiced() - throws IOException { - return this.analyseRecord(CgroupUtils.readFileByLine(Constants.getDir( - this.dir, BLKIO_THROTTLE_IO_SERVICED))); - } - - public Map> getThrottleIOServiceByte() - throws IOException { - return this.analyseRecord(CgroupUtils.readFileByLine(Constants.getDir( - this.dir, BLKIO_THROTTLE_IO_SERVICE_BYTES))); - } - - public Map getBlkioTime() throws IOException { - Map result = new HashMap(); - List strs = CgroupUtils.readFileByLine(Constants.getDir( - this.dir, BLKIO_TIME)); - for (String str : strs) { - String[] strArgs = str.split(" "); - result.put(new Device(strArgs[0]), Long.parseLong(strArgs[1])); - } - return result; - } - - public Map getBlkioSectors() throws IOException { - Map result = new HashMap(); - List strs = CgroupUtils.readFileByLine(Constants.getDir( - this.dir, BLKIO_SECTORS)); - for (String str : strs) { - String[] strArgs = str.split(" "); - result.put(new Device(strArgs[0]), Long.parseLong(strArgs[1])); - } - return result; - } - - public Map> getIOServiced() - throws IOException { - return this.analyseRecord(CgroupUtils.readFileByLine(Constants.getDir( - this.dir, BLKIO_IO_SERVICED))); - } - - public Map> getIOServiceBytes() - throws IOException { - return this.analyseRecord(CgroupUtils.readFileByLine(Constants.getDir( - this.dir, BLKIO_IO_SERVICE_BYTES))); - } - - public Map> getIOServiceTime() - throws IOException { - return this.analyseRecord(CgroupUtils.readFileByLine(Constants.getDir( - this.dir, BLKIO_IO_SERVICE_TIME))); - } - - public Map> getIOWaitTime() - throws IOException { - return this.analyseRecord(CgroupUtils.readFileByLine(Constants.getDir( - this.dir, BLKIO_IO_WAIT_TIME))); - } - - public Map> getIOMerged() throws IOException { - return this.analyseRecord(CgroupUtils.readFileByLine(Constants.getDir( - this.dir, BLKIO_IO_MERGED))); - } - - public Map> getIOQueued() throws IOException { - return this.analyseRecord(CgroupUtils.readFileByLine(Constants.getDir( - this.dir, BLKIO_IO_QUEUED))); - } - - public void resetStats() throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, BLKIO_RESET_STATS), "1"); - } - - private String makeContext(Device device, Object data) { - StringBuilder sb = new StringBuilder(); - sb.append(device.toString()).append(" ").append(data); - return sb.toString(); - } - - private Map> analyseRecord(List strs) { - Map> result = new HashMap>(); - for (String str : strs) { - String[] strArgs = str.split(" "); - if (strArgs.length != 3) - continue; - Device device = new Device(strArgs[0]); - RecordType key = RecordType.getType(strArgs[1]); - Long value = Long.parseLong(strArgs[2]); - Map record = result.get(device); - if (record == null) { - record = new HashMap(); - result.put(device, record); - } - record.put(key, value); - } - return result; - } - - public enum RecordType { - read, write, sync, async, total; - - public static RecordType getType(String type) { - if (type.equals("Read")) - return read; - else if (type.equals("Write")) - return write; - else if (type.equals("Sync")) - return sync; - else if (type.equals("Async")) - return async; - else if (type.equals("Total")) - return total; - else - return null; - } - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/CgroupCore.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/CgroupCore.java deleted file mode 100644 index 6a50cabbf..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/CgroupCore.java +++ /dev/null @@ -1,10 +0,0 @@ -package com.alibaba.jstorm.container.cgroup.core; - -import com.alibaba.jstorm.container.SubSystemType; - - -public interface CgroupCore { - - public SubSystemType getType(); - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/CpuCore.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/CpuCore.java deleted file mode 100644 index 67f0cfa33..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/CpuCore.java +++ /dev/null @@ -1,132 +0,0 @@ -package com.alibaba.jstorm.container.cgroup.core; - -import java.io.IOException; -import java.util.List; - -import com.alibaba.jstorm.container.CgroupUtils; -import com.alibaba.jstorm.container.Constants; -import com.alibaba.jstorm.container.SubSystemType; - -public class CpuCore implements CgroupCore { - - public static final String CPU_SHARES = "/cpu.shares"; - public static final String CPU_RT_RUNTIME_US = "/cpu.rt_runtime_us"; - public static final String CPU_RT_PERIOD_US = "/cpu.rt_period_us"; - public static final String CPU_CFS_PERIOD_US = "/cpu.cfs_period_us"; - public static final String CPU_CFS_QUOTA_US = "/cpu.cfs_quota_us"; - public static final String CPU_STAT = "/cpu.stat"; - - private final String dir; - - public CpuCore(String dir) { - this.dir = dir; - } - - @Override - public SubSystemType getType() { - // TODO Auto-generated method stub - return SubSystemType.cpu; - } - - public void setCpuShares(int weight) throws IOException { - CgroupUtils.writeFileByLine(Constants.getDir(this.dir, CPU_SHARES), - String.valueOf(weight)); - } - - public int getCpuShares() throws IOException { - return Integer.parseInt(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, CPU_SHARES)).get(0)); - } - - public void setCpuRtRuntimeUs(long us) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, CPU_RT_RUNTIME_US), - String.valueOf(us)); - } - - public long getCpuRtRuntimeUs() throws IOException { - return Long.parseLong(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, CPU_RT_RUNTIME_US)).get(0)); - } - - public void setCpuRtPeriodUs(long us) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, CPU_RT_PERIOD_US), - String.valueOf(us)); - } - - public Long getCpuRtPeriodUs() throws IOException { - return Long.parseLong(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, CPU_RT_PERIOD_US)).get(0)); - } - - public void setCpuCfsPeriodUs(long us) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, CPU_CFS_PERIOD_US), - String.valueOf(us)); - } - - public Long getCpuCfsPeriodUs(long us) throws IOException { - return Long.parseLong(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, CPU_CFS_PERIOD_US)).get(0)); - } - - public void setCpuCfsQuotaUs(long us) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, CPU_CFS_QUOTA_US), - String.valueOf(us)); - } - - public Long getCpuCfsQuotaUs(long us) throws IOException { - return Long.parseLong(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, CPU_CFS_QUOTA_US)).get(0)); - } - - public Stat getCpuStat() throws IOException { - return new Stat(CgroupUtils.readFileByLine(Constants.getDir(this.dir, - CPU_STAT))); - } - - public static class Stat { - public final int nrPeriods; - public final int nrThrottled; - public final int throttledTime; - - public Stat(List statStr) { - this.nrPeriods = Integer.parseInt(statStr.get(0).split(" ")[1]); - this.nrThrottled = Integer.parseInt(statStr.get(1).split(" ")[1]); - this.throttledTime = Integer.parseInt(statStr.get(2).split(" ")[1]); - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + nrPeriods; - result = prime * result + nrThrottled; - result = prime * result + throttledTime; - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - Stat other = (Stat) obj; - if (nrPeriods != other.nrPeriods) - return false; - if (nrThrottled != other.nrThrottled) - return false; - if (throttledTime != other.throttledTime) - return false; - return true; - } - - - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/CpuacctCore.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/CpuacctCore.java deleted file mode 100644 index 346c5c6a9..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/CpuacctCore.java +++ /dev/null @@ -1,59 +0,0 @@ -package com.alibaba.jstorm.container.cgroup.core; - -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import com.alibaba.jstorm.container.CgroupUtils; -import com.alibaba.jstorm.container.Constants; -import com.alibaba.jstorm.container.SubSystemType; - -public class CpuacctCore implements CgroupCore { - - public static final String CPUACCT_USAGE = "/cpuacct.usage"; - public static final String CPUACCT_STAT = "/cpuacct.stat"; - public static final String CPUACCT_USAGE_PERCPU = "/cpuacct.usage_percpu"; - - private final String dir; - - public CpuacctCore(String dir) { - this.dir = dir; - } - - @Override - public SubSystemType getType() { - // TODO Auto-generated method stub - return SubSystemType.cpuacct; - } - - public Long getCpuUsage() throws IOException { - return Long.parseLong(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, CPUACCT_USAGE)).get(0)); - } - - public Map getCpuStat() throws IOException { - List strs = CgroupUtils.readFileByLine(Constants.getDir( - this.dir, CPUACCT_STAT)); - Map result = new HashMap(); - result.put(StatType.user, Long.parseLong(strs.get(0).split(" ")[1])); - result.put(StatType.system, Long.parseLong(strs.get(1).split(" ")[1])); - return result; - } - - public Long[] getPerCpuUsage() throws IOException { - String str = CgroupUtils.readFileByLine( - Constants.getDir(this.dir, CPUACCT_USAGE_PERCPU)).get(0); - String[] strArgs = str.split(" "); - Long[] result = new Long[strArgs.length]; - for (int i = 0; i < result.length; i++) { - result[i] = Long.parseLong(strArgs[i]); - } - return result; - } - - public enum StatType { - user, system; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/CpusetCore.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/CpusetCore.java deleted file mode 100644 index 25b72eba3..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/CpusetCore.java +++ /dev/null @@ -1,232 +0,0 @@ -package com.alibaba.jstorm.container.cgroup.core; - -import java.io.IOException; -import java.util.LinkedList; - -import com.alibaba.jstorm.container.CgroupUtils; -import com.alibaba.jstorm.container.Constants; -import com.alibaba.jstorm.container.SubSystemType; - -public class CpusetCore implements CgroupCore { - - public static final String CPUSET_CPUS = "/cpuset.cpus"; - public static final String CPUSET_MEMS = "/cpuset.mems"; - public static final String CPUSET_MEMORY_MIGRATE = "/cpuset.memory_migrate"; - public static final String CPUSET_CPU_EXCLUSIVE = "/cpuset.cpu_exclusive"; - public static final String CPUSET_MEM_EXCLUSIVE = "/cpuset.mem_exclusive"; - public static final String CPUSET_MEM_HARDWALL = "/cpuset.mem_hardwall"; - public static final String CPUSET_MEMORY_PRESSURE = "/cpuset.memory_pressure"; - public static final String CPUSET_MEMORY_PRESSURE_ENABLED = "/cpuset.memory_pressure_enabled"; - public static final String CPUSET_MEMORY_SPREAD_PAGE = "/cpuset.memory_spread_page"; - public static final String CPUSET_MEMORY_SPREAD_SLAB = "/cpuset.memory_spread_slab"; - public static final String CPUSET_SCHED_LOAD_BALANCE = "/cpuset.sched_load_balance"; - public static final String CPUSET_SCHED_RELAX_DOMAIN_LEVEL = "/cpuset.sched_relax_domain_level"; - - private final String dir; - - public CpusetCore(String dir) { - this.dir = dir; - } - - @Override - public SubSystemType getType() { - // TODO Auto-generated method stub - return SubSystemType.cpuset; - } - - public void setCpus(int[] nums) throws IOException { - StringBuilder sb = new StringBuilder(); - for (int num : nums) { - sb.append(num); - sb.append(','); - } - sb.deleteCharAt(sb.length() - 1); - CgroupUtils.writeFileByLine(Constants.getDir(this.dir, CPUSET_CPUS), - sb.toString()); - } - - public int[] getCpus() throws IOException { - String output = CgroupUtils.readFileByLine( - Constants.getDir(this.dir, CPUSET_CPUS)).get(0); - return parseNums(output); - } - - public void setMems(int[] nums) throws IOException { - StringBuilder sb = new StringBuilder(); - for (int num : nums) { - sb.append(num); - sb.append(','); - } - sb.deleteCharAt(sb.length() - 1); - CgroupUtils.writeFileByLine(Constants.getDir(this.dir, CPUSET_MEMS), - sb.toString()); - } - - public int[] getMems() throws IOException { - String output = CgroupUtils.readFileByLine( - Constants.getDir(this.dir, CPUSET_MEMS)).get(0); - return parseNums(output); - } - - public void setMemMigrate(boolean flag) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, CPUSET_MEMORY_MIGRATE), - String.valueOf(flag ? 1 : 0)); - } - - public boolean isMemMigrate() throws IOException { - int output = Integer.parseInt(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, CPUSET_MEMORY_MIGRATE)).get(0)); - return output > 0; - } - - public void setCpuExclusive(boolean flag) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, CPUSET_CPU_EXCLUSIVE), - String.valueOf(flag ? 1 : 0)); - } - - public boolean isCpuExclusive() throws IOException { - int output = Integer.parseInt(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, CPUSET_CPU_EXCLUSIVE)).get(0)); - return output > 0; - } - - public void setMemExclusive(boolean flag) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, CPUSET_MEM_EXCLUSIVE), - String.valueOf(flag ? 1 : 0)); - } - - public boolean isMemExclusive() throws IOException { - int output = Integer.parseInt(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, CPUSET_MEM_EXCLUSIVE)).get(0)); - return output > 0; - } - - public void setMemHardwall(boolean flag) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, CPUSET_MEM_HARDWALL), - String.valueOf(flag ? 1 : 0)); - } - - public boolean isMemHardwall() throws IOException { - int output = Integer.parseInt(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, CPUSET_MEM_HARDWALL)).get(0)); - return output > 0; - } - - public int getMemPressure() throws IOException { - String output = CgroupUtils.readFileByLine( - Constants.getDir(this.dir, CPUSET_MEMORY_PRESSURE)).get(0); - return Integer.parseInt(output); - } - - public void setMemPressureEnabled(boolean flag) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, CPUSET_MEMORY_PRESSURE_ENABLED), - String.valueOf(flag ? 1 : 0)); - } - - public boolean isMemPressureEnabled() throws IOException { - int output = Integer.parseInt(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, CPUSET_MEMORY_PRESSURE_ENABLED)) - .get(0)); - return output > 0; - } - - public void setMemSpreadPage(boolean flag) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, CPUSET_MEMORY_SPREAD_PAGE), - String.valueOf(flag ? 1 : 0)); - } - - public boolean isMemSpreadPage() throws IOException { - int output = Integer.parseInt(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, CPUSET_MEMORY_SPREAD_PAGE)).get(0)); - return output > 0; - } - - public void setMemSpreadSlab(boolean flag) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, CPUSET_MEMORY_SPREAD_SLAB), - String.valueOf(flag ? 1 : 0)); - } - - public boolean isMemSpreadSlab() throws IOException { - int output = Integer.parseInt(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, CPUSET_MEMORY_SPREAD_SLAB)).get(0)); - return output > 0; - } - - public void setSchedLoadBlance(boolean flag) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, CPUSET_SCHED_LOAD_BALANCE), - String.valueOf(flag ? 1 : 0)); - } - - public boolean isSchedLoadBlance() throws IOException { - int output = Integer.parseInt(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, CPUSET_SCHED_LOAD_BALANCE)).get(0)); - return output > 0; - } - - public void setSchedRelaxDomainLevel(int value) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, CPUSET_SCHED_RELAX_DOMAIN_LEVEL), - String.valueOf(value)); - } - - public int getSchedRelaxDomainLevel() throws IOException { - String output = CgroupUtils.readFileByLine( - Constants.getDir(this.dir, CPUSET_SCHED_RELAX_DOMAIN_LEVEL)) - .get(0); - return Integer.parseInt(output); - } - - public static int[] parseNums(String outputStr) { - char[] output = outputStr.toCharArray(); - LinkedList numList = new LinkedList(); - int value = 0; - int start = 0; - boolean isHyphen = false; - for (char ch : output) { - if (ch == ',') { - if (isHyphen) { - for (; start <= value; start++) { - numList.add(start); - } - isHyphen = false; - } else { - numList.add(value); - } - value = 0; - } else if (ch == '-') { - isHyphen = true; - start = value; - value = 0; - } else { - value = value * 10 + (ch - '0'); - } - } - if (output[output.length - 1] != ',') { - if (isHyphen) { - for (; start <= value; start++) { - numList.add(start); - } - } else { - numList.add(value); - } - } - - int[] nums = new int[numList.size()]; - int index = 0; - for (int num : numList) { - nums[index] = num; - index++; - } - - return nums; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/DevicesCore.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/DevicesCore.java deleted file mode 100644 index af79f85e9..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/DevicesCore.java +++ /dev/null @@ -1,170 +0,0 @@ -package com.alibaba.jstorm.container.cgroup.core; - -import java.io.IOException; -import java.util.List; - -import com.alibaba.jstorm.container.CgroupUtils; -import com.alibaba.jstorm.container.Constants; -import com.alibaba.jstorm.container.SubSystemType; -import com.alibaba.jstorm.container.cgroup.Device; - -public class DevicesCore implements CgroupCore { - - private final String dir; - - public static final String DEVICES_ALLOW = "/devices.allow"; - public static final String DEVICES_DENY = "/devices.deny"; - public static final String DEVICES_LIST = "/devices.list"; - - public static final char TYPE_ALL = 'a'; - public static final char TYPE_BLOCK = 'b'; - public static final char TYPE_CHAR = 'c'; - - public static final int ACCESS_READ = 1; - public static final int ACCESS_WRITE = 2; - public static final int ACCESS_CREATE = 4; - - public static final char ACCESS_READ_CH = 'r'; - public static final char ACCESS_WRITE_CH = 'w'; - public static final char ACCESS_CREATE_CH = 'm'; - - public DevicesCore(String dir) { - this.dir = dir; - } - - @Override - public SubSystemType getType() { - // TODO Auto-generated method stub - return SubSystemType.devices; - } - - public static class Record { - Device device; - char type; - int accesses; - - public Record(char type, Device device, int accesses) { - this.type = type; - this.device = device; - this.accesses = accesses; - } - - public Record(String output) { - if (output.contains("*")) { - System.out.println("Pre:" + output); - output = output.replaceAll("\\*", "-1"); - System.out.println("After:" + output); - } - String[] splits = output.split("[: ]"); - type = splits[0].charAt(0); - int major = Integer.parseInt(splits[1]); - int minor = Integer.parseInt(splits[2]); - device = new Device(major, minor); - accesses = 0; - for (char c : splits[3].toCharArray()) { - if (c == ACCESS_READ_CH) { - accesses |= ACCESS_READ; - } - if (c == ACCESS_CREATE_CH) { - accesses |= ACCESS_CREATE; - } - if (c == ACCESS_WRITE_CH) { - accesses |= ACCESS_WRITE; - } - } - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append(type); - sb.append(' '); - sb.append(device.major); - sb.append(':'); - sb.append(device.minor); - sb.append(' '); - sb.append(getAccessesFlag(accesses)); - - return sb.toString(); - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + accesses; - result = prime * result - + ((device == null) ? 0 : device.hashCode()); - result = prime * result + type; - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - Record other = (Record) obj; - if (accesses != other.accesses) - return false; - if (device == null) { - if (other.device != null) - return false; - } else if (!device.equals(other.device)) - return false; - if (type != other.type) - return false; - return true; - } - - public static Record[] parseRecordList(List output) { - Record[] records = new Record[output.size()]; - for (int i = 0, l = output.size(); i < l; i++) { - records[i] = new Record(output.get(i)); - } - - return records; - } - - public static StringBuilder getAccessesFlag(int accesses) { - StringBuilder sb = new StringBuilder(); - if ((accesses & ACCESS_READ) != 0) { - sb.append(ACCESS_READ_CH); - } - if ((accesses & ACCESS_WRITE) != 0) { - sb.append(ACCESS_WRITE_CH); - } - if ((accesses & ACCESS_CREATE) != 0) { - sb.append(ACCESS_CREATE_CH); - } - return sb; - } - } - - private void setPermission(String prop, char type, Device device, - int accesses) throws IOException { - Record record = new Record(type, device, accesses); - CgroupUtils.writeFileByLine(Constants.getDir(this.dir, prop), - record.toString()); - } - - public void setAllow(char type, Device device, int accesses) - throws IOException { - setPermission(DEVICES_ALLOW, type, device, accesses); - } - - public void setDeny(char type, Device device, int accesses) - throws IOException { - setPermission(DEVICES_DENY, type, device, accesses); - } - - public Record[] getList() throws IOException { - List output = CgroupUtils.readFileByLine(Constants.getDir( - this.dir, DEVICES_LIST)); - return Record.parseRecordList(output); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/FreezerCore.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/FreezerCore.java deleted file mode 100644 index 66eedb0c1..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/FreezerCore.java +++ /dev/null @@ -1,50 +0,0 @@ -package com.alibaba.jstorm.container.cgroup.core; - -import java.io.IOException; - -import com.alibaba.jstorm.container.CgroupUtils; -import com.alibaba.jstorm.container.Constants; -import com.alibaba.jstorm.container.SubSystemType; - -public class FreezerCore implements CgroupCore { - - public static final String FREEZER_STATE = "/freezer.state"; - - private final String dir; - - public FreezerCore(String dir) { - this.dir = dir; - } - - @Override - public SubSystemType getType() { - // TODO Auto-generated method stub - return SubSystemType.freezer; - } - - public void setState(State state) throws IOException { - CgroupUtils.writeFileByLine(Constants.getDir(this.dir, FREEZER_STATE), - state.name().toUpperCase()); - } - - public State getState() throws IOException { - return State.getStateValue(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, FREEZER_STATE)).get(0)); - } - - public enum State { - frozen, freezing, thawed; - - public static State getStateValue(String state) { - if (state.equals("FROZEN")) - return frozen; - else if (state.equals("FREEZING")) - return freezing; - else if (state.equals("THAWED")) - return thawed; - else - return null; - } - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/MemoryCore.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/MemoryCore.java deleted file mode 100644 index fa24dea67..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/MemoryCore.java +++ /dev/null @@ -1,204 +0,0 @@ -package com.alibaba.jstorm.container.cgroup.core; - -import java.io.IOException; - -import com.alibaba.jstorm.container.CgroupUtils; -import com.alibaba.jstorm.container.Constants; -import com.alibaba.jstorm.container.SubSystemType; - -public class MemoryCore implements CgroupCore { - - public static final String MEMORY_STAT = "/memory.stat"; - public static final String MEMORY_USAGE_IN_BYTES = "/memory.usage_in_bytes"; - public static final String MEMORY_MEMSW_USAGE_IN_BYTES = "/memory.memsw.usage_in_bytes"; - public static final String MEMORY_MAX_USAGE_IN_BYTES = "/memory.max_usage_in_bytes"; - public static final String MEMORY_MEMSW_MAX_USAGE_IN_BYTES = "/memory.memsw.max_usage_in_bytes"; - public static final String MEMORY_LIMIT_IN_BYTES = "/memory.limit_in_bytes"; - public static final String MEMORY_MEMSW_LIMIT_IN_BYTES = "/memory.memsw.limit_in_bytes"; - public static final String MEMORY_FAILCNT = "/memory.failcnt"; - public static final String MEMORY_MEMSW_FAILCNT = "/memory.memsw.failcnt"; - public static final String MEMORY_FORCE_EMPTY = "/memory.force_empty"; - public static final String MEMORY_SWAPPINESS = "/memory.swappiness"; - public static final String MEMORY_USE_HIERARCHY = "/memory.use_hierarchy"; - public static final String MEMORY_OOM_CONTROL = "/memory.oom_control"; - - private final String dir; - - public MemoryCore(String dir) { - this.dir = dir; - } - - @Override - public SubSystemType getType() { - // TODO Auto-generated method stub - return SubSystemType.memory; - } - - public static class Stat { - public final long cacheSize; - public final long rssSize; - public final long mappedFileSize; - public final long pgpginNum; - public final long pgpgoutNum; - public final long swapSize; - public final long activeAnonSize; - public final long inactiveAnonSize; - public final long activeFileSize; - public final long inactiveFileSize; - public final long unevictableSize; - public final long hierarchicalMemoryLimitSize; - public final long hierarchicalMemswLimitSize; - public final long totalCacheSize; - public final long totalRssSize; - public final long totalMappedFileSize; - public final long totalPgpginNum; - public final long totalPgpgoutNum; - public final long totalSwapSize; - public final long totalActiveAnonSize; - public final long totalInactiveAnonSize; - public final long totalActiveFileSize; - public final long totalInactiveFileSize; - public final long totalUnevictableSize; - public final long totalHierarchicalMemoryLimitSize; - public final long totalHierarchicalMemswLimitSize; - - public Stat(String output) { - String[] splits = output.split("\n"); - this.cacheSize = Long.parseLong(splits[0]); - this.rssSize = Long.parseLong(splits[1]); - this.mappedFileSize = Long.parseLong(splits[2]); - this.pgpginNum = Long.parseLong(splits[3]); - this.pgpgoutNum = Long.parseLong(splits[4]); - this.swapSize = Long.parseLong(splits[5]); - this.inactiveAnonSize = Long.parseLong(splits[6]); - this.activeAnonSize = Long.parseLong(splits[7]); - this.inactiveFileSize = Long.parseLong(splits[8]); - this.activeFileSize = Long.parseLong(splits[9]); - this.unevictableSize = Long.parseLong(splits[10]); - this.hierarchicalMemoryLimitSize = Long.parseLong(splits[11]); - this.hierarchicalMemswLimitSize = Long.parseLong(splits[12]); - this.totalCacheSize = Long.parseLong(splits[13]); - this.totalRssSize = Long.parseLong(splits[14]); - this.totalMappedFileSize = Long.parseLong(splits[15]); - this.totalPgpginNum = Long.parseLong(splits[16]); - this.totalPgpgoutNum = Long.parseLong(splits[17]); - this.totalSwapSize = Long.parseLong(splits[18]); - this.totalInactiveAnonSize = Long.parseLong(splits[19]); - this.totalActiveAnonSize = Long.parseLong(splits[20]); - this.totalInactiveFileSize = Long.parseLong(splits[21]); - this.totalActiveFileSize = Long.parseLong(splits[22]); - this.totalUnevictableSize = Long.parseLong(splits[23]); - this.totalHierarchicalMemoryLimitSize = Long.parseLong(splits[24]); - this.totalHierarchicalMemswLimitSize = Long.parseLong(splits[25]); - } - } - - public Stat getStat() throws IOException { - String output = CgroupUtils.readFileByLine( - Constants.getDir(this.dir, MEMORY_STAT)).get(0); - Stat stat = new Stat(output); - return stat; - } - - public long getPhysicalUsage() throws IOException { - return Long.parseLong(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, MEMORY_USAGE_IN_BYTES)).get(0)); - } - - public long getWithSwapUsage() throws IOException { - return Long - .parseLong(CgroupUtils - .readFileByLine( - Constants.getDir(this.dir, - MEMORY_MEMSW_USAGE_IN_BYTES)).get(0)); - } - - public long getMaxPhysicalUsage() throws IOException { - return Long.parseLong(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, MEMORY_MAX_USAGE_IN_BYTES)).get(0)); - } - - public long getMaxWithSwapUsage() throws IOException { - return Long.parseLong(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, MEMORY_MEMSW_MAX_USAGE_IN_BYTES)) - .get(0)); - } - - public void setPhysicalUsageLimit(long value) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, MEMORY_LIMIT_IN_BYTES), - String.valueOf(value)); - } - - public long getPhysicalUsageLimit() throws IOException { - return Long.parseLong(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, MEMORY_LIMIT_IN_BYTES)).get(0)); - } - - public void setWithSwapUsageLimit(long value) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, MEMORY_MEMSW_LIMIT_IN_BYTES), - String.valueOf(value)); - } - - public long getWithSwapUsageLimit() throws IOException { - return Long - .parseLong(CgroupUtils - .readFileByLine( - Constants.getDir(this.dir, - MEMORY_MEMSW_LIMIT_IN_BYTES)).get(0)); - } - - public int getPhysicalFailCount() throws IOException { - return Integer.parseInt(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, MEMORY_FAILCNT)).get(0)); - } - - public int getWithSwapFailCount() throws IOException { - return Integer.parseInt(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, MEMORY_MEMSW_FAILCNT)).get(0)); - } - - public void clearForceEmpty() throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, MEMORY_FORCE_EMPTY), - String.valueOf(0)); - } - - public void setSwappiness(int value) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, MEMORY_SWAPPINESS), - String.valueOf(value)); - } - - public int getSwappiness() throws IOException { - return Integer.parseInt(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, MEMORY_SWAPPINESS)).get(0)); - } - - public void setUseHierarchy(boolean flag) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, MEMORY_USE_HIERARCHY), - String.valueOf(flag ? 1 : 0)); - } - - public boolean isUseHierarchy() throws IOException { - int output = Integer.parseInt(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, MEMORY_USE_HIERARCHY)).get(0)); - return output > 0; - } - - public void setOomControl(boolean flag) throws IOException { - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, MEMORY_OOM_CONTROL), - String.valueOf(flag ? 1 : 0)); - } - - public boolean isOomControl() throws IOException { - String output = CgroupUtils.readFileByLine( - Constants.getDir(this.dir, MEMORY_OOM_CONTROL)).get(0); - output = output.split("\n")[0].split("[\\s]")[1]; - int value = Integer.parseInt(output); - return value > 0; - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/NetClsCore.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/NetClsCore.java deleted file mode 100644 index ba935b093..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/NetClsCore.java +++ /dev/null @@ -1,57 +0,0 @@ -package com.alibaba.jstorm.container.cgroup.core; - -import java.io.IOException; - -import com.alibaba.jstorm.container.CgroupUtils; -import com.alibaba.jstorm.container.Constants; -import com.alibaba.jstorm.container.SubSystemType; -import com.alibaba.jstorm.container.cgroup.Device; - -public class NetClsCore implements CgroupCore { - - public static final String NET_CLS_CLASSID = "/net_cls.classid"; - - private final String dir; - - public NetClsCore(String dir) { - this.dir = dir; - } - - @Override - public SubSystemType getType() { - // TODO Auto-generated method stub - return SubSystemType.net_cls; - } - - private StringBuilder toHex(int num) { - String hex = num + ""; - StringBuilder sb = new StringBuilder(); - int l = hex.length(); - if (l > 4) { - hex = hex.substring(l - 4 - 1, l); - } - for (; l < 4; l++) { - sb.append('0'); - } - sb.append(hex); - return sb; - } - - public void setClassId(int major, int minor) throws IOException { - StringBuilder sb = new StringBuilder("0x"); - sb.append(toHex(major)); - sb.append(toHex(minor)); - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, NET_CLS_CLASSID), sb.toString()); - } - - public Device getClassId() throws IOException { - String output = CgroupUtils.readFileByLine( - Constants.getDir(this.dir, NET_CLS_CLASSID)).get(0); - output = Integer.toHexString(Integer.parseInt(output)); - int major = Integer.parseInt(output.substring(0, output.length() - 4)); - int minor = Integer.parseInt(output.substring(output.length() - 4)); - return new Device(major, minor); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/NetPrioCore.java b/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/NetPrioCore.java deleted file mode 100644 index 664d2f5ae..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/container/cgroup/core/NetPrioCore.java +++ /dev/null @@ -1,54 +0,0 @@ -package com.alibaba.jstorm.container.cgroup.core; - -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import com.alibaba.jstorm.container.CgroupUtils; -import com.alibaba.jstorm.container.Constants; -import com.alibaba.jstorm.container.SubSystemType; - -public class NetPrioCore implements CgroupCore { - - public static final String NET_PRIO_PRIOIDX = "/net_prio.prioidx"; - public static final String NET_PRIO_IFPRIOMAP = "/net_prio.ifpriomap"; - - private final String dir; - - public NetPrioCore(String dir) { - this.dir = dir; - } - - @Override - public SubSystemType getType() { - // TODO Auto-generated method stub - return SubSystemType.net_prio; - } - - public int getPrioId() throws IOException { - return Integer.parseInt(CgroupUtils.readFileByLine( - Constants.getDir(this.dir, NET_PRIO_PRIOIDX)).get(0)); - } - - public void setIfPrioMap(String iface, int priority) throws IOException { - StringBuilder sb = new StringBuilder(); - sb.append(iface); - sb.append(' '); - sb.append(priority); - CgroupUtils.writeFileByLine( - Constants.getDir(this.dir, NET_PRIO_IFPRIOMAP), sb.toString()); - } - - public Map getIfPrioMap() throws IOException { - Map result = new HashMap(); - List strs = CgroupUtils.readFileByLine(Constants.getDir( - this.dir, NET_PRIO_IFPRIOMAP)); - for (String str : strs) { - String[] strArgs = str.split(" "); - result.put(strArgs[0], Integer.valueOf(strArgs[1])); - } - return result; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/DefaultInimbus.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/DefaultInimbus.java deleted file mode 100644 index 2126fcf3f..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/DefaultInimbus.java +++ /dev/null @@ -1,55 +0,0 @@ -package com.alibaba.jstorm.daemon.nimbus; - -import java.util.Collection; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -import backtype.storm.scheduler.INimbus; -import backtype.storm.scheduler.IScheduler; -import backtype.storm.scheduler.SupervisorDetails; -import backtype.storm.scheduler.Topologies; -import backtype.storm.scheduler.WorkerSlot; - -public class DefaultInimbus implements INimbus { - - @Override - public void prepare(Map stormConf, String schedulerLocalDir) { - // TODO Auto-generated method stub - - } - - @Override - public Collection allSlotsAvailableForScheduling( - Collection existingSupervisors, - Topologies topologies, Set topologiesMissingAssignments) { - // TODO Auto-generated method stub - Collection result = new HashSet(); - for (SupervisorDetails detail : existingSupervisors) { - for (Integer port : detail.getAllPorts()) - result.add(new WorkerSlot(detail.getId(), port)); - } - return result; - } - - @Override - public void assignSlots(Topologies topologies, - Map> newSlotsByTopologyId) { - // TODO Auto-generated method stub - - } - - @Override - public String getHostName( - Map existingSupervisors, String nodeId) { - // TODO Auto-generated method stub - return null; - } - - @Override - public IScheduler getForcedScheduler() { - // TODO Auto-generated method stub - return null; - } - -} 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 deleted file mode 100644 index 864519bed..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusData.java +++ /dev/null @@ -1,234 +0,0 @@ -package com.alibaba.jstorm.daemon.nimbus; - -import java.util.HashMap; -import java.util.Map; -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; - -import org.apache.commons.lang.StringUtils; - -import backtype.storm.generated.ThriftResourceType; -import backtype.storm.scheduler.INimbus; -import backtype.storm.utils.TimeCacheMap; - -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.Cluster; -import com.alibaba.jstorm.cluster.StormClusterState; -import com.alibaba.jstorm.cluster.StormConfig; -import com.alibaba.jstorm.task.TkHbCacheTime; -import com.alibaba.jstorm.utils.TimeUtils; - -/** - * All nimbus data - * - */ -public class NimbusData { - - private Map conf; - - private StormClusterState stormClusterState; - - // Map> - private ConcurrentHashMap> taskHeartbeatsCache; - - // TODO two kind of value:Channel/BufferFileInputStream - private TimeCacheMap downloaders; - private TimeCacheMap uploaders; - - private int startTime; - - private final ScheduledExecutorService scheduExec; - - private AtomicInteger submittedCount; - - private Object submitLock = new Object(); - - private StatusTransition statusTransition; - - private static final int SCHEDULE_THREAD_NUM = 8; - - private final INimbus inimubs; - - private Map>> groupToTopology; - - private Map> groupToResource; - - private Map> groupToUsedResource; - - private final boolean localMode; - - private volatile boolean isLeader; - - private AtomicBoolean isShutdown = new AtomicBoolean(false); - - @SuppressWarnings({ "unchecked", "rawtypes" }) - public NimbusData(Map conf, TimeCacheMap downloaders, - TimeCacheMap uploaders, INimbus inimbus) - throws Exception { - this.conf = conf; - this.downloaders = downloaders; - this.uploaders = uploaders; - - this.submittedCount = new AtomicInteger(0); - - this.stormClusterState = Cluster.mk_storm_cluster_state(conf); - - this.taskHeartbeatsCache = new ConcurrentHashMap>(); - - this.scheduExec = Executors.newScheduledThreadPool(SCHEDULE_THREAD_NUM); - - this.statusTransition = new StatusTransition(this); - - this.startTime = TimeUtils.current_time_secs(); - - this.inimubs = inimbus; - - this.groupToTopology = new HashMap>>(); - - this.groupToResource = new ConcurrentHashMap>(); - - this.groupToUsedResource = new ConcurrentHashMap>(); - - new ReentrantLock(); - - localMode = StormConfig.local_mode(conf); - } - - /** - * Just for test - */ - public NimbusData() { - scheduExec = Executors.newScheduledThreadPool(6); - - inimubs = null; - conf = new HashMap(); - localMode = false; - } - - public int uptime() { - return (TimeUtils.current_time_secs() - startTime); - } - - public Map getConf() { - return conf; - } - - public void setConf(Map conf) { - this.conf = conf; - } - - public StormClusterState getStormClusterState() { - return stormClusterState; - } - - public void setStormClusterState(StormClusterState stormClusterState) { - this.stormClusterState = stormClusterState; - } - - public ConcurrentHashMap> getTaskHeartbeatsCache() { - return taskHeartbeatsCache; - } - - public void setTaskHeartbeatsCache( - ConcurrentHashMap> taskHeartbeatsCache) { - this.taskHeartbeatsCache = taskHeartbeatsCache; - } - - public TimeCacheMap getDownloaders() { - return downloaders; - } - - public void setDownloaders(TimeCacheMap downloaders) { - this.downloaders = downloaders; - } - - public TimeCacheMap getUploaders() { - return uploaders; - } - - public void setUploaders(TimeCacheMap uploaders) { - this.uploaders = uploaders; - } - - public int getStartTime() { - return startTime; - } - - public void setStartTime(int startTime) { - this.startTime = startTime; - } - - public AtomicInteger getSubmittedCount() { - return submittedCount; - } - - public void setSubmittedCount(AtomicInteger submittedCount) { - this.submittedCount = submittedCount; - } - - public Object getSubmitLock() { - return submitLock; - } - - public ScheduledExecutorService getScheduExec() { - return scheduExec; - } - - public StatusTransition getStatusTransition() { - return statusTransition; - } - - public void cleanup() { - try { - stormClusterState.disconnect(); - } catch (Exception e) { - // TODO Auto-generated catch block - - } - try { - scheduExec.shutdown(); - }catch(Exception e) { - } - - uploaders.cleanup(); - downloaders.cleanup(); - } - - public INimbus getInimubs() { - return inimubs; - } - - public Map> getGroupToResource() { - return groupToResource; - } - - public Map>> getGroupToTopology() { - return groupToTopology; - } - - public Map> getGroupToUsedResource() { - return groupToUsedResource; - } - - public boolean isLocalMode() { - return localMode; - } - - public boolean isLeader() { - return isLeader; - } - - 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 deleted file mode 100644 index 6cfe2327d..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusServer.java +++ /dev/null @@ -1,405 +0,0 @@ -package com.alibaba.jstorm.daemon.nimbus; - -import java.io.IOException; -import java.nio.channels.Channel; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.log4j.Logger; -import org.apache.thrift7.protocol.TBinaryProtocol; -import org.apache.thrift7.server.THsHaServer; -import org.apache.thrift7.transport.TNonblockingServerSocket; -import org.apache.thrift7.transport.TTransportException; - -import backtype.storm.Config; -import backtype.storm.generated.Nimbus; -import backtype.storm.generated.Nimbus.Iface; -import backtype.storm.scheduler.INimbus; -import backtype.storm.utils.BufferFileInputStream; -import backtype.storm.utils.TimeCacheMap; -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.callback.AsyncLoopThread; -import com.alibaba.jstorm.client.ConfigExtension; -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.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; - -/** - * - * NimbusServer work flow: 1. cleanup interrupted topology delete - * /storm-local-dir/nimbus/topologyid/stormdis delete - * /storm-zk-root/storms/topologyid - * - * 2. set /storm-zk-root/storms/topology stats as run - * - * 3. start one thread, every nimbus.monitor.reeq.secs set - * /storm-zk-root/storms/ all topology as monitor. when the topology's status is - * monitor, nimubs would reassign workers 4. start one threa, every - * nimubs.cleanup.inbox.freq.secs cleanup useless jar - * - * @author version 1: Nathan Marz version 2: Lixin/Chenjun version 3: Longda - * - */ -public class NimbusServer { - - private static final Logger LOG = Logger.getLogger(NimbusServer.class); - - private NimbusData data; - - private ServiceHandler serviceHandler; - - private TopologyAssign topologyAssign; - - private THsHaServer thriftServer; - - private FollowerRunnable follower; - - private Httpserver hs; - - private UploadMetricFromZK uploadMetric; - - private List smartThreads = new ArrayList(); - - public static void main(String[] args) throws Exception { - // read configuration files - @SuppressWarnings("rawtypes") - Map config = Utils.readStormConfig(); - - JStormServerUtils.startTaobaoJvmMonitor(); - - NimbusServer instance = new NimbusServer(); - - INimbus iNimbus = new DefaultInimbus(); - - instance.launchServer(config, iNimbus); - - } - - private void createPid(Map conf) throws Exception { - String pidDir = StormConfig.masterPids(conf); - - JStormServerUtils.createPid(pidDir); - } - - @SuppressWarnings("rawtypes") - private void launchServer(final Map conf, INimbus inimbus) { - LOG.info("Begin to start nimbus with conf " + conf); - - try { - // 1. check whether mode is distributed or not - StormConfig.validate_distributed_mode(conf); - - createPid(conf); - - initShutdownHook(); - - inimbus.prepare(conf, StormConfig.masterInimbus(conf)); - - data = createNimbusData(conf, inimbus); - - initFollowerThread(conf); - - int port = ConfigExtension.getNimbusDeamonHttpserverPort(conf); - hs = new Httpserver(port, conf); - hs.start(); - - initContainerHBThread(conf); - - while (!data.isLeader()) - Utils.sleep(5000); - - - initUploadMetricThread(data); - - init(conf); - } catch (Throwable e) { - LOG.error("Fail to run nimbus ", e); - } finally { - cleanup(); - } - - LOG.info("Quit nimbus"); - } - - public ServiceHandler launcherLocalServer(final Map conf, INimbus inimbus) - throws Exception { - LOG.info("Begin to start nimbus on local model"); - - StormConfig.validate_local_mode(conf); - - inimbus.prepare(conf, StormConfig.masterInimbus(conf)); - - data = createNimbusData(conf, inimbus); - - init(conf); - - return serviceHandler; - } - - private void initContainerHBThread(Map conf) throws IOException { - AsyncLoopThread thread = SyncContainerHb.mkNimbusInstance(conf); - if (thread != null) { - smartThreads.add(thread); - } - } - - private void init(Map conf) throws Exception { - - NimbusUtils.cleanupCorruptTopologies(data); - - initTopologyAssign(); - - initTopologyStatus(); - - initCleaner(conf); - - serviceHandler = new ServiceHandler(data); - - if (!data.isLocalMode()) { - - initMonitor(conf); - - initThrift(conf); - - } - } - - @SuppressWarnings("rawtypes") - private NimbusData createNimbusData(Map conf, INimbus inimbus) - throws Exception { - - TimeCacheMap.ExpiredCallback expiredCallback = new TimeCacheMap.ExpiredCallback() { - @Override - public void expire(Object key, Object val) { - try { - LOG.info("Close file " + String.valueOf(key)); - if (val != null) { - if (val instanceof Channel) { - Channel channel = (Channel) val; - channel.close(); - } else if (val instanceof BufferFileInputStream) { - BufferFileInputStream is = (BufferFileInputStream) val; - is.close(); - } - } - } catch (IOException e) { - LOG.error(e.getMessage(), e); - } - - } - }; - - int file_copy_expiration_secs = JStormUtils.parseInt( - conf.get(Config.NIMBUS_FILE_COPY_EXPIRATION_SECS), 30); - TimeCacheMap uploaders = new TimeCacheMap( - file_copy_expiration_secs, expiredCallback); - TimeCacheMap downloaders = new TimeCacheMap( - file_copy_expiration_secs, expiredCallback); - - // Callback callback=new TimerCallBack(); - // StormTimer timer=Timer.mkTimerTimer(callback); - NimbusData data = new NimbusData(conf, downloaders, uploaders, inimbus); - - return data; - - } - - private void initTopologyAssign() { - topologyAssign = TopologyAssign.getInstance(); - topologyAssign.init(data); - } - - private void initTopologyStatus() throws Exception { - // get active topology in ZK - List active_ids = data.getStormClusterState().active_storms(); - - if (active_ids != null) { - - for (String topologyid : active_ids) { - // set the topology status as startup - // in fact, startup won't change anything - NimbusUtils.transition(data, topologyid, false, - StatusType.startup); - } - - } - - LOG.info("Successfully init topology status"); - } - - @SuppressWarnings("rawtypes") - private void initMonitor(Map conf) { - final ScheduledExecutorService scheduExec = data.getScheduExec(); - - // Schedule Nimbus monitor - MonitorRunnable r1 = new MonitorRunnable(data); - - int monitor_freq_secs = JStormUtils.parseInt( - conf.get(Config.NIMBUS_MONITOR_FREQ_SECS), 10); - scheduExec.scheduleAtFixedRate(r1, 0, monitor_freq_secs, - TimeUnit.SECONDS); - - LOG.info("Successfully init Monitor thread"); - } - - /** - * Right now, every 600 seconds, nimbus will clean jar under - * /LOCAL-DIR/nimbus/inbox, which is the uploading topology directory - * - * @param conf - * @throws IOException - */ - @SuppressWarnings("rawtypes") - private void initCleaner(Map conf) throws IOException { - final ScheduledExecutorService scheduExec = data.getScheduExec(); - - // Schedule Nimbus inbox cleaner/nimbus/inbox jar - String dir_location = StormConfig.masterInbox(conf); - int inbox_jar_expiration_secs = JStormUtils.parseInt( - conf.get(Config.NIMBUS_INBOX_JAR_EXPIRATION_SECS), 3600); - CleanRunnable r2 = new CleanRunnable(dir_location, - inbox_jar_expiration_secs); - - int cleanup_inbox_freq_secs = JStormUtils.parseInt( - conf.get(Config.NIMBUS_CLEANUP_INBOX_FREQ_SECS), 600); - - scheduExec.scheduleAtFixedRate(r2, 0, cleanup_inbox_freq_secs, - TimeUnit.SECONDS); - - LOG.info("Successfully init " + dir_location + " cleaner"); - } - - @SuppressWarnings("rawtypes") - private void initThrift(Map conf) throws TTransportException { - Integer thrift_port = JStormUtils.parseInt(conf - .get(Config.NIMBUS_THRIFT_PORT)); - TNonblockingServerSocket socket = new TNonblockingServerSocket( - thrift_port); - - Integer maxReadBufSize = JStormUtils.parseInt(conf - .get(Config.NIMBUS_THRIFT_MAX_BUFFER_SIZE)); - - THsHaServer.Args args = new THsHaServer.Args(socket); - args.workerThreads(ServiceHandler.THREAD_NUM); - args.protocolFactory(new TBinaryProtocol.Factory(false, true, - maxReadBufSize)); - - args.processor(new Nimbus.Processor(serviceHandler)); - args.maxReadBufferBytes = maxReadBufSize; - - thriftServer = new THsHaServer(args); - - LOG.info("Successfully started nimbus: started Thrift server..."); - thriftServer.serve(); - } - - private void initFollowerThread(Map conf) { - follower = new FollowerRunnable(data, 5000); - Thread thread = new Thread(follower); - thread.setDaemon(true); - thread.start(); - LOG.info("Successfully init Follower thread"); - } - - private void initShutdownHook() { - Runtime.getRuntime().addShutdownHook(new Thread() { - public void run() { - NimbusServer.this.cleanup(); - } - - }); - } - - private void initUploadMetricThread(NimbusData data) { - ScheduledExecutorService scheduleService = data.getScheduExec(); - - MetricSendClient client; - if (ConfigExtension.isAlimonitorMetricsPost(data.getConf())) { - client = new AlimonitorClient(AlimonitorClient.DEFAUT_ADDR, - AlimonitorClient.DEFAULT_PORT, true); - } else { - client = new MetricSendClient(); - } - - uploadMetric = new UploadMetricFromZK(data, client); - - scheduleService.scheduleWithFixedDelay(uploadMetric, 120, 60, TimeUnit.SECONDS); - - LOG.info("Successfully init metrics uploading thread"); - } - - public void cleanup() { - if (data.getIsShutdown().getAndSet(true) == true) { - LOG.info("Notify to quit nimbus"); - return; - } - - LOG.info("Begin to shutdown nimbus"); - - for (AsyncLoopThread t : smartThreads) { - - t.cleanup(); - JStormUtils.sleepMs(10); - t.interrupt(); -// try { -// t.join(); -// } catch (InterruptedException e) { -// LOG.error("join thread", e); -// } - LOG.info("Successfully cleanup " + t.getThread().getName()); - } - - if (serviceHandler != null) { - serviceHandler.shutdown(); - } - - if (topologyAssign != null) { - topologyAssign.cleanup(); - LOG.info("Successfully shutdown TopologyAssign thread"); - } - - if (follower != null) { - follower.clean(); - LOG.info("Successfully shutdown follower thread"); - } - - if (uploadMetric != null) { - uploadMetric.clean(); - LOG.info("Successfully shutdown UploadMetric thread"); - } - - if (data != null) { - data.cleanup(); - LOG.info("Successfully shutdown NimbusData"); - } - - if (thriftServer != null) { - thriftServer.stop(); - LOG.info("Successfully shutdown thrift server"); - } - - if (hs != null) { - hs.shutdown(); - LOG.info("Successfully shutdown httpserver"); - } - - LOG.info("Successfully shutdown nimbus"); - // make sure shutdown nimbus - JStormUtils.halt_process(0, "!!!Shutdown!!!"); - - } - -} 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 deleted file mode 100644 index a4a8f878c..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/NimbusUtils.java +++ /dev/null @@ -1,798 +0,0 @@ -package com.alibaba.jstorm.daemon.nimbus; - -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; -import java.util.Date; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -import java.util.TreeMap; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; -import backtype.storm.generated.Bolt; -import backtype.storm.generated.ComponentCommon; -import backtype.storm.generated.ErrorInfo; -import backtype.storm.generated.NotAliveException; -import backtype.storm.generated.SpoutSpec; -import backtype.storm.generated.StateSpoutSpec; -import backtype.storm.generated.StormTopology; -import backtype.storm.generated.SupervisorSummary; -import backtype.storm.generated.TaskMetricData; -import backtype.storm.generated.TaskSummary; -import backtype.storm.generated.TopologySummary; -import backtype.storm.generated.WorkerMetricData; -import backtype.storm.generated.WorkerSummary; -import backtype.storm.utils.ThriftTopologyUtils; - -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.Cluster; -import com.alibaba.jstorm.cluster.StormClusterState; -import com.alibaba.jstorm.cluster.StormConfig; -import com.alibaba.jstorm.cluster.StormMonitor; -import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; -import com.alibaba.jstorm.daemon.worker.WorkerMetricInfo; -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; -import com.alibaba.jstorm.task.error.TaskError; -import com.alibaba.jstorm.task.heartbeat.TaskHeartbeat; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.PathUtils; -import com.alibaba.jstorm.utils.TimeUtils; - -public class NimbusUtils { - - private static Logger LOG = Logger.getLogger(NimbusUtils.class); - - /** - * add coustom KRYO serialization - * - */ - private static Map mapifySerializations(List sers) { - Map rtn = new HashMap(); - if (sers != null) { - int size = sers.size(); - for (int i = 0; i < size; i++) { - if (sers.get(i) instanceof Map) { - rtn.putAll((Map) sers.get(i)); - } else { - rtn.put(sers.get(i), null); - } - } - - } - return rtn; - } - - /** - * Normalize stormConf - * - * - * - * @param conf - * @param stormConf - * @param topology - * @return - * @throws Exception - */ - @SuppressWarnings("rawtypes") - public static Map normalizeConf(Map conf, Map stormConf, - StormTopology topology) throws Exception { - - List kryoRegisterList = new ArrayList(); - List kryoDecoratorList = new ArrayList(); - - Map totalConf = new HashMap(); - totalConf.putAll(conf); - totalConf.putAll(stormConf); - - Object totalRegister = totalConf.get(Config.TOPOLOGY_KRYO_REGISTER); - if (totalRegister != null) { - LOG.info("topology:" + stormConf.get(Config.TOPOLOGY_NAME) - + ", TOPOLOGY_KRYO_REGISTER" - + totalRegister.getClass().getName()); - - JStormUtils.mergeList(kryoRegisterList, totalRegister); - } - - Object totalDecorator = totalConf.get(Config.TOPOLOGY_KRYO_DECORATORS); - if (totalDecorator != null) { - LOG.info("topology:" + stormConf.get(Config.TOPOLOGY_NAME) - + ", TOPOLOGY_KRYO_DECORATOR" - + totalDecorator.getClass().getName()); - JStormUtils.mergeList(kryoDecoratorList, totalDecorator); - } - - Set cids = ThriftTopologyUtils.getComponentIds(topology); - for (Iterator it = cids.iterator(); it.hasNext();) { - String componentId = (String) it.next(); - - ComponentCommon common = ThriftTopologyUtils.getComponentCommon( - topology, componentId); - String json = common.get_json_conf(); - if (json == null) { - continue; - } - Map mtmp = (Map) JStormUtils.from_json(json); - if (mtmp == null) { - StringBuilder sb = new StringBuilder(); - - sb.append("Failed to deserilaize " + componentId); - sb.append(" json configuration: "); - sb.append(json); - LOG.info(sb.toString()); - throw new Exception(sb.toString()); - } - - Object componentKryoRegister = mtmp - .get(Config.TOPOLOGY_KRYO_REGISTER); - - if (componentKryoRegister != null) { - LOG.info("topology:" + stormConf.get(Config.TOPOLOGY_NAME) - + ", componentId:" + componentId - + ", TOPOLOGY_KRYO_REGISTER" - + componentKryoRegister.getClass().getName()); - - JStormUtils.mergeList(kryoRegisterList, componentKryoRegister); - } - - Object componentDecorator = mtmp - .get(Config.TOPOLOGY_KRYO_DECORATORS); - if (componentDecorator != null) { - LOG.info("topology:" + stormConf.get(Config.TOPOLOGY_NAME) - + ", componentId:" + componentId - + ", TOPOLOGY_KRYO_DECORATOR" - + componentDecorator.getClass().getName()); - JStormUtils.mergeList(kryoDecoratorList, componentDecorator); - } - - } - - Map kryoRegisterMap = mapifySerializations(kryoRegisterList); - List decoratorList = JStormUtils.distinctList(kryoDecoratorList); - - Integer ackerNum = JStormUtils.parseInt(totalConf - .get(Config.TOPOLOGY_ACKER_EXECUTORS)); - if (ackerNum == null) { - ackerNum = Integer.valueOf(1); - } - - Map rtn = new HashMap(); - rtn.putAll(stormConf); - rtn.put(Config.TOPOLOGY_KRYO_DECORATORS, decoratorList); - rtn.put(Config.TOPOLOGY_KRYO_REGISTER, kryoRegisterMap); - rtn.put(Config.TOPOLOGY_ACKER_EXECUTORS, ackerNum); - rtn.put(Config.TOPOLOGY_MAX_TASK_PARALLELISM, - totalConf.get(Config.TOPOLOGY_MAX_TASK_PARALLELISM)); - return rtn; - } - - public static Integer componentParalism(Map stormConf, - ComponentCommon common) { - Map mergeMap = new HashMap(); - mergeMap.putAll(stormConf); - - String jsonConfString = common.get_json_conf(); - if (jsonConfString != null) { - Map componentMap = (Map) JStormUtils.from_json(jsonConfString); - mergeMap.putAll(componentMap); - } - - Integer taskNum = common.get_parallelism_hint(); - if (taskNum == null) { - taskNum = Integer.valueOf(1); - } - - // don't get taskNum from component configuraiton - // skip .setTaskNum - // Integer taskNum = null; - // Object taskNumObject = mergeMap.get(Config.TOPOLOGY_TASKS); - // if (taskNumObject != null) { - // taskNum = JStormUtils.parseInt(taskNumObject); - // } else { - // taskNum = common.get_parallelism_hint(); - // if (taskNum == null) { - // taskNum = Integer.valueOf(1); - // } - // } - - Object maxTaskParalismObject = mergeMap - .get(Config.TOPOLOGY_MAX_TASK_PARALLELISM); - if (maxTaskParalismObject == null) { - return taskNum; - } else { - int maxTaskParalism = JStormUtils.parseInt(maxTaskParalismObject); - - return Math.min(maxTaskParalism, taskNum); - } - - } - - - /** - * finalize component's task paralism - * - * @param topology - * @param fromConf means if the paralism is read from conf file - * instead of reading from topology code - * @return - */ - public static StormTopology normalizeTopology(Map stormConf, - 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(); - - ComponentCommon common = null; - if (component instanceof Bolt) { - common = ((Bolt) component).get_common(); - if (fromConf) { - Integer paraNum = ConfigExtension.getBoltParallelism(stormConf, componentName); - 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) { - 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) { - LOG.info("Set " + componentName + " as " + paraNum); - common.set_parallelism_hint(paraNum); - } - } - } - - Map componentMap = new HashMap(); - - String jsonConfString = common.get_json_conf(); - if (jsonConfString != null) { - componentMap - .putAll((Map) JStormUtils.from_json(jsonConfString)); - } - - Integer taskNum = componentParalism(stormConf, common); - - componentMap.put(Config.TOPOLOGY_TASKS, taskNum); - // change the executor's task number - common.set_parallelism_hint(taskNum); - LOG.info("Set " + componentName + " parallelism " + taskNum); - - common.set_json_conf(JStormUtils.to_json(componentMap)); - } - - return ret; - } - - /** - * clean the topology which is in ZK but not in local dir - * - * @throws Exception - * - */ - public static void cleanupCorruptTopologies(NimbusData data) - throws Exception { - - StormClusterState stormClusterState = data.getStormClusterState(); - - // get /local-storm-dir/nimbus/stormdist path - String master_stormdist_root = StormConfig.masterStormdistRoot(data - .getConf()); - - // listdir /local-storm-dir/nimbus/stormdist - List code_ids = PathUtils - .read_dir_contents(master_stormdist_root); - - // get topology in ZK /storms - List active_ids = data.getStormClusterState().active_storms(); - if (active_ids != null && active_ids.size() > 0) { - if (code_ids != null) { - // clean the topology which is in ZK but not in local dir - active_ids.removeAll(code_ids); - } - - for (String corrupt : active_ids) { - LOG.info("Corrupt topology " - + corrupt - + " has state on zookeeper but doesn't have a local dir on Nimbus. Cleaning up..."); - - /** - * Just removing the /STORMS is enough - * - */ - stormClusterState.remove_storm(corrupt); - } - } - - LOG.info("Successfully cleanup all old toplogies"); - - } - - - public static boolean isTaskDead(NimbusData data, String topologyId, - Integer taskId) { - String idStr = " topology:" + topologyId + ",taskid:" + taskId; - - Integer zkReportTime = null; - - StormClusterState stormClusterState = data.getStormClusterState(); - TaskHeartbeat zkTaskHeartbeat = null; - try { - zkTaskHeartbeat = stormClusterState.task_heartbeat(topologyId, - taskId); - if (zkTaskHeartbeat != null) { - zkReportTime = zkTaskHeartbeat.getTimeSecs(); - } - } catch (Exception e) { - LOG.error("Failed to get ZK task hearbeat " + idStr, e); - return true; - } - - Map taskHBs = data.getTaskHeartbeatsCache() - .get(topologyId); - if (taskHBs == null) { - LOG.info("No task heartbeat cache " + topologyId); - - // update task hearbeat cache - taskHBs = new HashMap(); - - data.getTaskHeartbeatsCache().put(topologyId, taskHBs); - } - - TkHbCacheTime taskHB = taskHBs.get(taskId); - if (taskHB == null) { - LOG.info("No task heartbeat cache " + idStr); - - if (zkTaskHeartbeat == null) { - LOG.info("No ZK task hearbeat " + idStr); - return true; - } - - taskHB = new TkHbCacheTime(); - taskHB.update(zkTaskHeartbeat); - - taskHBs.put(taskId, taskHB); - - return false; - } - - if (zkReportTime == null) { - LOG.debug("No ZK task heartbeat " + idStr); - // Task hasn't finish init - int nowSecs = TimeUtils.current_time_secs(); - int assignSecs = taskHB.getTaskAssignedTime(); - - int waitInitTimeout = JStormUtils.parseInt(data.getConf().get( - Config.NIMBUS_TASK_LAUNCH_SECS)); - - if (nowSecs - assignSecs > waitInitTimeout) { - LOG.info(idStr + " failed to init "); - return true; - } else { - return false; - } - - } - - // the left is zkReportTime isn't null - // task has finished initialization - int nimbusTime = taskHB.getNimbusTime(); - int reportTime = taskHB.getTaskReportedTime(); - - int nowSecs = TimeUtils.current_time_secs(); - if (nimbusTime == 0) { - // taskHB no entry, first time - // update taskHB - taskHB.setNimbusTime(nowSecs); - taskHB.setTaskReportedTime(zkReportTime); - - LOG.info("Update taskheartbeat to nimbus cache " + idStr); - return false; - } - - if (reportTime != zkReportTime.intValue()) { - // zk has been updated the report time - taskHB.setNimbusTime(nowSecs); - taskHB.setTaskReportedTime(zkReportTime); - - LOG.debug(idStr + ",nimbusTime " + nowSecs + ",zkReport:" - + zkReportTime + ",report:" + reportTime); - return false; - } - - // the following is (zkReportTime == reportTime) - int taskHBTimeout = JStormUtils.parseInt(data.getConf().get( - Config.NIMBUS_TASK_TIMEOUT_SECS)); - - if (nowSecs - nimbusTime > taskHBTimeout) { - // task is dead - long ts = ((long) nimbusTime) * 1000; - Date lastTaskHBDate = new Date(ts); - StringBuilder sb = new StringBuilder(); - - sb.append(idStr); - sb.append(" last tasktime is "); - sb.append(nimbusTime); - sb.append(":").append(lastTaskHBDate); - sb.append(",current "); - sb.append(nowSecs); - sb.append(":").append(new Date(((long) nowSecs) * 1000)); - LOG.info(sb.toString()); - return true; - } - - return false; - - } - - public static void updateTaskHbStartTime(NimbusData data, - Assignment assignment, String topologyId) { - Map taskHBs = data.getTaskHeartbeatsCache() - .get(topologyId); - - if (taskHBs == null) { - taskHBs = new HashMap(); - data.getTaskHeartbeatsCache().put(topologyId, taskHBs); - } - - Map taskStartTimes = assignment - .getTaskStartTimeSecs(); - for (Entry entry : taskStartTimes.entrySet()) { - Integer taskId = entry.getKey(); - Integer taskStartTime = entry.getValue(); - - TkHbCacheTime taskHB = taskHBs.get(taskId); - if (taskHB == null) { - taskHB = new TkHbCacheTime(); - taskHBs.put(taskId, taskHB); - } - - taskHB.setTaskAssignedTime(taskStartTime); - } - - return; - } - - public static void transitionName(NimbusData data, String topologyName, - boolean errorOnNoTransition, StatusType transition_status, - T... args) throws Exception { - StormClusterState stormClusterState = data.getStormClusterState(); - String topologyId = Cluster.get_topology_id(stormClusterState, - topologyName); - if (topologyId == null) { - throw new NotAliveException(topologyName); - } - transition(data, topologyId, errorOnNoTransition, transition_status, - args); - } - - public static void transition(NimbusData data, String topologyid, - boolean errorOnNoTransition, StatusType transition_status, - T... args) { - try { - data.getStatusTransition().transition(topologyid, - errorOnNoTransition, transition_status, args); - } catch (Exception e) { - // TODO Auto-generated catch block - LOG.error("Failed to do status transition,", e); - } - } - - public static TopologySummary mkTopologySummary(Assignment assignment, - String topologyId, String topologyName, String status, - int uptime_secs, Map lastErrTimeStamp) { - - int num_workers = assignment.getWorkers().size(); - int num_tasks = 0; - - for (ResourceWorkerSlot worker : assignment.getWorkers()) { - num_tasks = num_tasks + worker.getTasks().size(); - } - - long currentTimeSecs = System.currentTimeMillis() / 1000; - String errorInfo = ""; - if (lastErrTimeStamp != null) - { - for (Entry entry : lastErrTimeStamp.entrySet()) { - if ((currentTimeSecs - Long.valueOf(entry.getValue())) < entry.getKey()) { - errorInfo = "Y"; - break; - } - } - } - - TopologySummary ret = new TopologySummary(topologyId, topologyName, - status, uptime_secs, num_tasks, num_workers, errorInfo); - - return ret; - } - - public static SupervisorSummary mkSupervisorSummary( - SupervisorInfo supervisorInfo, String supervisorId, - Map supervisorToUsedSlotNum) { - Integer usedNum = supervisorToUsedSlotNum.get(supervisorId); - - SupervisorSummary summary = new SupervisorSummary( - supervisorInfo.getHostName(), supervisorId, - supervisorInfo.getUptimeSecs(), supervisorInfo.getWorkerPorts() - .size(), usedNum == null ? 0 : usedNum); - - return summary; - } - - public static List mkSupervisorSummaries( - Map supervisorInfos, - Map assignments) { - - Map supervisorToLeftSlotNum = new HashMap(); - for (Entry entry : assignments.entrySet()) { - Set workers = entry.getValue().getWorkers(); - - for (ResourceWorkerSlot worker : workers) { - - String supervisorId = worker.getNodeId(); - SupervisorInfo supervisorInfo = supervisorInfos - .get(supervisorId); - if (supervisorInfo == null) { - continue; - } - Integer slots = supervisorToLeftSlotNum.get(supervisorId); - if (slots == null) { - slots = 0; - supervisorToLeftSlotNum.put(supervisorId, slots); - } - supervisorToLeftSlotNum.put(supervisorId, ++slots); - } - } - - List ret = new ArrayList(); - for (Entry entry : supervisorInfos.entrySet()) { - String supervisorId = entry.getKey(); - SupervisorInfo supervisorInfo = entry.getValue(); - - SupervisorSummary summary = mkSupervisorSummary(supervisorInfo, - supervisorId, supervisorToLeftSlotNum); - - ret.add(summary); - } - - Collections.sort(ret, new Comparator() { - - @Override - public int compare(SupervisorSummary o1, SupervisorSummary o2) { - - return o1.get_host().compareTo(o2.get_host()); - } - - }); - return ret; - } - - public static TaskSummary mkSimpleTaskSummary(ResourceWorkerSlot resource, - int taskId, String component, String componentType, String host, int uptime) { - TaskSummary ret = new TaskSummary(); - - ret.set_task_id(taskId); - ret.set_component_id(component); - ret.set_component_type(componentType); - ret.set_host(host); - ret.set_port(resource.getPort()); - ret.set_uptime_secs(uptime); - ret.set_errors(new ArrayList()); - - return ret; - } - - public static Map mkTaskSummary( - StormClusterState zkClusterState, Assignment assignment, - Map taskToComponent, String topologyId) - throws Exception { - - Map taskSummaries = new TreeMap(); - - Set workers = assignment.getWorkers(); - - for (ResourceWorkerSlot worker : workers) { - for (Integer taskId : worker.getTasks()) { - TaskSummary taskSummary = new TaskSummary(); - - taskSummary.set_task_id(taskId); - taskSummary.set_component_id(taskToComponent.get(taskId)); - taskSummary.set_host(worker.getHostname()); - taskSummary.set_port(worker.getPort()); - - List errors = zkClusterState.task_errors(topologyId, - taskId); - List newErrors = new ArrayList(); - if (errors != null) { - int size = errors.size(); - for (int i = 0; i < size; i++) { - TaskError e = (TaskError) errors.get(i); - newErrors.add(new ErrorInfo(e.getError(), e - .getTimSecs())); - } - } - taskSummary.set_errors(newErrors); - - TaskHeartbeat heartbeat = zkClusterState.task_heartbeat( - topologyId, taskId); - if (heartbeat == null) { - LOG.warn("Topology " + topologyId + " task " + taskId - + " hasn't been started"); - taskSummary.set_status(ConfigExtension.TASK_STATUS_STARTING); - } else { - taskSummary.set_uptime_secs(heartbeat.getUptimeSecs()); - taskSummary.set_stats(heartbeat.getStats().getTaskStats()); - taskSummary.set_status(ConfigExtension.TASK_STATUS_ACTIVE); - } - - taskSummaries.put(taskId, taskSummary); - } - } - - return taskSummaries; - } - - public static List mkWorkerSummary(String topology, - Assignment assignment, Map taskSumm) { - Set workers = assignment.getWorkers(); - List result = new ArrayList(); - for (ResourceWorkerSlot worker : workers) { - WorkerSummary workerSumm = new WorkerSummary(); - workerSumm.set_topology(topology); - workerSumm.set_port(worker.getPort()); - List tasks = new ArrayList(); - workerSumm.set_tasks(tasks); - for (Integer taskId : worker.getTasks()) { - TaskSummary task = taskSumm.get(taskId); - if (task == null) - continue; - tasks.add(task); - } - result.add(workerSumm); - } - return result; - } - - public static void updateMetricMonitorStatus(StormClusterState clusterState, - String topologyId, boolean isEnable) throws Exception { - StormMonitor stormMonitor = new StormMonitor(isEnable); - clusterState.set_storm_monitor(topologyId, stormMonitor); - } - - public static void updateMetricsInfo(NimbusData data, String topologyId, - Assignment assignment) { - List taskList = new ArrayList(); - List workerList = new ArrayList(); - - StormClusterState clusterState = data.getStormClusterState(); - - Set workerSlotSet = assignment.getWorkers(); - - for (ResourceWorkerSlot workerSlot : workerSlotSet) { - String workerId = workerSlot.getHostname() + ":" + workerSlot.getPort(); - workerList.add(workerId); - - taskList.addAll(workerSlot.getTasks()); - } - - try { - //Remove the obsolete tasks of metrics monitor in ZK - List metricTaskList = clusterState.get_metric_taskIds(topologyId); - for (String task : metricTaskList) { - Integer taskId = Integer.valueOf(task); - if(taskList.contains(taskId) == false) - clusterState.remove_metric_task(topologyId, String.valueOf(taskId)); - } - - //Remove the obsolete workers of metrics monitor in ZK - List metricWorkerList = clusterState.get_metric_workerIds(topologyId); - for (String workerId : metricWorkerList) { - if (workerList.contains(workerId) == false) - clusterState.remove_metric_worker(topologyId, workerId); - } - - //Remove the obsolete user workers of metrics monitor in ZK - List metricUserList = clusterState.get_metric_users(topologyId); - for (String workerId : metricUserList) { - if (workerList.contains(workerId) == false) - clusterState.remove_metric_user(topologyId, workerId); - } - } catch (Exception e) { - LOG.error("Failed to update metrics info when rebalance or reassignment, topologyId=" + - topologyId, e); - } - } - - public static void updateTaskMetricData(TaskMetricData metricData, TaskMetricInfo metricInfo) { - metricData.set_task_id(Integer.valueOf(metricInfo.getTaskId())); - metricData.set_component_id(metricInfo.getComponent()); - metricData.set_gauge(metricInfo.getGaugeData()); - metricData.set_counter(metricInfo.getCounterData()); - metricData.set_meter(metricInfo.getMeterData()); - metricData.set_timer(metricInfo.getTimerData()); - metricData.set_histogram(metricInfo.getHistogramData()); - } - - public static void updateWorkerMetricData(WorkerMetricData metricData, WorkerMetricInfo metricInfo) { - metricData.set_hostname(metricInfo.getHostName()); - metricData.set_port(metricInfo.getPort()); - metricData.set_gauge(metricInfo.getGaugeData()); - metricData.set_counter(metricInfo.getCounterData()); - metricData.set_meter(metricInfo.getMeterData()); - metricData.set_timer(metricInfo.getTimerData()); - metricData.set_histogram(metricInfo.getHistogramData()); - - //Add cpu and Mem into gauge map - Map gaugeMap = metricData.get_gauge(); - gaugeMap.put(MetricDef.CPU_USED_RATIO, metricInfo.getUsedCpu()); - gaugeMap.put(MetricDef.MEMORY_USED,((Long) metricInfo.getUsedMem()).doubleValue()); - } - - public static String getNimbusVersion() { - String ret = null; - - String path = System.getProperty("jstorm.home") + "/RELEASE"; - File file = new File(path); - FileReader reader = null; - Closeable resource = reader; - - try{ - reader = new FileReader(file); - BufferedReader bufferedReader = new BufferedReader(reader); - resource = bufferedReader; - ret = bufferedReader.readLine(); - } catch (Exception e) { - LOG.warn("Failed to get nimbus version", e); - } finally { - if (resource != null) { - try { - resource.close(); - } catch (Exception e) { - LOG.error("Failed to close the reader of RELEASE", e); - } - } - } - - return ret; - } -} 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 deleted file mode 100644 index 4314e6d9e..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/ServiceHandler.java +++ /dev/null @@ -1,1353 +0,0 @@ -package com.alibaba.jstorm.daemon.nimbus; - -import java.io.File; -import java.io.FileNotFoundException; -import java.io.FileOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.Channels; -import java.nio.channels.WritableByteChannel; -import java.security.InvalidParameterException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -import java.util.TreeMap; -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; - -import backtype.storm.Config; -import backtype.storm.daemon.Shutdownable; -import backtype.storm.generated.AlreadyAliveException; -import backtype.storm.generated.Bolt; -import backtype.storm.generated.ClusterSummary; -import backtype.storm.generated.ComponentCommon; -import backtype.storm.generated.InvalidTopologyException; -import backtype.storm.generated.KillOptions; -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; -import backtype.storm.generated.StateSpoutSpec; -import backtype.storm.generated.StormTopology; -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.TopologyMetricInfo; -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.callback.impl.RemoveTransitionCallback; -import com.alibaba.jstorm.cluster.Cluster; -import com.alibaba.jstorm.cluster.Common; -import com.alibaba.jstorm.cluster.DaemonCommon; -import com.alibaba.jstorm.cluster.StormBase; -import com.alibaba.jstorm.cluster.StormClusterState; -import com.alibaba.jstorm.cluster.StormConfig; -import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; -import com.alibaba.jstorm.daemon.worker.WorkerMetricInfo; -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.NetWorkUtils; -import com.alibaba.jstorm.utils.PathUtils; -import com.alibaba.jstorm.utils.Thrift; -import com.alibaba.jstorm.utils.TimeUtils; - -/** - * Thrift callback, all commands handling entrance - * - * @author version 1: lixin, version 2:Longda - * - */ -public class ServiceHandler implements Iface, Shutdownable, DaemonCommon { - private final static Logger LOG = Logger.getLogger(ServiceHandler.class); - - public final static int THREAD_NUM = 64; - - private NimbusData data; - - private Map conf; - - public ServiceHandler(NimbusData data) { - this.data = data; - conf = data.getConf(); - } - - @Override - public void submitTopology(String name, String uploadedJarLocation, - String jsonConf, StormTopology topology) - throws AlreadyAliveException, InvalidTopologyException, - TopologyAssignException, TException { - SubmitOptions options = new SubmitOptions(TopologyInitialStatus.ACTIVE); - - submitTopologyWithOpts(name, uploadedJarLocation, jsonConf, topology, - options); - } - - private void makeAssignment(String topologyName, String topologyId, - TopologyInitialStatus status) throws FailedAssignTopologyException { - TopologyAssignEvent assignEvent = new TopologyAssignEvent(); - assignEvent.setTopologyId(topologyId); - assignEvent.setScratch(false); - assignEvent.setTopologyName(topologyName); - assignEvent.setOldStatus(Thrift - .topologyInitialStatusToStormStatus(status)); - - TopologyAssign.push(assignEvent); - - boolean isSuccess = assignEvent.waitFinish(); - if (isSuccess == true) { - LOG.info("Finish submit for " + topologyName); - } else { - throw new FailedAssignTopologyException( - assignEvent.getErrorMsg()); - } - } - - /** - * Submit one Topology - * - * @param topologyname - * String: topology name - * @param uploadedJarLocation - * String: already uploaded jar path - * @param jsonConf - * String: jsonConf serialize all toplogy configuration to Json - * @param topology - * StormTopology: topology Object - */ - @SuppressWarnings("unchecked") - @Override - public void submitTopologyWithOpts(String topologyname, - String uploadedJarLocation, String jsonConf, - StormTopology topology, SubmitOptions options) - throws AlreadyAliveException, InvalidTopologyException, - TopologyAssignException, TException { - LOG.info("Receive " + topologyname + ", uploadedJarLocation:" - + uploadedJarLocation); - // @@@ Move validate topologyname in client code - try { - checkTopologyActive(data, topologyname, false); - } catch (AlreadyAliveException e) { - LOG.info(topologyname + " is already exist "); - throw e; - } 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 = Common.TopologyNameToId(topologyname, counter); - - try { - - Map serializedConf = (Map) JStormUtils - .from_json(jsonConf); - if (serializedConf == null) { - LOG.warn("Failed to serialized Configuration"); - throw new InvalidTopologyException( - "Failed to serilaze topology configuration"); - } - - serializedConf.put(Config.TOPOLOGY_ID, topologyId); - serializedConf.put(Config.TOPOLOGY_NAME, topologyname); - - Map stormConf; - - 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, true); - - // this validates the structure of the topology - Common.validate_basic(normalizedTopology, totalStormConf, - topologyId); - // don't need generate real topology, so skip Common.system_topology - // Common.system_topology(totalStormConf, topology); - - StormClusterState stormClusterState = data.getStormClusterState(); - - // create /local-dir/nimbus/topologyId/xxxx files - setupStormCode(conf, topologyId, uploadedJarLocation, stormConf, - normalizedTopology); - - // generate TaskInfo for every bolt or spout in ZK - // /ZK/tasks/topoologyId/xxx - setupZkTaskInfo(conf, topologyId, stormClusterState); - - // make assignments for a topology - LOG.info("Submit for " + topologyname + " with conf " - + serializedConf); - makeAssignment(topologyname, topologyId, options.get_initial_status()); - - } 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); - sb.append(", uploadedJarLocation:" + uploadedJarLocation + "\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); - sb.append(", uploadedJarLocation:" + uploadedJarLocation + "\n"); - LOG.error(sb.toString(), e); - throw new InvalidParameterException(sb.toString()); - } catch (InvalidTopologyException e) { - LOG.error("Topology is invalid. " + e.get_msg()); - throw e; - } 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); - sb.append(", uploadedJarLocation:" + uploadedJarLocation + "\n"); - LOG.error(sb.toString(), e); - throw new TopologyAssignException(sb.toString()); - } - - } - - /** - * kill topology - * - * @param topologyname - * String topology name - */ - @Override - public void killTopology(String name) throws NotAliveException, TException { - killTopologyWithOpts(name, new KillOptions()); - - } - - @Override - public void killTopologyWithOpts(String topologyName, KillOptions options) - throws NotAliveException, TException { - try { - - checkTopologyActive(data, topologyName, true); - Integer wait_amt = null; - if (options.is_set_wait_secs()) { - wait_amt = options.get_wait_secs(); - } - NimbusUtils.transitionName(data, topologyName, true, - StatusType.kill, wait_amt); - } catch (NotAliveException e) { - String errMsg = "KillTopology Error, no this topology " - + topologyName; - LOG.error(errMsg, e); - throw new NotAliveException(errMsg); - } catch (Exception e) { - String errMsg = "Failed to kill topology " + topologyName; - LOG.error(errMsg, e); - throw new TException(errMsg); - } - - } - - /** - * set topology status as active - * - * @param topologyname - * - */ - @Override - public void activate(String topologyName) throws NotAliveException, - TException { - try { - NimbusUtils.transitionName(data, topologyName, true, - StatusType.activate); - } catch (NotAliveException e) { - String errMsg = "Activate Error, no this topology " + topologyName; - LOG.error(errMsg, e); - throw new NotAliveException(errMsg); - } catch (Exception e) { - String errMsg = "Failed to active topology " + topologyName; - LOG.error(errMsg, e); - throw new TException(errMsg); - } - - } - - /** - * set topology stauts as deactive - * - * @param topologyname - * - */ - @Override - public void deactivate(String topologyName) throws NotAliveException, - TException { - - try { - NimbusUtils.transitionName(data, topologyName, true, - StatusType.inactivate); - } catch (NotAliveException e) { - String errMsg = "Deactivate Error, no this topology " - + topologyName; - LOG.error(errMsg, e); - throw new NotAliveException(errMsg); - } catch (Exception e) { - String errMsg = "Failed to deactivate topology " + topologyName; - LOG.error(errMsg, e); - throw new TException(errMsg); - } - - } - - /** - * rebalance one topology - * - * @@@ rebalance options hasn't implements - * - * It is used to let workers wait several seconds to finish jobs - * - * @param topologyname - * String - * @param options - * RebalanceOptions - */ - @Override - public void rebalance(String topologyName, RebalanceOptions options) - throws NotAliveException, TException, InvalidTopologyException { - - try { - - checkTopologyActive(data, topologyName, true); - Integer wait_amt = null; - if (options != null && options.is_set_wait_secs()) { - wait_amt = options.get_wait_secs(); - } - - NimbusUtils.transitionName(data, topologyName, true, - StatusType.rebalance, wait_amt); - } catch (NotAliveException e) { - String errMsg = "Rebalance Error, no this topology " + topologyName; - LOG.error(errMsg, e); - throw new NotAliveException(errMsg); - } catch (Exception e) { - String errMsg = "Failed to rebalance topology " + topologyName; - LOG.error(errMsg, e); - throw new TException(errMsg); - } - - } - - @Override - 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; - try { - 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 - // 2. Deactivate - deactivate(name); - JStormUtils.sleepMs(5000); - 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 (Exception e) { - // TODO Auto-generated catch block - LOG.error("Fail to upload jar " + libName, e); - throw new TException(e); - } - } - - /** - * prepare to uploading topology jar, return the file location - * - * @throws - */ - @Override - public String beginFileUpload() throws TException { - - String fileLoc = null; - try { - String path = null; - String key = UUID.randomUUID().toString(); - path = StormConfig.masterInbox(conf) + "/" + key; - FileUtils.forceMkdir(new File(path)); - FileUtils.cleanDirectory(new File(path)); - fileLoc = path + "/stormjar-" + key + ".jar"; - - data.getUploaders().put(fileLoc, - Channels.newChannel(new FileOutputStream(fileLoc))); - LOG.info("Begin upload file from client to " + fileLoc); - return path; - } catch (FileNotFoundException e) { - LOG.error("File not found: " + fileLoc, e); - throw new TException(e); - } catch (IOException e) { - LOG.error("Upload file error: " + fileLoc, e); - throw new TException(e); - } - } - - /** - * uploading topology jar data - */ - @Override - public void uploadChunk(String location, ByteBuffer chunk) - throws TException { - TimeCacheMap uploaders = data.getUploaders(); - Object obj = uploaders.get(location); - if (obj == null) { - throw new TException( - "File for that location does not exist (or timed out) " - + location); - } - try { - if (obj instanceof WritableByteChannel) { - WritableByteChannel channel = (WritableByteChannel) obj; - channel.write(chunk); - uploaders.put(location, channel); - } else { - throw new TException("Object isn't WritableByteChannel for " - + location); - } - } catch (IOException e) { - String errMsg = " WritableByteChannel write filed when uploadChunk " - + location; - LOG.error(errMsg); - throw new TException(e); - } - - } - - @Override - public void finishFileUpload(String location) throws TException { - - TimeCacheMap uploaders = data.getUploaders(); - Object obj = uploaders.get(location); - if (obj == null) { - throw new TException( - "File for that location does not exist (or timed out)"); - } - try { - if (obj instanceof WritableByteChannel) { - WritableByteChannel channel = (WritableByteChannel) obj; - channel.close(); - uploaders.remove(location); - LOG.info("Finished uploading file from client: " + location); - } else { - throw new TException("Object isn't WritableByteChannel for " - + location); - } - } catch (IOException e) { - LOG.error(" WritableByteChannel close failed when finishFileUpload " - + location); - } - - } - - @Override - public String beginFileDownload(String file) throws TException { - BufferFileInputStream is = null; - String id = null; - try { - int bufferSize = JStormUtils - .parseInt(conf.get(Config.NIMBUS_THRIFT_MAX_BUFFER_SIZE), - 1024 * 1024) / 2; - - is = new BufferFileInputStream(file, bufferSize); - id = UUID.randomUUID().toString(); - data.getDownloaders().put(id, is); - } catch (FileNotFoundException e) { - LOG.error(e + "file:" + file + " not found"); - throw new TException(e); - } - - return id; - } - - @Override - public ByteBuffer downloadChunk(String id) throws TException { - TimeCacheMap downloaders = data.getDownloaders(); - Object obj = downloaders.get(id); - if (obj == null) { - throw new TException("Could not find input stream for that id"); - } - - try { - if (obj instanceof BufferFileInputStream) { - - BufferFileInputStream is = (BufferFileInputStream) obj; - byte[] ret = is.read(); - if (ret != null) { - downloaders.put(id, (BufferFileInputStream) is); - return ByteBuffer.wrap(ret); - } - } else { - throw new TException("Object isn't BufferFileInputStream for " - + id); - } - } catch (IOException e) { - LOG.error("BufferFileInputStream read failed when downloadChunk ", - e); - throw new TException(e); - } - byte[] empty = {}; - return ByteBuffer.wrap(empty); - } - - /** - * get cluster's summary, it will contain SupervisorSummary and - * TopologySummary - * - * @return ClusterSummary - */ - @Override - public ClusterSummary getClusterInfo() throws TException { - - try { - - StormClusterState stormClusterState = data.getStormClusterState(); - - Map assignments = new HashMap(); - - // get nimbus running time - int uptime = data.uptime(); - - // get TopologySummary - List topologySummaries = new ArrayList(); - - // 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(); - - Assignment assignment = stormClusterState.assignment_info( - topologyId, null); - if (assignment == null) { - LOG.error("Failed to get assignment of " + topologyId); - continue; - } - assignments.put(topologyId, assignment); - - Map lastErrTimeStamp = null; - try { - lastErrTimeStamp = stormClusterState.topo_lastErr_time(topologyId); - } catch (Exception e) { - LOG.error("Failed to get last error timestamp map for "+ topologyId + - ", and begin to remove the corrupt data", e); - try { - stormClusterState.remove_lastErr_time(topologyId); - } catch (Exception rmErr) { - LOG.error("Failed to remove last error timestamp in ZK for " + topologyId, rmErr); - } - } - - TopologySummary topology = NimbusUtils.mkTopologySummary( - assignment, topologyId, base.getStormName(), - base.getStatusString(), - TimeUtils.time_delta(base.getLanchTimeSecs()), lastErrTimeStamp); - - topologySummaries.add(topology); - - } - - // all supervisors - Map supervisorInfos = Cluster - .allSupervisorInfo(stormClusterState, null); - - // generate SupervisorSummaries - List supervisorSummaries = NimbusUtils - .mkSupervisorSummaries(supervisorInfos, assignments); - - ClusterSummary ret = new ClusterSummary(supervisorSummaries, uptime, - topologySummaries); - // set cluster version - ret.set_version(NimbusUtils.getNimbusVersion()); - - return ret; - - } catch (TException e) { - LOG.info("Failed to get ClusterSummary ", e); - throw e; - } catch (Exception e) { - LOG.info("Failed to get ClusterSummary ", e); - throw new TException(e); - } - } - - @Override - public SupervisorWorkers getSupervisorWorkers(String host) - throws NotAliveException, TException { - try { - StormClusterState stormClusterState = data.getStormClusterState(); - - String supervisorId = null; - SupervisorInfo supervisorInfo = null; - - String ip = NetWorkUtils.host2Ip(host); - String hostName = NetWorkUtils.ip2Host(host); - - // all supervisors - Map supervisorInfos = Cluster - .allSupervisorInfo(stormClusterState, null); - - for (Entry entry : supervisorInfos - .entrySet()) { - - SupervisorInfo info = entry.getValue(); - if (info.getHostName().equals(hostName) - || info.getHostName().equals(ip)) { - supervisorId = entry.getKey(); - supervisorInfo = info; - break; - } - } - - if (supervisorId == null) { - throw new TException("No supervisor of " + host); - } - - Map assignments = new HashMap(); - - // 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(); - - Assignment assignment = stormClusterState.assignment_info( - topologyId, null); - if (assignment == null) { - LOG.error("Failed to get assignment of " + topologyId); - continue; - } - assignments.put(topologyId, assignment); - - } - - Map portWorkerSummarys = new TreeMap(); - Map supervisorToUsedSlotNum = new HashMap(); - for (Entry entry : assignments.entrySet()) { - String topologyId = entry.getKey(); - Assignment assignment = entry.getValue(); - - Map taskToComponent = Cluster - .topology_task_info(stormClusterState, topologyId); - Map taskToComponentType = Cluster - .topology_task_compType(stormClusterState, topologyId); - - Set workers = assignment.getWorkers(); - - for (ResourceWorkerSlot worker : workers) { - if (supervisorId.equals(worker.getNodeId()) == false) { - continue; - } - Integer slotNum = supervisorToUsedSlotNum - .get(supervisorId); - if (slotNum == null) { - slotNum = 0; - supervisorToUsedSlotNum.put(supervisorId, slotNum); - } - supervisorToUsedSlotNum.put(supervisorId, ++slotNum); - - Integer port = worker.getPort(); - WorkerSummary workerSummary = portWorkerSummarys - .get(port); - if (workerSummary == null) { - workerSummary = new WorkerSummary(); - workerSummary.set_port(port); - workerSummary.set_topology(topologyId); - workerSummary - .set_tasks(new ArrayList()); - - portWorkerSummarys.put(port, workerSummary); - } - - for (Integer taskId : worker.getTasks()) { - - String componentName = taskToComponent.get(taskId); - String componentType = taskToComponentType.get(taskId); - int uptime = TimeUtils.time_delta(assignment - .getTaskStartTimeSecs().get(taskId)); - List tasks = workerSummary.get_tasks(); - - TaskSummary taskSummary = NimbusUtils - .mkSimpleTaskSummary(worker, taskId, - componentName, componentType, host, uptime); - - tasks.add(taskSummary); - } - } - } - - List wokersList = new ArrayList(); - wokersList.addAll(portWorkerSummarys.values()); - - SupervisorSummary supervisorSummary = NimbusUtils - .mkSupervisorSummary(supervisorInfo, supervisorId, - supervisorToUsedSlotNum); - return new SupervisorWorkers(supervisorSummary, wokersList); - - } catch (TException e) { - LOG.info("Failed to get ClusterSummary ", e); - throw e; - } catch (Exception e) { - LOG.info("Failed to get ClusterSummary ", e); - 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 - * - * @return TopologyInfo - */ - @Override - public TopologyInfo getTopologyInfo(String topologyId) - throws NotAliveException, TException { - - StormClusterState stormClusterState = data.getStormClusterState(); - - try { - - // get topology's StormBase - StormBase base = stormClusterState.storm_base(topologyId, null); - if (base == null) { - throw new NotAliveException("No topology of " + topologyId); - } - return getTopologyInfo(topologyId, base); - } catch (TException e) { - LOG.info("Failed to get topologyInfo " + topologyId, e); - throw e; - } catch (Exception e) { - LOG.info("Failed to get topologyInfo " + topologyId, e); - throw new TException("Failed to get topologyInfo" + 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 - * - * @param id - * String: topology id - * @return String - */ - @Override - public String getTopologyConf(String id) throws NotAliveException, - TException { - String rtn; - try { - Map topologyConf = StormConfig - .read_nimbus_topology_conf(conf, id); - rtn = JStormUtils.to_json(topologyConf); - } catch (IOException e) { - // TODO Auto-generated catch block - LOG.info("Failed to get configuration of " + id, e); - throw new TException(e); - } - return rtn; - } - - /** - * get StormTopology throw deserialize local files - * - * @param id - * String: topology id - * @return StormTopology - */ - @Override - public StormTopology getTopology(String id) throws NotAliveException, - TException { - StormTopology topology = null; - try { - StormTopology stormtopology = StormConfig - .read_nimbus_topology_code(conf, id); - if (stormtopology == null) { - throw new TException("topology:" + id + "is null"); - } - - Map topologyConf = (Map) StormConfig - .read_nimbus_topology_conf(conf, id); - - topology = Common.system_topology(topologyConf, stormtopology); - } catch (Exception e) { - LOG.error("Failed to get topology " + id + ",", e); - throw new TException("Failed to get system_topology"); - } - return topology; - } - - /** - * Shutdown the nimbus - */ - @Override - public void shutdown() { - LOG.info("Begin to shut down master"); - // Timer.cancelTimer(nimbus.getTimer()); - - LOG.info("Successfully shut down master"); - - } - - @Override - public boolean waiting() { - // @@@ TODO - return false; - } - - /** - * check whether the topology is bActive? - * - * @param nimbus - * @param topologyName - * @param bActive - * @throws Exception - */ - public void checkTopologyActive(NimbusData nimbus, String topologyName, - boolean bActive) throws Exception { - if (isTopologyActive(nimbus.getStormClusterState(), topologyName) != bActive) { - if (bActive) { - throw new NotAliveException(topologyName + " is not alive"); - } else { - throw new AlreadyAliveException(topologyName - + " is already active"); - } - } - } - - /** - * whether the topology is active by topology name - * - * @param stormClusterState - * see Cluster_clj - * @param topologyName - * @return boolean if the storm is active, return true, otherwise return - * false; - * @throws Exception - */ - public boolean isTopologyActive(StormClusterState stormClusterState, - String topologyName) throws Exception { - boolean rtn = false; - if (Cluster.get_topology_id(stormClusterState, topologyName) != null) { - rtn = true; - } - return rtn; - } - - /** - * create local topology files /local-dir/nimbus/topologyId/stormjar.jar - * /local-dir/nimbus/topologyId/stormcode.ser - * /local-dir/nimbus/topologyId/stormconf.ser - * - * @param conf - * @param topologyId - * @param tmpJarLocation - * @param stormConf - * @param topology - * @throws IOException - */ - private void setupStormCode(Map conf, String topologyId, - String tmpJarLocation, Map stormConf, - StormTopology topology) throws IOException { - // local-dir/nimbus/stormdist/topologyId - String stormroot = StormConfig.masterStormdistRoot(conf, topologyId); - - FileUtils.forceMkdir(new File(stormroot)); - FileUtils.cleanDirectory(new File(stormroot)); - - // copy jar to /local-dir/nimbus/topologyId/stormjar.jar - setupJar(conf, tmpJarLocation, stormroot); - - // serialize to file /local-dir/nimbus/topologyId/stormcode.ser - FileUtils.writeByteArrayToFile( - new File(StormConfig.stormcode_path(stormroot)), - Utils.serialize(topology)); - - // serialize to file /local-dir/nimbus/topologyId/stormconf.ser - FileUtils.writeByteArrayToFile( - 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 - * - * @param conf - * @param tmpJarLocation - * @param stormroot - * @throws IOException - */ - private void setupJar(Map conf, String tmpJarLocation, - String stormroot) throws IOException { - if (!StormConfig.local_mode(conf)) { - 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(); - - return ; - } - } - - /** - * generate TaskInfo for every bolt or spout in ZK /ZK/tasks/topoologyId/xxx - * - * @param conf - * @param topologyId - * @param stormClusterState - * @throws Exception - */ - public void setupZkTaskInfo(Map conf, String topologyId, - StormClusterState stormClusterState) throws Exception { - - // mkdir /ZK/taskbeats/topoologyId - stormClusterState.setup_heartbeats(topologyId); - - Map taskToComponetId = mkTaskComponentAssignments( - conf, topologyId); - if (taskToComponetId == null || taskToComponetId.size() == 0) { - throw new InvalidTopologyException("Failed to generate TaskIDs map"); - } - - for (Entry entry : taskToComponetId.entrySet()) { - // key is taskid, value is taskinfo - stormClusterState.set_task(topologyId, entry.getKey(), entry.getValue()); - } - } - - /** - * generate a taskid(Integer) for every task - * - * @param conf - * @param topologyid - * @return Map: from taskid to componentid - * @throws IOException - * @throws InvalidTopologyException - */ - public Map mkTaskComponentAssignments( - Map conf, String topologyid) throws IOException, - InvalidTopologyException { - - // @@@ here exist a little problem, - // we can directly pass stormConf from Submit method - Map stormConf = StormConfig.read_nimbus_topology_conf( - conf, topologyid); - - StormTopology stopology = StormConfig.read_nimbus_topology_code(conf, - topologyid); - - // use TreeMap to make task as sequence - Map rtn = new TreeMap(); - - StormTopology topology = Common.system_topology(stormConf, stopology); - - Integer count = 0; - count = mkTaskMaker(stormConf, topology.get_bolts(), rtn, count); - count = mkTaskMaker(stormConf, topology.get_spouts(), rtn, count); - count = mkTaskMaker(stormConf, topology.get_state_spouts(), rtn, count); - - return rtn; - } - - @SuppressWarnings({ "rawtypes", "unchecked" }) - public Integer mkTaskMaker(Map stormConf, - Map cidSpec, Map rtn, Integer cnt) { - if (cidSpec == null) { - LOG.warn("Component map is empty"); - return cnt; - } - - Set entrySet = cidSpec.entrySet(); - for (Iterator it = entrySet.iterator(); it.hasNext();) { - Entry entry = (Entry) it.next(); - Object obj = entry.getValue(); - - ComponentCommon common = null; - String componentType = "bolt"; - if (obj instanceof Bolt) { - common = ((Bolt) obj).get_common(); - componentType = "bolt"; - } else if (obj instanceof SpoutSpec) { - common = ((SpoutSpec) obj).get_common(); - componentType = "spout"; - } else if (obj instanceof StateSpoutSpec) { - common = ((StateSpoutSpec) obj).get_common(); - componentType = "spout"; - } - - if (common == null) { - throw new RuntimeException("No ComponentCommon of " - + entry.getKey()); - } - - int declared = Thrift.parallelismHint(common); - Integer parallelism = declared; - // Map tmp = (Map) Utils_clj.from_json(common.get_json_conf()); - - Map newStormConf = new HashMap(stormConf); - // newStormConf.putAll(tmp); - Integer maxParallelism = JStormUtils.parseInt(newStormConf - .get(Config.TOPOLOGY_MAX_TASK_PARALLELISM)); - if (maxParallelism != null) { - parallelism = Math.min(maxParallelism, declared); - } - - for (int i = 0; i < parallelism; i++) { - cnt++; - TaskInfo taskInfo = new TaskInfo((String) entry.getKey(), componentType); - rtn.put(cnt, taskInfo); - } - } - return cnt; - } - - @Override - public String getNimbusConf() throws TException { - return null; - } - - @Override - public StormTopology getUserTopology(String id) throws NotAliveException, - TException { - return null; - } - - @Override - public void metricMonitor(String topologyName, MonitorOptions options) throws NotAliveException, - TException { - boolean isEnable = options.is_isEnable(); - StormClusterState clusterState = data.getStormClusterState(); - - try { - String topologyId = Cluster.get_topology_id(clusterState, topologyName); - if (null != topologyId) { - NimbusUtils.updateMetricMonitorStatus(clusterState, topologyId, isEnable); - } else { - throw new NotAliveException("Failed to update metricsMonitor status as " + topologyName + " is not alive"); - } - } catch(Exception e) { - String errMsg = "Failed to update metricsMonitor " + topologyName; - LOG.error(errMsg, e); - throw new TException(e); - } - - } - - @Override - public TopologyMetricInfo getTopologyMetric(String topologyId) throws NotAliveException, TException{ - LOG.debug("Nimbus service handler, getTopologyMetric, topology ID: " + topologyId); - - TopologyMetricInfo topologyMetricInfo = new TopologyMetricInfo(); - - StormClusterState clusterState = data.getStormClusterState(); - - topologyMetricInfo.set_topology_id(topologyId); - try { - //update task metrics list - Map taskInfoList = clusterState.task_info_list(topologyId); - List taskMetricList = clusterState.get_task_metric_list(topologyId); - for(TaskMetricInfo taskMetricInfo : taskMetricList) { - TaskMetricData taskMetricData = new TaskMetricData(); - NimbusUtils.updateTaskMetricData(taskMetricData, taskMetricInfo); - TaskInfo taskInfo = taskInfoList.get(Integer.parseInt(taskMetricInfo.getTaskId())); - String componentId = taskInfo.getComponentId(); - taskMetricData.set_component_id(componentId); - - topologyMetricInfo.add_to_task_metric_list(taskMetricData); - } - - //update worker metrics list - List workerMetricList = clusterState.get_worker_metric_list(topologyId); - for(WorkerMetricInfo workerMetricInfo : workerMetricList) { - WorkerMetricData workerMetricData = new WorkerMetricData(); - NimbusUtils.updateWorkerMetricData(workerMetricData, workerMetricInfo); - - topologyMetricInfo.add_to_worker_metric_list(workerMetricData); - } - - } catch(Exception e) { - String errMsg = "Failed to get topology Metric Data " + topologyId; - LOG.error(errMsg, e); - throw new TException(e); - } - - 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 deleted file mode 100644 index acd1d82cb..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/StatusTransition.java +++ /dev/null @@ -1,258 +0,0 @@ -package com.alibaba.jstorm.daemon.nimbus; - -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -import org.apache.log4j.Logger; - -import com.alibaba.jstorm.callback.Callback; -import com.alibaba.jstorm.callback.impl.ActiveTransitionCallback; -import com.alibaba.jstorm.callback.impl.DoRebalanceTransitionCallback; -import com.alibaba.jstorm.callback.impl.InactiveTransitionCallback; -import com.alibaba.jstorm.callback.impl.KillTransitionCallback; -import com.alibaba.jstorm.callback.impl.ReassignTransitionCallback; -import com.alibaba.jstorm.callback.impl.RebalanceTransitionCallback; -import com.alibaba.jstorm.callback.impl.RemoveTransitionCallback; -import com.alibaba.jstorm.cluster.StormBase; -import com.alibaba.jstorm.cluster.StormStatus; - -/** - * Status changing - * - * @author version1: lixin version2: Longda - * - * - * - */ -public class StatusTransition { - - private final static Logger LOG = Logger.getLogger(StatusTransition.class); - - private NimbusData data; - - private Map topologyLocks = new ConcurrentHashMap(); - - public StatusTransition(NimbusData data) { - this.data = data; - - } - - public void transition(String topologyid, boolean errorOnNoTransition, - StatusType changeStatus, T... args) throws Exception { - // lock outside - Object lock = topologyLocks.get(topologyid); - if (lock == null) { - 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); - - // update the lock times - topologyLocks.put(topologyid, lock); - } - } - - /** - * Changing status - * - * @param topologyId - * @param errorOnNTransition - * if it is true, failure will throw exception - * @param args - * -- will be used in the status changing callback - * - */ - public void transitionLock(String topologyid, - boolean errorOnNoTransition, StatusType changeStatus, T... args) - throws Exception { - - // get ZK's topology node's data, which is StormBase - StormBase stormbase = data.getStormClusterState().storm_base( - topologyid, null); - if (stormbase == null) { - - LOG.error("Cannot apply event changing status " - + changeStatus.getStatus() + " to " + topologyid - + " because failed to get StormBase from ZK"); - return; - } - - StormStatus currentStatus = stormbase.getStatus(); - if (currentStatus == null) { - LOG.error("Cannot apply event changing status " - + changeStatus.getStatus() + " to " + topologyid - + " because topologyStatus is null in ZK"); - return; - } - - // > - Map> callbackMap = stateTransitions( - topologyid, currentStatus); - - // get current changingCallbacks - Map changingCallbacks = callbackMap - .get(currentStatus.getStatusType()); - - if (changingCallbacks == null - || changingCallbacks.containsKey(changeStatus) == false - || changingCallbacks.get(changeStatus) == null) { - String msg = "No transition for event: changing status:" - + changeStatus.getStatus() + ", current status: " - + currentStatus.getStatusType() + " topology-id: " - + topologyid; - LOG.info(msg); - if (errorOnNoTransition) { - throw new RuntimeException(msg); - } - return; - } - - Callback callback = changingCallbacks.get(changeStatus); - - Object obj = callback.execute(args); - if (obj != null && obj instanceof StormStatus) { - StormStatus newStatus = (StormStatus) obj; - // update status to ZK - data.getStormClusterState().update_storm(topologyid, newStatus); - LOG.info("Successfully updated " + topologyid + " as status " - + newStatus); - } - - LOG.info("Successfully apply event changing status " - + changeStatus.getStatus() + " to " + topologyid); - return; - - } - - /** - * generate status changing map - * - * - * - * @param topologyid - * @param status - * @return - * - * Map> means - * Map> - */ - - private Map> stateTransitions( - String topologyid, StormStatus currentStatus) { - - /** - * - * 1. Status: this status will be stored in ZK - * killed/inactive/active/rebalancing 2. action: - * - * monitor -- every Config.NIMBUS_MONITOR_FREQ_SECS seconds will trigger - * this only valid when current status is active inactivate -- client - * will trigger this action, only valid when current status is active - * activate -- client will trigger this action only valid when current - * status is inactive startup -- when nimbus startup, it will trigger - * this action only valid when current status is killed/rebalancing kill - * -- client kill topology will trigger this action, only valid when - * current status is active/inactive/killed remove -- 30 seconds after - * client submit kill command, it will do this action, only valid when - * current status is killed rebalance -- client submit rebalance - * command, only valid when current status is active/deactive - * do_rebalance -- 30 seconds after client submit rebalance command, it - * will do this action, only valid when current status is rebalance - */ - - Map> rtn = new HashMap>(); - - // current status is active - Map activeMap = new HashMap(); - activeMap.put(StatusType.monitor, new ReassignTransitionCallback(data, - topologyid)); - activeMap.put(StatusType.inactivate, new InactiveTransitionCallback()); - activeMap.put(StatusType.startup, null); - activeMap.put(StatusType.activate, null); - activeMap.put(StatusType.kill, new KillTransitionCallback(data, - topologyid)); - activeMap.put(StatusType.remove, null); - activeMap.put(StatusType.rebalance, new RebalanceTransitionCallback( - data, topologyid, currentStatus)); - activeMap.put(StatusType.do_rebalance, null); - - rtn.put(StatusType.active, activeMap); - - // current status is inactive - Map inactiveMap = new HashMap(); - - inactiveMap.put(StatusType.monitor, new ReassignTransitionCallback( - data, topologyid, new StormStatus(StatusType.inactive))); - inactiveMap.put(StatusType.inactivate, null); - inactiveMap.put(StatusType.startup, null); - inactiveMap.put(StatusType.activate, new ActiveTransitionCallback()); - inactiveMap.put(StatusType.kill, new KillTransitionCallback(data, - topologyid)); - inactiveMap.put(StatusType.remove, null); - inactiveMap.put(StatusType.rebalance, new RebalanceTransitionCallback( - data, topologyid, currentStatus)); - inactiveMap.put(StatusType.do_rebalance, null); - - rtn.put(StatusType.inactive, inactiveMap); - - // current status is killed - Map killedMap = new HashMap(); - - killedMap.put(StatusType.monitor, null); - killedMap.put(StatusType.inactivate, null); - killedMap.put(StatusType.startup, new KillTransitionCallback(data, - topologyid)); - killedMap.put(StatusType.activate, null); - killedMap.put(StatusType.kill, new KillTransitionCallback(data, - topologyid)); - killedMap.put(StatusType.remove, new RemoveTransitionCallback(data, - topologyid)); - killedMap.put(StatusType.rebalance, null); - killedMap.put(StatusType.do_rebalance, null); - rtn.put(StatusType.killed, killedMap); - - // current status is under rebalancing - Map rebalancingMap = new HashMap(); - - StatusType rebalanceOldStatus = StatusType.active; - if (currentStatus.getOldStatus() != null) { - rebalanceOldStatus = currentStatus.getOldStatus().getStatusType(); - // fix double rebalance, make the status always as rebalacing - if (rebalanceOldStatus == StatusType.rebalance) { - rebalanceOldStatus = StatusType.active; - } - } - - rebalancingMap.put(StatusType.monitor, null); - rebalancingMap.put(StatusType.inactivate, null); - rebalancingMap.put(StatusType.startup, new RebalanceTransitionCallback( - data, topologyid, new StormStatus(rebalanceOldStatus))); - rebalancingMap.put(StatusType.activate, null); - rebalancingMap.put(StatusType.kill, null); - rebalancingMap.put(StatusType.remove, null); - rebalancingMap - .put(StatusType.rebalance, new RebalanceTransitionCallback( - data, topologyid, currentStatus)); - rebalancingMap.put(StatusType.do_rebalance, - new DoRebalanceTransitionCallback(data, topologyid, - new StormStatus(rebalanceOldStatus))); - rtn.put(StatusType.rebalancing, rebalancingMap); - - /** - * @@@ just handling 4 kind of status, maybe add later - */ - - return rtn; - - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/StatusType.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/StatusType.java deleted file mode 100644 index f8fd36285..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/StatusType.java +++ /dev/null @@ -1,48 +0,0 @@ -package com.alibaba.jstorm.daemon.nimbus; - - -/** - * topology status: - * - * 1. Status: this status will be stored in ZK - * killed/inactive/active/rebalancing 2. action: - * - * monitor -- every Config.NIMBUS_MONITOR_FREQ_SECS seconds will trigger this - * only valid when current status is active inactivate -- client will trigger - * this action, only valid when current status is active activate -- client will - * trigger this action only valid when current status is inactive startup -- - * when nimbus startup, it will trigger this action only valid when current - * status is killed/rebalancing kill -- client kill topology will trigger this - * action, only valid when current status is active/inactive/killed remove -- 30 - * seconds after client submit kill command, it will do this action, only valid - * when current status is killed rebalance -- client submit rebalance command, - * only valid when current status is active/deactive do_rebalance -- 30 seconds - * after client submit rebalance command, it will do this action, only valid - * when current status is rebalance - * - * - * - */ - -public enum StatusType { - - // status - active("active"), inactive("inactive"), rebalancing("rebalancing"), - killed("killed"), - - // actions - activate("activate"), inactivate("inactivate"), - monitor("monitor"), startup("startup"), - kill("kill"), remove("remove"), - rebalance("rebalance"), do_rebalance("do-rebalance"); - - private String status; - - StatusType(String status) { - this.status = status; - } - - public String getStatus() { - return status; - } -} 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 deleted file mode 100644 index 67e71307b..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/TopologyAssign.java +++ /dev/null @@ -1,834 +0,0 @@ -package com.alibaba.jstorm.daemon.nimbus; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -import java.util.TreeMap; -import java.util.concurrent.LinkedBlockingQueue; - -import org.apache.log4j.Logger; - -import backtype.storm.generated.StormTopology; -import backtype.storm.scheduler.WorkerSlot; - -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.Cluster; -import com.alibaba.jstorm.cluster.StormBase; -import com.alibaba.jstorm.cluster.StormClusterState; -import com.alibaba.jstorm.cluster.StormConfig; -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.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.PathUtils; -import com.alibaba.jstorm.utils.TimeUtils; - -public class TopologyAssign implements Runnable { - - private final static Logger LOG = Logger.getLogger(TopologyAssign.class); - - /** - * private constructor function to avoid multiple instance - */ - private TopologyAssign() { - - } - - private static TopologyAssign instance = null; - - public static TopologyAssign getInstance() { - synchronized (TopologyAssign.class) { - if (instance == null) { - instance = new TopologyAssign(); - } - return instance; - - } - } - - protected NimbusData nimbusData; - - protected Map schedulers; - - private Thread thread; - - public static final String DEFAULT_SCHEDULER_NAME = "default"; - - public void init(NimbusData nimbusData) { - this.nimbusData = nimbusData; - this.schedulers = new HashMap(); - - IToplogyScheduler defaultScheduler = new DefaultTopologyScheduler(); - defaultScheduler.prepare(nimbusData.getConf()); - - schedulers.put(DEFAULT_SCHEDULER_NAME, defaultScheduler); - - thread = new Thread(this); - thread.setName("TopologyAssign"); - thread.setDaemon(true); - thread.start(); - } - - public void cleanup() { - runFlag = false; - thread.interrupt(); - } - - protected static LinkedBlockingQueue queue = new LinkedBlockingQueue(); - - public static void push(TopologyAssignEvent event) { - queue.offer(event); - } - - volatile boolean runFlag = false; - - public void run() { - LOG.info("TopologyAssign thread has been started"); - runFlag = true; - - while (runFlag) { - TopologyAssignEvent event; - try { - event = queue.take(); - } catch (InterruptedException e1) { - continue; - } - if (event == null) { - continue; - } - - boolean isSuccess = doTopologyAssignment(event); - - if (isSuccess == false) { - } else { - try { - cleanupDisappearedTopology(); - } catch (Exception e) { - LOG.error("Failed to do cleanup disappear topology ", e); - continue; - } - } - } - - } - - /** - * Create/Update topology assignment set topology status - * - * @param event - * @return - */ - protected boolean doTopologyAssignment(TopologyAssignEvent event) { - Assignment assignment = null; - try { - assignment = mkAssignment(event); - - setTopologyStatus(event); - } catch (Throwable e) { - LOG.error("Failed to assign topology " + event.getTopologyId(), e); - event.fail(e.getMessage()); - return false; - } - - backupAssignment(assignment, event); - event.done(); - return true; - } - - /** - * cleanup the topologies which are not in ZK /topology, but in other place - * - * @param nimbusData - * @param active_topologys - * @throws Exception - */ - public void cleanupDisappearedTopology() throws Exception { - StormClusterState clusterState = nimbusData.getStormClusterState(); - - List active_topologys = clusterState.active_storms(); - if (active_topologys == null) { - return; - } - - Set cleanupIds = get_cleanup_ids(clusterState, active_topologys); - - for (String topologyId : cleanupIds) { - - LOG.info("Cleaning up " + topologyId); - - clusterState.try_remove_storm(topologyId); - // - nimbusData.getTaskHeartbeatsCache().remove(topologyId); - - // get /nimbus/stormdist/topologyId - String master_stormdist_root = StormConfig.masterStormdistRoot( - nimbusData.getConf(), topologyId); - try { - // delete topologyId local dir - PathUtils.rmr(master_stormdist_root); - } catch (IOException e) { - LOG.warn("Failed to delete " + master_stormdist_root + ",", e); - } - } - } - - /** - * get topology ids which need to be cleanup - * - * @param clusterState - * @return - * @throws Exception - */ - private Set get_cleanup_ids(StormClusterState clusterState, - List active_topologys) throws Exception { - - - 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()); - // listdir /local-dir/nimbus/stormdist - List code_ids = PathUtils - .read_dir_contents(master_stormdist_root); - - // 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); - } - return to_cleanup_ids; - } - - /** - * start a topology: set active status of the topology - * - * @param topologyName - * @param stormClusterState - * @param topologyId - * @throws Exception - */ - public void setTopologyStatus(TopologyAssignEvent event) throws Exception { - StormClusterState stormClusterState = nimbusData.getStormClusterState(); - - String topologyId = event.getTopologyId(); - String topologyName = event.getTopologyName(); - String group = event.getGroup(); - - StormStatus status = new StormStatus(StatusType.active); - if (event.getOldStatus() != null) { - status = event.getOldStatus(); - } - - StormBase stormBase = stormClusterState.storm_base(topologyId, null); - if (stormBase == null) { - stormBase = new StormBase(topologyName, - TimeUtils.current_time_secs(), status, group); - stormClusterState.activate_storm(topologyId, stormBase); - - } else { - - stormClusterState.update_storm(topologyId, status); - - // here exist one hack operation - // when monitor/rebalance/startup topologyName is null - if (topologyName == null) { - event.setTopologyName(stormBase.getStormName()); - } - } - - LOG.info("Update " + topologyId + " " + status); - - } - - protected TopologyAssignContext prepareTopologyAssign( - TopologyAssignEvent event) throws Exception { - TopologyAssignContext ret = new TopologyAssignContext(); - - String topologyId = event.getTopologyId(); - - Map nimbusConf = nimbusData.getConf(); - Map topologyConf = StormConfig - .read_nimbus_topology_conf(nimbusConf, topologyId); - - StormTopology rawTopology = StormConfig.read_nimbus_topology_code( - nimbusConf, topologyId); - ret.setRawTopology(rawTopology); - - Map stormConf = new HashMap(); - stormConf.putAll(nimbusConf); - stormConf.putAll(topologyConf); - ret.setStormConf(stormConf); - - StormClusterState stormClusterState = nimbusData.getStormClusterState(); - - // get all running supervisor, don't need callback to watch supervisor - Map supInfos = Cluster.allSupervisorInfo( - stormClusterState, null); - if (supInfos.size() == 0) { - throw new FailedAssignTopologyException( - "Failed to make assignment " + topologyId - + ", due to no alive supervisor"); - } - - Map taskToComponent = Cluster.topology_task_info( - stormClusterState, topologyId); - ret.setTaskToComponent(taskToComponent); - - // get taskids /ZK/tasks/topologyId - Set allTaskIds = taskToComponent.keySet(); - if (allTaskIds == null || allTaskIds.size() == 0) { - String errMsg = "Failed to get all task ID list from /ZK-dir/tasks/" - + topologyId; - LOG.warn(errMsg); - throw new IOException(errMsg); - } - ret.setAllTaskIds(allTaskIds); - - Set aliveTasks = new HashSet(); - // unstoppedTasks are tasks which are alive on no supervisor's(dead) - // machine - Set unstoppedTasks = new HashSet(); - Set deadTasks = new HashSet(); - Set unstoppedWorkers = new HashSet(); - - Assignment existingAssignment = stormClusterState.assignment_info( - topologyId, null); - if (existingAssignment != null) { - aliveTasks = getAliveTasks(topologyId, allTaskIds); - unstoppedTasks = getUnstoppedSlots(aliveTasks, supInfos, - existingAssignment); - - deadTasks.addAll(allTaskIds); - deadTasks.removeAll(aliveTasks); - } - - ret.setDeadTaskIds(deadTasks); - ret.setUnstoppedTaskIds(unstoppedTasks); - - // Step 2: get all slots resource, free slots/ alive slots/ unstopped - // slots - getFreeSlots(supInfos, stormClusterState); - ret.setCluster(supInfos); - - if (existingAssignment == null) { - ret.setAssignType(TopologyAssignContext.ASSIGN_TYPE_NEW); - - try { - AssignmentBak lastAssignment = stormClusterState - .assignment_bak(event.getTopologyName()); - if (lastAssignment != null) { - ret.setOldAssignment(lastAssignment.getAssignment()); - } - } catch (Exception e) { - LOG.warn("Fail to get old assignment", e); - } - } else { - ret.setOldAssignment(existingAssignment); - if (event.isScratch()) { - ret.setAssignType(TopologyAssignContext.ASSIGN_TYPE_REBALANCE); - unstoppedWorkers = getUnstoppedWorkers(unstoppedTasks, - existingAssignment); - ret.setUnstoppedWorkers(unstoppedWorkers); - } else { - ret.setAssignType(TopologyAssignContext.ASSIGN_TYPE_MONITOR); - unstoppedWorkers = getUnstoppedWorkers(aliveTasks, - existingAssignment); - ret.setUnstoppedWorkers(unstoppedWorkers); - } - } - - return ret; - } - - /** - * make assignments for a topology The nimbus core function, this function - * has been totally rewrite - * - * @param nimbusData - * NimbusData - * @param topologyId - * String - * @param isScratch - * Boolean: isScratch is false unless rebalancing the topology - * @throws Exception - */ - public Assignment mkAssignment(TopologyAssignEvent event) throws Exception { - String topologyId = event.getTopologyId(); - - LOG.info("Determining assignment for " + topologyId); - - TopologyAssignContext context = prepareTopologyAssign(event); - - Set assignments = null; - - if (!StormConfig.local_mode(nimbusData.getConf())) { - - IToplogyScheduler scheduler = schedulers - .get(DEFAULT_SCHEDULER_NAME); - - assignments = scheduler.assignTasks(context); - - } else { - assignments = mkLocalAssignment(context); - } - Assignment assignment = null; - - Map nodeHost = getTopologyNodeHost( - context.getCluster(), context.getOldAssignment(), assignments); - - Map startTimes = getTaskStartTimes(context, - nimbusData, topologyId, context.getOldAssignment(), assignments); - - String codeDir = StormConfig.masterStormdistRoot(nimbusData.getConf(), - topologyId); - - assignment = new Assignment(codeDir, assignments, nodeHost, startTimes); - - StormClusterState stormClusterState = nimbusData.getStormClusterState(); - - stormClusterState.set_assignment(topologyId, assignment); - - // update task heartbeat's start time - NimbusUtils.updateTaskHbStartTime(nimbusData, assignment, topologyId); - - // Update metrics information in ZK when rebalance or reassignment - // Only update metrics monitor status when creating topology - if (context.getAssignType() == TopologyAssignContext.ASSIGN_TYPE_REBALANCE - || context.getAssignType() == TopologyAssignContext.ASSIGN_TYPE_MONITOR) - NimbusUtils.updateMetricsInfo(nimbusData, topologyId, assignment); - else - metricsMonitor(event); - - LOG.info("Successfully make assignment for topology id " + topologyId - + ": " + assignment); - - return assignment; - } - - private static Set mkLocalAssignment( - TopologyAssignContext context) { - Set result = new HashSet(); - Map cluster = context.getCluster(); - if (cluster.size() != 1) - throw new RuntimeException(); - SupervisorInfo localSupervisor = null; - String supervisorId = null; - for (Entry entry : cluster.entrySet()) { - supervisorId = entry.getKey(); - localSupervisor = entry.getValue(); - } - int port = localSupervisor.getWorkerPorts().iterator().next(); - ResourceWorkerSlot worker = new ResourceWorkerSlot(supervisorId, port); - worker.setTasks(new HashSet(context.getAllTaskIds())); - worker.setHostname(localSupervisor.getHostName()); - result.add(worker); - return result; - } - - /** - * @param existingAssignment - * @param taskWorkerSlot - * @return - * @throws Exception - */ - public static Map getTaskStartTimes( - TopologyAssignContext context, NimbusData nimbusData, - String topologyId, Assignment existingAssignment, - Set workers) throws Exception { - - Map startTimes = new TreeMap(); - - if (context.getAssignType() == TopologyAssignContext.ASSIGN_TYPE_NEW) { - int nowSecs = TimeUtils.current_time_secs(); - for (ResourceWorkerSlot worker : workers) { - for (Integer changedTaskId : worker.getTasks()) { - startTimes.put(changedTaskId, nowSecs); - } - } - - return startTimes; - } - - Set oldWorkers = new HashSet(); - - if (existingAssignment != null) { - Map taskStartTimeSecs = existingAssignment - .getTaskStartTimeSecs(); - if (taskStartTimeSecs != null) { - startTimes.putAll(taskStartTimeSecs); - } - - if (existingAssignment.getWorkers() != null) { - oldWorkers = existingAssignment.getWorkers(); - } - } - - StormClusterState zkClusterState = nimbusData.getStormClusterState(); - Set changeTaskIds = getChangeTaskIds(oldWorkers, workers); - int nowSecs = TimeUtils.current_time_secs(); - for (Integer changedTaskId : changeTaskIds) { - startTimes.put(changedTaskId, nowSecs); - - zkClusterState.remove_task_heartbeat(topologyId, changedTaskId); - } - - LOG.info("Task assignment has been changed " + changeTaskIds); - return startTimes; - } - - public static Map getTopologyNodeHost( - Map supervisorMap, - Assignment existingAssignment, Set workers) { - - // the following is that remove unused node from allNodeHost - Set usedNodes = new HashSet(); - for (ResourceWorkerSlot worker : workers) { - - usedNodes.add(worker.getNodeId()); - } - - // map - Map allNodeHost = new HashMap(); - - if (existingAssignment != null) { - allNodeHost.putAll(existingAssignment.getNodeHost()); - } - - // get alive supervisorMap Map - Map nodeHost = SupervisorInfo - .getNodeHost(supervisorMap); - if (nodeHost != null) { - allNodeHost.putAll(nodeHost); - } - - Map ret = new HashMap(); - - for (String supervisorId : usedNodes) { - if (allNodeHost.containsKey(supervisorId)) { - ret.put(supervisorId, allNodeHost.get(supervisorId)); - } else { - LOG.warn("Node " + supervisorId - + " doesn't in the supervisor list"); - } - } - - return ret; - } - - /** - * get all taskids which should be reassigned - * - * @param taskToWorkerSlot - * @param newtaskToWorkerSlot - * @return Set taskid which should reassigned - */ - public static Set getChangeTaskIds( - Set oldWorkers, Set workers) { - - Set rtn = new HashSet(); - for (ResourceWorkerSlot worker : workers) { - if (!oldWorkers.contains(worker)) - rtn.addAll(worker.getTasks()); - } - return rtn; - } - - /** - * sort slots, the purpose is to ensure that the tasks are assigned in - * balancing - * - * @param allSlots - * @return List - */ - public static List sortSlots(Set allSlots, - int needSlotNum) { - - Map> nodeMap = new HashMap>(); - - // group by first - for (WorkerSlot np : allSlots) { - String node = np.getNodeId(); - - List list = nodeMap.get(node); - if (list == null) { - list = new ArrayList(); - nodeMap.put(node, list); - } - - list.add(np); - - } - - for (Entry> entry : nodeMap.entrySet()) { - List ports = entry.getValue(); - - Collections.sort(ports, new Comparator() { - - @Override - public int compare(WorkerSlot first, WorkerSlot second) { - String firstNode = first.getNodeId(); - String secondNode = second.getNodeId(); - if (firstNode.equals(secondNode) == false) { - return firstNode.compareTo(secondNode); - } else { - return first.getPort() - second.getPort(); - } - - } - - }); - } - - // interleave - List> splitup = new ArrayList>( - nodeMap.values()); - - Collections.sort(splitup, new Comparator>() { - - public int compare(List o1, List o2) { - return o2.size() - o1.size(); - } - }); - - List sortedFreeSlots = JStormUtils.interleave_all(splitup); - - if (sortedFreeSlots.size() <= needSlotNum) { - return sortedFreeSlots; - - } - - // sortedFreeSlots > needSlotNum - return sortedFreeSlots.subList(0, needSlotNum); - } - - /** - * Get unstopped slots from alive task list - * - * @param aliveAssigned - * @param supInfos - * @return - */ - public Set getUnstoppedSlots(Set aliveTasks, - Map supInfos, Assignment existAssignment) { - Set ret = new HashSet(); - - Set oldWorkers = existAssignment.getWorkers(); - - Set aliveSupervisors = supInfos.keySet(); - - for (ResourceWorkerSlot worker : oldWorkers) { - for (Integer taskId : worker.getTasks()) { - if (aliveTasks.contains(taskId) == false) { - // task is dead - continue; - } - - String oldTaskSupervisorId = worker.getNodeId(); - - if (aliveSupervisors.contains(oldTaskSupervisorId) == false) { - // supervisor is dead - ret.add(taskId); - continue; - } - } - } - - return ret; - - } - - private Set getUnstoppedWorkers( - Set aliveTasks, Assignment existAssignment) { - Set ret = new HashSet(); - for (ResourceWorkerSlot worker : existAssignment.getWorkers()) { - boolean alive = true; - for (Integer task : worker.getTasks()) { - if (!aliveTasks.contains(task)) { - alive = false; - break; - } - } - if (alive) { - ret.add(worker); - } - } - return ret; - } - - /** - * Get free resources - * - * @param supervisorInfos - * @param stormClusterState - * @throws Exception - */ - public static void getFreeSlots( - Map supervisorInfos, - StormClusterState stormClusterState) throws Exception { - - Map assignments = Cluster.get_all_assignment( - stormClusterState, null); - - for (Entry entry : assignments.entrySet()) { - String topologyId = entry.getKey(); - Assignment assignment = entry.getValue(); - - Set workers = assignment.getWorkers(); - - for (ResourceWorkerSlot worker : workers) { - - SupervisorInfo supervisorInfo = supervisorInfos.get(worker - .getNodeId()); - if (supervisorInfo == null) { - // the supervisor is dead - continue; - } - supervisorInfo.getWorkerPorts().remove(worker.getPort()); - } - } - - } - - /** - * find all alived taskid Does not assume that clocks are synchronized. Task - * heartbeat is only used so that nimbus knows when it's received a new - * heartbeat. All timing is done by nimbus and tracked through - * task-heartbeat-cache - * - * @param conf - * @param topologyId - * @param stormClusterState - * @param taskIds - * @param taskStartTimes - * @param taskHeartbeatsCache - * --Map>> - * @return Set : taskid - * @throws Exception - */ - public Set getAliveTasks(String topologyId, Set taskIds) - throws Exception { - - Set aliveTasks = new HashSet(); - - // taskIds is the list from ZK /ZK-DIR/tasks/topologyId - for (int taskId : taskIds) { - - boolean isDead = NimbusUtils.isTaskDead(nimbusData, topologyId, - taskId); - if (isDead == false) { - aliveTasks.add(taskId); - } - - } - - return aliveTasks; - } - - /** - * Backup the toplogy's Assignment to ZK - * - * @@@ Question Do we need to do backup operation every time? - * @param assignment - * @param event - */ - public void backupAssignment(Assignment assignment, - TopologyAssignEvent event) { - String topologyId = event.getTopologyId(); - String topologyName = event.getTopologyName(); - try { - - StormClusterState zkClusterState = nimbusData - .getStormClusterState(); - // one little problem, get tasks twice when assign one topology - HashMap tasks = Cluster.topology_task_info( - zkClusterState, topologyId); - - Map> componentTasks = JStormUtils - .reverse_map(tasks); - - for (Entry> entry : componentTasks.entrySet()) { - List keys = entry.getValue(); - - Collections.sort(keys); - - } - - AssignmentBak assignmentBak = new AssignmentBak(componentTasks, - assignment); - zkClusterState.backup_assignment(topologyName, assignmentBak); - - } catch (Exception e) { - LOG.warn("Failed to backup " + topologyId + " assignment " - + assignment, e); - } - } - - public void metricsMonitor(TopologyAssignEvent event) { - String topologyId = event.getTopologyId(); - try { - Map conf = nimbusData.getConf(); - boolean isEnable = ConfigExtension.isEnablePerformanceMetrics(conf); - StormClusterState zkClusterState = nimbusData - .getStormClusterState(); - StormMonitor monitor = new StormMonitor(isEnable); - zkClusterState.set_storm_monitor(topologyId, monitor); - } catch (Exception e) { - LOG.warn( - "Failed to update metrics monitor status of " + topologyId, - e); - } - } - - /** - * @param args - */ - public static void main(String[] args) { - // TODO Auto-generated method stub - - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/TopologyAssignEvent.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/TopologyAssignEvent.java deleted file mode 100644 index 9004505c9..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/TopologyAssignEvent.java +++ /dev/null @@ -1,91 +0,0 @@ -package com.alibaba.jstorm.daemon.nimbus; - -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; - -import com.alibaba.jstorm.cluster.StormStatus; - -public class TopologyAssignEvent { - - // unit is minutes - private static final int DEFAULT_WAIT_TIME = 2; - private String topologyId; - private String topologyName; // if this field has been set, it is create - private String group; - // topology - private boolean isScratch; - private StormStatus oldStatus; // if this field has been set, it is - // rebalance - private CountDownLatch latch = new CountDownLatch(1); - private boolean isSuccess = false; - private String errorMsg; - - public String getTopologyId() { - return topologyId; - } - - public void setTopologyId(String topologyId) { - this.topologyId = topologyId; - } - - public boolean isScratch() { - return isScratch; - } - - public void setScratch(boolean isScratch) { - this.isScratch = isScratch; - } - - public StormStatus getOldStatus() { - return oldStatus; - } - - public void setOldStatus(StormStatus oldStatus) { - this.oldStatus = oldStatus; - } - - public String getTopologyName() { - return topologyName; - } - - public void setTopologyName(String topologyName) { - this.topologyName = topologyName; - } - - public boolean waitFinish() { - try { - latch.await(DEFAULT_WAIT_TIME, TimeUnit.MINUTES); - } catch (InterruptedException e) { - - } - return isSuccess; - } - - public boolean isFinish() { - return latch.getCount() == 0; - } - - public void done() { - isSuccess = true; - latch.countDown(); - } - - public void fail(String errorMsg) { - isSuccess = false; - this.errorMsg = errorMsg; - latch.countDown(); - } - - public String getErrorMsg() { - return errorMsg; - } - - public String getGroup() { - return group; - } - - public void setGroup(String group) { - this.group = group; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/TransitionZkCallback.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/TransitionZkCallback.java deleted file mode 100644 index 6f83b5fed..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/nimbus/TransitionZkCallback.java +++ /dev/null @@ -1,24 +0,0 @@ -package com.alibaba.jstorm.daemon.nimbus; - -import com.alibaba.jstorm.callback.RunnableCallback; - -/** - * This is ZK watch callback When supervisor Zk dir has been changed, it will - * trigger this callback Set the status as monitor - * - */ -public class TransitionZkCallback extends RunnableCallback { - - private NimbusData data; - private String topologyid; - - public TransitionZkCallback(NimbusData data, String topologyid) { - this.data = data; - this.topologyid = topologyid; - } - - @Override - public void run() { - NimbusUtils.transition(data, topologyid, false, StatusType.monitor); - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/CgroupManager.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/CgroupManager.java deleted file mode 100644 index 2a8b12ce8..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/CgroupManager.java +++ /dev/null @@ -1,102 +0,0 @@ -package com.alibaba.jstorm.daemon.supervisor; - -import java.io.File; -import java.io.IOException; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -import org.apache.log4j.Logger; - -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.container.CgroupCenter; -import com.alibaba.jstorm.container.Hierarchy; -import com.alibaba.jstorm.container.SubSystemType; -import com.alibaba.jstorm.container.SystemOperation; -import com.alibaba.jstorm.container.cgroup.CgroupCommon; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.container.cgroup.core.CgroupCore; -import com.alibaba.jstorm.container.cgroup.core.CpuCore; - -public class CgroupManager { - - public static final Logger LOG = Logger.getLogger(CgroupManager.class); - - public static final String JSTORM_HIERARCHY_NAME = "jstorm_cpu"; - - public static final int ONE_CPU_SLOT = 1024; - - private CgroupCenter center; - - private Hierarchy h; - - private CgroupCommon rootCgroup; - - private static final String JSTORM_CPU_HIERARCHY_DIR = "/cgroup/cpu"; - private static String root_dir; - public CgroupManager(Map conf) { - LOG.info("running on cgroup mode"); - - // Cgconfig service is used to create the corresponding cpu hierarchy "/cgroup/cpu" - root_dir = ConfigExtension.getCgroupRootDir(conf); - if(root_dir == null) - throw new RuntimeException( - "Check configuration file. The supervisor.cgroup.rootdir is missing."); - - File file = new File(JSTORM_CPU_HIERARCHY_DIR + "/" + root_dir); - if(!file.exists()) { - LOG.error(JSTORM_CPU_HIERARCHY_DIR + "/" + root_dir + " is not existing."); - throw new RuntimeException( - "Check if cgconfig service starts or /etc/cgconfig.conf is consistent with configuration file."); - } - center = CgroupCenter.getInstance(); - if (center == null) - throw new RuntimeException( - "Cgroup error, please check /proc/cgroups"); - this.prepareSubSystem(); - } - - public String startNewWorker(int cpuNum, String workerId) - throws SecurityException, IOException { - CgroupCommon workerGroup = new CgroupCommon(workerId, h, - this.rootCgroup); - this.center.create(workerGroup); - CgroupCore cpu = workerGroup.getCores().get(SubSystemType.cpu); - CpuCore cpuCore = (CpuCore) cpu; - cpuCore.setCpuShares(cpuNum * ONE_CPU_SLOT); - StringBuilder sb = new StringBuilder(); - sb.append("cgexec -g cpu:").append(workerGroup.getName()).append(" "); - return sb.toString(); - } - - public void shutDownWorker(String workerId, boolean isKilled) { - CgroupCommon workerGroup = new CgroupCommon(workerId, h, - this.rootCgroup); - try { - if (isKilled == false) { - for (Integer pid : workerGroup.getTasks()) { - JStormUtils.kill(pid); - } - JStormUtils.sleepMs(1500); - } - center.delete(workerGroup); - }catch(Exception e) { - LOG.info("No task of " + workerId); - } - - } - - public void close() throws IOException { - this.center.delete(this.rootCgroup); - } - - private void prepareSubSystem() { - h = center.busy(SubSystemType.cpu); - if (h == null) { - Set types = new HashSet(); - types.add(SubSystemType.cpu); - h = new Hierarchy(JSTORM_HIERARCHY_NAME, types,JSTORM_CPU_HIERARCHY_DIR); - } - rootCgroup = new CgroupCommon(root_dir, h, h.getRootCgroups()); - } -} 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 deleted file mode 100644 index 8276fd30b..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/Heartbeat.java +++ /dev/null @@ -1,127 +0,0 @@ -package com.alibaba.jstorm.daemon.supervisor; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.StormClusterState; -import com.alibaba.jstorm.cluster.StormConfig; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.NetWorkUtils; -import com.alibaba.jstorm.utils.TimeUtils; -import com.alibaba.jstorm.utils.JStormServerUtils; - -/** - * supervisor Heartbeat, just write SupervisorInfo to ZK - */ -class Heartbeat extends RunnableCallback { - - private static Logger LOG = Logger.getLogger(Heartbeat.class); - - private static final int CPU_THREADHOLD = 4; - private static final long MEM_THREADHOLD = 8 * JStormUtils.SIZE_1_G; - - private Map conf; - - private StormClusterState stormClusterState; - - private String supervisorId; - - private String myHostName; - - private final int startTime; - - private final int frequence; - - private SupervisorInfo supervisorInfo; - - /** - * @param conf - * @param stormClusterState - * @param supervisorId - * @param myHostName - */ - @SuppressWarnings({ "rawtypes", "unchecked" }) - public Heartbeat(Map conf, StormClusterState stormClusterState, - String supervisorId, AtomicBoolean active) { - - String myHostName = JStormServerUtils.getHostName(conf); - - this.stormClusterState = stormClusterState; - this.supervisorId = supervisorId; - this.conf = conf; - this.myHostName = myHostName; - this.startTime = TimeUtils.current_time_secs(); - this.active = active; - this.frequence = JStormUtils.parseInt(conf - .get(Config.SUPERVISOR_HEARTBEAT_FREQUENCY_SECS)); - - initSupervisorInfo(conf); - - LOG.info("Successfully init supervisor heartbeat thread, " + supervisorInfo); - } - - private void initSupervisorInfo(Map conf) { - List portList = JStormUtils.getSupervisorPortList(conf); - - if (!StormConfig.local_mode(conf)) { - Set ports = JStormUtils.listToSet(portList); - supervisorInfo = new SupervisorInfo(myHostName, supervisorId, ports); - } else { - Set ports = JStormUtils.listToSet(portList.subList(0, 1)); - supervisorInfo = new SupervisorInfo(myHostName, supervisorId, ports); - } - } - - @SuppressWarnings("unchecked") - public void update() { - - supervisorInfo.setTimeSecs(TimeUtils.current_time_secs()); - supervisorInfo - .setUptimeSecs((int) (TimeUtils.current_time_secs() - startTime)); - - try { - stormClusterState - .supervisor_heartbeat(supervisorId, supervisorInfo); - } catch (Exception e) { - LOG.error("Failed to update SupervisorInfo to ZK", e); - - } - } - - private AtomicBoolean active = null; - - private Integer result; - - @Override - public Object getResult() { - return result; - } - - @Override - public void run() { - update(); - if (active.get()) { - result = frequence; - } else { - result = -1; - } - } - - public int getStartTime() { - return startTime; - } - - public SupervisorInfo getSupervisorInfo() { - return supervisorInfo; - } - -} 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 deleted file mode 100644 index 8c60b7336..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/Httpserver.java +++ /dev/null @@ -1,460 +0,0 @@ -package com.alibaba.jstorm.daemon.supervisor; - -import java.io.BufferedReader; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.io.OutputStream; -import java.io.RandomAccessFile; -import java.net.BindException; -import java.net.HttpURLConnection; -import java.net.InetSocketAddress; -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; -import java.util.Map; -import java.util.concurrent.Executor; -import java.util.concurrent.Executors; - -import org.apache.commons.io.IOUtils; -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; -import com.alibaba.jstorm.utils.FileAttribute; -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; -import com.sun.net.httpserver.HttpExchange; -import com.sun.net.httpserver.HttpHandler; -import com.sun.net.httpserver.HttpServer; - -public class Httpserver implements Shutdownable { - - private static Logger LOG = Logger.getLogger(Httpserver.class); - - private HttpServer hs; - private int port; - private Map conf; - - public Httpserver(int port, Map conf) { - this.port = port; - this.conf = 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); // +++ - - } - - public void handlFailure(HttpExchange t, String errorMsg) - throws IOException { - LOG.error(errorMsg); - - byte[] data = errorMsg.getBytes(); - t.sendResponseHeaders(HttpURLConnection.HTTP_BAD_REQUEST, - data.length); - OutputStream os = t.getResponseBody(); - os.write(data); - os.close(); - } - - 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); - if (StringUtils.isBlank(cmd) == true) { - handlFailure(t, "Bad Request, Not set command type"); - return; - } - - if (HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD_SHOW.equals(cmd)) { - handleShowLog(t, paramMap); - return; - } else if (HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD_LIST - .equals(cmd)) { - handleListDir(t, paramMap); - return; - }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(); - - for (String param : StringUtils.split(uriRawQuery, "&")) { - String[] pair = StringUtils.split(param, "="); - if (pair.length == 2) { - paramMap.put(pair[0], pair[1]); - } - } - - return paramMap; - } - - private void handleShowLog(HttpExchange t, Map paramMap) - throws IOException { - Pair logPair = queryLog(t, paramMap); - if (logPair == null) { - return; - } - - String size = String.format( - HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_SIZE_FORMAT, - logPair.getFirst()); - byte[] sizeByts = size.getBytes(); - - byte[] logData = logPair.getSecond(); - - t.sendResponseHeaders(HttpURLConnection.HTTP_OK, sizeByts.length - + logData.length); - OutputStream os = t.getResponseBody(); - os.write(sizeByts); - os.write(logData); - os.close(); - } - - private Pair queryLog(HttpExchange t, - Map paramMap) throws IOException { - - String fileParam = paramMap - .get(HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_LOGFILE); - if (StringUtils.isBlank(fileParam)) { - handlFailure(t, "Bad Request, Params Error, no log file name."); - return null; - } - - String logFile = Joiner.on(File.separator).join(logDir, fileParam); - accessCheck(logFile); - FileChannel fc = null; - MappedByteBuffer fout = null; - long fileSize = 0; - byte[] ret = "Failed to get data".getBytes(); - try { - fc = new RandomAccessFile(logFile, "r").getChannel(); - - fileSize = fc.size(); - - long position = fileSize - - HttpserverUtils.HTTPSERVER_LOGVIEW_PAGESIZE; - try { - String posStr = paramMap - .get(HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_POS); - if (StringUtils.isBlank(posStr) == false) { - long pos = Long.valueOf(posStr); - - position = pos; - } - } catch (Exception e) { - LOG.warn("Invalide position " + position); - } - if (position < 0) { - position = 0L; - } - - long size = Math.min(fileSize - position, - HttpserverUtils.HTTPSERVER_LOGVIEW_PAGESIZE); - - LOG.info("logview " + logFile + ", position=" + position - + ", size=" + size); - fout = fc.map(FileChannel.MapMode.READ_ONLY, position, size); - - ret = new byte[(int) size]; - fout.get(ret); - String str = new String(ret, ConfigExtension.getLogViewEncoding(conf)); - return new Pair(fileSize, str.getBytes()); - - } catch (FileNotFoundException e) { - LOG.warn(e); - handlFailure(t, "Bad Request, Failed to find " + fileParam); - return null; - - } catch (IOException e) { - LOG.warn(e); - handlFailure(t, "Bad Request, Failed to open " + fileParam); - return null; - } finally { - fout = null; - if (fc != null) { - IOUtils.closeQuietly(fc); - } - } - - } - - byte[] getJSonFiles(String dir) throws Exception { - Map fileMap = new HashMap(); - - - String path = logDir; - if (dir != null) { - path = path + File.separator + dir; - } - accessCheck(path); - - LOG.info("List dir " + path); - - File file = new File(path); - - String[] files = file.list(); - - for (String fileName : files) { - String logFile = Joiner.on(File.separator).join(path, - fileName); - - FileAttribute fileAttribute = new FileAttribute(); - fileAttribute.setFileName(fileName); - - File subFile = new File(logFile); - - Date modify = new Date(subFile.lastModified()); - fileAttribute.setModifyTime(TimeFormat.getSecond(modify)); - - if (subFile.isFile()) { - fileAttribute.setIsDir(String.valueOf(false)); - fileAttribute.setSize(String.valueOf(subFile.length())); - - - fileMap.put(logFile, fileAttribute); - } else if (subFile.isDirectory()) { - fileAttribute.setIsDir(String.valueOf(true)); - fileAttribute.setSize(String.valueOf(4096)); - - fileMap.put(logFile, fileAttribute); - } - - - } - - String fileJsonStr = JStormUtils.to_json(fileMap); - return fileJsonStr.getBytes(); - } - - void handleListDir(HttpExchange t, Map paramMap) - throws IOException { - byte[] filesJson = "Failed to get file list".getBytes(); - - try { - 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; - } - - t.sendResponseHeaders(HttpURLConnection.HTTP_OK, filesJson.length); - OutputStream os = t.getResponseBody(); - os.write(filesJson); - os.close(); - } - - void handleJstack(StringBuffer sb, Integer pid) { - - String cmd = "jstack " + pid; - - try { - LOG.info("Begin to execute " + cmd); - Process process = JStormUtils.launch_process(cmd, - new HashMap(), false); - - // Process process = Runtime.getRuntime().exec(sb.toString()); - - InputStream stdin = process.getInputStream(); - BufferedReader reader = new BufferedReader(new InputStreamReader( - stdin)); - - JStormUtils.sleepMs(1000); - - // if (process.exitValue() != 0) { - // LOG.info("Failed to execute " + sb.toString()); - // return null; - // } - - String str; - while ((str = reader.readLine()) != null) { - if (StringUtils.isBlank(str)) { - // LOG.info(str + " is Blank"); - continue; - } - - sb.append(str).append("\r\n"); - } - - LOG.info("Successfully get output of " + cmd); - return ; - } catch (IOException e) { - LOG.info("Failed to execute " + cmd, e); - sb.append("Failed to execute " + cmd); - return ; - } catch (Exception e) { - LOG.info(e.getCause(), e); - sb.append("Failed to execute " + cmd + ", " + e.getCause()); - return ; - } - } - - void handleJstack(HttpExchange t, Map paramMap) - throws IOException { - String workerPort = paramMap.get( - HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_WORKER_PORT); - if (workerPort == null) { - handlFailure(t, "Not set worker's port"); - return ; - } - - LOG.info("Begin to get jstack of " + workerPort); - StringBuffer sb = new StringBuffer(); - List pids = Worker.getOldPortPids(workerPort); - for (Integer pid : pids) { - sb.append("!!!!!!!!!!!!!!!!!!\r\n"); - sb.append("WorkerPort:" + workerPort + ", pid:" + pid); - sb.append("\r\n!!!!!!!!!!!!!!!!!!\r\n"); - - handleJstack(sb, pid); - } - - byte[] data = sb.toString().getBytes(); - t.sendResponseHeaders(HttpURLConnection.HTTP_OK, data.length); - OutputStream os = t.getResponseBody(); - 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() { - int numHandler = 3; - InetSocketAddress socketAddr = new InetSocketAddress(port); - Executor executor = Executors.newFixedThreadPool(numHandler); - - try { - hs = HttpServer.create(socketAddr, 0); - hs.createContext(HttpserverUtils.HTTPSERVER_CONTEXT_PATH_LOGVIEW, - new LogHandler(conf)); - hs.setExecutor(executor); - hs.start(); - - } catch (BindException e) { - LOG.info("Httpserver Already start!"); - hs = null; - return; - } catch (IOException e) { - LOG.error("Httpserver Start Failed", e); - hs = null; - return; - } - LOG.info("Success start httpserver at port:" + port); - - } - - @Override - public void shutdown() { - if (hs != null) { - hs.stop(0); - LOG.info("Successfully stop http server"); - } - - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SandBoxMaker.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SandBoxMaker.java deleted file mode 100644 index 60be31712..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SandBoxMaker.java +++ /dev/null @@ -1,195 +0,0 @@ -package com.alibaba.jstorm.daemon.supervisor; - -import java.io.BufferedReader; -import java.io.BufferedWriter; -import java.io.File; -import java.io.FileWriter; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.io.LineNumberReader; -import java.io.PrintWriter; -import java.util.HashMap; -import java.util.Map; -import java.util.Map.Entry; -import java.util.UUID; - -import org.apache.commons.lang.StringUtils; -import org.apache.log4j.Logger; - -import backtype.storm.Config; -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.StormConfig; - -/** - * Right now generate java sandbox policy through template file - * - * In the future, generating java sandbox policy will through hardcode - * - * @author longda - * @version - */ -public class SandBoxMaker { - private static final Logger LOG = Logger.getLogger(SandBoxMaker.class); - - public static final String SANBOX_TEMPLATE_NAME = "sandbox.policy"; - - public static final String JSTORM_HOME_KEY = "%JSTORM_HOME%"; - - public static final String CLASS_PATH_KEY = "%CLASS_PATH%"; - - public static final String LOCAL_DIR_KEY = "%JSTORM_LOCAL_DIR%"; - - // this conf should only be Supervisor Conf - private final Map conf; - - private final boolean isEnable; - - - private final Map replaceBaseMap = new HashMap(); - - public SandBoxMaker(Map conf) { - this.conf = conf; - - isEnable = ConfigExtension.isJavaSandBoxEnable(conf); - - LOG.info("Java Sandbox Policy :" + String.valueOf(isEnable)); - - String jstormHome = System.getProperty("jstorm.home"); - if (jstormHome == null) { - jstormHome = "./"; - } - - replaceBaseMap.put(JSTORM_HOME_KEY, jstormHome); - - replaceBaseMap.put(LOCAL_DIR_KEY, (String)conf.get(Config.STORM_LOCAL_DIR)); - - - LOG.info("JSTORM_HOME is " + jstormHome); - } - - private String genClassPath(String classPathLine) { - StringBuilder sb = new StringBuilder(); - - String[] classPathes = classPathLine.split(":"); - for (String classpath: classPathes) { - if (StringUtils.isBlank(classpath)) { - continue; - } - - File file = new File(classpath); - if (file.isDirectory()) { - sb.append(" permission java.io.FilePermission \""); - sb.append(classpath).append(File.separator).append("**"); - sb.append("\", \"read\";\n"); - }else { - sb.append(" permission java.io.FilePermission \""); - sb.append(classpath); - sb.append("\", \"read\";\n"); - } - - } - - return sb.toString(); - } - - private String replaceLine(String line, Map replaceMap) { - - for (Entryentry : replaceMap.entrySet()) { - if (line.contains(CLASS_PATH_KEY)) { - return genClassPath(entry.getValue()); - }else if (line.contains(entry.getKey())) { - return line.replace(entry.getKey(), entry.getValue()); - } - } - - return line; - } - - public String generatePolicyFile(Map replaceMap) throws IOException { - // dynamic generate policy file, no static file - String tmpPolicy = StormConfig.supervisorTmpDir(conf) + File.separator - + UUID.randomUUID().toString(); - - InputStream inputStream = SandBoxMaker.class.getClassLoader() - .getResourceAsStream(SANBOX_TEMPLATE_NAME); - - PrintWriter writer = new PrintWriter(new BufferedWriter(new FileWriter( - tmpPolicy))); - - try { - - InputStreamReader inputReader = new InputStreamReader(inputStream); - - BufferedReader reader = new BufferedReader(new LineNumberReader( - inputReader)); - - String line = null; - while ((line = reader.readLine()) != null) { - String replaced = replaceLine(line, replaceMap); - - writer.println(replaced); - } - - - return tmpPolicy; - } catch (Exception e) { - LOG.error("Failed to generate policy file\n", e); - throw new IOException(e); - } finally { - if (inputStream != null) { - inputStream.close(); - } - if (writer != null) { - writer.close(); - } - - } - } - - /** - * Generate command string - * - * @param workerId - * @return - * @throws IOException - */ - public String sandboxPolicy(String workerId, Map replaceMap) throws IOException { - if (isEnable == false) { - return ""; - } - - replaceMap.putAll(replaceBaseMap); - - String tmpPolicy = generatePolicyFile(replaceMap); - - File file = new File(tmpPolicy); - String policyPath = StormConfig.worker_root(conf, workerId) + File.separator + SANBOX_TEMPLATE_NAME; - File dest = new File(policyPath); - file.renameTo(dest); - - StringBuilder sb = new StringBuilder(); - sb.append(" -Djava.security.manager -Djava.security.policy="); - sb.append(policyPath); - - return sb.toString(); - - } - - public static void main(String[] args) { - Map conf = Utils.readStormConfig(); - - conf.put("java.sandbox.enable", Boolean.valueOf(true)); - - SandBoxMaker maker = new SandBoxMaker(conf); - - try { - System.out.println("sandboxPolicy:" + maker.sandboxPolicy("simple", new HashMap())); - } catch (IOException e) { - e.printStackTrace(); - } - } - -} 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 deleted file mode 100644 index 2bd28f05f..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/ShutdownWork.java +++ /dev/null @@ -1,140 +0,0 @@ -package com.alibaba.jstorm.daemon.supervisor; - -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.concurrent.ConcurrentHashMap; - -import org.apache.log4j.Logger; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.StormConfig; -import com.alibaba.jstorm.daemon.worker.ProcessSimulator; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.PathUtils; - -public class ShutdownWork extends RunnableCallback { - - private static Logger LOG = Logger.getLogger(ShutdownWork.class); - - /** - * shutdown all workers - * - * @param conf - * @param supervisorId - * @param removed - * @param workerThreadPids - * @param cgroupManager - */ - public void shutWorker(Map conf, String supervisorId, - Map removed, - ConcurrentHashMap workerThreadPids, - CgroupManager cgroupManager) { - - Map> workerId2Pids = new HashMap>(); - - boolean localMode = false; - - int maxWaitTime = 0; - - for (Entry entry : removed.entrySet()) { - String workerId = entry.getKey(); - String topologyId = entry.getValue(); - - LOG.info("Begin to shut down " + topologyId + ":" + workerId); - try { - - List pids = getPid(conf, workerId); - workerId2Pids.put(workerId, pids); - - String threadPid = workerThreadPids.get(workerId); - - // local mode - if (threadPid != null) { - ProcessSimulator.killProcess(threadPid); - localMode = true; - continue; - } - - for (String pid : pids) { - JStormUtils.process_killed(Integer.parseInt(pid)); - } - - maxWaitTime = ConfigExtension - .getTaskCleanupTimeoutSec(conf); - // The best design is get getTaskCleanupTimeoutSec from - // topology configuration, but topology configuration is likely - // to be deleted before kill worker, so in order to simplify - // the logical, just get task.cleanup.timeout.sec from - // supervisor configuration - - } catch (Exception e) { - LOG.info("Failed to shutdown ", e); - } - - } - - JStormUtils.sleepMs(maxWaitTime * 1000); - - for (Entry> entry : workerId2Pids.entrySet()) { - String workerId = entry.getKey(); - List pids = entry.getValue(); - - if (localMode == false) { - for (String pid : pids) { - - JStormUtils.ensure_process_killed(Integer.parseInt(pid)); - if (cgroupManager != null) { - cgroupManager.shutDownWorker(workerId, true); - } - } - } - - tryCleanupWorkerDir(conf, workerId); - LOG.info("Successfully shut down " + workerId); - } - } - - /** - * clean the directory , subdirectories of STORM-LOCAL-DIR/workers/workerId - * - * - * @param conf - * @param workerId - * @throws IOException - */ - public void tryCleanupWorkerDir(Map conf, String workerId) { - try { - // delete heartbeat dir LOCAL_DIR/workers/workid/heartbeats - PathUtils.rmr(StormConfig.worker_heartbeats_root(conf, workerId)); - // delete pid dir, LOCAL_DIR/workers/workerid/pids - PathUtils.rmr(StormConfig.worker_pids_root(conf, workerId)); - // delete workerid dir, LOCAL_DIR/worker/workerid - PathUtils.rmr(StormConfig.worker_root(conf, workerId)); - } catch (Exception e) { - LOG.warn(e + "Failed to cleanup worker " + 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/StateHeartbeat.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/StateHeartbeat.java deleted file mode 100644 index a65b4e8f4..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/StateHeartbeat.java +++ /dev/null @@ -1,36 +0,0 @@ -package com.alibaba.jstorm.daemon.supervisor; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; - -import com.alibaba.jstorm.daemon.worker.State; -import com.alibaba.jstorm.daemon.worker.WorkerHeartbeat; - -/** - * Worker's state and Hearbeat - * - * @author Xin.Zhou - */ -public class StateHeartbeat { - private State state; - private WorkerHeartbeat hb; - - public StateHeartbeat(State state, WorkerHeartbeat hb) { - this.state = state; - this.hb = hb; - } - - public State getState() { - return this.state; - } - - public WorkerHeartbeat getHeartbeat() { - return this.hb; - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this, - ToStringStyle.SHORT_PREFIX_STYLE); - } -} 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 deleted file mode 100644 index fddcc842b..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/Supervisor.java +++ /dev/null @@ -1,249 +0,0 @@ -package com.alibaba.jstorm.daemon.supervisor; - -import java.io.File; -import java.util.Map; -import java.util.UUID; -import java.util.Vector; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.commons.io.FileUtils; -import org.apache.log4j.Logger; - -import backtype.storm.Config; -import backtype.storm.messaging.IContext; -import backtype.storm.utils.LocalState; -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.callback.AsyncLoopThread; -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.Cluster; -import com.alibaba.jstorm.cluster.Common; -import com.alibaba.jstorm.cluster.StormClusterState; -import com.alibaba.jstorm.cluster.StormConfig; -import com.alibaba.jstorm.daemon.worker.hearbeat.SyncContainerHb; -import com.alibaba.jstorm.daemon.worker.metrics.AlimonitorClient; -import com.alibaba.jstorm.daemon.worker.metrics.MetricSendClient; -import com.alibaba.jstorm.daemon.worker.metrics.UploadSupervMetric; -import com.alibaba.jstorm.event.EventManager; -import com.alibaba.jstorm.event.EventManagerImp; -import com.alibaba.jstorm.event.EventManagerPusher; -import com.alibaba.jstorm.utils.JStormServerUtils; -import com.alibaba.jstorm.utils.JStormUtils; - -/** - * - * - * Supevisor workflow 1. write SupervisorInfo to ZK - * - * 2. Every 10 seconds run SynchronizeSupervisor 2.1 download new topology 2.2 - * release useless worker 2.3 assgin new task to - * /local-dir/supervisor/localstate 2.4 add one syncProcesses event - * - * 3. Every supervisor.monitor.frequency.secs run SyncProcesses 3.1 kill useless - * worker 3.2 start new worker - * - * 4. create heartbeat thread every supervisor.heartbeat.frequency.secs, write - * SupervisorInfo to ZK - */ - -public class Supervisor { - - private static Logger LOG = Logger.getLogger(Supervisor.class); - - AtomicBoolean active; - - /** - * create and start one supervisor - * - * @param conf - * : configurationdefault.yaml storm.yaml - * @param sharedContext - * : null (right now) - * @return SupervisorManger: which is used to shutdown all workers and - * supervisor - */ - @SuppressWarnings("rawtypes") - public SupervisorManger mkSupervisor(Map conf, IContext sharedContext) - throws Exception { - - LOG.info("Starting Supervisor with conf " + conf); - - active = new AtomicBoolean(true); - - /** - * Step 1: cleanup all files in /storm-local-dir/supervisor/tmp - */ - String path = StormConfig.supervisorTmpDir(conf); - FileUtils.cleanDirectory(new File(path)); - - /* - * Step 2: create ZK operation instance StromClusterState - */ - - StormClusterState stormClusterState = Cluster - .mk_storm_cluster_state(conf); - - /* - * Step 3, create LocalStat LocalStat is one KV database 4.1 create - * LocalState instance 4.2 get supervisorId, if no supervisorId, create - * one - */ - - LocalState localState = StormConfig.supervisorState(conf); - - String supervisorId = (String) localState.get(Common.LS_ID); - if (supervisorId == null) { - supervisorId = UUID.randomUUID().toString(); - localState.put(Common.LS_ID, supervisorId); - } - - Vector threads = new Vector(); - - // Step 5 create HeartBeat - // every supervisor.heartbeat.frequency.secs, write SupervisorInfo to ZK - // sync hearbeat to nimbus - Heartbeat hb = new Heartbeat(conf, stormClusterState, supervisorId, - active); - hb.update(); - AsyncLoopThread heartbeat = new AsyncLoopThread(hb, false, null, - Thread.MIN_PRIORITY, true); - threads.add(heartbeat); - - // Sync heartbeat to Apsara Container - AsyncLoopThread syncContainerHbThread = SyncContainerHb.mkSupervisorInstance(conf); - if (syncContainerHbThread != null) { - threads.add(syncContainerHbThread); - } - - // Step 6 create and start sync Supervisor thread - // every supervisor.monitor.frequency.secs second run SyncSupervisor - EventManager processEventManager = new EventManagerImp(false); - ConcurrentHashMap workerThreadPids = new ConcurrentHashMap(); - SyncProcessEvent syncProcessEvent = new SyncProcessEvent(supervisorId, - conf, localState, workerThreadPids, sharedContext); - - EventManager syncSupEventManager = new EventManagerImp(false); - SyncSupervisorEvent syncSupervisorEvent = new SyncSupervisorEvent( - supervisorId, conf, processEventManager, syncSupEventManager, - stormClusterState, localState, syncProcessEvent); - - int syncFrequence = JStormUtils.parseInt(conf - .get(Config.SUPERVISOR_MONITOR_FREQUENCY_SECS)); - EventManagerPusher syncSupervisorPusher = new EventManagerPusher( - syncSupEventManager, syncSupervisorEvent, active, syncFrequence); - AsyncLoopThread syncSupervisorThread = new AsyncLoopThread( - syncSupervisorPusher); - threads.add(syncSupervisorThread); - - // Step 7 start sync process thread - // every supervisor.monitor.frequency.secs run SyncProcesses - // skip thread to do syncProcess, due to nimbus will check whether - // worker is dead or not, if dead, it will reassign a new worker - // - // int syncProcessFrequence = syncFrequence/2; - // EventManagerPusher syncProcessPusher = new EventManagerPusher( - // processEventManager, syncProcessEvent, active, - // syncProcessFrequence); - // AsyncLoopThread syncProcessThread = new - // AsyncLoopThread(syncProcessPusher); - // threads.add(syncProcessThread); - - - 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); - } - - - // SupervisorManger which can shutdown all supervisor and workers - return new SupervisorManger(conf, supervisorId, active, threads, - syncSupEventManager, processEventManager, httpserver, - stormClusterState, workerThreadPids); - } - - - /** - * shutdown - * - * @param supervisor - */ - 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); - - JStormServerUtils.createPid(pidDir); - } - - /** - * start supervisor - */ - public void run() { - - SupervisorManger supervisorManager = null; - try { - Map conf = Utils.readStormConfig(); - - StormConfig.validate_distributed_mode(conf); - - createPid(conf); - - supervisorManager = mkSupervisor(conf, null); - - JStormUtils.redirectOutput("/dev/null"); - - initShutdownHook(supervisorManager); - - } catch (Exception e) { - LOG.error("Failed to start supervisor\n", e); - System.exit(1); - } - - while (supervisorManager.isFinishShutdown() == false) { - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - - } - } - } - - /** - * supervisor daemon enter entrance - * - * @param args - */ - public static void main(String[] args) { - - JStormServerUtils.startTaobaoJvmMonitor(); - - Supervisor instance = new Supervisor(); - - instance.run(); - - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SupervisorDaemon.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SupervisorDaemon.java deleted file mode 100644 index fc431be19..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SupervisorDaemon.java +++ /dev/null @@ -1,13 +0,0 @@ -package com.alibaba.jstorm.daemon.supervisor; - -import java.util.Map; - -public interface SupervisorDaemon { - - public String getId(); - - public Map getConf(); - - public void ShutdownAllWorkers(); - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SupervisorInfo.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SupervisorInfo.java deleted file mode 100644 index 0363a2185..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SupervisorInfo.java +++ /dev/null @@ -1,153 +0,0 @@ -package com.alibaba.jstorm.daemon.supervisor; - -import java.io.Serializable; -import java.util.HashMap; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; - -/** - * Object stored in ZK /ZK-DIR/supervisors - * - * @author Xin.Zhou/Longda - */ -public class SupervisorInfo implements Serializable { - - private static final long serialVersionUID = -8384417078907518922L; - - private final String hostName; - private final String supervisorId; - - private Integer timeSecs; - private Integer uptimeSecs; - - private Set workerPorts; - - public SupervisorInfo(String hostName, String supervisorId, - Set workerPorts) { - this.hostName = hostName; - this.supervisorId = supervisorId; - this.workerPorts = workerPorts; - } - - public String getHostName() { - return hostName; - } - - public String getSupervisorId() { - return supervisorId; - } - - public int getTimeSecs() { - return timeSecs; - } - - public void setTimeSecs(int timeSecs) { - this.timeSecs = timeSecs; - } - - public int getUptimeSecs() { - return uptimeSecs; - } - - public void setUptimeSecs(int uptimeSecs) { - this.uptimeSecs = uptimeSecs; - } - - public Set getWorkerPorts() { - return workerPorts; - } - - public void setWorkerPorts(Set workerPorts) { - this.workerPorts = workerPorts; - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result - + ((hostName == null) ? 0 : hostName.hashCode()); - result = prime * result - + ((supervisorId == null) ? 0 : supervisorId.hashCode()); - result = prime * result - + ((timeSecs == null) ? 0 : timeSecs.hashCode()); - result = prime * result - + ((uptimeSecs == null) ? 0 : uptimeSecs.hashCode()); - result = prime * result - + ((workerPorts == null) ? 0 : workerPorts.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - SupervisorInfo other = (SupervisorInfo) obj; - if (hostName == null) { - if (other.hostName != null) - return false; - } else if (!hostName.equals(other.hostName)) - return false; - if (supervisorId == null) { - if (other.supervisorId != null) - return false; - } else if (!supervisorId.equals(other.supervisorId)) - return false; - if (timeSecs == null) { - if (other.timeSecs != null) - return false; - } else if (!timeSecs.equals(other.timeSecs)) - return false; - if (uptimeSecs == null) { - if (other.uptimeSecs != null) - return false; - } else if (!uptimeSecs.equals(other.uptimeSecs)) - return false; - if (workerPorts == null) { - if (other.workerPorts != null) - return false; - } else if (!workerPorts.equals(other.workerPorts)) - return false; - return true; - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this, - ToStringStyle.SHORT_PREFIX_STYLE); - } - - /** - * get Map - * - * @param stormClusterState - * @param callback - * @return - */ - public static Map getNodeHost( - Map supInfos) { - - Map rtn = new HashMap(); - - for (Entry entry : supInfos.entrySet()) { - - SupervisorInfo superinfo = entry.getValue(); - - String supervisorid = entry.getKey(); - - rtn.put(supervisorid, superinfo.getHostName()); - - } - - return rtn; - } - -} 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 deleted file mode 100644 index dad799668..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SupervisorManger.java +++ /dev/null @@ -1,177 +0,0 @@ -package com.alibaba.jstorm.daemon.supervisor; - -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Vector; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.log4j.Logger; - -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; - -/** - * supervisor shutdown manager which can shutdown supervisor - */ -public class SupervisorManger extends ShutdownWork implements Shutdownable, - SupervisorDaemon, DaemonCommon, Runnable { - - private static Logger LOG = Logger.getLogger(SupervisorManger.class); - - // private Supervisor supervisor; - - private Map conf; - - private String supervisorId; - - private AtomicBoolean active; - - private Vector threads; - - private EventManager processesEventManager; - - private EventManager eventManager; - - private Httpserver httpserver; - - private StormClusterState stormClusterState; - - private ConcurrentHashMap workerThreadPidsAtom; - - private volatile boolean isFinishShutdown = false; - - public SupervisorManger(Map conf, String supervisorId, - AtomicBoolean active, Vector threads, - EventManager processesEventManager, EventManager eventManager, - Httpserver httpserver, StormClusterState stormClusterState, - ConcurrentHashMap workerThreadPidsAtom) { - this.conf = conf; - this.supervisorId = supervisorId; - this.active = active; - this.threads = threads; - this.processesEventManager = processesEventManager; - this.eventManager = eventManager; - this.httpserver = httpserver; - this.stormClusterState = stormClusterState; - this.workerThreadPidsAtom = workerThreadPidsAtom; - - Runtime.getRuntime().addShutdownHook(new Thread(this)); - } - - @Override - public void shutdown() { - if (active.getAndSet(false) == false) { - LOG.info("Supervisor has been shutdown before " + supervisorId); - return ; - } - LOG.info("Shutting down supervisor " + supervisorId); - - int size = threads.size(); - for (AsyncLoopThread thread : threads) { - thread.cleanup(); - JStormUtils.sleepMs(10); - thread.interrupt(); -// try { -// thread.join(); -// } catch (InterruptedException e) { -// LOG.error(e.getMessage(), e); -// } - LOG.info("Successfully shutdown thread:" + thread.getThread().getName()); - } - eventManager.shutdown(); - processesEventManager.shutdown(); - try { - stormClusterState.disconnect(); - } catch (Exception e) { - // TODO Auto-generated catch block - LOG.error("Failed to shutdown ZK client", e); - } - if (httpserver != null) { - httpserver.shutdown(); - } - - - // if (this.cgroupManager != null) - // try { - // this.cgroupManager.close(); - // } catch (IOException e) { - // // TODO Auto-generated catch block - // LOG.error("Fail to close cgroup", e); - // } - - isFinishShutdown = true; - - JStormUtils.halt_process(0, "!!!Shutdown!!!"); - } - - @Override - public void ShutdownAllWorkers() { - LOG.info("Begin to shutdown all workers"); - String path; - try { - path = StormConfig.worker_root(conf); - } catch (IOException e1) { - // TODO Auto-generated catch block - LOG.error("Failed to get Local worker dir", e1); - return; - } - List myWorkerIds = PathUtils.read_dir_contents(path); - HashMap workerId2topologyIds = new HashMap(); - - for (String workerId : myWorkerIds) { - workerId2topologyIds.put(workerId, null); - } - - shutWorker(conf, supervisorId, workerId2topologyIds, workerThreadPidsAtom, null); - } - - @Override - public Map getConf() { - return conf; - } - - @Override - public String getId() { - return supervisorId; - } - - @Override - public boolean waiting() { - if (!active.get()) { - return true; - } - - Boolean bThread = true; - int size = threads.size(); - for (int i = 0; i < size; i++) { - if (!(Boolean) threads.elementAt(i).isSleeping()) { - bThread = false; - return false; - } - } - boolean bManagers = true; - if (eventManager.waiting() && processesEventManager.waiting()) { - bManagers = false; - return false; - } - return true; - } - - public void run() { - shutdown(); - } - - public boolean isFinishShutdown() { - return isFinishShutdown; - } -} 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 deleted file mode 100644 index b539c12ac..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SyncProcessEvent.java +++ /dev/null @@ -1,882 +0,0 @@ -package com.alibaba.jstorm.daemon.supervisor; - -import java.io.File; -import java.io.IOException; -import java.util.Collection; -import java.util.Date; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -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 com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.Common; -import com.alibaba.jstorm.cluster.StormConfig; -import com.alibaba.jstorm.daemon.worker.ProcessSimulator; -import com.alibaba.jstorm.daemon.worker.State; -import com.alibaba.jstorm.daemon.worker.Worker; -import com.alibaba.jstorm.daemon.worker.WorkerHeartbeat; -import com.alibaba.jstorm.daemon.worker.WorkerShutdown; -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; - -/** - * SyncProcesses (1) kill bad worker (2) start new worker - */ -class SyncProcessEvent extends ShutdownWork { - private static Logger LOG = Logger.getLogger(SyncProcessEvent.class); - - private LocalState localState; - - private Map conf; - - private ConcurrentHashMap workerThreadPids; - - private String supervisorId; - - private IContext sharedContext; - - private CgroupManager cgroupManager; - - 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; - - /** - * @param conf - * @param localState - * @param workerThreadPids - * @param supervisorId - * @param sharedContext - * @param workerThreadPidsReadLock - * @param workerThreadPidsWriteLock - */ - public SyncProcessEvent(String supervisorId, Map conf, - LocalState localState, - ConcurrentHashMap workerThreadPids, - IContext sharedContext) { - - this.supervisorId = supervisorId; - - this.conf = conf; - - this.localState = localState; - - this.workerThreadPids = workerThreadPids; - - // right now, sharedContext is null - this.sharedContext = sharedContext; - - this.sandBoxMaker = new SandBoxMaker(conf); - - this.workerIdToStartTimeAndPort = new HashMap>(); - - if (ConfigExtension.isEnableCgroup(conf)) { - cgroupManager = new CgroupManager(conf); - } - } - - /** - * @@@ Change the old logic In the old logic, it will store - * LS_LOCAL_ASSIGNMENTS Map into LocalState - * - * But I don't think LS_LOCAL_ASSIGNMENTS is useful, so remove this - * logic - */ - @SuppressWarnings("unchecked") - @Override - public void run() { - LOG.debug("Syncing processes"); - try { - - /** - * Step 1: get assigned tasks from localstat Map - */ - Map localAssignments = null; - try { - localAssignments = (Map) localState - .get(Common.LS_LOCAL_ASSIGNMENTS); - } catch (IOException e) { - LOG.error("Failed to get LOCAL_ASSIGNMENTS from LocalState", e); - throw e; - } - - if (localAssignments == null) { - localAssignments = new HashMap(); - } - LOG.debug("Assigned tasks: " + localAssignments); - - /** - * Step 2: get local WorkerStats from local_dir/worker/ids/heartbeat - * Map - */ - Map localWorkerStats = null; - try { - localWorkerStats = getLocalWorkerStats(conf, localState, - localAssignments); - } catch (Exception e) { - LOG.error("Failed to get Local worker stats"); - throw e; - } - LOG.debug("Allocated: " + localWorkerStats); - - /** - * Step 3: kill Invalid Workers and remove killed worker from - * localWorkerStats - */ - Set keepPorts = killUselessWorkers(localWorkerStats); - - // check new workers - checkNewWorkers(conf); - - // start new workers - startNewWorkers(keepPorts, localAssignments); - - } catch (Exception e) { - LOG.error("Failed Sync Process", e); - // throw e - } - - } - - /** - * mark all new Workers - * - * @param workerIds - * @pdOid 52b11418-7474-446d-bff5-0ecd68f4954f - */ - public void markAllNewWorkers(Map workerIds) { - - int startTime = TimeUtils.current_time_secs(); - - for (Entry entry : workerIds.entrySet()) { - - workerIdToStartTimeAndPort.put(entry.getValue(), - new Pair(startTime, entry.getKey())); - - } - } - - /** - * check new workers if the time is not > * - * SUPERVISOR_WORKER_START_TIMEOUT_SECS, otherwise info failed - * - * @param conf - * @pdOid f0a6ab43-8cd3-44e1-8fd3-015a2ec51c6a - */ - 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) { - 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 { - LOG.info("Successfully start worker " + workerId); - workers.add(workerId); - } - } - for (String workerId : workers) { - this.workerIdToStartTimeAndPort.remove(workerId); - } - } - - /** - * get localstat approved workerId's map - * - * @return Map [workerheart, state] is also a - * map, key is "workheartbeat" and "state" - * @param conf - * @param localState - * @param assignedTasks - * @throws IOException - * @pdOid 11c9bebb-d082-4c51-b323-dd3d5522a649 - */ - @SuppressWarnings("unchecked") - public Map getLocalWorkerStats(Map conf, - LocalState localState, Map assignedTasks) - throws Exception { - - Map workeridHbstate = new HashMap(); - - int now = TimeUtils.current_time_secs(); - - /** - * Get Map from - * local_dir/worker/ids/heartbeat - */ - Map idToHeartbeat = readWorkerHeartbeats(conf); - for (Map.Entry entry : idToHeartbeat - .entrySet()) { - - String workerid = entry.getKey().toString(); - - WorkerHeartbeat whb = entry.getValue(); - - State state = null; - - if (whb == null) { - - state = State.notStarted; - - } else if (matchesAssignment(whb, assignedTasks) == false) { - - // workerId isn't approved or - // isn't assigned task - state = State.disallowed; - - } else if ((now - whb.getTimeSecs()) > JStormUtils.parseInt(conf - .get(Config.SUPERVISOR_WORKER_TIMEOUT_SECS))) {// - - state = State.timedOut; - } else { - if (isWorkerDead(workerid)) { - state = State.timedOut; - }else { - state = State.valid; - } - } - - if (state != State.valid) { - LOG.info("Worker:" + workerid + " state:" + state - + " WorkerHeartbeat: " + whb - + " at supervisor time-secs " + now); - } else { - LOG.debug("Worker:" + workerid + " state:" + state - + " WorkerHeartbeat: " + whb - + " at supervisor time-secs " + now); - } - - workeridHbstate.put(workerid, new StateHeartbeat(state, whb)); - } - - return workeridHbstate; - } - - /** - * check whether the workerheartbeat is allowed in the assignedTasks - * - * @param whb - * : WorkerHeartbeat - * @param assignedTasks - * @return boolean if true, the assignments(LS-LOCAL-ASSIGNMENTS) is match - * with workerheart if fasle, is not matched - */ - public boolean matchesAssignment(WorkerHeartbeat whb, - Map assignedTasks) { - - boolean isMatch = true; - LocalAssignment localAssignment = assignedTasks.get(whb.getPort()); - - if (localAssignment == null) { - isMatch = false; - } else if (!whb.getTopologyId().equals(localAssignment.getTopologyId())) { - // topology id not equal - LOG.info("topology id not equal whb=" + whb.getTopologyId() - + ",localAssignment=" + localAssignment.getTopologyId()); - isMatch = false; - } else if (!(whb.getTaskIds().equals(localAssignment.getTaskIds()))) { - // task-id isn't equal - LOG.info("task-id isn't equal whb=" + whb.getTaskIds() - + ",localAssignment=" + localAssignment.getTaskIds()); - isMatch = false; - } - - return isMatch; - } - - /** - * get all workers heartbeats of the supervisor - * - * @param conf - * @return Map - * @throws IOException - * @throws IOException - */ - public Map readWorkerHeartbeats(Map conf) - throws Exception { - - Map workerHeartbeats = new HashMap(); - - // get the path: STORM-LOCAL-DIR/workers - String path = StormConfig.worker_root(conf); - - List workerIds = PathUtils.read_dir_contents(path); - - if (workerIds == null) { - LOG.info("No worker dir under " + path); - return workerHeartbeats; - - } - - for (String workerId : workerIds) { - - WorkerHeartbeat whb = readWorkerHeartbeat(conf, workerId); - - // ATTENTION: whb can be null - workerHeartbeats.put(workerId, whb); - } - return workerHeartbeats; - } - - /** - * get worker heartbeat by workerid - * - * @param conf - * @param workerId - * @returns WorkerHeartbeat - * @throws IOException - */ - public WorkerHeartbeat readWorkerHeartbeat(Map conf, String workerId) - throws Exception { - - try { - LocalState ls = StormConfig.worker_state(conf, workerId); - - return (WorkerHeartbeat) ls.get(Common.LS_WORKER_HEARTBEAT); - } catch (IOException e) { - LOG.error("Failed to get worker Heartbeat", e); - return null; - } - - } - - /** - * launch a worker in local mode - * - * @param conf - * @param sharedcontext - * @param topologyId - * @param supervisorId - * @param port - * @param workerId - * @param workerThreadPidsAtom - * @param workerThreadPidsAtomWriteLock - * @pdOid 405f44c7-bc1b-4e16-85cc-b59352b6ff5d - */ - public void launchWorker(Map conf, IContext sharedcontext, - String topologyId, String supervisorId, Integer port, - String workerId, - ConcurrentHashMap workerThreadPidsAtom) - throws Exception { - - String pid = UUID.randomUUID().toString(); - - WorkerShutdown worker = Worker.mk_worker(conf, sharedcontext, - topologyId, supervisorId, port, workerId, null); - - ProcessSimulator.registerProcess(pid, worker); - - 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; - // return classpath; - - String classpath = JStormUtils.current_classpath(); - - String[] classpathes = classpath.split(":"); - - Set classSet = new HashSet(); - - for (String classJar : classpathes) { - if (StringUtils.isBlank(classJar) == true) { - continue; - } - classSet.add(classJar); - } - - if (stormHome != null) { - List stormHomeFiles = PathUtils - .read_dir_contents(stormHome); - - for (String file : stormHomeFiles) { - if (file.endsWith(".jar")) { - classSet.add(stormHome + File.separator + file); - } - } - - List stormLibFiles = PathUtils.read_dir_contents(stormHome - + File.separator + "lib"); - for (String file : stormLibFiles) { - if (file.endsWith(".jar")) { - classSet.add(stormHome + File.separator + "lib" - + File.separator + file); - } - } - - } - - Set filterJars = setFilterJars(totalConf); - - StringBuilder sb = new StringBuilder(); - for (String jar : classSet) { - if (isKeyContain(filterJars, jar)) { - LOG.info("Remove " + jar); - continue; - } - sb.append(jar + ":"); - } - - if (ConfigExtension.isEnableTopologyClassLoader(totalConf)) { - return sb.toString().substring(0, sb.length() - 1); - } else { - sb.append(stormjar); - return sb.toString(); - } - - } - - public String getChildOpts(Map stormConf) { - String childopts = " "; - - if (stormConf.get(Config.TOPOLOGY_WORKER_CHILDOPTS) != null) { - childopts += (String) stormConf - .get(Config.TOPOLOGY_WORKER_CHILDOPTS); - } else if (ConfigExtension.getWorkerGc(stormConf) != null) { - childopts += ConfigExtension.getWorkerGc(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); - String gcPath = JStormUtils.getLogDir(); - - Date now = new Date(); - String nowStr = TimeFormat.getSecond(now); - - StringBuilder gc = new StringBuilder(); - - gc.append(" -Xloggc:"); - gc.append(gcPath); - gc.append(File.separator); - gc.append("%TOPOLOGYID%-worker-%ID%-"); - gc.append(nowStr); - gc.append("-gc.log -verbose:gc -XX:HeapDumpPath="); - gc.append(gcPath).append(File.separator).append("java-%TOPOLOGYID%-") - .append(nowStr).append(".hprof"); - gc.append(" "); - - return gc.toString(); - } - - /** - * launch a worker in distributed mode - * - * @param conf - * @param sharedcontext - * @param topologyId - * @param supervisorId - * @param port - * @param workerId - * @throws IOException - * @pdOid 6ea369dd-5ce2-4212-864b-1f8b2ed94abb - */ - public void launchWorker(Map conf, IContext sharedcontext, - String topologyId, String supervisorId, Integer port, - String workerId, LocalAssignment assignment) throws IOException { - - // STORM-LOCAL-DIR/supervisor/stormdist/topologyId - String stormroot = StormConfig.supervisor_stormdist_root(conf, - topologyId); - - // STORM-LOCAL-DIR/supervisor/stormdist/topologyId/stormjar.jar - String stormjar = StormConfig.stormjar_path(stormroot); - - // get supervisor conf - Map stormConf = StormConfig.read_supervisor_topology_conf(conf, - topologyId); - - Map totalConf = new HashMap(); - totalConf.putAll(conf); - totalConf.putAll(stormConf); - - // get classpath - // String[] param = new String[1]; - // param[0] = stormjar; - // String classpath = JStormUtils.add_to_classpath( - // JStormUtils.current_classpath(), param); - - // get child process parameter - - String stormhome = System.getProperty("jstorm.home"); - - long memSize = assignment.getMem(); - int cpuNum = assignment.getCpu(); - String childopts = getChildOpts(totalConf); - - childopts += getGcDumpParam(totalConf); - - Map environment = new HashMap(); - - if (ConfigExtension.getWorkerRedirectOutput(totalConf)) { - environment.put("REDIRECT", "true"); - } else { - environment.put("REDIRECT", "false"); - } - - environment.put("LD_LIBRARY_PATH", - (String) totalConf.get(Config.JAVA_LIBRARY_PATH)); - - StringBuilder commandSB = new StringBuilder(); - - try { - if (this.cgroupManager != null) { - commandSB - .append(cgroupManager.startNewWorker(cpuNum, workerId)); - } - } catch (Exception e) { - LOG.error("fail to prepare cgroup to workerId: " + workerId, e); - return; - } - - // commandSB.append("java -server -Xdebug -Xrunjdwp:transport=dt_socket,address=8000,server=y,suspend=n "); - commandSB.append("java -server "); - commandSB.append(" -Xms" + memSize); - commandSB.append(" -Xmx" + memSize + " "); - commandSB.append(" -Xmn" + memSize / 3 + " "); - commandSB.append(" -XX:PermSize=" + memSize / 16); - commandSB.append(" -XX:MaxPermSize=" + memSize / 8); - commandSB.append(" " + childopts); - commandSB.append(" " - + (assignment.getJvm() == null ? "" : assignment.getJvm())); - - commandSB.append(" -Djava.library.path="); - commandSB.append((String) totalConf.get(Config.JAVA_LIBRARY_PATH)); - - if (stormhome != null) { - commandSB.append(" -Djstorm.home="); - commandSB.append(stormhome); - } - - commandSB.append(getLogParameter(totalConf, stormhome, assignment.getTopologyName(), port)); - - String classpath = getClassPath(stormjar, stormhome, totalConf); - String workerClassPath = (String) totalConf - .get(Config.WORKER_CLASSPATH); - List otherLibs = (List) stormConf - .get(GenericOptionsParser.TOPOLOGY_LIB_NAME); - StringBuilder sb = new StringBuilder(); - if (otherLibs != null) { - for (String libName : otherLibs) { - sb.append(StormConfig.stormlib_path(stormroot, libName)) - .append(":"); - } - } - workerClassPath = workerClassPath + ":" + sb.toString(); - - Map policyReplaceMap = new HashMap(); - String realClassPath = classpath + ":" + workerClassPath; - policyReplaceMap.put(SandBoxMaker.CLASS_PATH_KEY, realClassPath); - commandSB - .append(sandBoxMaker.sandboxPolicy(workerId, policyReplaceMap)); - - // commandSB.append(" -Dlog4j.configuration=storm.log.properties"); - - commandSB.append(" -cp "); - // commandSB.append(workerClassPath + ":"); - commandSB.append(classpath); - if (!ConfigExtension.isEnableTopologyClassLoader(totalConf)) - commandSB.append(":").append(workerClassPath); - - commandSB.append(" com.alibaba.jstorm.daemon.worker.Worker "); - commandSB.append(topologyId); - - commandSB.append(" "); - commandSB.append(supervisorId); - - commandSB.append(" "); - commandSB.append(port); - - commandSB.append(" "); - commandSB.append(workerId); - - commandSB.append(" "); - commandSB.append(workerClassPath + ":" + stormjar); - - 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(cmd, environment, true); - } - - private Set killUselessWorkers( - Map localWorkerStats) { - Map removed = new HashMap(); - Set keepPorts = new HashSet(); - - for (Entry entry : localWorkerStats.entrySet()) { - - 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, null); - } - - StringBuilder sb = new StringBuilder(); - sb.append("Shutting down and clearing state for id "); - sb.append(workerid); - sb.append(";State:"); - sb.append(hbstate); - - LOG.info(sb); - } - } - - 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; - } - - private void startNewWorkers(Set keepPorts, - Map localAssignments) throws Exception { - /** - * Step 4: get reassigned tasks, which is in assignedTasks, but not in - * keeperPorts Map - */ - Map newWorkers = JStormUtils - .select_keys_pred(keepPorts, localAssignments); - - /** - * Step 5: generate new work ids - */ - Map newWorkerIds = new HashMap(); - - for (Entry entry : newWorkers.entrySet()) { - Integer port = entry.getKey(); - LocalAssignment assignment = entry.getValue(); - - String workerId = UUID.randomUUID().toString(); - - newWorkerIds.put(port, workerId); - - // create new worker Id directory - // LOCALDIR/workers/newworkid/pids - try { - StormConfig.worker_pids_root(conf, workerId); - } catch (IOException e1) { - LOG.error("Failed to create " + workerId + " localdir", e1); - throw e1; - } - - StringBuilder sb = new StringBuilder(); - sb.append("Launching worker with assiangment "); - sb.append(assignment.toString()); - sb.append(" for the supervisor "); - sb.append(supervisorId); - sb.append(" on port "); - sb.append(port); - sb.append(" with id "); - sb.append(workerId); - LOG.info(sb); - - try { - String clusterMode = StormConfig.cluster_mode(conf); - - if (clusterMode.equals("distributed")) { - launchWorker(conf, sharedContext, - assignment.getTopologyId(), supervisorId, port, - workerId, assignment); - } else if (clusterMode.equals("local")) { - launchWorker(conf, sharedContext, - assignment.getTopologyId(), supervisorId, port, - workerId, workerThreadPids); - } - } catch (Exception e) { - String errorMsg = "Failed to launchWorker workerId:" + workerId - + ":" + port; - LOG.error(errorMsg, e); - throw e; - } - - } - - /** - * FIXME, workerIds should be Set, not Collection, but here simplify the - * logic - */ - 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 { - 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.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/supervisor/SyncSupervisorEvent.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SyncSupervisorEvent.java deleted file mode 100644 index e02fe32e8..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/supervisor/SyncSupervisorEvent.java +++ /dev/null @@ -1,448 +0,0 @@ -package com.alibaba.jstorm.daemon.supervisor; - -import java.io.File; -import java.io.IOException; -import java.net.URL; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -import java.util.UUID; - -import org.apache.commons.io.FileUtils; -import org.apache.log4j.Logger; -import org.apache.thrift7.TException; - -import backtype.storm.utils.LocalState; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.cluster.Cluster; -import com.alibaba.jstorm.cluster.Common; -import com.alibaba.jstorm.cluster.StormBase; -import com.alibaba.jstorm.cluster.StormClusterState; -import com.alibaba.jstorm.cluster.StormConfig; -import com.alibaba.jstorm.event.EventManager; -import com.alibaba.jstorm.event.EventManagerZkPusher; -import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; -import com.alibaba.jstorm.task.Assignment; -import com.alibaba.jstorm.task.LocalAssignment; -import com.alibaba.jstorm.utils.JStormServerUtils; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.PathUtils; - -/** - * supervisor SynchronizeSupervisor workflow (1) writer local assignment to - * LocalState (2) download new Assignment's topology (3) remove useless Topology - * (4) push one SyncProcessEvent to SyncProcessEvent's EventManager - */ -class SyncSupervisorEvent extends RunnableCallback { - - private static final Logger LOG = Logger - .getLogger(SyncSupervisorEvent.class); - - // private Supervisor supervisor; - - private String supervisorId; - - private EventManager processEventManager; - - private EventManager syncSupEventManager; - - private StormClusterState stormClusterState; - - private LocalState localState; - - private Map conf; - - private SyncProcessEvent syncProcesses; - - /** - * @param conf - * @param processEventManager - * @param syncSupEventManager - * @param stormClusterState - * @param supervisorId - * @param localState - * @param syncProcesses - */ - public SyncSupervisorEvent(String supervisorId, Map conf, - EventManager processEventManager, EventManager syncSupEventManager, - StormClusterState stormClusterState, LocalState localState, - SyncProcessEvent syncProcesses) { - - this.syncProcesses = syncProcesses; - this.processEventManager = processEventManager; - this.syncSupEventManager = syncSupEventManager; - this.stormClusterState = stormClusterState; - this.conf = conf; - this.supervisorId = supervisorId; - this.localState = localState; - - } - - @Override - public void run() { - LOG.debug("Synchronizing supervisor"); - - try { - - RunnableCallback syncCallback = new EventManagerZkPusher(this, - syncSupEventManager); - - /** - * Step 1: get all assignments and register /ZK-dir/assignment and - * every assignment watch - * - */ - Map assignments = Cluster.get_all_assignment( - stormClusterState, syncCallback); - LOG.debug("Get all assignments " + assignments); - - /** - * Step 2: get topologyIds list from - * STORM-LOCAL-DIR/supervisor/stormdist/ - */ - List downloadedTopologyIds = StormConfig - .get_supervisor_toplogy_list(conf); - LOG.debug("Downloaded storm ids: " + downloadedTopologyIds); - - /** - * Step 3: get from ZK local node's - * assignment - */ - Map localAssignment = getLocalAssign( - stormClusterState, supervisorId, assignments); - - /** - * Step 4: writer local assignment to LocalState - */ - try { - LOG.debug("Writing local assignment " + localAssignment); - localState.put(Common.LS_LOCAL_ASSIGNMENTS, localAssignment); - } catch (IOException e) { - LOG.error("put LS_LOCAL_ASSIGNMENTS " + localAssignment - + " of localState failed"); - throw e; - } - - // Step 5: download code from ZK - - Map topologyCodes = getTopologyCodeLocations( - assignments, supervisorId); - - downloadTopology(topologyCodes, downloadedTopologyIds); - - /** - * Step 6: remove any downloaded useless topology - */ - removeUselessTopology(topologyCodes, downloadedTopologyIds); - - /** - * Step 7: push syncProcesses Event - */ - processEventManager.add(syncProcesses); - - } catch (Exception e) { - LOG.error("Failed to Sync Supervisor", e); - // throw new RuntimeException(e); - } - - } - - /** - * download code ; two cluster mode: local and distributed - * - * @param conf - * @param topologyId - * @param masterCodeDir - * @param clusterMode - * @throws IOException - */ - private void downloadStormCode(Map conf, String topologyId, - String masterCodeDir) throws IOException, TException { - String clusterMode = StormConfig.cluster_mode(conf); - - if (clusterMode.endsWith("distributed")) { - downloadDistributeStormCode(conf, topologyId, masterCodeDir); - } else if (clusterMode.endsWith("local")) { - downloadLocalStormCode(conf, topologyId, masterCodeDir); - - } - } - - private void downloadLocalStormCode(Map conf, String topologyId, - String masterCodeDir) throws IOException, TException { - - // STORM-LOCAL-DIR/supervisor/stormdist/storm-id - String stormroot = StormConfig.supervisor_stormdist_root(conf, - topologyId); - - FileUtils.copyDirectory(new File(masterCodeDir), new File(stormroot)); - - ClassLoader classloader = Thread.currentThread() - .getContextClassLoader(); - - String resourcesJar = resourcesJar(); - - URL url = classloader.getResource(StormConfig.RESOURCES_SUBDIR); - - String targetDir = stormroot + '/' + StormConfig.RESOURCES_SUBDIR; - - if (resourcesJar != null) { - - LOG.info("Extracting resources from jar at " + resourcesJar - + " to " + targetDir); - - JStormUtils.extract_dir_from_jar(resourcesJar, - StormConfig.RESOURCES_SUBDIR, stormroot);// extract dir - // from jar;; - // util.clj - } else if (url != null) { - - LOG.info("Copying resources at " + url.toString() + " to " - + targetDir); - - FileUtils.copyDirectory(new File(url.getFile()), (new File( - targetDir))); - - } - } - - /** - * Don't need synchronize, due to EventManager will execute serially - * - * @param conf - * @param topologyId - * @param masterCodeDir - * @throws IOException - * @throws TException - */ - private void downloadDistributeStormCode(Map conf, String topologyId, - String masterCodeDir) throws IOException, TException { - - // STORM_LOCAL_DIR/supervisor/tmp/(UUID) - String tmproot = StormConfig.supervisorTmpDir(conf) + File.separator - + UUID.randomUUID().toString(); - - // STORM_LOCAL_DIR/supervisor/stormdist/topologyId - String stormroot = StormConfig.supervisor_stormdist_root(conf, - topologyId); - - JStormServerUtils.downloadCodeFromMaster(conf, tmproot, masterCodeDir, - topologyId, true); - - // tmproot/stormjar.jar - String localFileJarTmp = StormConfig.stormjar_path(tmproot); - - // extract dir from jar - JStormUtils.extract_dir_from_jar(localFileJarTmp, - StormConfig.RESOURCES_SUBDIR, tmproot); - - FileUtils.moveDirectory(new File(tmproot), new File(stormroot)); - - } - - private String resourcesJar() { - - String path = System.getProperty("java.class.path"); - if (path == null) { - return null; - } - - String[] paths = path.split(File.pathSeparator); - - List jarPaths = new ArrayList(); - for (String s : paths) { - if (s.endsWith(".jar")) { - jarPaths.add(s); - } - } - - /** - * FIXME, this place seems exist problem - */ - List rtn = new ArrayList(); - int size = jarPaths.size(); - for (int i = 0; i < size; i++) { - if (JStormUtils.zipContainsDir(jarPaths.get(i), - StormConfig.RESOURCES_SUBDIR)) { - rtn.add(jarPaths.get(i)); - } - } - - if (rtn.size() == 0) - return null; - - return rtn.get(0); - } - - /** - * a port must be assigned one topology - * - * @param stormClusterState - * @param supervisorId - * @param callback - * @throws Exception - * @returns map: {port,LocalAssignment} - */ - private Map getLocalAssign( - StormClusterState stormClusterState, String supervisorId, - Map assignments) throws Exception { - - Map portLA = new HashMap(); - - for (Entry assignEntry : assignments.entrySet()) { - String topologyId = assignEntry.getKey(); - Assignment assignment = assignEntry.getValue(); - - Map portTasks = readMyTasks( - stormClusterState, topologyId, supervisorId, assignment); - if (portTasks == null) { - continue; - } - - // a port must be assigned one storm - for (Entry entry : portTasks.entrySet()) { - - Integer port = entry.getKey(); - - LocalAssignment la = entry.getValue(); - - if (!portLA.containsKey(port)) { - portLA.put(port, la); - } else { - throw new RuntimeException( - "Should not have multiple topologys assigned to one port"); - } - } - } - - return portLA; - } - - /** - * get local node's tasks - * - * @param stormClusterState - * @param topologyId - * @param supervisorId - * @param callback - * @return Map: {port, LocalAssignment} - * @throws Exception - */ - private Map readMyTasks( - StormClusterState stormClusterState, String topologyId, - String supervisorId, Assignment assignmenInfo) throws Exception { - - Map portTasks = new HashMap(); - - StormBase stormBase = stormClusterState.storm_base(topologyId, null); - if (stormBase == null) { - LOG.error("Failed to get StormBase of " + topologyId); - return portTasks; - } - - Set workers = assignmenInfo.getWorkers(); - if (workers == null) { - LOG.error("No worker of assignement's " + assignmenInfo); - return portTasks; - } - - for (ResourceWorkerSlot worker : workers) { - if (!supervisorId.equals(worker.getNodeId())) - continue; - portTasks.put(worker.getPort(), - new LocalAssignment(topologyId, worker.getTasks(), - stormBase.getStormName(), worker.getMemSize(), - worker.getCpu(), worker.getJvm())); - } - - return portTasks; - } - - /** - * get mastercodedir for every topology - * - * @param stormClusterState - * @param callback - * @throws Exception - * @returns Map: from zookeeper - */ - public static Map getTopologyCodeLocations( - Map assignments, String supervisorId) - throws Exception { - - Map rtn = new HashMap(); - for (Entry entry : assignments.entrySet()) { - String topologyid = entry.getKey(); - Assignment assignmenInfo = entry.getValue(); - - Set workers = assignmenInfo.getWorkers(); - for (ResourceWorkerSlot worker : workers) { - String node = worker.getNodeId(); - if (supervisorId.equals(node)) { - rtn.put(topologyid, assignmenInfo.getMasterCodeDir()); - break; - } - } - - } - return rtn; - } - - public void downloadTopology(Map topologyCodes, - List downloadedTopologyIds) throws Exception { - - for (Entry entry : topologyCodes.entrySet()) { - - String topologyId = entry.getKey(); - String masterCodeDir = entry.getValue(); - - if (!downloadedTopologyIds.contains(topologyId)) { - - LOG.info("Downloading code for storm id " + topologyId - + " from " + masterCodeDir); - - try { - downloadStormCode(conf, topologyId, masterCodeDir); - } catch (IOException e) { - LOG.error(e + " downloadStormCode failed " + "topologyId:" - + topologyId + "masterCodeDir:" + masterCodeDir); - throw e; - - } catch (TException e) { - LOG.error(e + " downloadStormCode failed " + "topologyId:" - + topologyId + "masterCodeDir:" + masterCodeDir); - throw e; - } - LOG.info("Finished downloading code for storm id " + topologyId - + " from " + masterCodeDir); - } - - } - } - - public void removeUselessTopology(Map topologyCodes, - List downloadedTopologyIds) { - for (String topologyId : downloadedTopologyIds) { - - if (!topologyCodes.containsKey(topologyId)) { - - LOG.info("Removing code for storm id " + topologyId); - - String path = null; - try { - path = StormConfig.supervisor_stormdist_root(conf, - topologyId); - PathUtils.rmr(path); - } catch (IOException e) { - String errMsg = "rmr the path:" + path + "failed\n"; - LOG.error(errMsg, e); - } - } - } - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/BatchTupleRunable.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/BatchTupleRunable.java deleted file mode 100644 index da2db625e..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/BatchTupleRunable.java +++ /dev/null @@ -1,127 +0,0 @@ -package com.alibaba.jstorm.daemon.worker; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.log4j.Logger; - -import backtype.storm.messaging.IConnection; -import backtype.storm.messaging.TaskMessage; -import backtype.storm.scheduler.WorkerSlot; -import backtype.storm.utils.DisruptorQueue; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.metric.MetricDef; -import com.alibaba.jstorm.metric.JStormTimer; -import com.alibaba.jstorm.metric.Metrics; -import com.alibaba.jstorm.utils.DisruptorRunable; -import com.alibaba.jstorm.utils.Pair; -import com.lmax.disruptor.EventHandler; - -//import com.alibaba.jstorm.message.zeroMq.ISendConnection; - -/** - * - * Tuple sender - * - * @author yannian - * - */ -public class BatchTupleRunable extends DisruptorRunable { - private final static Logger LOG = Logger.getLogger(BatchTupleRunable.class); - - private DisruptorQueue transferQueue; - private ConcurrentHashMap nodeportSocket; - private ConcurrentHashMap taskNodeport; - - - private Map> dispatchMap; - private DisruptorQueue sendingQueue; - - private final boolean isDirectSend = true; - - private static JStormTimer timer = Metrics.registerTimer(null, - MetricDef.BATCH_TUPLE_TIME, null, Metrics.MetricType.WORKER); - private DisruptorQueue queue; - - public BatchTupleRunable(WorkerData workerData) { - super(workerData.getTransferQueue(), timer, BatchTupleRunable.class.getSimpleName(), workerData.getActive()); - this.sendingQueue = workerData.getSendingQueue(); - this.nodeportSocket = workerData.getNodeportSocket(); - this.taskNodeport = workerData.getTaskNodeport(); - this.dispatchMap = new HashMap>(); - - this.queue = workerData.getTransferQueue(); - Metrics.registerQueue(null, MetricDef.BATCH_TUPLE_QUEUE, this.queue, null, Metrics.MetricType.WORKER); - - this.queue.consumerStarted(); - } - - public void handleOneEvent(TaskMessage felem) { - - int taskId = felem.task(); - byte[] tuple = felem.message(); - - WorkerSlot nodePort = taskNodeport.get(taskId); - if (nodePort == null) { - String errormsg = "can`t not found IConnection to " + taskId; - LOG.warn("DrainerRunable warn", new Exception(errormsg)); - return; - } - IConnection conn = nodeportSocket.get(nodePort); - if (conn == null) { - String errormsg = "can`t not found nodePort " + nodePort; - LOG.warn("DrainerRunable warn", new Exception(errormsg)); - return; - } - - if (conn.isClosed() == true) { - // if connection has been closed, just skip the package - return; - } - - if (isDirectSend) { - conn.send(felem); - return ; - } - - List list = dispatchMap.get(conn); - if (list == null) { - list = new ArrayList(); - dispatchMap.put(conn, list); - } - list.add(felem); - return ; - - } - - public void handleFinish() { - for (Entry> entry: dispatchMap.entrySet()) { - Pair> pair = - new Pair>( - entry.getKey(), entry.getValue()); - - sendingQueue.publish(pair); - } - - dispatchMap.clear(); - } - - @Override - public void handleEvent(Object event, boolean endOfBatch) - throws Exception { - - handleOneEvent((TaskMessage)event); - - if (endOfBatch == true && isDirectSend == false) { - handleFinish(); - } - - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/ContextMaker.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/ContextMaker.java deleted file mode 100644 index fc4e5f0db..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/ContextMaker.java +++ /dev/null @@ -1,107 +0,0 @@ -package com.alibaba.jstorm.daemon.worker; - -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.Map.Entry; - -import org.apache.log4j.Logger; - -import backtype.storm.generated.StormTopology; -import backtype.storm.generated.StreamInfo; -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Fields; -import backtype.storm.utils.ThriftTopologyUtils; - -import com.alibaba.jstorm.cluster.StormConfig; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.PathUtils; - -/** - * ContextMaker This class is used to create TopologyContext - * - * @author yannian/Longda - * - */ -public class ContextMaker { - private static Logger LOG = Logger.getLogger(ContextMaker.class); - - private WorkerData workerData; - - private String resourcePath; - private String pidDir; - private String codeDir; - private List workerTasks; - - @SuppressWarnings("rawtypes") - public ContextMaker(WorkerData workerData) { - /* - * Map stormConf, String topologyId, String workerId, HashMap tasksToComponent, Integer port, List workerTasks - */ - this.workerData = workerData; - this.workerTasks = JStormUtils.mk_list(workerData.getTaskids()); - - try { - Map stormConf = workerData.getStormConf(); - String topologyId = workerData.getTopologyId(); - String workerId = workerData.getWorkerId(); - - String distroot = StormConfig.supervisor_stormdist_root(stormConf, - topologyId); - - resourcePath = StormConfig - .supervisor_storm_resources_path(distroot); - - pidDir = StormConfig.worker_pids_root(stormConf, workerId); - - - String codePath = StormConfig.stormcode_path(distroot); - codeDir = PathUtils.parent_path(codePath); - - } catch (IOException e) { - LOG.error("Failed to create ContextMaker", e); - throw new RuntimeException(e); - } - } - - public TopologyContext makeTopologyContext(StormTopology topology, - Integer taskId, clojure.lang.Atom openOrPrepareWasCalled) { - - Map stormConf = workerData.getStormConf(); - String topologyId = workerData.getTopologyId(); - - HashMap> componentToStreamToFields = new HashMap>(); - - Set components = ThriftTopologyUtils.getComponentIds(topology); - for (String component : components) { - - Map streamToFieldsMap = new HashMap(); - - Map streamInfoMap = ThriftTopologyUtils - .getComponentCommon(topology, component).get_streams(); - for (Entry entry : streamInfoMap.entrySet()) { - String streamId = entry.getKey(); - StreamInfo streamInfo = entry.getValue(); - - streamToFieldsMap.put(streamId, - new Fields(streamInfo.get_output_fields())); - } - - componentToStreamToFields.put(component, streamToFieldsMap); - } - - return new TopologyContext(topology, stormConf, - workerData.getTasksToComponent(), - workerData.getComponentToSortedTasks(), - componentToStreamToFields, topologyId, resourcePath, pidDir, - taskId, workerData.getPort(), workerTasks, - workerData.getDefaultResources(), - workerData.getUserResources(), workerData.getExecutorData(), - workerData.getRegisteredMetrics(), openOrPrepareWasCalled); - - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/DrainerRunable.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/DrainerRunable.java deleted file mode 100644 index bca2dd6cf..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/DrainerRunable.java +++ /dev/null @@ -1,55 +0,0 @@ -package com.alibaba.jstorm.daemon.worker; - -import java.util.List; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.log4j.Logger; - -import backtype.storm.messaging.IConnection; -import backtype.storm.messaging.TaskMessage; -import backtype.storm.scheduler.WorkerSlot; -import backtype.storm.utils.DisruptorQueue; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.metric.MetricDef; -import com.alibaba.jstorm.metric.JStormTimer; -import com.alibaba.jstorm.metric.Metrics; -import com.alibaba.jstorm.utils.DisruptorRunable; -import com.alibaba.jstorm.utils.Pair; -import com.alibaba.jstorm.utils.RunCounter; -import com.lmax.disruptor.EventHandler; - -//import com.alibaba.jstorm.message.zeroMq.ISendConnection; - -/** - * - * Tuple sender - * - * @author yannian - * - */ -public class DrainerRunable extends DisruptorRunable{ - private final static Logger LOG = Logger.getLogger(DrainerRunable.class); - - private static JStormTimer timer = Metrics.registerTimer(null, - MetricDef.DRAINER_TIME, null, Metrics.MetricType.WORKER); - - public DrainerRunable(WorkerData workerData) { - super(workerData.getSendingQueue(), timer, - DrainerRunable.class.getSimpleName(), workerData.getActive()); - - Metrics.registerQueue(null, MetricDef.DRAINER_QUEUE, queue, null, Metrics.MetricType.WORKER); - } - - @Override - public void handleEvent(Object event, boolean endOfBatch) - throws Exception { - - Pair> pair = (Pair>)event; - - pair.getFirst().send(pair.getSecond()); - - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/ProcessSimulator.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/ProcessSimulator.java deleted file mode 100644 index cc2276f33..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/ProcessSimulator.java +++ /dev/null @@ -1,81 +0,0 @@ -package com.alibaba.jstorm.daemon.worker; - -import java.util.Collection; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; - -import org.apache.log4j.Logger; - -public class ProcessSimulator { - private static Logger LOG = Logger.getLogger(ProcessSimulator.class); - - protected static Object lock = new Object(); - - /** - * skip old function name: pid-counter - */ - - protected static ConcurrentHashMap processMap = new ConcurrentHashMap(); - - /** - * Register process handler old function name: register-process - * - * @param pid - * @param shutdownable - */ - public static void registerProcess(String pid, WorkerShutdown shutdownable) { - processMap.put(pid, shutdownable); - } - - /** - * Get process handle old function name: process-handle - * - * @param pid - * @return - */ - protected static WorkerShutdown getProcessHandle(String pid) { - return processMap.get(pid); - } - - /** - * Get all process handles old function name:all-processes - * - * @return - */ - protected static Collection GetAllProcessHandles() { - return processMap.values(); - } - - /** - * Kill pid handle old function name: KillProcess - * - * @param pid - */ - public static void killProcess(String pid) { - synchronized (lock) { - LOG.info("Begin killing process " + pid); - - WorkerShutdown shutdownHandle = getProcessHandle(pid); - - if (shutdownHandle != null) { - shutdownHandle.shutdown(); - } - - processMap.remove(pid); - - LOG.info("Successfully killing process " + pid); - } - } - - /** - * kill all handle old function name: kill-all-processes - */ - public static void killAllProcesses() { - Set pids = processMap.keySet(); - for (String pid : pids) { - killProcess(pid); - } - - LOG.info("Successfully kill all processes"); - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/RefreshActive.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/RefreshActive.java deleted file mode 100644 index 25aaf30b1..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/RefreshActive.java +++ /dev/null @@ -1,138 +0,0 @@ -package com.alibaba.jstorm.daemon.worker; - -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.cluster.StormBase; -import com.alibaba.jstorm.cluster.StormClusterState; -import com.alibaba.jstorm.cluster.StormMonitor; -import com.alibaba.jstorm.daemon.nimbus.StatusType; -import com.alibaba.jstorm.daemon.worker.metrics.MetricReporter; -import com.alibaba.jstorm.task.TaskShutdownDameon; -import com.alibaba.jstorm.utils.JStormUtils; - -/** - * Timely check whether topology is active or not and whether - * the metrics monitor is enable or disable from ZK - * - * @author yannian/Longda - * - */ -public class RefreshActive extends RunnableCallback { - private static Logger LOG = Logger.getLogger(RefreshActive.class); - - private WorkerData workerData; - - private AtomicBoolean active; - private Map conf; - private StormClusterState zkCluster; - private String topologyId; - private Integer frequence; - - // private Object lock = new Object(); - - @SuppressWarnings("rawtypes") - public RefreshActive(WorkerData workerData) { - this.workerData = workerData; - - this.active = workerData.getActive(); - this.conf = workerData.getConf(); - this.zkCluster = workerData.getZkCluster(); - this.topologyId = workerData.getTopologyId(); - this.frequence = JStormUtils.parseInt( - conf.get(Config.TASK_REFRESH_POLL_SECS), 10); - } - - @Override - public void run() { - - if (active.get() == false) { - return; - } - - - - try { - StatusType newTopologyStatus = StatusType.activate; - // /ZK-DIR/topology - StormBase base = zkCluster.storm_base(topologyId, this); - if (base == null) { - // @@@ normally the topology has been removed - LOG.warn("Failed to get StromBase from ZK of " + topologyId); - newTopologyStatus = StatusType.killed; - } else { - - newTopologyStatus = base.getStatus().getStatusType(); - } - - // Start metrics report if metrics monitor is enabled. - // Stop metrics report if metrics monitor is disabled. - try { - StormMonitor monitor = zkCluster.get_storm_monitor(topologyId); - if (null != monitor) { - boolean newMetricsMonitor = monitor.getMetrics(); - MetricReporter metricReporter = workerData.getMetricsReporter(); - boolean oldMetricsMonitor = metricReporter.isEnable(); - - if (oldMetricsMonitor != newMetricsMonitor) { - metricReporter.setEnable(newMetricsMonitor); - if (true == newMetricsMonitor) { - LOG.info("Start metrics reporter"); - } else { - LOG.info("Stop metrics reporter"); - } - } - } - } catch (Exception e) { - LOG.warn("Failed to get monitor status of topology " + topologyId); - LOG.debug(e); - } - - // Process the topology status change - StatusType oldTopologyStatus = workerData.getTopologyStatus(); - - if (newTopologyStatus.equals(oldTopologyStatus)) { - return; - } - - LOG.info("Old TopologyStatus:" + oldTopologyStatus - + ", new TopologyStatus:" + newTopologyStatus); - - List tasks = workerData.getShutdownTasks(); - if(tasks == null) { - LOG.info("Tasks aren't ready or begin to shutdown"); - return ; - } - - if (newTopologyStatus.equals(StatusType.active)) { - for (TaskShutdownDameon task : tasks) { - task.active(); - } - } else { - for (TaskShutdownDameon task : tasks) { - task.deactive(); - } - } - workerData.setTopologyStatus(newTopologyStatus); - } catch (Exception e) { - // TODO Auto-generated catch block - LOG.error("Failed to get topology from ZK ", e); - return; - } - - } - - @Override - public Object getResult() { - if (active.get()) { - return frequence; - } - return -1; - } -} 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 deleted file mode 100644 index 11b9a4b03..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/RefreshConnections.java +++ /dev/null @@ -1,214 +0,0 @@ -package com.alibaba.jstorm.daemon.worker; - -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; -import backtype.storm.messaging.IConnection; -import backtype.storm.messaging.IContext; -import backtype.storm.scheduler.WorkerSlot; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.cluster.StormClusterState; -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; - -/** - * - * Update current worker and other workers' zeroMQ connection. - * - * When worker shutdown/create, need update these connection - * - * @author yannian/Longda - * - */ -public class RefreshConnections extends RunnableCallback { - private static Logger LOG = Logger.getLogger(RefreshConnections.class); - - private WorkerData workerData; - - private AtomicBoolean active; - - @SuppressWarnings("rawtypes") - private Map conf; - - private StormClusterState zkCluster; - - private String topologyId; - - private Set outboundTasks; - - private ConcurrentHashMap nodeportSocket; - - private IContext context; - - private ConcurrentHashMap taskNodeport; - - private Integer frequence; - - private String supervisorId; - - private int taskTimeoutSecs; - - // private ReentrantReadWriteLock endpoint_socket_lock; - - @SuppressWarnings("rawtypes") - public RefreshConnections(WorkerData workerData, Set outbound_tasks) { - - this.workerData = workerData; - - this.active = workerData.getActive(); - this.conf = workerData.getConf(); - this.zkCluster = workerData.getZkCluster(); - this.topologyId = workerData.getTopologyId(); - this.outboundTasks = outbound_tasks; - this.nodeportSocket = workerData.getNodeportSocket(); - this.context = workerData.getContext(); - this.taskNodeport = workerData.getTaskNodeport(); - this.supervisorId = workerData.getSupervisorId(); - - // 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 - public void run() { - - if (active.get() == false) { - return; - } - - try { - // - // @@@ does lock need? - // endpoint_socket_lock.writeLock().lock(); - // - - synchronized (this) { - Assignment assignment = zkCluster.assignment_info(topologyId, - this); - if (assignment == null) { - String errMsg = "Failed to get Assignment of " + topologyId; - LOG.error(errMsg); - // throw new RuntimeException(errMsg); - return; - } - - Set workers = assignment.getWorkers(); - if (workers == null) { - String errMsg = "Failed to get taskToResource of " - + topologyId; - LOG.error(errMsg); - return; - } - workerData.getWorkerToResource().addAll(workers); - - Map my_assignment = new HashMap(); - - Map node = assignment.getNodeHost(); - - // only reserve outboundTasks - Set need_connections = new HashSet(); - - Set localNodeTasks = new HashSet(); - - if (workers != null && outboundTasks != null) { - for (ResourceWorkerSlot worker : workers) { - if (supervisorId.equals(worker.getNodeId())) - localNodeTasks.addAll(worker.getTasks()); - for (Integer id : worker.getTasks()) { - if (outboundTasks.contains(id)) { - my_assignment.put(id, worker); - need_connections.add(worker); - } - } - } - } - taskNodeport.putAll(my_assignment); - workerData.setLocalNodeTasks(localNodeTasks); - - // get which connection need to be remove or add - Set current_connections = nodeportSocket.keySet(); - Set new_connections = new HashSet(); - Set remove_connections = new HashSet(); - - for (WorkerSlot node_port : need_connections) { - if (!current_connections.contains(node_port)) { - new_connections.add(node_port); - } - } - - for (WorkerSlot node_port : current_connections) { - if (!need_connections.contains(node_port)) { - remove_connections.add(node_port); - } - } - - // create new connection - for (WorkerSlot nodePort : new_connections) { - - String host = node.get(nodePort.getNodeId()); - - int port = nodePort.getPort(); - - IConnection conn = context.connect(topologyId, host, port); - - nodeportSocket.put(nodePort, conn); - - LOG.info("Add connection to " + nodePort); - } - - // close useless connection - for (WorkerSlot node_port : remove_connections) { - 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); - throw new RuntimeException(e); - } - - // finally { - // endpoint_socket_lock.writeLock().unlock(); - // } - - } - - @Override - public Object getResult() { - if (active.get()) { - return frequence; - } - return -1; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/State.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/State.java deleted file mode 100644 index 9a7bdc194..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/State.java +++ /dev/null @@ -1,11 +0,0 @@ -package com.alibaba.jstorm.daemon.worker; - -/** - * Worker's status - * - * @author chenjun - * - */ -public enum State { - valid, disallowed, notStarted, timedOut; -} 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 deleted file mode 100644 index e01abaeb5..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/VirtualPortDispatch.java +++ /dev/null @@ -1,84 +0,0 @@ -package com.alibaba.jstorm.daemon.worker; - -import java.util.concurrent.ConcurrentHashMap; - -import org.apache.log4j.Logger; - -import backtype.storm.messaging.IConnection; -import backtype.storm.messaging.TaskMessage; -import backtype.storm.utils.DisruptorQueue; - -import com.alibaba.jstorm.metric.JStormTimer; -import com.alibaba.jstorm.metric.MetricDef; -import com.alibaba.jstorm.metric.Metrics; -import com.alibaba.jstorm.utils.DisruptorRunable; - -//import com.alibaba.jstorm.message.zeroMq.ISendConnection; - -/** - * Message dispatcher - * - * @author yannian/Longda - * - */ -public class VirtualPortDispatch extends DisruptorRunable { - private final static Logger LOG = Logger - .getLogger(VirtualPortDispatch.class); - - private ConcurrentHashMap deserializeQueues; - private IConnection recvConnection; - - private static JStormTimer timer = Metrics.registerTimer(null, - MetricDef.DISPATCH_TIME, null, Metrics.MetricType.WORKER); - - public VirtualPortDispatch(WorkerData workerData, - IConnection recvConnection, DisruptorQueue recvQueue) { - super(recvQueue, timer, VirtualPortDispatch.class.getSimpleName(), - workerData.getActive()); - - this.recvConnection = recvConnection; - this.deserializeQueues = workerData.getDeserializeQueues(); - - Metrics.registerQueue(null, MetricDef.DISPATCH_QUEUE, queue, null, Metrics.MetricType.WORKER); - } - - public void cleanup() { - LOG.info("Begin to shutdown VirtualPortDispatch"); - // don't need send shutdown command to every task - // due to every task has been shutdown by workerData.active - // at the same time queue has been fulll -// byte shutdownCmd[] = { TaskStatus.SHUTDOWN }; -// for (DisruptorQueue queue : deserializeQueues.values()) { -// -// queue.publish(shutdownCmd); -// } - - try { - recvConnection.close(); - }catch(Exception e) { - - } - recvConnection = null; - Metrics.unregister(null, MetricDef.DISPATCH_QUEUE, null, Metrics.MetricType.WORKER); - LOG.info("Successfully shudown VirtualPortDispatch"); - } - - @Override - public void handleEvent(Object event, boolean endOfBatch) - throws Exception { - TaskMessage message = (TaskMessage) event; - - int task = message.task(); - - DisruptorQueue queue = deserializeQueues.get(task); - if (queue == null) { - LOG.warn("Received invalid message directed at port " + task - + ". Dropping..."); - return; - } - - queue.publish(message.message()); - - } - -} 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 deleted file mode 100644 index e80579e26..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/Worker.java +++ /dev/null @@ -1,478 +0,0 @@ -package com.alibaba.jstorm.daemon.worker; - -import java.io.BufferedReader; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -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 org.apache.commons.lang.StringUtils; -import org.apache.log4j.Logger; - -import backtype.storm.Config; -import backtype.storm.generated.Grouping; -import backtype.storm.generated.StormTopology; -import backtype.storm.messaging.IConnection; -import backtype.storm.messaging.IContext; -import backtype.storm.task.TopologyContext; -import backtype.storm.utils.DisruptorQueue; -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.callback.AsyncLoopThread; -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.StormConfig; -import com.alibaba.jstorm.daemon.worker.hearbeat.SyncContainerHb; -import com.alibaba.jstorm.daemon.worker.hearbeat.WorkerHeartbeatRunable; -import com.alibaba.jstorm.daemon.worker.metrics.MetricReporter; -import com.alibaba.jstorm.task.Task; -import com.alibaba.jstorm.task.TaskShutdownDameon; -import com.alibaba.jstorm.task.heartbeat.TaskHeartbeatRunable; -import com.alibaba.jstorm.utils.JStormServerUtils; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.PathUtils; -import com.lmax.disruptor.WaitStrategy; -import com.lmax.disruptor.dsl.ProducerType; - -/** - * worker entrance - * - * @author yannian/Longda - * - */ -public class Worker { - - private static Logger LOG = Logger.getLogger(Worker.class); - - /** - * Why need workerData, it is for thread comeptition - */ - private WorkerData workerData; - - @SuppressWarnings({ "rawtypes", "unchecked" }) - public Worker(Map conf, IContext context, String topology_id, - String supervisor_id, int port, String worker_id, String jar_path) - throws Exception { - - workerData = new WorkerData(conf, context, topology_id, supervisor_id, - port, worker_id, jar_path); - - } - - /** - * get current task's output task list - * - * @param tasks_component - * @param mk_topology_context - * @param task_ids - * @throws Exception - */ - public Set worker_output_tasks() { - - ContextMaker context_maker = workerData.getContextMaker(); - Set task_ids = workerData.getTaskids(); - StormTopology topology = workerData.getSysTopology(); - - Set rtn = new HashSet(); - - for (Integer taskid : task_ids) { - TopologyContext context = context_maker.makeTopologyContext( - topology, taskid, null); - - // > - Map> targets = context - .getThisTargets(); - for (Map e : targets.values()) { - for (String componentId : e.keySet()) { - List tasks = context - .getComponentTasks(componentId); - rtn.addAll(tasks); - } - } - } - - return rtn; - } - - private RefreshConnections makeRefreshConnections() { - - // get output streams of every task - Set outboundTasks = worker_output_tasks(); - - workerData.initOutboundTaskStatus(outboundTasks); - - RefreshConnections refresh_connections = new RefreshConnections( - workerData, outboundTasks); - - return refresh_connections; - } - - private List createTasks() throws Exception { - List shutdowntasks = new ArrayList(); - - Set taskids = workerData.getTaskids(); - - for (int taskid : taskids) { - - TaskShutdownDameon t = Task.mk_task(workerData, taskid); - - shutdowntasks.add(t); - } - - return shutdowntasks; - } - - private AsyncLoopThread startDispatchThread() { - Map stormConf = workerData.getStormConf(); - - int queue_size = Utils.getInt( - stormConf.get(Config.TOPOLOGY_TRANSFER_BUFFER_SIZE), 1024); - WaitStrategy waitStrategy = (WaitStrategy) Utils - .newInstance((String) stormConf - .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); - DisruptorQueue recvQueue = DisruptorQueue.mkInstance("Dispatch", ProducerType.MULTI, - queue_size, waitStrategy); - // stop consumerStarted - //recvQueue.consumerStarted(); - - IContext context = workerData.getContext(); - String topologyId = workerData.getTopologyId(); - - IConnection recvConnection = context.bind(topologyId, - workerData.getPort()); - recvConnection.registerQueue(recvQueue); - - RunnableCallback recvDispather = new VirtualPortDispatch(workerData, - recvConnection, recvQueue); - - AsyncLoopThread vthread = new AsyncLoopThread(recvDispather, false, - Thread.MAX_PRIORITY, false); - - return vthread; - } - - public WorkerShutdown execute() throws Exception { - List threads = new ArrayList(); - - AsyncLoopThread dispatcher = startDispatchThread(); - threads.add(dispatcher); - - // create client before create task - // so create client connection before create task - // refresh connection - RefreshConnections refreshConn = makeRefreshConnections(); - AsyncLoopThread refreshconn = new AsyncLoopThread(refreshConn, false, - Thread.MIN_PRIORITY, true); - threads.add(refreshconn); - - // refresh ZK active status - RefreshActive refreshZkActive = new RefreshActive(workerData); - AsyncLoopThread refreshzk = new AsyncLoopThread(refreshZkActive, false, - Thread.MIN_PRIORITY, true); - threads.add(refreshzk); - - BatchTupleRunable batchRunable = new BatchTupleRunable(workerData); - AsyncLoopThread batch = new AsyncLoopThread(batchRunable, false, - Thread.MAX_PRIORITY, true); - threads.add(batch); - - // transferQueue, nodeportSocket, taskNodeport - DrainerRunable drainer = new DrainerRunable(workerData); - AsyncLoopThread dr = new AsyncLoopThread(drainer, false, - Thread.MAX_PRIORITY, true); - threads.add(dr); - - // Sync heartbeat to Apsara Container - AsyncLoopThread syncContainerHbThread = SyncContainerHb - .mkWorkerInstance(workerData.getConf()); - if (syncContainerHbThread != null) { - threads.add(syncContainerHbThread); - } - - MetricReporter metricReporter = workerData.getMetricsReporter(); - boolean isMetricsEnable = ConfigExtension - .isEnablePerformanceMetrics(workerData.getStormConf()); - metricReporter.setEnable(isMetricsEnable); - metricReporter.start(); - LOG.info("Start metrics reporter, enable performance metrics: " - + isMetricsEnable); - - // create task heartbeat - TaskHeartbeatRunable taskHB = new TaskHeartbeatRunable(workerData); - AsyncLoopThread taskHBThread = new AsyncLoopThread(taskHB); - threads.add(taskHBThread); - - // refresh hearbeat to Local dir - RunnableCallback heartbeat_fn = new WorkerHeartbeatRunable(workerData); - AsyncLoopThread hb = new AsyncLoopThread(heartbeat_fn, false, null, - Thread.NORM_PRIORITY, true); - threads.add(hb); - - // shutdown task callbacks - List shutdowntasks = createTasks(); - workerData.setShutdownTasks(shutdowntasks); - - // start dispatcher - dispatcher.start(); - - return new WorkerShutdown(workerData, shutdowntasks, threads, metricReporter); - - } - - /** - * create worker instance and run it - * - * @param conf - * @param mq_context - * @param topology_id - * @param supervisor_id - * @param port - * @param worker_id - * @return - * @throws Exception - */ - @SuppressWarnings("rawtypes") - public static WorkerShutdown mk_worker(Map conf, IContext context, - String topology_id, String supervisor_id, int port, - String worker_id, String jar_path) throws Exception { - - StringBuilder sb = new StringBuilder(); - sb.append("topologyId:" + topology_id + ", "); - sb.append("port:" + port + ", "); - sb.append("workerId:" + worker_id + ", "); - sb.append("jarPath:" + jar_path + "\n"); - - LOG.info("Begin to run worker:" + sb.toString()); - - Worker w = new Worker(conf, context, topology_id, supervisor_id, port, - worker_id, jar_path); - - w.redirectOutput(); - - return w.execute(); - } - - public void redirectOutput(){ - - if (System.getenv("REDIRECT") == null - || !System.getenv("REDIRECT").equals("true")) { - return; - } - - String DEFAULT_OUT_TARGET_FILE = JStormUtils.getLogFileName(); - if (DEFAULT_OUT_TARGET_FILE == null) { - DEFAULT_OUT_TARGET_FILE = "/dev/null"; - } else { - DEFAULT_OUT_TARGET_FILE += ".out"; - } - - String outputFile = ConfigExtension.getWorkerRedirectOutputFile(workerData.getStormConf()); - if (outputFile == null) { - outputFile = DEFAULT_OUT_TARGET_FILE; - }else { - try { - File file = new File(outputFile); - if (file.exists() == false) { - PathUtils.touch(outputFile); - }else { - if (file.isDirectory() == true) { - LOG.warn("Failed to write " + outputFile); - outputFile = DEFAULT_OUT_TARGET_FILE; - }else if (file.canWrite() == false) { - LOG.warn("Failed to write " + outputFile); - outputFile = DEFAULT_OUT_TARGET_FILE; - } - } - - }catch(Exception e) { - LOG.warn("Failed to touch " + outputFile, e); - outputFile = DEFAULT_OUT_TARGET_FILE; - } - } - - try { - JStormUtils.redirectOutput(outputFile); - }catch(Exception e) { - LOG.warn("Failed to redirect to " + outputFile, e); - } - - } - - /** - * Have one problem if the worker's start parameter length is longer than - * 4096, ps -ef|grep com.alibaba.jstorm.daemon.worker.Worker can't find - * worker - * - * @param port - */ - - public static List getOldPortPids(String port) { - String currPid = JStormUtils.process_pid(); - - List ret = new ArrayList(); - - StringBuilder sb = new StringBuilder(); - - sb.append("ps -Af "); - // sb.append(" | grep "); - // sb.append(Worker.class.getName()); - // sb.append(" |grep "); - // sb.append(port); - // sb.append(" |grep -v grep"); - - try { - LOG.info("Begin to execute " + sb.toString()); - Process process = JStormUtils.launch_process(sb.toString(), - new HashMap(), false); - - // Process process = Runtime.getRuntime().exec(sb.toString()); - - InputStream stdin = process.getInputStream(); - BufferedReader reader = new BufferedReader(new InputStreamReader( - stdin)); - - JStormUtils.sleepMs(1000); - - // if (process.exitValue() != 0) { - // LOG.info("Failed to execute " + sb.toString()); - // return null; - // } - - String str; - while ((str = reader.readLine()) != null) { - if (StringUtils.isBlank(str)) { - // LOG.info(str + " is Blank"); - continue; - } - - // LOG.info("Output:" + str); - if (str.contains(Worker.class.getName()) == false) { - continue; - } else if (str.contains(port) == false) { - continue; - } - - LOG.info("Find :" + str); - - String[] fields = StringUtils.split(str); - - boolean find = false; - int i = 0; - for (; i < fields.length; i++) { - String field = fields[i]; - LOG.debug("Filed, " + i+ ":" + field); - - if (field.contains(Worker.class.getName()) == true) { - if (i + 3 >= fields.length) { - LOG.info("Failed to find port "); - - }else if (fields[i + 3].equals(String.valueOf(port))) { - find = true; - } - - break; - } - } - - if (find == false) { - LOG.info("No old port worker"); - continue; - } - - if (fields.length >= 2) { - try { - if (currPid.equals(fields[1])) { - LOG.info("Skip kill myself"); - continue; - } - - Integer pid = Integer.valueOf(fields[1]); - - LOG.info("Find one process :" + pid.toString()); - ret.add(pid); - } catch (Exception e) { - LOG.error(e.getMessage(), e); - continue; - } - } - - } - - return ret; - } catch (IOException e) { - LOG.info("Failed to execute " + sb.toString()); - return ret; - } catch (Exception e) { - LOG.info(e.getCause(), e); - return ret; - } - } - - public static void killOldWorker(String port) { - - List oldPids = getOldPortPids(port); - for (Integer pid : oldPids) { - - JStormUtils.kill(pid); - } - - } - - /** - * worker entrance - * - * @param args - */ - @SuppressWarnings("rawtypes") - public static void main(String[] args) { - if (args.length < 5) { - StringBuilder sb = new StringBuilder(); - sb.append("The length of args is less than 5 "); - for (String arg : args) { - sb.append(arg + " "); - } - LOG.error(sb.toString()); - System.exit(-1); - } - - StringBuilder sb = new StringBuilder(); - - 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]; - - killOldWorker(port_str); - - Map conf = Utils.readStormConfig(); - StormConfig.validate_distributed_mode(conf); - - 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, - jar_path); - sd.join(); - - LOG.info("Successfully shutdown worker " + sb.toString()); - } catch (Throwable e) { - String errMsg = "Failed to create worker, " + sb.toString(); - LOG.error(errMsg, e); - JStormUtils.halt_process(-1, errMsg); - } - } - -} 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 deleted file mode 100644 index bd2bc4612..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerData.java +++ /dev/null @@ -1,459 +0,0 @@ -package com.alibaba.jstorm.daemon.worker; - -import java.net.URL; -import java.security.InvalidParameterException; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -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; -import org.apache.log4j.Logger; - -import backtype.storm.Config; -import backtype.storm.generated.StormTopology; -import backtype.storm.messaging.IConnection; -import backtype.storm.messaging.IContext; -import backtype.storm.messaging.TransportFactory; -import backtype.storm.scheduler.WorkerSlot; -import backtype.storm.utils.DisruptorQueue; -import backtype.storm.utils.Utils; -import backtype.storm.utils.WorkerClassLoader; - -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.Cluster; -import com.alibaba.jstorm.cluster.ClusterState; -import com.alibaba.jstorm.cluster.Common; -import com.alibaba.jstorm.cluster.StormClusterState; -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; -import com.alibaba.jstorm.zk.ZkTool; -import com.lmax.disruptor.WaitStrategy; -import com.lmax.disruptor.dsl.ProducerType; - -public class WorkerData { - private static Logger LOG = Logger.getLogger(WorkerData.class); - - // system configuration - - private Map conf; - // worker configuration - - private Map stormConf; - - // message queue - private IContext context; - - private final String topologyId; - private final String supervisorId; - private final Integer port; - private final String workerId; - // worker status :active/shutdown - private AtomicBoolean active; - - // Topology status - private StatusType topologyStatus; - - // ZK interface - private ClusterState zkClusterstate; - private StormClusterState zkCluster; - - // running taskId list in current worker - private Set taskids; - - // connection to other workers - private ConcurrentHashMap nodeportSocket; - // - private ConcurrentHashMap taskNodeport; - - private ConcurrentSkipListSet workerToResource; - - private Set localNodeTasks; - - private ConcurrentHashMap innerTaskTransfer; - private ConcurrentHashMap deserializeQueues; - - // - private HashMap tasksToComponent; - - private Map> componentToSortedTasks; - - private Map defaultResources; - private Map userResources; - private Map executorData; - private Map registeredMetrics; - - // raw topology is deserialized from local jar - // it doesn't contain acker - private StormTopology rawTopology; - // sys topology is the running topology in the worker - // it contain ackers - private StormTopology sysTopology; - - private ContextMaker contextMaker; - - // shutdown woker entrance - private final WorkerHaltRunable workHalt = new WorkerHaltRunable(); - - // sending tuple's queue - // private LinkedBlockingQueue transferQueue; - private DisruptorQueue transferQueue; - - private DisruptorQueue sendingQueue; - - 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, - String supervisor_id, int port, String worker_id, String jar_path) - throws Exception { - - this.conf = conf; - this.context = context; - this.topologyId = topology_id; - this.supervisorId = supervisor_id; - this.port = port; - this.workerId = worker_id; - - this.active = new AtomicBoolean(true); - this.topologyStatus = StatusType.active; - - if (StormConfig.cluster_mode(conf).equals("distributed")) { - String pidDir = StormConfig.worker_pids_root(conf, worker_id); - JStormServerUtils.createPid(pidDir); - } - - // create zk interface - this.zkClusterstate = ZkTool.mk_distributed_cluster_state(conf); - this.zkCluster = Cluster.mk_storm_cluster_state(zkClusterstate); - - Map rawConf = StormConfig.read_supervisor_topology_conf(conf, - topology_id); - this.stormConf = new HashMap(); - this.stormConf.putAll(conf); - this.stormConf.putAll(rawConf); - - 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(); - for (String path : paths) { - if (StringUtils.isBlank(path)) - continue; - URL url = new URL("File:" + path); - urls.add(url); - } - 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); - throw new InvalidParameterException(); - } - - if (this.context == null) { - this.context = TransportFactory.makeContext(stormConf); - } - - boolean disruptorUseSleep = ConfigExtension - .isDisruptorUseSleep(stormConf); - DisruptorQueue.setUseSleep(disruptorUseSleep); - boolean isLimited = ConfigExtension.getTopologyBufferSizeLimited(stormConf); - DisruptorQueue.setLimited(isLimited); - LOG.info("Disruptor use sleep:" + disruptorUseSleep + ", limited size:" + isLimited); - - // this.transferQueue = new LinkedBlockingQueue(); - int buffer_size = Utils.getInt(conf - .get(Config.TOPOLOGY_TRANSFER_BUFFER_SIZE)); - WaitStrategy waitStrategy = (WaitStrategy) Utils - .newInstance((String) conf - .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); - this.transferQueue = DisruptorQueue.mkInstance("TotalTransfer", ProducerType.MULTI, - buffer_size, waitStrategy); - this.transferQueue.consumerStarted(); - this.sendingQueue = DisruptorQueue.mkInstance("TotalSending", ProducerType.MULTI, - buffer_size, waitStrategy); - this.sendingQueue.consumerStarted(); - - - this.nodeportSocket = new ConcurrentHashMap(); - this.taskNodeport = new ConcurrentHashMap(); - this.workerToResource = new ConcurrentSkipListSet(); - this.innerTaskTransfer = new ConcurrentHashMap(); - this.deserializeQueues = new ConcurrentHashMap(); - - Assignment assignment = zkCluster.assignment_info(topologyId, null); - if (assignment == null) { - String errMsg = "Failed to get Assignment of " + topologyId; - LOG.error(errMsg); - throw new RuntimeException(errMsg); - } - workerToResource.addAll(assignment.getWorkers()); - - // get current worker's task list - - this.taskids = assignment.getCurrentWorkerTasks(supervisorId, port); - if (taskids.size() == 0) { - throw new RuntimeException("No tasks running current workers"); - } - LOG.info("Current worker taskList:" + taskids); - - // deserialize topology code from local dir - rawTopology = StormConfig.read_supervisor_topology_code(conf, - topology_id); - sysTopology = Common.system_topology(stormConf, rawTopology); - - generateMaps(); - - 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"); - - } - - /** - * private ConcurrentHashMap taskNodeport; private - * HashMap tasksToComponent; private Map> componentToSortedTasks; private Map> componentToStreamToFields; private Map - * defaultResources; private Map userResources; private - * Map executorData; private Map registeredMetrics; - * - * @throws Exception - */ - private void generateMaps() throws Exception { - this.tasksToComponent = Cluster.topology_task_info(zkCluster, - topologyId); - LOG.info("Map:" + tasksToComponent); - - this.componentToSortedTasks = JStormUtils.reverse_map(tasksToComponent); - for (java.util.Map.Entry> entry : componentToSortedTasks - .entrySet()) { - List tasks = entry.getValue(); - - Collections.sort(tasks); - } - - this.defaultResources = new HashMap(); - this.userResources = new HashMap(); - this.executorData = new HashMap(); - this.registeredMetrics = new HashMap(); - } - - public Map getConf() { - return conf; - } - - public AtomicBoolean getActive() { - return active; - } - - public void setActive(AtomicBoolean active) { - this.active = active; - } - - public StatusType getTopologyStatus() { - return topologyStatus; - } - - public void setTopologyStatus(StatusType topologyStatus) { - this.topologyStatus = topologyStatus; - } - - public Map getStormConf() { - return stormConf; - } - - public IContext getContext() { - return context; - } - - public String getTopologyId() { - return topologyId; - } - - public String getSupervisorId() { - return supervisorId; - } - - public Integer getPort() { - return port; - } - - public String getWorkerId() { - return workerId; - } - - public ClusterState getZkClusterstate() { - return zkClusterstate; - } - - public StormClusterState getZkCluster() { - return zkCluster; - } - - public Set getTaskids() { - return taskids; - } - - public ConcurrentHashMap getNodeportSocket() { - return nodeportSocket; - } - - public ConcurrentHashMap getTaskNodeport() { - return taskNodeport; - } - - public ConcurrentSkipListSet getWorkerToResource() { - return workerToResource; - } - - public ConcurrentHashMap getInnerTaskTransfer() { - return innerTaskTransfer; - } - - public ConcurrentHashMap getDeserializeQueues() { - return deserializeQueues; - } - - public HashMap getTasksToComponent() { - return tasksToComponent; - } - - public StormTopology getRawTopology() { - return rawTopology; - } - - public StormTopology getSysTopology() { - return sysTopology; - } - - public ContextMaker getContextMaker() { - return contextMaker; - } - - public WorkerHaltRunable getWorkHalt() { - return workHalt; - } - - public DisruptorQueue getTransferQueue() { - return transferQueue; - } - - // public LinkedBlockingQueue getTransferQueue() { - // return transferQueue; - // } - - public DisruptorQueue getSendingQueue() { - return sendingQueue; - } - - public Map> getComponentToSortedTasks() { - return componentToSortedTasks; - } - - public Map getDefaultResources() { - return defaultResources; - } - - public Map getUserResources() { - return userResources; - } - - public Map getExecutorData() { - return executorData; - } - - public Map getRegisteredMetrics() { - return registeredMetrics; - } - - public List getShutdownTasks() { - return shutdownTasks; - } - - public void setShutdownTasks(List shutdownTasks) { - this.shutdownTasks = shutdownTasks; - } - - public Set getLocalNodeTasks() { - return localNodeTasks; - } - - public void setLocalNodeTasks(Set localNodeTasks) { - this.localNodeTasks = localNodeTasks; - } - - public void setMetricsReporter(MetricReporter reporter) { - this.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/WorkerHaltRunable.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerHaltRunable.java deleted file mode 100644 index c1a8b541f..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerHaltRunable.java +++ /dev/null @@ -1,13 +0,0 @@ -package com.alibaba.jstorm.daemon.worker; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.utils.JStormUtils; - -public class WorkerHaltRunable extends RunnableCallback { - - @Override - public void run() { - JStormUtils.halt_process(1, "Task died"); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerHeartbeat.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerHeartbeat.java deleted file mode 100644 index 282782811..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerHeartbeat.java +++ /dev/null @@ -1,72 +0,0 @@ -package com.alibaba.jstorm.daemon.worker; - -import java.io.Serializable; -import java.util.HashSet; -import java.util.Set; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; - -/** - * Worker's Heartbeat data woker will update the object to - * /LOCAL-DIR/workers/${woker-id}/heartbeats - * - * @author yannian/Longda - * - */ -public class WorkerHeartbeat implements Serializable { - - private static final long serialVersionUID = -914166726205534892L; - private int timeSecs; - private String topologyId; - private Set taskIds; - private Integer port; - - public WorkerHeartbeat(int timeSecs, String topologyId, - Set taskIds, Integer port) { - - this.timeSecs = timeSecs; - this.topologyId = topologyId; - this.taskIds = new HashSet(taskIds); - this.port = port; - - } - - public int getTimeSecs() { - return timeSecs; - } - - public void setTimeSecs(int timeSecs) { - this.timeSecs = timeSecs; - } - - public String getTopologyId() { - return topologyId; - } - - public void setTopologyId(String topologyId) { - this.topologyId = topologyId; - } - - public Set getTaskIds() { - return taskIds; - } - - public void setTaskIds(Set taskIds) { - this.taskIds = new HashSet(taskIds); - } - - public Integer getPort() { - return port; - } - - public void setPort(Integer port) { - this.port = port; - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this, - ToStringStyle.SHORT_PREFIX_STYLE); - } -} 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 deleted file mode 100644 index f19a9c328..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerMetricInfo.java +++ /dev/null @@ -1,156 +0,0 @@ -package com.alibaba.jstorm.daemon.worker; - -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.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.codahale.metrics.Timer; - - -/** - * /storm-zk-root/Monitor/{topologyid}/{workerid} data - */ -public class WorkerMetricInfo implements Serializable { - private static Logger LOG = Logger.getLogger(WorkerMetricInfo.class); - - private static final long serialVersionUID = 7745327094257659471L; - - private String hostName; - private Integer port; - - private long usedMem; - private double usedCpu; - - private Map gaugeData; - private Map counterData; - private Map meterData; - private Map timerData; - private Map histogramData; - - private static final String METRIC_SEPERATOR = "-"; - - public WorkerMetricInfo(String hostName, Integer port) { - this.hostName = hostName; - this.port = port; - - this.gaugeData = new HashMap(); - this.counterData = new HashMap(); - this.meterData = new HashMap(); - this.timerData = new HashMap(); - this.histogramData = new HashMap(); - } - - public void setHostName(String hostName) { - this.hostName = hostName; - } - - public String getHostName() { - return this.hostName; - } - - public void setPort(Integer port) { - this.port = port; - } - - public Integer getPort() { - return this.port; - } - - public void setUsedMem(long usedMem) { - this.usedMem = usedMem; - } - - public long getUsedMem() { - return this.usedMem; - } - - public void setUsedCpu(double usedCpu) { - this.usedCpu = usedCpu; - } - - public double getUsedCpu() { - return this.usedCpu; - } - - public Map getGaugeData() { - return gaugeData; - } - - public Map getCounterData() { - return counterData; - } - - public Map getMeterData() { - return meterData; - } - - public Map getTimerData() { - return timerData; - } - - public Map getHistogramData() { - return histogramData; - } - - // There are some metrics that have same metric name, but just different prefix. - // e.g for netty_send_time, full metric name is dest_ip:port-name - // The metrics with same metric name will be sum here. - public void updateMetricData(MetricInfo metricInfo) { - String name = metricInfo.getName(); - Metric metric = metricInfo.getMetric(); - LOG.debug("Metric name=" + name); - if (metric instanceof QueueGauge) { - //covert to % - float queueRatio = (((QueueGauge) metric).getValue())*100; - sum(gaugeData, name, (double)queueRatio); - } else if (metric instanceof Gauge) { - Double value = JStormUtils.convertToDouble(((Gauge) metric).getValue()); - if (value == null) { - LOG.warn("gauge value is null or unknow type."); - } else { - sum(gaugeData, name, value); - } - } else if (metric instanceof Timer) { - Snapshot snapshot = ((Timer) metric).getSnapshot(); - //covert from ns to ms - sum(timerData, name, (snapshot.getMean())/1000000); - } else if (metric instanceof Counter) { - Double value = ((Long) ((Counter) metric).getCount()).doubleValue(); - sum(counterData, name, value); - } else if (metric instanceof Meter) { - sum(meterData, name, ((Meter) metric).getMeanRate()); - } else if (metric instanceof Histogram) { - Snapshot snapshot = ((Histogram)metric).getSnapshot(); - sum(histogramData, name, snapshot.getMean()); - } else { - LOG.warn("Unknown metric type, name:" + name); - } - } - - private void sum(Map dataMap, String name, Double value) { - Double currentValue = dataMap.get(name); - if (currentValue != null) - value = value + currentValue; - 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 deleted file mode 100644 index c2d2e216c..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/WorkerShutdown.java +++ /dev/null @@ -1,166 +0,0 @@ -package com.alibaba.jstorm.daemon.worker; - -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; - -import backtype.storm.messaging.IConnection; -import backtype.storm.messaging.IContext; -import backtype.storm.scheduler.WorkerSlot; - -import com.alibaba.jstorm.callback.AsyncLoopThread; -import com.alibaba.jstorm.cluster.ClusterState; -import com.alibaba.jstorm.cluster.StormClusterState; -import com.alibaba.jstorm.daemon.worker.metrics.MetricReporter; -import com.alibaba.jstorm.task.ShutdownableDameon; -import com.alibaba.jstorm.task.TaskShutdownDameon; -import com.alibaba.jstorm.utils.JStormUtils; - -/** - * Shutdown worker - * - * @author yannian/Longda - * - */ -public class WorkerShutdown implements ShutdownableDameon { - private static Logger LOG = Logger.getLogger(WorkerShutdown.class); - - public static final String HOOK_SIGNAL = "USR2"; - - private List shutdowntasks; - private AtomicBoolean active; - private ConcurrentHashMap nodeportSocket; - private IContext context; - private List threads; - private StormClusterState zkCluster; - private ClusterState cluster_state; - private MetricReporter metricReporter; - private ScheduledExecutorService threadPool; - - // active nodeportSocket context zkCluster zkClusterstate - public WorkerShutdown(WorkerData workerData, - List _shutdowntasks, - List _threads, MetricReporter metricReporter) { - - this.shutdowntasks = _shutdowntasks; - this.threads = _threads; - - this.active = workerData.getActive(); - this.nodeportSocket = workerData.getNodeportSocket(); - 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)); - - // PreCleanupTasks preCleanupTasks = new PreCleanupTasks(); - // // install signals - // Signal sig = new Signal(HOOK_SIGNAL); - // Signal.handle(sig, preCleanupTasks); - } - - @Override - public void shutdown() { - - if (active.getAndSet(false) == false) { - LOG.info("Worker has been shutdown already"); - return; - } - - threadPool.shutdown(); - metricReporter.shutdown(); - - // shutdown tasks - for (ShutdownableDameon task : shutdowntasks) { - task.shutdown(); - } - - // shutdown worker's demon thread - // refreshconn, refreshzk, hb, drainer - for (AsyncLoopThread t : threads) { - LOG.info("Begin to shutdown " + t.getThread().getName()); - t.cleanup(); - JStormUtils.sleepMs(100); - t.interrupt(); - // try { - // t.join(); - // } catch (InterruptedException e) { - // LOG.error("join thread", e); - // } - LOG.info("Successfully " + t.getThread().getName()); - } - - // send data to close connection - for (WorkerSlot k : nodeportSocket.keySet()) { - IConnection value = nodeportSocket.get(k); - value.close(); - } - - context.term(); - - // close ZK client - try { - zkCluster.disconnect(); - cluster_state.close(); - } catch (Exception e) { - // TODO Auto-generated catch block - LOG.info("Shutdown error,", e); - } - - JStormUtils.halt_process(0, "!!!Shutdown!!!"); - } - - public void join() throws InterruptedException { - for (TaskShutdownDameon task : shutdowntasks) { - task.join(); - } - for (AsyncLoopThread t : threads) { - t.join(); - } - - } - - public boolean waiting() { - Boolean isExistsWait = false; - for (ShutdownableDameon task : shutdowntasks) { - if (task.waiting()) { - isExistsWait = true; - break; - } - } - for (AsyncLoopThread thr : threads) { - if (thr.isSleeping()) { - isExistsWait = true; - break; - } - } - return isExistsWait; - } - - @Override - public void run() { - // TODO Auto-generated method stub - shutdown(); - } - - // class PreCleanupTasks implements SignalHandler { - // - // @Override - // public void handle(Signal arg0) { - // LOG.info("Receive " + arg0.getName() + ", begin to do pre_cleanup job"); - // - // for (ShutdownableDameon task : shutdowntasks) { - // task.shutdown(); - // } - // - // LOG.info("Successfully do pre_cleanup job"); - // } - // - // } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/hearbeat/SyncContainerHb.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/hearbeat/SyncContainerHb.java deleted file mode 100644 index f51a064e4..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/hearbeat/SyncContainerHb.java +++ /dev/null @@ -1,361 +0,0 @@ -package com.alibaba.jstorm.daemon.worker.hearbeat; - -import java.io.File; -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -import org.apache.commons.lang.StringUtils; -import org.apache.log4j.Logger; - -import com.alibaba.jstorm.callback.AsyncLoopThread; -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.StormConfig; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.PathUtils; - -public class SyncContainerHb extends RunnableCallback { - private final static Logger LOG = Logger.getLogger(SyncContainerHb.class); - - private String readDir; - private String writeDir; - private int timeoutSeconds = 60; - private int frequence = 10; - private int reserverNum = 10; - private int noContainerHbTimes = 0; - private boolean isFirstRead = true; - private static final int SECOND_MILLISCOND = 1000; - private static final int MAX_NO_CONTAINER_HB_TIMES = 30; - - public void removeOld(List fileList, String dir) { - if (fileList.size() <= reserverNum) { - // don't need remove old files - return; - } - - int removeNum = fileList.size() - reserverNum; - for (int i = 0; i < removeNum; i++) { - String fileName = fileList.get(i); - - String filePath = dir + File.separator + fileName; - try { - PathUtils.rmpath(filePath); - }catch (Exception e) { - LOG.error("Failed to delete " + filePath, e); - } - LOG.info("Remove hearbeat file " + filePath); - } - } - - public void checkNoContainerHbTimes() { - noContainerHbTimes++; - if (noContainerHbTimes >= MAX_NO_CONTAINER_HB_TIMES) { - LOG.info("It's long time no container hearbeat"); - throw new RuntimeException("It's long time no container hearbeat"); - } - } - - public void handlReadDir() { - if (StringUtils.isBlank(readDir) == true) { - return; - } - - File file = new File(readDir); - if (file.exists() == false) { - LOG.info(readDir + " doesn't exist right now"); - checkNoContainerHbTimes(); - return; - } else if (file.isDirectory() == false) { - String msg = readDir + " isn't dir"; - LOG.error(msg); - throw new RuntimeException(msg); - } - - String[] files = file.list(); - if (files.length == 0) { - LOG.info(readDir + " doesn't contain hearbeat files right now"); - checkNoContainerHbTimes(); - return; - } - - noContainerHbTimes = 0; - List fileList = JStormUtils.mk_list(files); - Collections.sort(fileList); - - // removeOld(fileList); - - String biggest = fileList.get(fileList.size() - 1); - - long now = System.currentTimeMillis() / SECOND_MILLISCOND; - long hb = 0; - - try { - hb = Long.valueOf(biggest); - } catch (Exception e) { - LOG.info("Heartbeat file " + biggest - + " isn't a valid file, remove it"); - - String path = readDir + File.separator + biggest; - try { - PathUtils.rmpath(path); - }catch (Exception e1) { - LOG.error("Failed to delete " + path, e1); - } - - } - - if (now - hb > timeoutSeconds) { - if (isFirstRead == true) { - checkNoContainerHbTimes(); - return; - } - - StringBuilder sb = new StringBuilder(); - - sb.append("It's long time no container's hearbeat, "); - sb.append("ContainerDir:").append(readDir); - sb.append(",last hearbeat:").append(biggest); - LOG.error(sb.toString()); - - throw new RuntimeException(sb.toString()); - } else { - isFirstRead = false; - LOG.info("Receive container hearbeat " + biggest); - } - - } - - public void handleWriteDir() { - if (StringUtils.isBlank(writeDir) == true) { - return; - } - - String seconds = String.valueOf(System.currentTimeMillis() - / SECOND_MILLISCOND); - - String path = writeDir + File.separator + seconds; - - try { - PathUtils.touch(path); - LOG.info("Successfully touch " + path); - } catch (IOException e) { - LOG.error("Failed to touch " + path, e); - throw new RuntimeException("Failed to touch " + path); - } - - File file = new File(writeDir); - String[] files = file.list(); - if (files.length == 0) { - LOG.info(readDir + " doesn't contain hearbeat files right now"); - return; - } - - List fileList = JStormUtils.mk_list(files); - Collections.sort(fileList); - - removeOld(fileList, writeDir); - - } - - @Override - public void run() { - - handleWriteDir(); - - handlReadDir(); - - } - - @Override - public void cleanup() { - frequence = -1; - LOG.info("Shutdown sync container thread"); - } - - public Object getResult() { - return frequence; - } - - public String getReadDir() { - return readDir; - } - - public void resetReadHeatbeats() { - File file = new File(readDir); - - if (file.exists() == false) { - LOG.info("Read hearbeat directory hasn't been created " + readDir); - return; - } else if (file.isDirectory() == false) { - LOG.error(readDir + " isn't a directory "); - throw new RuntimeException(readDir + " isn't a directory "); - } - - String[] files = file.list(); - for (String fileName : files) { - String path = readDir + File.separator + fileName; - - try { - PathUtils.rmr(path); - } catch (IOException e) { - // TODO Auto-generated catch block - LOG.error("Failed to remove " + path, e); - } - } - - LOG.info("Successfully reset read heatbeats " + readDir); - } - - public void setReadDir(String readDir) { - this.readDir = readDir; - if (StringUtils.isBlank(readDir) == true) { - LOG.warn("ReadDir is empty"); - return ; - } else { - LOG.info("ReadDir is " + readDir); - } - - } - - public int getTimeoutSeconds() { - return timeoutSeconds; - } - - public void setTimeoutSeconds(int timeoutSeconds) { - this.timeoutSeconds = timeoutSeconds; - } - - public int getFrequence() { - return frequence; - } - - public void setFrequence(int frequence) { - this.frequence = frequence; - } - - public String getWriteDir() { - return writeDir; - } - - public void setWriteDir(String writeDir) { - this.writeDir = writeDir; - if (StringUtils.isBlank(writeDir) == true) { - LOG.warn("writeDir is empty"); - return; - } else { - LOG.info("writeDir is " + writeDir); - } - - File file = new File(writeDir); - - if (file.exists() == false) { - file.mkdirs(); - LOG.info("Create Directory " + writeDir); - return; - } else if (file.isDirectory() == false) { - LOG.error(writeDir + " isn't a directory "); - throw new RuntimeException(writeDir + " isn't a directory "); - } - - return; - } - - public int getReserverNum() { - return reserverNum; - } - - public void setReserverNum(int reserverNum) { - this.reserverNum = reserverNum; - } - - public static AsyncLoopThread mkInstance(String containerHbDir, - String hbDir, int timeout, int frequence) { - SyncContainerHb syncContainerHbThread = new SyncContainerHb(); - - syncContainerHbThread.setReadDir(containerHbDir); - syncContainerHbThread.setWriteDir(hbDir); - syncContainerHbThread.setTimeoutSeconds(timeout); - syncContainerHbThread.setFrequence(frequence); - - StringBuilder sb = new StringBuilder(); - sb.append("Run process under Apsara/Yarn container"); - sb.append("ContainerDir:").append(containerHbDir); - sb.append("MyDir:").append(hbDir); - sb.append(", timeout:").append(timeout); - sb.append(",frequence:").append(frequence); - LOG.info(sb.toString()); - - AsyncLoopThread thread = new AsyncLoopThread(syncContainerHbThread, - true, Thread.NORM_PRIORITY, true); - - return thread; - } - - public static AsyncLoopThread mkNimbusInstance(Map conf) throws IOException { - boolean isEnable = ConfigExtension.isEnableContainerNimbus(); - if (isEnable == false) { - LOG.info("Run nimbus without Apsara/Yarn container"); - return null; - } - - String containerHbDir = ConfigExtension.getContainerNimbusHearbeat(); - String hbDir = StormConfig.masterHearbeatForContainer(conf); - int timeout = ConfigExtension.getContainerHeartbeatTimeoutSeconds(conf); - int frequence = ConfigExtension.getContainerHeartbeatFrequence(conf); - - return mkInstance(containerHbDir, hbDir, timeout, frequence); - - } - - public static AsyncLoopThread mkSupervisorInstance(Map conf) - throws IOException { - boolean isEnableContainer = ConfigExtension.isEnableContainerSupervisor(); - if (isEnableContainer) { - String containerHbDir = ConfigExtension - .getContainerSupervisorHearbeat(); - String hbDir = StormConfig.supervisorHearbeatForContainer(conf); - int timeout = ConfigExtension.getContainerHeartbeatTimeoutSeconds(conf); - int frequence = ConfigExtension.getContainerHeartbeatFrequence(conf); - - return mkInstance(containerHbDir, hbDir, timeout, frequence); - } - - boolean isWorkerAutomaticStop = ConfigExtension.isWorkerStopWithoutSupervisor(conf); - if (isWorkerAutomaticStop) { - String containerHbDir = null; - String hbDir = StormConfig.supervisorHearbeatForContainer(conf); - int timeout = ConfigExtension.getContainerHeartbeatTimeoutSeconds(conf); - int frequence = ConfigExtension.getContainerHeartbeatFrequence(conf); - - return mkInstance(containerHbDir, hbDir, timeout, frequence); - } - - - LOG.info("Run Supervisor without Apsara/Yarn container"); - return null; - - } - - public static AsyncLoopThread mkWorkerInstance(Map conf) - throws IOException { - boolean isEnableContainer = ConfigExtension.isEnableContainerSupervisor(); - boolean isWorkerAutomaticStop = ConfigExtension.isWorkerStopWithoutSupervisor(conf); - if (isEnableContainer == false && isWorkerAutomaticStop == false) { - LOG.info("Run worker without Apsara/Yarn container"); - return null; - } - - String containerHbDir = StormConfig.supervisorHearbeatForContainer(conf); - String hbDir = null; - int timeout = ConfigExtension.getContainerHeartbeatTimeoutSeconds(conf); - int frequence = ConfigExtension.getContainerHeartbeatFrequence(conf); - - return mkInstance(containerHbDir, hbDir, timeout, frequence); - - - - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/hearbeat/WorkerHeartbeatRunable.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/hearbeat/WorkerHeartbeatRunable.java deleted file mode 100644 index df1b7261a..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/hearbeat/WorkerHeartbeatRunable.java +++ /dev/null @@ -1,101 +0,0 @@ -package com.alibaba.jstorm.daemon.worker.hearbeat; - -import java.io.IOException; -import java.util.Map; -import java.util.concurrent.CopyOnWriteArraySet; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; -import backtype.storm.utils.LocalState; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.cluster.Common; -import com.alibaba.jstorm.cluster.StormConfig; -import com.alibaba.jstorm.daemon.worker.WorkerData; -import com.alibaba.jstorm.daemon.worker.WorkerHeartbeat; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.TimeUtils; - -/** - * worker Heartbeat - * - * @author yannian/Longda - * - */ -public class WorkerHeartbeatRunable extends RunnableCallback { - private static Logger LOG = Logger.getLogger(WorkerHeartbeatRunable.class); - - private WorkerData workerData; - - private AtomicBoolean active; - private Map conf; - private String worker_id; - private Integer port; - private String topologyId; - private CopyOnWriteArraySet task_ids; - // private Object lock = new Object(); - - private Integer frequence; - - public WorkerHeartbeatRunable(WorkerData workerData) { - - this.workerData = workerData; - - this.conf = workerData.getConf(); - this.worker_id = workerData.getWorkerId(); - this.port = workerData.getPort(); - this.topologyId = workerData.getTopologyId(); - this.task_ids = new CopyOnWriteArraySet( - workerData.getTaskids()); - this.active = workerData.getActive(); - - String key = Config.WORKER_HEARTBEAT_FREQUENCY_SECS; - frequence = JStormUtils.parseInt(conf.get(key), 10); - } - - /** - * do hearbeat, update LocalState - * - * @throws IOException - */ - - public void doHeartbeat() throws IOException { - - int currtime = TimeUtils.current_time_secs(); - WorkerHeartbeat hb = new WorkerHeartbeat(currtime, topologyId, - task_ids, port); - - LOG.debug("Doing heartbeat:" + worker_id + ",port:" + port + ",hb" - + hb.toString()); - - LocalState state = StormConfig.worker_state(conf, worker_id); - state.put(Common.LS_WORKER_HEARTBEAT, hb); - - } - - @Override - public void run() { - - if (active.get() == false) { - return; - - } - try { - doHeartbeat(); - } catch (IOException e) { - LOG.error("work_heart_beat_fn fail", e); - throw new RuntimeException(e); - } - - } - - @Override - public Object getResult() { - if (this.active.get()) { - return frequence; - } - return -1; - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/AlimonitorClient.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/AlimonitorClient.java deleted file mode 100644 index b3b23e582..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/AlimonitorClient.java +++ /dev/null @@ -1,267 +0,0 @@ -package com.alibaba.jstorm.daemon.worker.metrics; - -import java.net.URLEncoder; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.http.HttpEntity; -import org.apache.http.NameValuePair; -import org.apache.http.client.entity.UrlEncodedFormEntity; -import org.apache.http.client.methods.CloseableHttpResponse; -import org.apache.http.client.methods.HttpGet; -import org.apache.http.client.methods.HttpPost; -import org.apache.http.impl.client.CloseableHttpClient; -import org.apache.http.impl.client.HttpClientBuilder; -import org.apache.http.message.BasicNameValuePair; -import org.apache.http.util.EntityUtils; -import org.apache.log4j.Logger; - -import backtype.storm.utils.Utils; - - - -public class AlimonitorClient extends MetricSendClient { - - public static Logger LOG = Logger.getLogger(AlimonitorClient.class); - - // Send to localhost:15776 by default - public static final String DEFAUT_ADDR = "127.0.0.1"; - public static final String DEFAULT_PORT = "15776"; - public static final int DEFAUTL_FLAG = 0; - public static final String DEFAULT_ERROR_INFO = ""; - - private final String COLLECTION_FLAG = "collection_flag"; - private final String ERROR_INFO = "error_info"; - private final String MSG = "MSG"; - - private String port; - private String requestIP; - private String monitorName; - private int collectionFlag; - private String errorInfo; - - private boolean post; - - public AlimonitorClient() { - } - - public AlimonitorClient(String requestIP, String port, boolean post) { - this.requestIP = requestIP; - this.port = port; - this.post = post; - this.monitorName = null; - this.collectionFlag = 0; - this.errorInfo = null; - } - - public void setIpAddr(String ipAddr) { - this.requestIP = ipAddr; - } - - public void setPort(String port) { - this.port = port; - } - - public void setMonitorName(String monitorName) { - this.monitorName = monitorName; - } - - public void setCollectionFlag(int flag) { - this.collectionFlag = flag; - } - - public void setErrorInfo(String msg) { - this.errorInfo = msg; - } - - public void setPostFlag(boolean post) { - this.post = post; - } - - public String buildURL() { - return "http://" + requestIP + ":" + port + "/passive"; - } - - public String buildRqstAddr() { - return "http://" + requestIP + ":" + port + "/passive?name=" + monitorName + "&msg="; - } - - @Override - public boolean send(Map msg) { - try { - if(monitorName == null) { - LOG.warn("monitor name is null"); - return false; - } - return sendRequest(collectionFlag, errorInfo, msg); - } catch (Exception e) { - LOG.error("Failed to sendRequest", e); - return false; - } - } - - @Override - public boolean send(List> msg) { - try { - if(monitorName == null) { - LOG.warn("monitor name is null"); - return false; - } - return sendRequest(collectionFlag, errorInfo, msg); - } catch (Exception e) { - LOG.error("Failed to sendRequest", e); - return false; - } - } - - public Map buildAliMonitorMsg(int collection_flag, String error_message) { - // Json format of the message sent to Alimonitor - // { - // "collection_flag":int, - // "error_info":string, - // "MSG": ojbect | array - // } - Map ret = new HashMap(); - ret.put(COLLECTION_FLAG, collection_flag); - ret.put(ERROR_INFO, error_message); - ret.put(MSG, null); - - return ret; - } - - private void addMsgData(Map jsonObj, Map map) { - jsonObj.put(MSG, map); - } - - private void addMsgData(Map jsonObj, List> mapList) { -// JSONArray jsonArray = new JSONArray(); -// for(Map map : mapList) { -// jsonArray.add(map); -// } - - jsonObj.put(MSG, mapList); - } - - private boolean sendRequest(int collection_flag, String error_message, - Map msg) throws Exception { - boolean ret = false; - - if (msg.size() == 0) return ret; - - Map jsonObj = buildAliMonitorMsg(collection_flag, error_message); - addMsgData(jsonObj, msg); - String jsonMsg = jsonObj.toString(); - LOG.info(jsonMsg); - - if (post == true) { - String url = buildURL(); - ret = httpPost(url, jsonMsg); - } else { - String request = buildRqstAddr(); - StringBuilder postAddr= new StringBuilder(); - postAddr.append(request); - postAddr.append(URLEncoder.encode(jsonMsg)); - - ret = httpGet(postAddr); - } - - return ret; - } - - private boolean sendRequest(int collection_flag, String error_message, - List> msgList) throws Exception { - boolean ret = false; - - if (msgList.size() == 0) return ret; - - Map jsonObj = buildAliMonitorMsg(collection_flag, error_message); - addMsgData(jsonObj, msgList); - - String jsonMsg = Utils.to_json(jsonObj); - LOG.info(jsonMsg); - - if (post == true) { - String url = buildURL(); - ret = httpPost(url, jsonMsg); - } else { - String request = buildRqstAddr(); - StringBuilder postAddr= new StringBuilder(); - postAddr.append(request); - postAddr.append(URLEncoder.encode(jsonMsg)); - - ret = httpGet(postAddr); - } - - return ret; - } - - private boolean httpGet(StringBuilder postAddr) { - boolean ret = false; - - CloseableHttpClient httpClient = HttpClientBuilder.create().build(); - CloseableHttpResponse response = null; - - try { - HttpGet request = new HttpGet(postAddr.toString()); - response = httpClient.execute(request); - HttpEntity entity = response.getEntity(); - if (entity != null) { - LOG.info(EntityUtils.toString(entity)); - } - EntityUtils.consume(entity); - ret = true; - } catch (Exception e) { - LOG.error("Exception when sending http request to alimonitor", e); - } finally { - try { - if (response != null) - response.close(); - httpClient.close(); - } catch (Exception e) { - LOG.error("Exception when closing httpclient", e); - } - } - - return ret; - } - - private boolean httpPost(String url, String msg) { - boolean ret = false; - - CloseableHttpClient httpClient = HttpClientBuilder.create().build(); - CloseableHttpResponse response = null; - - try { - HttpPost request = new HttpPost(url); - List nvps = new ArrayList (); - nvps.add(new BasicNameValuePair("name", monitorName)); - nvps.add(new BasicNameValuePair("msg", msg)); - request.setEntity(new UrlEncodedFormEntity(nvps)); - response = httpClient.execute(request); - HttpEntity entity = response.getEntity(); - if (entity != null) { - LOG.info(EntityUtils.toString(entity)); - } - EntityUtils.consume(entity); - ret = true; - } catch (Exception e) { - LOG.error("Exception when sending http request to alimonitor", e); - } finally { - try { - if (response != null) - response.close(); - httpClient.close(); - } catch (Exception e) { - LOG.error("Exception when closing httpclient", e); - } - } - - return ret; - } - - public void close() { - } -} - 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 deleted file mode 100644 index abaf59710..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/MetricKVMsg.java +++ /dev/null @@ -1,254 +0,0 @@ -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 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); - - public enum MetricType{ - count, min, max, mean, median, stddev, p75, p95, p98, p99, p999, mean_rate, m1, m5, m15 - } - - 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> timerMapKV = new HashMap>(); // count data of Timer - private Map> meterMapKV = new HashMap>(); // count data of Meter - private Map countMap = new HashMap(); - - public Map convertToKVMap() { - Map ret = new HashMap(); - - addGaugeToKVMap(ret); - addCounterToKVMap(ret); - addHistogramToKVMap(ret); - addTimerToKVMap(ret); - addMeterToKVMap(ret); - - return ret; - } - - public void addGaugeToKVMap(Map kVMap) { - for (Entry entry : gaugeMapKV.entrySet()) { - kVMap.put(entry.getKey(), entry.getValue()); - } - } - - public void addCounterToKVMap(Map kVMap) { - for (Entry entry : counterMapKV.entrySet()) { - kVMap.put(entry.getKey(), entry.getValue()); - } - } - - public void addHistogramToKVMap(Map kVMap) { - for (Entry> entry : histogramMapKV.entrySet()) { - String name = entry.getKey(); - Map typeMap = entry.getValue(); - - for (Entry typeEntry : typeMap.entrySet()) { - kVMap.put(name+ "_" + typeEntry.getKey().toString(), typeEntry.getValue()); - } - } - } - - public void addTimerToKVMap(Map kVMap) { - for (Entry> entry : timerMapKV.entrySet()) { - String name = entry.getKey(); - Map typeMap = entry.getValue(); - - for (Entry typeEntry : typeMap.entrySet()) { - kVMap.put(name+ "_" + typeEntry.getKey().toString(), typeEntry.getValue()); - } - } - } - - public void addMeterToKVMap(Map kVMap) { - for (Entry> entry : meterMapKV.entrySet()) { - String name = entry.getKey(); - Map typeMap = entry.getValue(); - - for (Entry typeEntry : typeMap.entrySet()) { - kVMap.put(name+ "_" + typeEntry.getKey().toString(), typeEntry.getValue()); - } - } - } - - public void countGangeMetric(Map gaugeMap ){ - //count value of Gauge - for(Entry entry : gaugeMap.entrySet()){ - String taskMetricName = entry.getKey(); - String userDefName = taskMetricName.substring(taskMetricName.indexOf(":") + 1); - Double value = entry.getValue().getValue(); - if(gaugeMapKV.containsKey(userDefName)){ - value = value + gaugeMapKV.get(userDefName); - } - gaugeMapKV.put(userDefName, value); - } - } - - public void countCounterMetric(Map counterMap){ - for (Entry entry : counterMap.entrySet()) { - String taskMetricName = entry.getKey(); - String userDefName = taskMetricName.substring(taskMetricName.indexOf(":") + 1); - Long value = entry.getValue().getValue(); - if(counterMapKV.containsKey(userDefName)){ - value = value + counterMapKV.get(userDefName); - } - counterMapKV.put(userDefName, value); - } - } - - public void countHistogramMetric(Map histogramMap){ - //only count: minValue, maxValue ,aveValue - for (Entry entry : histogramMap.entrySet()) { - String taskMetricName = entry.getKey(); - String userDefName = taskMetricName.substring(taskMetricName.indexOf(":") + 1); - - double maxValue = entry.getValue().getMax(); - double minValue = entry.getValue().getMin(); - double meanValue = entry.getValue().getMean(); - - Map temMap = histogramMapKV.get(userDefName); - if(temMap == null){ - 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.min)); - meanValue += (temMap.get(MetricType.mean) == null ? 0.0 : temMap.get(MetricType.mean)); - - temMap.put(MetricType.max, maxValue); - temMap.put(MetricType.min, minValue); - temMap.put(MetricType.mean, meanValue); - } - } - - public void countTimerMetric(Map timerMap){ - //only count: mean time - for(Entry entry:timerMap.entrySet()){ - String taskMetricName = entry.getKey(); - String userDefName = taskMetricName.substring(taskMetricName.indexOf(":") + 1); - double meanValue = (double)entry.getValue().getMean(); - - Map temMap = timerMapKV.get(userDefName); - if (temMap == null) { - temMap = new HashMap(); - 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); - } - } - - public void calcAvgTimer() { - for (Entry> entry: timerMapKV.entrySet()) { - String userDefName = entry.getKey(); - Map valueMap = entry.getValue(); - Integer count = countMap.get(userDefName); - 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; - valueMap.put(MetricType.mean, convertDurationFromNsToMs(meanValue)); - } - } - - public void countMeterMetric(Map meterMap){ - //only count: meanRate - for(Entry entry:meterMap.entrySet()){ - String taskMetricName = entry.getKey(); - String userDefName = taskMetricName.substring(taskMetricName.indexOf(":")+1); - - Double meanRate = entry.getValue().getMeanRate(); - Map temMap = meterMapKV.get(userDefName); - if (temMap == null) { - temMap = new HashMap(); - meterMapKV.put(userDefName, temMap); - } - - meanRate += (temMap.get(MetricType.mean) == null ? 0.0 : temMap.get(MetricType.mean)); - temMap.put(MetricType.mean, meanRate); - meterMapKV.put(userDefName, temMap); - } - } - - public Map> getTimerKVMap() { - return this.timerMapKV; - } - - public void emptyCountMap() { - countMap.clear(); - } - - 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/MetricReporter.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/MetricReporter.java deleted file mode 100644 index d6e37a869..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/MetricReporter.java +++ /dev/null @@ -1,135 +0,0 @@ -package com.alibaba.jstorm.daemon.worker.metrics; - -import java.util.Random; -import java.util.concurrent.TimeUnit; - -import org.slf4j.LoggerFactory; - -import com.codahale.metrics.ConsoleReporter; -import com.codahale.metrics.Gauge; -import com.codahale.metrics.Meter; -import com.codahale.metrics.Slf4jReporter; -import com.codahale.metrics.Snapshot; -import com.codahale.metrics.Timer; -import com.alibaba.jstorm.daemon.worker.WorkerData; -import com.alibaba.jstorm.metric.JStormHistogram; -import com.alibaba.jstorm.metric.JStormTimer; -import com.alibaba.jstorm.metric.Metrics; - -public class MetricReporter { - - final StormMetricReporter reporter1Minute; - final Slf4jReporter reporter10Minute; - - WorkerData workerData; - - private boolean isEnable; - - public MetricReporter(WorkerData workerData) { - this.workerData = workerData; - - reporter1Minute = StormMetricReporter.forRegistry(Metrics.getMetrics()) - .outputTo(LoggerFactory.getLogger(MetricReporter.class)) - .convertRatesTo(TimeUnit.SECONDS) - .convertDurationsTo(TimeUnit.MILLISECONDS) - .setWorkerData(workerData).build(); - - reporter10Minute = Slf4jReporter.forRegistry(Metrics.getJstack()) - .outputTo(LoggerFactory.getLogger(MetricReporter.class)) - .convertRatesTo(TimeUnit.SECONDS) - .convertDurationsTo(TimeUnit.MILLISECONDS).build(); - - } - - public void start() { - reporter1Minute.start(1, TimeUnit.MINUTES); - reporter10Minute.start(10, TimeUnit.MINUTES); - - } - - public void stop() { - reporter1Minute.stop(); - reporter10Minute.stop(); - - } - - public void shutdown() { - reporter10Minute.close(); - reporter1Minute.close(); - } - - public boolean isEnable() { - return isEnable; - } - - public void setEnable(boolean isEnable) { - this.isEnable = isEnable; - JStormTimer.setEnable(isEnable); - JStormHistogram.setEnable(isEnable); - } - - private static class LatencyRatio implements Gauge { - Timer timer; - - protected LatencyRatio(Timer base) { - timer = base; - } - - @Override - public Double getValue() { - Snapshot snapshot = timer.getSnapshot(); - return snapshot.getMedian() / 1000000; - } - - } - - public static void main(String[] args) { - final Random random = new Random(); - random.setSeed(System.currentTimeMillis()); - - Thread thread = new Thread(new Runnable() { - - final JStormTimer timer = Metrics.registerTimer("timer"); - final Meter meter = Metrics.registerMeter("meter"); - LatencyRatio latency = Metrics.getMetrics().register("latency", new LatencyRatio(timer.getInstance())); - - @Override - public void run() { - System.out.println("Begin to run"); - int counter = 0; - while (counter++ < 40000) { - meter.mark(); - timer.start(); - - int rand = random.nextInt(10); - - try { - Thread.sleep(rand * 1); - } catch (InterruptedException e) { - - } finally { - timer.stop(); - } - - try { - Thread.sleep(2); - } catch (InterruptedException e) { - } - if (counter % 1000 == 0) { - System.out.println("Done " + counter); - } - - } - } - }); - - Metrics.getMetrics().registerAll(Metrics.getJstack()); - final ConsoleReporter reporter = ConsoleReporter - .forRegistry(Metrics.getMetrics()) - .convertRatesTo(TimeUnit.SECONDS) - .convertDurationsTo(TimeUnit.MILLISECONDS).build(); - reporter.start(1, TimeUnit.MINUTES); - - thread.start(); - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/MetricSendClient.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/MetricSendClient.java deleted file mode 100644 index 4027049b0..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/MetricSendClient.java +++ /dev/null @@ -1,18 +0,0 @@ -package com.alibaba.jstorm.daemon.worker.metrics; - -import java.util.Map; -import java.util.List; - -public class MetricSendClient { - - public MetricSendClient() { - } - - public boolean send(Map msg) { - return true; - } - - public boolean send(List> msgList) { - return true; - } -} \ No newline at end of file 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 deleted file mode 100644 index c541ece14..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/StormMetricReporter.java +++ /dev/null @@ -1,433 +0,0 @@ -package com.alibaba.jstorm.daemon.worker.metrics; - -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.HashMap; -import java.util.SortedMap; -import java.util.Map.Entry; -import java.util.concurrent.TimeUnit; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.slf4j.Marker; - -import com.codahale.metrics.Counter; -import com.codahale.metrics.CsvReporter; -import com.codahale.metrics.Gauge; -import com.codahale.metrics.Histogram; -import com.codahale.metrics.Meter; -import com.codahale.metrics.Metric; -import com.codahale.metrics.MetricFilter; -import com.codahale.metrics.MetricRegistry; -import com.codahale.metrics.Snapshot; -import com.codahale.metrics.Timer; -import com.codahale.metrics.ScheduledReporter; -import com.esotericsoftware.minlog.Log; -import com.alibaba.jstorm.daemon.worker.WorkerData; -import com.alibaba.jstorm.daemon.worker.WorkerMetricInfo; -import com.alibaba.jstorm.client.metric.MetricCallback; -import com.alibaba.jstorm.cluster.StormClusterState; -import com.alibaba.jstorm.container.SystemOperation; -import com.alibaba.jstorm.metric.MetricInfo; -import com.alibaba.jstorm.metric.Metrics; -import com.alibaba.jstorm.metric.UserDefMetric; -import com.alibaba.jstorm.metric.UserDefMetricData; -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 { - /** - * Returns a new {@link Builder} for {@link StormMetricReporter}. - * - * @param registry the registry to report - * @return a {@link Builder} instance for a {@link StormMetricReporter} - */ - public static Builder forRegistry(MetricRegistry registry) { - return new Builder(registry); - } - - /** - * A builder for {@link CsvReporter} instances. Defaults to logging to {@code metrics}, not - * using a marker, converting rates to events/second, converting durations to milliseconds, and - * not filtering metrics. - */ - public static class Builder { - private final MetricRegistry registry; - private Logger logger; - private Marker marker; - private TimeUnit rateUnit; - private TimeUnit durationUnit; - private MetricFilter filter; - private WorkerData workerData; - - private Builder(MetricRegistry registry) { - this.registry = registry; - this.logger = LoggerFactory.getLogger("metrics"); - this.marker = null; - this.rateUnit = TimeUnit.SECONDS; - this.durationUnit = TimeUnit.MILLISECONDS; - this.filter = MetricFilter.ALL; - this.workerData = null; - } - - /** - * Log metrics to the given logger. - * - * @param logger an SLF4J {@link Logger} - * @return {@code this} - */ - public Builder outputTo(Logger logger) { - this.logger = logger; - return this; - } - - /** - * Mark all logged metrics with the given marker. - * - * @param marker an SLF4J {@link Marker} - * @return {@code this} - */ - public Builder markWith(Marker marker) { - this.marker = marker; - return this; - } - - /** - * Convert rates to the given time unit. - * - * @param rateUnit a unit of time - * @return {@code this} - */ - public Builder convertRatesTo(TimeUnit rateUnit) { - this.rateUnit = rateUnit; - return this; - } - - /** - * Convert durations to the given time unit. - * - * @param durationUnit a unit of time - * @return {@code this} - */ - public Builder convertDurationsTo(TimeUnit durationUnit) { - this.durationUnit = durationUnit; - return this; - } - - /** - * Only report metrics which match the given filter. - * - * @param filter a {@link MetricFilter} - * @return {@code this} - */ - public Builder filter(MetricFilter filter) { - this.filter = filter; - return this; - } - - public Builder setWorkerData(WorkerData Data) { - this.workerData = Data; - return this; - } - /** - * Builds a {@link StormMetricReporter} with the given properties. - * - * @return a {@link StormMetricReporter} - */ - public StormMetricReporter build() { - return new StormMetricReporter(registry, logger, marker, rateUnit, - durationUnit, filter, workerData); - } - } - - private final Logger logger; - private final Marker marker; - private WorkerData workerData; - - private StormMetricReporter(MetricRegistry registry, - Logger logger, - Marker marker, - TimeUnit rateUnit, - TimeUnit durationUnit, - MetricFilter filter, - WorkerData workerData) { - super(registry, "logger-reporter", filter, rateUnit, durationUnit); - this.logger = logger; - this.marker = marker; - this.workerData = workerData; - } - - @Override - public void report(SortedMap gauges, - SortedMap counters, - SortedMap histograms, - SortedMap meters, - SortedMap timers) { - boolean metricPerf = workerData.getMetricsReporter().isEnable(); - - processMetricData(gauges, counters, histograms, meters, timers, metricPerf); - - // update internal metrics data of jstorm task and worker, - // and user define metrics data to ZK - updateMetricsDataToZK(metricPerf); - } - - private void doCallback(Map metrics) { - Map callbacks = Metrics.getUserDefMetric().getCallbacks(); - String name = ""; - try{ - for (Entry entry : metrics.entrySet()) { - name = entry.getKey(); - MetricCallback callback = callbacks.get(entry.getValue()); - if (callback != null) - callback.callback(entry.getValue()); - } - } catch (Exception e) { - logger.error("Error when excuting the callbacks defined by user. CallBack Name=" + name, e); - } - } - - private void processMetricData(SortedMap gauges, - SortedMap counters, - SortedMap histograms, - SortedMap meters, - SortedMap timers, - boolean metricPerf) { - UserDefMetric userDefMetric = Metrics.getUserDefMetric(); - - for (Entry entry : gauges.entrySet()) { - logGauge(entry.getKey(), entry.getValue()); - } - - for (Entry entry : counters.entrySet()) { - logCounter(entry.getKey(), entry.getValue()); - } - - for (Entry entry : meters.entrySet()) { - logMeter(entry.getKey(), entry.getValue()); - } - - if (metricPerf == true) { - for (Entry entry : histograms.entrySet()) { - logHistogram(entry.getKey(), entry.getValue()); - } - - for (Entry entry : timers.entrySet()) { - Map timerMap = userDefMetric.getTimer(); - logTimer(entry.getKey(), entry.getValue()); - } - } - - } - - private void logTimer(String name, Timer timer) { - final Snapshot snapshot = timer.getSnapshot(); - - logger.info(marker, - "type=TIMER, name={}, count={}, min={}, max={}, mean={}, stddev={}, median={}, " + - "p75={}, p95={}, p98={}, p99={}, p999={}, mean_rate={}, m1={}, m5={}, " + - "m15={}, rate_unit={}, duration_unit={}", - name, - timer.getCount(), - convertDuration(snapshot.getMin()), - convertDuration(snapshot.getMax()), - convertDuration(snapshot.getMean()), - convertDuration(snapshot.getStdDev()), - convertDuration(snapshot.getMedian()), - convertDuration(snapshot.get75thPercentile()), - convertDuration(snapshot.get95thPercentile()), - convertDuration(snapshot.get98thPercentile()), - convertDuration(snapshot.get99thPercentile()), - convertDuration(snapshot.get999thPercentile()), - convertRate(timer.getMeanRate()), - convertRate(timer.getOneMinuteRate()), - convertRate(timer.getFiveMinuteRate()), - convertRate(timer.getFifteenMinuteRate()), - getRateUnit(), - getDurationUnit()); - - } - - private void logMeter(String name, Meter meter) { - logger.info(marker, - "type=METER, name={}, count={}, mean_rate={}, m1={}, m5={}, m15={}, rate_unit={}", - name, - meter.getCount(), - convertRate(meter.getMeanRate()), - convertRate(meter.getOneMinuteRate()), - convertRate(meter.getFiveMinuteRate()), - convertRate(meter.getFifteenMinuteRate()), - getRateUnit()); - } - - private void logHistogram(String name, Histogram histogram) { - final Snapshot snapshot = histogram.getSnapshot(); - logger.info(marker, - "type=HISTOGRAM, name={}, count={}, min={}, max={}, mean={}, stddev={}, " + - "median={}, p75={}, p95={}, p98={}, p99={}, p999={}", - name, - histogram.getCount(), - snapshot.getMin(), - snapshot.getMax(), - snapshot.getMean(), - snapshot.getStdDev(), - snapshot.getMedian(), - snapshot.get75thPercentile(), - snapshot.get95thPercentile(), - snapshot.get98thPercentile(), - snapshot.get99thPercentile(), - snapshot.get999thPercentile()); - } - - private void logCounter(String name, Counter counter) { - logger.info(marker, "type=COUNTER, name={}, count={}", name, counter.getCount()); - } - - private void logGauge(String name, Gauge gauge) { - logger.info(marker, "type=GAUGE, name={}, value={}", name, gauge.getValue()); - } - - @Override - protected String getRateUnit() { - return "events/" + super.getRateUnit(); - } - - private void updateMetricsDataToZK(boolean metricPerf) { - Map> taskMetricMap = Metrics.getTaskMetricMap(); - List workerMetricList = Metrics.getWorkerMetricList(); - - updateTaskMetricsToZK(taskMetricMap, metricPerf); - updateWorkerMetricsToZK(workerMetricList, metricPerf); - updateUserDefMetricsToZK(metricPerf); - } - - private void updateTaskMetricsToZK(Map> metricMap, boolean metricPerf) { - StormClusterState clusterState = workerData.getZkCluster(); - String topologyId = workerData.getTopologyId(); - - for(Entry> entry : metricMap.entrySet()) { - String taskId = entry.getKey(); - List MetricList = entry.getValue(); - - try { - 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); - } - - List errors = taskMetricInfo.anyQueueFull(); - if (errors.size() > 0) { - for (String error : errors) - clusterState.report_task_error(topologyId, Integer.valueOf(taskId), error); - } - - clusterState.update_task_metric(topologyId, taskId, taskMetricInfo); - } catch(Exception e) { - logger.error(marker, "Failed to update metrics data in ZK for topo-{} task-{}.", - topologyId, taskId, e); - } - } - } - - public Double getCpuUsage() { - Double value = 0.0; - String output = null; - try { - String pid = JStormUtils.process_pid(); - output = SystemOperation.exec("top -b -n 1 | grep " + pid); - String subStr = output.substring(output.indexOf("S") + 1); - for(int i = 0; i < subStr.length(); i++) { - char ch = subStr.charAt(i); - if (ch != ' ') { - subStr = subStr.substring(i); - break; - } - } - String usedCpu = subStr.substring(0, subStr.indexOf(" ")); - value = Double.valueOf(usedCpu); - } catch (Exception e) { - logger.warn("Failed to get cpu usage ratio."); - if (output != null) - logger.warn("Output string is \"" + output + "\""); - value = 0.0; - } - - return value; - } - - private void updateWorkerMetricsToZK(List metricList, boolean metricPerf) { - StormClusterState clusterState = workerData.getZkCluster(); - String topologyId = workerData.getTopologyId(); - String hostName; - - hostName = NetWorkUtils.ip(); - String workerId = hostName + ":" + workerData.getPort(); - - WorkerMetricInfo workerMetricInfo = new WorkerMetricInfo(hostName, workerData.getPort()); - try { - //Set metrics data - for(MetricInfo metricInfo : metricList) { - if(metricPerf == false && ((metricInfo.getMetric() instanceof Timer) || - (metricInfo.getMetric() instanceof Histogram))) - continue; - workerMetricInfo.updateMetricData(metricInfo); - } - - //Set cpu & memory usage - Runtime rt=Runtime.getRuntime(); - long usedMem = rt.totalMemory() - rt.freeMemory(); - workerMetricInfo.setUsedMem(usedMem); - - workerMetricInfo.setUsedCpu(getCpuUsage()); - - clusterState.update_worker_metric(topologyId, workerId, workerMetricInfo); - } catch(Exception e) { - logger.error(marker, "Failed to update metrics data in ZK for topo-{} idStr-{}.", - topologyId, workerId, e); - } - } - - private void updateUserDefMetricsToZK(boolean metricPerf) { - StormClusterState clusterState = workerData.getZkCluster(); - String topologyId = workerData.getTopologyId(); - String hostName =JStormServerUtils.getHostName(workerData.getConf()); - String workerId = hostName + ":" + workerData.getPort(); - - UserDefMetric userDefMetric = Metrics.getUserDefMetric(); - UserDefMetricData userDefMetricData = new UserDefMetricData(); - userDefMetricData.updateFromGauge(userDefMetric.getGauge()); - userDefMetricData.updateFromCounter(userDefMetric.getCounter()); - 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) { - userDefMetricData.updateFromHistogramData(userDefMetric.getHistogram()); - userDefMetricData.updateFromTimerData(userDefMetric.getTimer()); - } - - try { - clusterState.update_userDef_metric(topologyId, workerId, userDefMetricData); - } catch(Exception e) { - logger.error(marker, "Failed to update user define metrics data in ZK for topo-{} idStr-{}.", - topologyId, workerId, e); - } - - //Do callbacks defined by user - doCallback(userDefMetric.getGauge()); - doCallback(userDefMetric.getCounter()); - doCallback(userDefMetric.getMeter()); - if (metricPerf) { - doCallback(userDefMetric.getHistogram()); - doCallback(userDefMetric.getTimer()); - } - } - -} \ No newline at end of file diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/TopoCommStatsInfo.java b/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/TopoCommStatsInfo.java deleted file mode 100644 index 165f02f9f..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/TopoCommStatsInfo.java +++ /dev/null @@ -1,229 +0,0 @@ -package com.alibaba.jstorm.daemon.worker.metrics; - -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.Map; -import java.util.Map.Entry; - -import org.apache.log4j.Logger; - -import com.alibaba.jstorm.task.heartbeat.TaskHeartbeat; -import com.alibaba.jstorm.stats.CommonStatsData; - -public class TopoCommStatsInfo { - private static final Logger LOG = Logger.getLogger(TopoCommStatsInfo.class); - - private String topologyId; - private String topologyName; - // Map> - private Map> spoutHbMap; - private Map> boltHbMap; - - //*********** Statistic data ****************// - // Topology Data - private CommStatsData topoStatsData; - // Spout Data - private Map spoutStatsDataMap; - // Bolt Data - private Map boltStatsDataMap; - - public TopoCommStatsInfo(String topologyId, String topologyName) { - this.topologyId = topologyId; - this.topologyName = topologyName; - topoStatsData = new CommStatsData(); - spoutHbMap = new HashMap>(); - boltHbMap = new HashMap>(); - spoutStatsDataMap = new HashMap(); - boltStatsDataMap = new HashMap(); - } - - public String getTopoId() { - return topologyId; - } - - public String getTopoName() { - return topologyName; - } - - public Map> getSpoutList() { - return spoutHbMap; - } - - public Map> getBoltList() { - return boltHbMap; - } - - public CommStatsData getTopoStatsData() { - return topoStatsData; - } - - public Map getSpoutStatsData() { - return spoutStatsDataMap; - } - - public Map getBoltStatsData() { - return boltStatsDataMap; - } - - public void addToSpoutList(String componentId, String taskId, TaskHeartbeat taskHb) { - Map taskMap = spoutHbMap.get(componentId); - if (taskMap == null) { - taskMap = new HashMap(); - spoutHbMap.put(componentId, taskMap); - } - taskMap.put(taskId, taskHb); - } - - public void addToBoltList(String componentId, String taskId, TaskHeartbeat taskHb) { - Map taskMap = boltHbMap.get(componentId); - if (taskMap == null) { - taskMap = new HashMap(); - boltHbMap.put(componentId, taskMap); - } - taskMap.put(taskId, taskHb); - } - - public void buildTopoStatsData() { - topoStatsData.resetData(); - Double latency = 0.0; - for (Entry spoutEntry : spoutStatsDataMap.entrySet()) { - CommStatsData statsData = spoutEntry.getValue(); - topoStatsData.updateSendTps(statsData.getSendTps()); - topoStatsData.updateRecvTps(statsData.getRecvTps()); - topoStatsData.updateFailed(statsData.getFailed()); - latency += statsData.getLatency(); - } - latency = latency/(spoutStatsDataMap.size()); - topoStatsData.updateLatency(latency); - - for (Entry boltEntry : boltStatsDataMap.entrySet()) { - CommStatsData statsData = boltEntry.getValue(); - topoStatsData.updateSendTps(statsData.getSendTps()); - topoStatsData.updateRecvTps(statsData.getRecvTps()); - topoStatsData.updateFailed(statsData.getFailed()); - } - } - - public void buildSpoutStatsData() { - updateStatsData(spoutHbMap, spoutStatsDataMap); - } - - public void buildBoltStatsData() { - updateStatsData(boltHbMap, boltStatsDataMap); - } - - public void updateStatsData(Map> HbMap, Map statsDataMap) { - for (Entry> Entry : HbMap.entrySet()) { - String componentId = Entry.getKey(); - Map compList = Entry.getValue(); - - CommStatsData comStatsData = statsDataMap.get(componentId); - if (comStatsData == null) { - comStatsData = new CommStatsData(); - statsDataMap.put(componentId, comStatsData); - } - comStatsData.resetData(); - - for (Entry compEntry : compList.entrySet()) { - TaskHeartbeat taskHb = compEntry.getValue(); - CommonStatsData statsData = taskHb.getStats(); - comStatsData.updateStatsData(statsData); - } - double avgLatency = (comStatsData.getLatency())/(compList.size()); - comStatsData.updateLatency(avgLatency); - } - } - - - public class CommStatsData { - private static final String TOPOLOGYNAME = "TopologyName"; - private static final String COMPONTENT= "Component"; - private static final String SEND_TPS = "send_tps"; - private static final String RECV_TPS = "recv_tps"; - private static final String FAILED = "failed"; - private static final String LATENCY = "process_latency"; - - private Double sendTps; - private Double recvTps; - private Long failed; - private Double latency; - - public CommStatsData() { - resetData(); - } - - public Double getSendTps() { - return sendTps; - } - - public Double getRecvTps() { - return recvTps; - } - - public Long getFailed() { - return failed; - } - - public Double getLatency() { - return latency; - } - - public void updateSendTps(Double tps) { - sendTps += tps; - } - - public void updateRecvTps(Double tps) { - recvTps += tps; - } - - public void updateFailed(Long fail) { - failed += fail; - } - - public void updateLatency(Double latency) { - this.latency = latency; - } - - public void updateStatsData(CommonStatsData commStatsData) { - sendTps += commStatsData.get_total_send_tps(); - recvTps += commStatsData.get_total_recv_tps(); - failed += commStatsData.get_total_failed(); - latency += commStatsData.get_avg_latency(); - } - - public void updateStatsData(CommStatsData commStatsData) { - sendTps += commStatsData.getSendTps(); - recvTps += commStatsData.getRecvTps(); - failed += commStatsData.getFailed(); - latency += commStatsData.getLatency(); - } - - public void resetData() { - sendTps = 0.0; - recvTps = 0.0; - failed = 0l; - latency = 0.0; - } - - public Map convertToKVMap(String topoloygName,String componentId) { - Map ret = new HashMap(); - ret.put(TOPOLOGYNAME, topoloygName); - ret.put( COMPONTENT, componentId); - ret.put(SEND_TPS, sendTps); - ret.put( RECV_TPS, recvTps); - ret.put(FAILED, failed); - ret.put(LATENCY, latency); - - return ret; - } - - public void printValue() { - LOG.info("send_tps: " + sendTps); - LOG.info("recv_tps: " + recvTps); - LOG.info("failed: " + failed); - LOG.info("latency: " + latency); - } - } -} - - \ No newline at end of file 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 deleted file mode 100644 index 6409834b6..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/UploadMetricFromZK.java +++ /dev/null @@ -1,242 +0,0 @@ -package com.alibaba.jstorm.daemon.worker.metrics; - -import java.util.List; -import java.util.ArrayList; -import java.util.LinkedList; -import java.util.Map; -import java.util.Map.Entry; -import java.util.HashMap; - -import org.apache.log4j.Logger; - -import com.alibaba.jstorm.daemon.nimbus.NimbusData; -import com.alibaba.jstorm.task.heartbeat.TaskHeartbeat; -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.StormBase; -import com.alibaba.jstorm.cluster.StormClusterState; -import com.alibaba.jstorm.stats.CommonStatsData; -import com.alibaba.jstorm.task.TaskInfo; -import com.alibaba.jstorm.metric.MetricDef; -import com.alibaba.jstorm.metric.UserDefMetricData; -import com.codahale.metrics.Gauge; -import com.codahale.metrics.Metric; - -public class UploadMetricFromZK implements Runnable { - - private static final Logger LOG = Logger.getLogger(UploadMetricFromZK.class); - - private NimbusData data; - private StormClusterState clusterState; - - private MetricSendClient client; - - private Map topologyMap; - - public UploadMetricFromZK(NimbusData data, MetricSendClient client) { - this.data = data; - this.client = client; - clusterState = data.getStormClusterState(); - topologyMap = new HashMap(); - } - - @Override - public void run() { - uploadCommStats(); - uploadUseDefMetric(clusterState); - } - - // remove obsolete topology - private boolean rmObsoleteTopo() { - boolean ret = true; - List obsoleteTopos = new ArrayList(); - try { - List topologys = clusterState.active_storms(); - - for (Entry entry : topologyMap.entrySet()) { - if (topologys.contains(entry.getKey()) == false) { - obsoleteTopos.add(entry.getKey()); - } - } - - for (String topo : obsoleteTopos) { - topologyMap.remove(topo); - } - } catch (Exception e) { - LOG.warn("Faild to update topology list.", e); - ret = false; - } - - return ret; - } - - private void uploadCommStats() { - // Remove obsolete topology firstly. new topology will be - // added when uploading the common statistic data - rmObsoleteTopo(); - - List> listMapMsg=new ArrayList>(); - - try { - TopoCommStatsInfo ret; - List topologys = clusterState.heartbeat_storms(); - - for (String topologyId : topologys) { - if (topologyMap.containsKey(topologyId) == false) { - StormBase base = clusterState.storm_base(topologyId, null); - if (base == null) { - topologyMap.remove(topologyId); - continue; - } else { - topologyMap.put(topologyId, new TopoCommStatsInfo(topologyId, base.getStormName())); - } - } - // Get common statistic data from taskbeats in ZK for a topology - ret = getCommStatsData(topologyId); - - if (ret != null) { - // Build topology, spout and bolt statis data from the - // statis data of all tasks - buildCommStatsData(ret); - // Build statistic data message of remote monitor server - buildComonSendMsg(ret,listMapMsg); - - } - } - - if(listMapMsg.size() > 0) { - // Send statistic data to remote monitor server - sendCommStatsData(listMapMsg); - } - } catch (Exception e) { - LOG.warn("Failed to upload comm statistic data to Alimonitor.", e); - } - } - - public void uploadUseDefMetric(StormClusterState clusterState) { - try { - List active_topologys = clusterState.active_storms(); - if (active_topologys == null) { - return; - } - - Map totalMsg = new HashMap(); - - for (String topologyId : active_topologys) { - MetricKVMsg topologyMetricMsg = MetricKVMsg.getMetricKVMsg(topologyId, clusterState); - Map ret = topologyMetricMsg.convertToKVMap(); - if(ret.size() >0) totalMsg.putAll(ret); - } - - if(totalMsg.size() > 0) { - // For Alimonitor Client only - if (client instanceof AlimonitorClient) { - ((AlimonitorClient) client).setMonitorName( - ConfigExtension.getAlmonUserMetricName(data.getConf())); - ((AlimonitorClient) client).setCollectionFlag(0); - ((AlimonitorClient) client).setErrorInfo(""); - } - client.send(totalMsg); - } - } catch (Exception e) { - LOG.warn("Failed to upload user define metric data", e); - } - } - - public void clean() { - - } - - private TopoCommStatsInfo getCommStatsData(String topologyId) { - try - { - String taskId; - String componentId; - TaskHeartbeat taskHb; - - TopoCommStatsInfo commStatsInfo = topologyMap.get(topologyId); - if (commStatsInfo == null) {LOG.warn("commStatsInfo is null, topoId=" + topologyId);} - - Map heartbeats = clusterState.task_heartbeat(topologyId); - if (heartbeats == null || heartbeats.size() == 0) return null; - - for (Entry entry : heartbeats.entrySet()) { - taskId = entry.getKey(); - taskHb = entry.getValue(); - - TaskInfo taskInfo = clusterState.task_info(topologyId, Integer.parseInt(taskId)); - if (taskInfo == null ) { - LOG.warn("Task information can not be found in ZK for task-" + taskId); - continue; - } - componentId = taskInfo.getComponentId(); - - //update taskHb into the corresponding component map - if (taskHb.getComponentType().equals("spout")) { - commStatsInfo.addToSpoutList(componentId, taskId, taskHb); - } else { - commStatsInfo.addToBoltList(componentId, taskId, taskHb); - } - } - - return commStatsInfo; - - } catch (Exception e) { - LOG.warn("getCommStatsData, failed to read data from ZK.", e); - return null; - } - } - - private void buildCommStatsData(TopoCommStatsInfo commStatsInfo) { - commStatsInfo.buildBoltStatsData(); - commStatsInfo.buildSpoutStatsData(); - commStatsInfo.buildTopoStatsData(); - } - - private void sendCommStatsData(List> listMapMsg) { - - try { - // For Alimonitor Client only - if (client instanceof AlimonitorClient) { - ((AlimonitorClient) client).setMonitorName( - ConfigExtension.getAlmonTopoMetricName(data.getConf())); - ((AlimonitorClient) client).setCollectionFlag(0); - ((AlimonitorClient) client).setErrorInfo(""); - } - client.send(listMapMsg); - } catch (Exception e) { - LOG.warn("Error when sending common statistic data.", e); - } - } - - private void buildComonSendMsg(TopoCommStatsInfo commStatsInfo,List> listMapMsg) { - String topoloygName = commStatsInfo.getTopoName(); - - Map jsonMsg; - - try { - //build topology statistic data - TopoCommStatsInfo.CommStatsData topoStatsData = commStatsInfo.getTopoStatsData(); - jsonMsg = topoStatsData.convertToKVMap(topoloygName,topoloygName); - listMapMsg.add(jsonMsg); - //build spout statistic data - Map spoutStatsData = commStatsInfo.getSpoutStatsData(); - for (Entry entry : spoutStatsData.entrySet()) { - String componentId = entry.getKey(); - jsonMsg = entry.getValue().convertToKVMap(topoloygName,componentId); - listMapMsg.add(jsonMsg); - } - - //build bolt statistic data - Map boltStatsData = commStatsInfo.getBoltStatsData(); - for (Entry entry : boltStatsData.entrySet()) { - String componentId = entry.getKey(); - jsonMsg = entry.getValue().convertToKVMap(topoloygName,componentId); - listMapMsg.add(jsonMsg); - } - } catch (Exception e) { - LOG.warn("Error when bulding common statistic data message.", e); - } - } - - -} \ No newline at end of file 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 deleted file mode 100644 index 0b09f2c22..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/metrics/UploadSupervMetric.java +++ /dev/null @@ -1,235 +0,0 @@ -package com.alibaba.jstorm.daemon.worker.metrics; - -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.Map.Entry; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.HashMap; -import java.util.Set; -import java.util.HashSet; - -import org.apache.log4j.Logger; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.StormBase; -import com.alibaba.jstorm.cluster.StormClusterState; -import com.alibaba.jstorm.cluster.StormMonitor; -import com.alibaba.jstorm.daemon.supervisor.Supervisor; -import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; -import com.alibaba.jstorm.daemon.worker.WorkerMetricInfo; -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); - - private AtomicBoolean active; - private Integer result; - private int frequence; - - private Map conf; - private String supervisorId; - private String hostName; - private StormClusterState cluster; - private MetricSendClient client; - - List> jsonMsgTasks = new ArrayList>(); - List> jsonMsgWorkers = new ArrayList>(); - - public UploadSupervMetric(Map conf, StormClusterState cluster, String supervisorId, - AtomicBoolean active, int frequence, MetricSendClient client) { - this.active = active; - this.frequence = frequence; - this.result = null; - this.conf = conf; - this.cluster = cluster; - this.supervisorId = supervisorId; - this.client = client; - try { - SupervisorInfo supervisorInfo = cluster.supervisor_info(supervisorId); - this.hostName = supervisorInfo.getHostName(); - } catch (Exception e) { - LOG.error("Failed to get hostname for supervisorID=" + supervisorId); - } - } - - @Override - public Object getResult() { - return result; - } - - @Override - public void run() { - sendMetricsData();; - if (active.get()) { - this.result = frequence; - } else { - this.result = -1; - - } - } - - - public void sendMetricsData() { - - try { - List topologys = cluster.active_storms(); - - for (String topologyId : topologys) { - StormMonitor monitor = null; - boolean metricPerf = true; - Assignment assignment = null; - - try { - monitor = cluster.get_storm_monitor(topologyId); - if (monitor != null) metricPerf = monitor.getMetrics(); - - assignment = cluster.assignment_info(topologyId, null); - } catch (Exception e) { - LOG.error("Error when retrieving monitor status and assignment info " - + "for " + topologyId, e); - continue; - } - - if (assignment != null) { - Set taskSet = new HashSet(); - Set workerSet = new HashSet(); - //Retrieve task set - Set tempTaskSet = assignment.getCurrentSuperviosrTasks(supervisorId); - taskSet.addAll(tempTaskSet); - - //Retrieve worker set - Set tempWorkerSet = assignment.getCurrentSuperviosrWorkers(supervisorId); - workerSet.addAll(tempWorkerSet); - - //Build KV Map for AliMonitor - buildTaskJsonMsg(topologyId, taskSet, metricPerf); - buildWorkerJsonMsg(topologyId, workerSet, metricPerf); - } - } - - if (jsonMsgTasks.size() != 0) { - if (client instanceof AlimonitorClient) { - ((AlimonitorClient) client).setMonitorName( - ConfigExtension.getAlmonTaskMetricName(conf)); - ((AlimonitorClient) client).setCollectionFlag(0); - ((AlimonitorClient) client).setErrorInfo(""); - } - client.send(jsonMsgTasks); - } - - if (jsonMsgWorkers.size() != 0) { - if (client instanceof AlimonitorClient) { - ((AlimonitorClient) client).setMonitorName( - ConfigExtension.getAlmonWorkerMetricName(conf)); - ((AlimonitorClient) client).setCollectionFlag(0); - ((AlimonitorClient) client).setErrorInfo(""); - } - client.send(jsonMsgWorkers); - } - - jsonMsgTasks.clear(); - jsonMsgWorkers.clear(); - - } catch (Exception e) { - LOG.error("Failed to upload worker&task metrics data", e); - jsonMsgTasks.clear(); - jsonMsgWorkers.clear(); - } - } - - public void buildTaskJsonMsg(String topologyId, Set taskSet, boolean metricPerf) { - for (Integer taskId : taskSet) { - try { - TaskMetricInfo taskMetric = cluster.get_task_metric(topologyId, taskId); - if (taskMetric == null) continue; - - // Task KV structure - Map taskKV = new HashMap(); - taskKV.put("Topology_Name", topologyId); - taskKV.put("Task_Id", String.valueOf(taskId)); - taskKV.put("Component", taskMetric.getComponent()); - taskKV.putAll(taskMetric.getGaugeData()); - taskKV.putAll(taskMetric.getCounterData()); - taskKV.putAll(taskMetric.getMeterData()); - if (metricPerf == true) { - taskKV.putAll(taskMetric.getTimerData()); - 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); - } - } - } - - public void buildWorkerJsonMsg(String topologyId, Set workerSet, boolean metricPerf) { - String workerId = null; - for (Integer port: workerSet) { - try { - workerId = hostName + ":" + port; - WorkerMetricInfo workerMetric = cluster.get_worker_metric(topologyId, workerId); - if (workerMetric == null) continue; - - Map workerKV = new HashMap(); - - workerKV.put("Topology_Name", topologyId); - workerKV.put("Port", String.valueOf(port)); - workerKV.put(MetricDef.MEMORY_USED, workerMetric.getUsedMem()); - workerKV.put(MetricDef.CPU_USED_RATIO, workerMetric.getUsedCpu()); - - workerKV.putAll(workerMetric.getGaugeData()); - workerKV.putAll(workerMetric.getCounterData()); - workerKV.putAll(workerMetric.getMeterData()); - - if (metricPerf == true) - { - workerKV.putAll(workerMetric.getTimerData()); - workerKV.putAll(workerMetric.getHistogramData()); - } - - jsonMsgWorkers.add(workerKV); - } catch (Exception e) { - LOG.error("Failed to buildWorkerJsonMsg, workerId=" + workerId + ", e=" + e); - } - } - } - - 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() { - } -} \ No newline at end of file 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 deleted file mode 100644 index 0cbcfc33b..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/timer/RotatingMapTrigger.java +++ /dev/null @@ -1,64 +0,0 @@ -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 deleted file mode 100644 index 4394e3d8f..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TickTupleTrigger.java +++ /dev/null @@ -1,39 +0,0 @@ -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 deleted file mode 100644 index c6e40e02b..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/daemon/worker/timer/TimerTrigger.java +++ /dev/null @@ -1,118 +0,0 @@ -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/drpc/ClearThread.java b/jstorm-server/src/main/java/com/alibaba/jstorm/drpc/ClearThread.java deleted file mode 100644 index b31afe8e3..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/drpc/ClearThread.java +++ /dev/null @@ -1,66 +0,0 @@ -package com.alibaba.jstorm.drpc; - -import java.util.Map.Entry; -import java.util.concurrent.Semaphore; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; -import backtype.storm.generated.DRPCExecutionException; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.TimeUtils; - -public class ClearThread extends RunnableCallback { - private static final Logger LOG = Logger.getLogger(ClearThread.class); - - private final int REQUEST_TIMEOUT_SECS; - private static final int TIMEOUT_CHECK_SECS = 5; - - private Drpc drpcService; - private AtomicBoolean active; - - public ClearThread(Drpc drpc) { - drpcService = drpc; - active = drpc.getIsActive(); - - REQUEST_TIMEOUT_SECS = JStormUtils.parseInt( - drpcService.getConf().get(Config.DRPC_REQUEST_TIMEOUT_SECS), 60); - LOG.info("Drpc timeout seconds is " + REQUEST_TIMEOUT_SECS); - } - - @Override - public void run() { - - for (Entry e : drpcService.getIdtoStart().entrySet()) { - if (TimeUtils.time_delta(e.getValue()) > REQUEST_TIMEOUT_SECS) { - String id = e.getKey(); - - drpcService.getIdtoResult().put(id, new DRPCExecutionException( - "Request timed out")); - Semaphore s = drpcService.getIdtoSem().get(id); - if (s != null) { - s.release(); - } - drpcService.cleanup(id); - LOG.info("Clear request " + id); - } - } - - JStormUtils.sleepMs(10); - - } - - public Object getResult() { - if (active.get() == true ) { - return TIMEOUT_CHECK_SECS; - }else { - LOG.info("Quit Drpc clear thread "); - return -1; - } - - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/drpc/Drpc.java b/jstorm-server/src/main/java/com/alibaba/jstorm/drpc/Drpc.java deleted file mode 100644 index a01bba454..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/drpc/Drpc.java +++ /dev/null @@ -1,294 +0,0 @@ -package com.alibaba.jstorm.drpc; - -import java.util.Map; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.Semaphore; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.log4j.Logger; -import org.apache.thrift7.TException; -import org.apache.thrift7.protocol.TBinaryProtocol; -import org.apache.thrift7.server.THsHaServer; -import org.apache.thrift7.transport.TNonblockingServerSocket; - -import backtype.storm.Config; -import backtype.storm.daemon.Shutdownable; -import backtype.storm.generated.DRPCExecutionException; -import backtype.storm.generated.DRPCRequest; -import backtype.storm.generated.DistributedRPC; -import backtype.storm.generated.DistributedRPCInvocations; - -import com.alibaba.jstorm.callback.AsyncLoopThread; -import com.alibaba.jstorm.cluster.StormConfig; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.TimeUtils; - -/** - * Drpc - * - * @author yannian - * - */ -public class Drpc implements DistributedRPC.Iface, - DistributedRPCInvocations.Iface, Shutdownable { - - private static final Logger LOG = Logger.getLogger(Drpc.class); - - public static void main(String[] args) throws Exception { - LOG.info("Begin to start Drpc server"); - - final Drpc service = new Drpc(); - - service.init(); - } - - private Map conf; - - private THsHaServer handlerServer; - - private THsHaServer invokeServer; - - private AsyncLoopThread clearThread; - - private AtomicBoolean isActive = new AtomicBoolean(true); - - private THsHaServer initHandlerServer(Map conf, final Drpc service) - throws Exception { - int port = JStormUtils.parseInt(conf.get(Config.DRPC_PORT)); - int workerThreadNum = JStormUtils.parseInt(conf.get(Config.DRPC_WORKER_THREADS)); - int queueSize = JStormUtils.parseInt(conf.get(Config.DRPC_QUEUE_SIZE)); - - TNonblockingServerSocket socket = new TNonblockingServerSocket(port); - THsHaServer.Args targs = new THsHaServer.Args(socket); - targs.workerThreads(64); - targs.protocolFactory(new TBinaryProtocol.Factory()); - targs.processor(new DistributedRPC.Processor( - service)); - - ThreadPoolExecutor executor = new ThreadPoolExecutor(workerThreadNum, - workerThreadNum, 60, TimeUnit.SECONDS, - new ArrayBlockingQueue(queueSize)); - targs.executorService(executor); - - THsHaServer handlerServer = new THsHaServer(targs); - LOG.info("Successfully init Handler Server " + port); - - return handlerServer; - } - - private THsHaServer initInvokeServer(Map conf, final Drpc service) - throws Exception { - int port = JStormUtils.parseInt(conf.get(Config.DRPC_INVOCATIONS_PORT)); - - TNonblockingServerSocket socket = new TNonblockingServerSocket(port); - THsHaServer.Args targsInvoke = new THsHaServer.Args(socket); - targsInvoke.workerThreads(64); - targsInvoke.protocolFactory(new TBinaryProtocol.Factory()); - targsInvoke - .processor(new DistributedRPCInvocations.Processor( - service)); - - THsHaServer invokeServer = new THsHaServer(targsInvoke); - - LOG.info("Successfully init Invoke Server " + port); - return invokeServer; - } - - private void initThrift() throws Exception { - - handlerServer = initHandlerServer(conf, this); - - invokeServer = initInvokeServer(conf, this); - - Runtime.getRuntime().addShutdownHook(new Thread() { - public void run() { - Drpc.this.shutdown(); - handlerServer.stop(); - invokeServer.stop(); - } - - }); - - LOG.info("Starting Distributed RPC servers..."); - new Thread(new Runnable() { - @Override - public void run() { - invokeServer.serve(); - } - }).start(); - handlerServer.serve(); - } - - private void initClearThread() { - clearThread = new AsyncLoopThread(new ClearThread(this)); - LOG.info("Successfully start clear thread"); - } - - public void init() throws Exception { - conf = StormConfig.read_storm_config(); - LOG.info("Configuration is \n" + conf); - - initClearThread(); - - initThrift(); - } - - public Drpc() { - - } - - @Override - public void shutdown() { - isActive.set(false); - - clearThread.interrupt(); - - try { - clearThread.join(); - } catch (InterruptedException e) { - } - LOG.info("Successfully cleanup clear thread"); - - invokeServer.stop(); - LOG.info("Successfully stop invokeServer"); - - handlerServer.stop(); - LOG.info("Successfully stop handlerServer"); - - } - - - private AtomicInteger ctr = new AtomicInteger(0); - private ConcurrentHashMap idtoSem = new ConcurrentHashMap(); - private ConcurrentHashMap idtoResult = new ConcurrentHashMap(); - private ConcurrentHashMap idtoStart = new ConcurrentHashMap(); - private ConcurrentHashMap> requestQueues = new ConcurrentHashMap>(); - - public void cleanup(String id) { - LOG.info("clean id " + id + " @ " + (System.currentTimeMillis())); - - idtoSem.remove(id); - idtoResult.remove(id); - idtoStart.remove(id); - } - - @Override - public String execute(String function, String args) - throws DRPCExecutionException, TException { - LOG.info("Received DRPC request for " + function + " " + args + " at " - + (System.currentTimeMillis())); - int idinc = this.ctr.incrementAndGet(); - int maxvalue = 1000000000; - int newid = idinc % maxvalue; - if (idinc != newid) { - this.ctr.compareAndSet(idinc, newid); - } - - String strid = String.valueOf(newid); - Semaphore sem = new Semaphore(0); - - DRPCRequest req = new DRPCRequest(args, strid); - this.idtoStart.put(strid, TimeUtils.current_time_secs()); - this.idtoSem.put(strid, sem); - ConcurrentLinkedQueue queue = acquireQueue(function); - queue.add(req); - LOG.info("Waiting for DRPC request for " + function + " " + args - + " at " + (System.currentTimeMillis())); - try { - sem.acquire(); - } catch (InterruptedException e) { - LOG.error("acquire fail ", e); - } - LOG.info("Acquired for DRPC request for " + function + " " + args - + " at " + (System.currentTimeMillis())); - - Object result = this.idtoResult.get(strid); - LOG.info("Returning for DRPC request for " + function + " " + args - + " at " + (System.currentTimeMillis())); - - this.cleanup(strid); - - if (result instanceof DRPCExecutionException) { - throw (DRPCExecutionException) result; - } - return String.valueOf(result); - } - - @Override - public void result(String id, String result) throws TException { - Semaphore sem = this.idtoSem.get(id); - LOG.info("Received result " + result + " for id " + id + " at " - + (System.currentTimeMillis())); - if (sem != null) { - this.idtoResult.put(id, result); - sem.release(); - } - - } - - @Override - public DRPCRequest fetchRequest(String functionName) throws TException { - - ConcurrentLinkedQueue queue = acquireQueue(functionName); - DRPCRequest req = queue.poll(); - if (req != null) { - LOG.info("Fetched request for " + functionName + " at " - + (System.currentTimeMillis())); - return req; - }else { - return new DRPCRequest("", ""); - } - - - } - - @Override - public void failRequest(String id) throws TException { - Semaphore sem = this.idtoSem.get(id); - LOG.info("failRequest result for id " + id + " at " - + (System.currentTimeMillis())); - if (sem != null) { - this.idtoResult.put(id, - new DRPCExecutionException("Request failed")); - sem.release(); - } - } - - - - private ConcurrentLinkedQueue acquireQueue(String function) { - ConcurrentLinkedQueue reqQueue = requestQueues.get(function); - if (reqQueue == null) { - reqQueue = new ConcurrentLinkedQueue(); - requestQueues.put(function, reqQueue); - } - return reqQueue; - } - - public ConcurrentHashMap getIdtoSem() { - return idtoSem; - } - - public ConcurrentHashMap getIdtoResult() { - return idtoResult; - } - - public ConcurrentHashMap getIdtoStart() { - return idtoStart; - } - - public AtomicBoolean getIsActive() { - return isActive; - } - - public Map getConf() { - return conf; - } - - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/event/EventManager.java b/jstorm-server/src/main/java/com/alibaba/jstorm/event/EventManager.java deleted file mode 100644 index 713749d97..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/event/EventManager.java +++ /dev/null @@ -1,11 +0,0 @@ -package com.alibaba.jstorm.event; - -import com.alibaba.jstorm.callback.RunnableCallback; - -public interface EventManager { - public void add(RunnableCallback event_fn); - - public boolean waiting(); - - public void shutdown(); -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/event/EventManagerImp.java b/jstorm-server/src/main/java/com/alibaba/jstorm/event/EventManagerImp.java deleted file mode 100644 index 7d2de10e7..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/event/EventManagerImp.java +++ /dev/null @@ -1,79 +0,0 @@ -package com.alibaba.jstorm.event; - -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; - -import com.alibaba.jstorm.callback.RunnableCallback; - -import backtype.storm.utils.Time; - -/** - * Event Manager, drop one event from queue, then execute the event. - */ -public class EventManagerImp implements EventManager { - private AtomicInteger added = new AtomicInteger(); - private AtomicInteger processed = new AtomicInteger(); - private AtomicBoolean isrunning = new AtomicBoolean(true); - private Thread runningThread; - private LinkedBlockingQueue queue = new LinkedBlockingQueue(); - - public EventManagerImp(boolean _daemon) { - - Runnable runner = new EventManagerImpExecute(this); - - this.runningThread = new Thread(runner); - this.runningThread.setDaemon(_daemon); - this.runningThread.start(); - } - - public boolean isRunning() { - return isrunning.get(); - } - - public RunnableCallback poll() throws InterruptedException { - RunnableCallback event = queue.poll(); - return event; - } - - public RunnableCallback take() throws InterruptedException { - RunnableCallback event = queue.take(); - return event; - } - - public void proccessinc() { - processed.incrementAndGet(); - } - - @Override - public void add(RunnableCallback event_fn) { - if (!this.isRunning()) { - // throw new RuntimeException( - // "Cannot add events to a shutdown event manager"); - - return; - } - added.incrementAndGet(); - queue.add(event_fn); - } - - @Override - public boolean waiting() { - return (processed.get() == added.get()); - - // return Time.isThreadWaiting(runningThread) || - // (processed.get() == added.get()); - - } - - @Override - public void shutdown() { - isrunning.set(false); - runningThread.interrupt(); - try { - runningThread.join(); - } catch (InterruptedException e) { - } - - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/event/EventManagerImpExecute.java b/jstorm-server/src/main/java/com/alibaba/jstorm/event/EventManagerImpExecute.java deleted file mode 100644 index 69a83e6a9..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/event/EventManagerImpExecute.java +++ /dev/null @@ -1,57 +0,0 @@ -package com.alibaba.jstorm.event; - -import org.apache.log4j.Logger; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.utils.JStormUtils; - -public class EventManagerImpExecute implements Runnable { - private static Logger LOG = Logger.getLogger(EventManagerImpExecute.class); - - public EventManagerImpExecute(EventManagerImp manager) { - this.manager = manager; - } - - EventManagerImp manager; - Exception error = null; - - @Override - public void run() { - try { - while (manager.isRunning()) { - RunnableCallback r = null; - try { - r = manager.take(); - } catch (InterruptedException e) { - // LOG.info("Failed to get ArgsRunable from EventManager queue"); - } - - if (r == null) { - try { - Thread.sleep(10); - } catch (InterruptedException e) { - - } - continue; - } - - r.run(); - Exception e = r.error(); - if (e != null) { - throw e; - } - manager.proccessinc(); - - } - - } catch (InterruptedException e) { - error = e; - LOG.error("Event Manager interrupted", e); - } catch (Exception e) { - LOG.error("Error when processing event ", e); - JStormUtils.halt_process(20, "Error when processing an event"); - } - - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/event/EventManagerPusher.java b/jstorm-server/src/main/java/com/alibaba/jstorm/event/EventManagerPusher.java deleted file mode 100644 index a474d6a5c..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/event/EventManagerPusher.java +++ /dev/null @@ -1,47 +0,0 @@ -package com.alibaba.jstorm.event; - -import java.util.concurrent.atomic.AtomicBoolean; - -import com.alibaba.jstorm.callback.RunnableCallback; - -/** - * put event to eventManager queue - */ -public class EventManagerPusher extends RunnableCallback { - - private EventManager eventManager; - - private RunnableCallback event; - - private AtomicBoolean active; - - private Integer result; - - private int frequence; - - public EventManagerPusher(EventManager eventManager, - RunnableCallback event, AtomicBoolean active, int frequence) { - this.eventManager = eventManager; - this.event = event; - this.active = active; - this.result = null; - this.frequence = frequence; - } - - @Override - public Object getResult() { - return result; - } - - @Override - public void run() { - eventManager.add(event); - if (active.get()) { - this.result = frequence; - } else { - this.result = -1; - - } - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/event/EventManagerZkPusher.java b/jstorm-server/src/main/java/com/alibaba/jstorm/event/EventManagerZkPusher.java deleted file mode 100644 index 53c49a676..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/event/EventManagerZkPusher.java +++ /dev/null @@ -1,25 +0,0 @@ -package com.alibaba.jstorm.event; - -import com.alibaba.jstorm.callback.RunnableCallback; - -public class EventManagerZkPusher extends RunnableCallback { - - private EventManager eventManager; - - private RunnableCallback cb; - - /** - * @param cb - * @param eventManager - */ - public EventManagerZkPusher(RunnableCallback cb, EventManager eventManager) { - this.eventManager = eventManager; - this.cb = cb; - } - - @Override - public void run() { - eventManager.add(cb); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/ControlMessage.java b/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/ControlMessage.java deleted file mode 100644 index ce2e7ee0e..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/ControlMessage.java +++ /dev/null @@ -1,62 +0,0 @@ -package com.alibaba.jstorm.message.netty; - -import org.jboss.netty.buffer.ChannelBuffer; -import org.jboss.netty.buffer.ChannelBufferOutputStream; -import org.jboss.netty.buffer.ChannelBuffers; -import org.jboss.netty.channel.Channel; - -enum ControlMessage { - EOB_MESSAGE((short) -201), OK_RESPONSE((short) -200); - - private short code; - private long timeStamp; - - // private constructor - private ControlMessage(short code) { - this.code = code; - } - - /** - * Return a control message per an encoded status code - * - * @param encoded - * @return - */ - static ControlMessage mkMessage(short encoded) { - for (ControlMessage cm : ControlMessage.values()) { - if (encoded == cm.code) - return cm; - } - return null; - } - - int encodeLength() { - return 10; // short + long - } - - /** - * encode the current Control Message into a channel buffer - * - * @throws Exception - */ - ChannelBuffer buffer() throws Exception { - ChannelBufferOutputStream bout = new ChannelBufferOutputStream( - ChannelBuffers.directBuffer(encodeLength())); - write(bout); - bout.close(); - return bout.buffer(); - } - - void write(ChannelBufferOutputStream bout) throws Exception { - bout.writeShort(code); - bout.writeLong(System.currentTimeMillis()); - } - - long getTimeStamp() { - return timeStamp; - } - - void setTimeStamp(long timeStamp) { - this.timeStamp = timeStamp; - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/MessageBatch.java b/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/MessageBatch.java deleted file mode 100644 index d433d2e57..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/MessageBatch.java +++ /dev/null @@ -1,178 +0,0 @@ -package com.alibaba.jstorm.message.netty; - -import java.util.ArrayList; - -import org.jboss.netty.buffer.ChannelBuffer; -import org.jboss.netty.buffer.ChannelBufferOutputStream; -import org.jboss.netty.buffer.ChannelBuffers; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import backtype.storm.messaging.TaskMessage; - -class MessageBatch { - private static final Logger LOG = LoggerFactory - .getLogger(MessageBatch.class); - private int buffer_size; - private ArrayList msgs; - private int encoded_length; - - MessageBatch(int buffer_size) { - this.buffer_size = buffer_size; - msgs = new ArrayList(); - encoded_length = ControlMessage.EOB_MESSAGE.encodeLength(); - } - - void add(Object obj) { - if (obj == null) - throw new RuntimeException("null object forbidded in message batch"); - - if (obj instanceof TaskMessage) { - TaskMessage msg = (TaskMessage) obj; - msgs.add(msg); - encoded_length += msgEncodeLength(msg); - return; - } - - if (obj instanceof ControlMessage) { - ControlMessage msg = (ControlMessage) obj; - msgs.add(msg); - encoded_length += msg.encodeLength(); - return; - } - - throw new RuntimeException("Unsuppoted object type " - + obj.getClass().getName()); - } - - void remove(Object obj) { - if (obj == null) - return; - - if (obj instanceof TaskMessage) { - TaskMessage msg = (TaskMessage) obj; - msgs.remove(msg); - encoded_length -= msgEncodeLength(msg); - return; - } - - if (obj instanceof ControlMessage) { - ControlMessage msg = (ControlMessage) obj; - msgs.remove(msg); - encoded_length -= msg.encodeLength(); - return; - } - } - - Object get(int index) { - return msgs.get(index); - } - - /** - * try to add a TaskMessage to a batch - * - * @param taskMsg - * @return false if the msg could not be added due to buffer size limit; - * true otherwise - */ - boolean tryAdd(TaskMessage taskMsg) { - if ((encoded_length + msgEncodeLength(taskMsg)) > buffer_size) - return false; - add(taskMsg); - return true; - } - - private int msgEncodeLength(TaskMessage taskMsg) { - if (taskMsg == null) - return 0; - - int size = 6; // INT + SHORT - if (taskMsg.message() != null) - size += taskMsg.message().length; - return size; - } - - /** - * Has this batch used up allowed buffer size - * - * @return - */ - boolean isFull() { - return encoded_length >= buffer_size; - } - - /** - * true if this batch doesn't have any messages - * - * @return - */ - boolean isEmpty() { - return msgs.isEmpty(); - } - - /** - * # of msgs in this batch - * - * @return - */ - int size() { - return msgs.size(); - } - - - - public int getEncoded_length() { - return encoded_length; - } - - /** - * create a buffer containing the encoding of this batch - */ - ChannelBuffer buffer() throws Exception { - ChannelBufferOutputStream bout = new ChannelBufferOutputStream( - ChannelBuffers.directBuffer(encoded_length)); - - for (Object msg : msgs) - if (msg instanceof TaskMessage) - writeTaskMessage(bout, (TaskMessage) msg); - else { - // LOG.debug("Write one non-TaskMessage {}", msg ); - ((ControlMessage) msg).write(bout); - } - - // add a END_OF_BATCH indicator - ControlMessage.EOB_MESSAGE.write(bout); - // LOG.debug("ControlMessage.EOB_MESSAGE " ); - - bout.close(); - - return bout.buffer(); - } - - /** - * write a TaskMessage into a stream - * - * Each TaskMessage is encoded as: task ... short(2) len ... int(4) payload - * ... byte[] * - */ - private void writeTaskMessage(ChannelBufferOutputStream bout, - TaskMessage message) throws Exception { - int payload_len = 0; - if (message.message() != null) - payload_len = message.message().length; - - int task_id = message.task(); - if (task_id > Short.MAX_VALUE) - throw new RuntimeException("Task ID should not exceed " - + Short.MAX_VALUE); - - bout.writeShort((short) task_id); - bout.writeInt(payload_len); - if (payload_len > 0) - bout.write(message.message()); - - // @@@ TESTING CODE - // LOG.info("Write one message taskid:{}, len:{}, data:{}", task_id - // , payload_len, JStormUtils.toPrintableString(message.message()) ); - } -} \ No newline at end of file diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/MessageDecoder.java b/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/MessageDecoder.java deleted file mode 100644 index ea9f6bdf3..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/MessageDecoder.java +++ /dev/null @@ -1,147 +0,0 @@ -package com.alibaba.jstorm.message.netty; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.HashMap; -import java.net.SocketAddress; -import java.net.InetSocketAddress; - -import org.jboss.netty.buffer.ChannelBuffer; -import org.jboss.netty.channel.Channel; -import org.jboss.netty.channel.ChannelHandlerContext; -import org.jboss.netty.handler.codec.frame.FrameDecoder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import backtype.storm.messaging.TaskMessage; - -import com.alibaba.jstorm.metric.MetricDef; -import com.alibaba.jstorm.metric.JStormHistogram; -import com.alibaba.jstorm.metric.JStormTimer; -import com.alibaba.jstorm.metric.Metrics; - -public class MessageDecoder extends FrameDecoder { - private static final Logger LOG = LoggerFactory - .getLogger(MessageDecoder.class); - - private static JStormTimer timer = null; - private static Map networkTransmitTimeMap = null; - - public MessageDecoder(boolean isServer) { - if (isServer) { - if (timer == null) - timer = Metrics.registerTimer(null, MetricDef.NETTY_SERV_DECODE_TIME, - null, Metrics.MetricType.WORKER); - } - - if (networkTransmitTimeMap == null) - networkTransmitTimeMap = new HashMap(); - } - - /* - * Each ControlMessage is encoded as: code (<0) ... short(2) Each - * TaskMessage is encoded as: task (>=0) ... short(2) len ... int(4) payload - * ... byte[] * - */ - protected Object decode(ChannelHandlerContext ctx, Channel channel, - ChannelBuffer buf) throws Exception { - // Make sure that we have received at least a short - long available = buf.readableBytes(); - // Length of control message is 10. - // Minimum length of a task message is 6(short taskId, int length). - if (available < 6) { - // need more data - return null; - } - - - if (timer != null) timer.start(); - try { - // Mark the current buffer position before reading task/len field - // because the whole frame might not be in the buffer yet. - // We will reset the buffer position to the marked position if - // there's not enough bytes in the buffer. - buf.markReaderIndex(); - - // read the short field - short code = buf.readShort(); - available -= 2; - - // case 1: Control message - ControlMessage ctrl_msg = ControlMessage.mkMessage(code); - if (ctrl_msg != null) { - if (available < 8) { - // The time stamp bytes were not received yet - return null. - buf.resetReaderIndex(); - return null; - } - long timeStamp = buf.readLong(); - available -= 8; - if (ctrl_msg == ControlMessage.EOB_MESSAGE) { - InetSocketAddress sockAddr = (InetSocketAddress)(channel.getRemoteAddress()); - String remoteAddr = sockAddr.getHostName() + ":" + sockAddr.getPort(); - - long interval = System.currentTimeMillis() - timeStamp; - if (interval < 0) interval = 0; - - JStormHistogram netTransTime = networkTransmitTimeMap.get(remoteAddr); - if (netTransTime == null) { - netTransTime = Metrics.registerHistograms(remoteAddr, MetricDef.NETWORK_MSG_TRANS_TIME, - null, Metrics.MetricType.WORKER); - networkTransmitTimeMap.put(remoteAddr, netTransTime); - } - - netTransTime.update(interval); - } - - return ctrl_msg; - } - - // case 2: task Message - short task = code; - - // Make sure that we have received at least an integer (length) - if (available < 4) { - // need more data - buf.resetReaderIndex(); - - return null; - } - - // Read the length field. - int length = buf.readInt(); - if (length <= 0) { - LOG.info("Receive one message whose TaskMessage's message length is {}", length); - return new TaskMessage(task, null); - } - - // Make sure if there's enough bytes in the buffer. - available -= 4; - if (available < length) { - // The whole bytes were not received yet - return null. - buf.resetReaderIndex(); - - return null; - } - - // There's enough bytes in the buffer. Read it. - ChannelBuffer payload = buf.readBytes(length); - - // Successfully decoded a frame. - // Return a TaskMessage object - - byte[] rawBytes = payload.array(); - // @@@ TESTING CODE - // LOG.info("Receive task:{}, length: {}, data:{}", - // task, length, JStormUtils.toPrintableString(rawBytes)); - - TaskMessage ret = new TaskMessage(task, rawBytes); - - return ret; - } finally { - if (timer != null) timer.stop(); - } - - } -} \ No newline at end of file diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/MessageEncoder.java b/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/MessageEncoder.java deleted file mode 100644 index 6deb81308..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/MessageEncoder.java +++ /dev/null @@ -1,23 +0,0 @@ -package com.alibaba.jstorm.message.netty; - -import org.jboss.netty.channel.Channel; -import org.jboss.netty.channel.ChannelHandlerContext; -import org.jboss.netty.handler.codec.oneone.OneToOneEncoder; - -public class MessageEncoder extends OneToOneEncoder { - @Override - protected Object encode(ChannelHandlerContext ctx, Channel channel, - Object obj) throws Exception { - if (obj instanceof ControlMessage) { - return ((ControlMessage) obj).buffer(); - } - - if (obj instanceof MessageBatch) { - return ((MessageBatch) obj).buffer(); - } - - throw new RuntimeException("Unsupported encoding of object of class " - + obj.getClass().getName()); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/NettyClient.java b/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/NettyClient.java deleted file mode 100644 index c608376a2..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/NettyClient.java +++ /dev/null @@ -1,450 +0,0 @@ -package com.alibaba.jstorm.message.netty; - -import java.net.InetSocketAddress; -import java.net.SocketAddress; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; - -import org.jboss.netty.bootstrap.ClientBootstrap; -import org.jboss.netty.channel.Channel; -import org.jboss.netty.channel.ChannelFactory; -import org.jboss.netty.channel.ChannelFuture; -import org.jboss.netty.channel.ChannelFutureListener; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import backtype.storm.Config; -import backtype.storm.messaging.IConnection; -import backtype.storm.messaging.TaskMessage; -import backtype.storm.utils.DisruptorQueue; -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.metric.MetricDef; -import com.alibaba.jstorm.metric.JStormHistogram; -import com.alibaba.jstorm.metric.JStormTimer; -import com.alibaba.jstorm.metric.Metrics; -import com.alibaba.jstorm.utils.JStormServerUtils; -import com.alibaba.jstorm.utils.JStormUtils; -import com.codahale.metrics.Gauge; - -class NettyClient implements IConnection { - private static final Logger LOG = LoggerFactory - .getLogger(NettyClient.class); - public static final String PREFIX = "Netty-Client-"; - protected String name; - - protected final int max_retries; - protected final int base_sleep_ms; - protected final int max_sleep_ms; - protected final int timeoutSecond; - protected final int MAX_SEND_PENDING; - - protected AtomicInteger retries; - - protected AtomicReference channelRef; - protected ClientBootstrap bootstrap; - protected final InetSocketAddress remote_addr; - protected final ChannelFactory factory; - - protected final int buffer_size; - protected final AtomicBoolean being_closed; - - protected AtomicLong pendings; - protected int messageBatchSize; - protected AtomicReference messageBatchRef; - - protected ScheduledExecutorService scheduler; - - protected String address; - protected JStormTimer sendTimer; - protected JStormHistogram histogram; - - protected ReconnectRunnable reconnector; - protected ChannelFactory clientChannelFactory; - - protected Set closingChannel; - - protected AtomicBoolean isConnecting = new AtomicBoolean(false); - - @SuppressWarnings("rawtypes") - NettyClient(Map storm_conf, ChannelFactory factory, - ScheduledExecutorService scheduler, String host, int port, - ReconnectRunnable reconnector) { - this.factory = factory; - this.scheduler = scheduler; - this.reconnector = reconnector; - - retries = new AtomicInteger(0); - channelRef = new AtomicReference(null); - being_closed = new AtomicBoolean(false); - pendings = new AtomicLong(0); - - // Configure - buffer_size = Utils.getInt(storm_conf - .get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE)); - max_retries = Math.min(30, Utils.getInt(storm_conf - .get(Config.STORM_MESSAGING_NETTY_MAX_RETRIES))); - base_sleep_ms = Utils.getInt(storm_conf - .get(Config.STORM_MESSAGING_NETTY_MIN_SLEEP_MS)); - max_sleep_ms = Utils.getInt(storm_conf - .get(Config.STORM_MESSAGING_NETTY_MAX_SLEEP_MS)); - - timeoutSecond = JStormUtils.parseInt( - storm_conf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS), 30); - MAX_SEND_PENDING = (int) ConfigExtension - .getNettyMaxSendPending(storm_conf); - - this.messageBatchSize = Utils.getInt( - storm_conf.get(Config.STORM_NETTY_MESSAGE_BATCH_SIZE), 262144); - messageBatchRef = new AtomicReference(); - - // Start the connection attempt. - remote_addr = new InetSocketAddress(host, port); - name = remote_addr.toString(); - - address = JStormServerUtils.getName(host, port); - sendTimer = Metrics.registerTimer(address, MetricDef.NETTY_CLI_SEND_TIME, - null, Metrics.MetricType.WORKER); - histogram = Metrics.registerHistograms(address, MetricDef.NETTY_CLI_BATCH_SIZE, - null, Metrics.MetricType.WORKER); - Metrics.register(address, MetricDef.NETTY_CLI_SEND_PENDING, new Gauge() { - - @Override - public Long getValue() { - return pendings.get(); - } - }, null, Metrics.MetricType.WORKER); - - closingChannel = new HashSet(); - } - - public void start() { - bootstrap = new ClientBootstrap(clientChannelFactory); - bootstrap.setOption("tcpNoDelay", true); - bootstrap.setOption("reuserAddress", true); - bootstrap.setOption("sendBufferSize", buffer_size); - bootstrap.setOption("keepAlive", true); - - // Set up the pipeline factory. - bootstrap.setPipelineFactory(new StormClientPipelineFactory(this)); - reconnect(); - } - - /** - * The function can't be synchronized, otherwise it will be deadlock - * - */ - public void doReconnect() { - if (channelRef.get() != null ) { - -// if (channelRef.get().isWritable()) { -// LOG.info("already exist a writable channel, give up reconnect, {}", -// channelRef.get()); -// return; -// } - return; - } - - if (isClosed() == true) { - return; - } - - if (isConnecting.getAndSet(true)) { - LOG.info("Connect twice {}", name()); - return ; - } - - long sleepMs = getSleepTimeMs(); - LOG.info("Reconnect ... [{}], {}, sleep {}ms", retries.get(), name, - sleepMs); - ChannelFuture future = bootstrap.connect(remote_addr); - future.addListener(new ChannelFutureListener() { - public void operationComplete(ChannelFuture future) - throws Exception { - isConnecting.set(false); - Channel channel = future.getChannel(); - if (future.isSuccess()) { - // do something else - LOG.info("Connection established, channel = :{}", - channel); - setChannel(channel); - //handleResponse(); - } else { - LOG.info( - "Failed to reconnect ... [{}], {}, channel = {}, cause = {}", - retries.get(), name, channel, - future.getCause()); - reconnect(); - } - } - }); - JStormUtils.sleepMs(sleepMs); - - return; - - } - - public void reconnect() { - reconnector.pushEvent(this); - } - - /** - * # of milliseconds to wait per exponential back-off policy - */ - private int getSleepTimeMs() { - - int sleepMs = base_sleep_ms * retries.incrementAndGet(); - if (sleepMs > 1000) { - sleepMs = 1000; - } - return sleepMs; - } - - /** - * Enqueue a task message to be sent to server - */ - @Override - public void send(List messages) { - LOG.warn("Should be overload"); - } - - @Override - public void send(TaskMessage message) { - LOG.warn("Should be overload"); - } - - protected synchronized void flushRequest(Channel channel, - final MessageBatch requests) { - if (requests == null || requests.isEmpty()) - return; - - histogram.update(requests.getEncoded_length()); - pendings.incrementAndGet(); - ChannelFuture future = channel.write(requests); - future.addListener(new ChannelFutureListener() { - public void operationComplete(ChannelFuture future) - throws Exception { - - pendings.decrementAndGet(); - if (!future.isSuccess()) { - Channel channel = future.getChannel(); - if (isClosed() == false) { - LOG.info("Failed to send requests to " + name + ": " + - channel.toString() + ":", - future.getCause() ); - } - - - - if (null != channel) { - - exceptionChannel(channel); - } - } else { - // LOG.debug("{} request(s) sent", requests.size()); - } - } - }); - } - - /** - * gracefully close this client. - * - * We will send all existing requests, and then invoke close_n_release() - * method - */ - public synchronized void close() { - LOG.info("Close netty connection to {}", name()); - if (isClosed() == true) { - return; - } - being_closed.set(true); - - Metrics.unregister(address, MetricDef.NETTY_CLI_SEND_TIME, null, Metrics.MetricType.WORKER); - Metrics.unregister(address, MetricDef.NETTY_CLI_BATCH_SIZE, null, Metrics.MetricType.WORKER); - Metrics.unregister(address, MetricDef.NETTY_CLI_SEND_PENDING, null, Metrics.MetricType.WORKER); - - Channel channel = channelRef.get(); - if (channel == null) { - LOG.info("Channel {} has been closed before", name()); - return; - } - - if (channel.isWritable()) { - MessageBatch toBeFlushed = messageBatchRef.getAndSet(null); - flushRequest(channel, toBeFlushed); - } - - // wait for pendings to exit - final long timeoutMilliSeconds = 10 * 1000; - final long start = System.currentTimeMillis(); - - LOG.info("Waiting for pending batchs to be sent with " + name() - + "..., timeout: {}ms, pendings: {}", timeoutMilliSeconds, - pendings.get()); - - while (pendings.get() != 0) { - try { - long delta = System.currentTimeMillis() - start; - if (delta > timeoutMilliSeconds) { - LOG.error( - "Timeout when sending pending batchs with {}..., there are still {} pending batchs not sent", - name(), pendings.get()); - break; - } - Thread.sleep(1000); // sleep 1s - } catch (InterruptedException e) { - break; - } - } - - close_n_release(); - - } - - /** - * close_n_release() is invoked after all messages have been sent. - */ - void close_n_release() { - if (channelRef.get() != null) { - setChannel(null); - } - - } - - /** - * Avoid channel double close - * - * @param channel - */ - void closeChannel(final Channel channel) { - synchronized (this) { - if (closingChannel.contains(channel)) { - LOG.info(channel.toString() + " is already closed"); - return ; - } - - closingChannel.add(channel); - } - - LOG.debug(channel.toString() + " begin to closed"); - ChannelFuture closeFuture = channel.close(); - closeFuture.addListener(new ChannelFutureListener() { - public void operationComplete(ChannelFuture future) - throws Exception { - - synchronized (this) { - closingChannel.remove(channel); - } - LOG.debug(channel.toString() + " finish closed"); - } - }); - } - - - void disconnectChannel(Channel channel) { - if (isClosed()) { - return ; - } - - if (channel == channelRef.get()) { - setChannel(null); - reconnect(); - }else { - closeChannel(channel); - } - - } - - void exceptionChannel(Channel channel) { - if (channel == channelRef.get()) { - setChannel(null); - } else { - closeChannel(channel); - } - } - - void setChannel(Channel newChannel) { - final Channel oldChannel = channelRef.getAndSet(newChannel); - - if (newChannel != null) { - retries.set(0); - } - - final String oldLocalAddres = (oldChannel == null) ? "null" : oldChannel - .getLocalAddress().toString(); - String newLocalAddress = (newChannel == null) ? "null" : newChannel - .getLocalAddress().toString(); - LOG.info("Use new channel {} replace old channel {}", newLocalAddress, - oldLocalAddres); - - // avoid one netty client use too much connection, close old one - if (oldChannel != newChannel && oldChannel != null) { - - closeChannel(oldChannel); - LOG.info("Successfully close old channel " + oldLocalAddres); -// scheduler.schedule(new Runnable() { -// -// @Override -// public void run() { -// -// } -// }, 10, TimeUnit.SECONDS); - - - // @@@ todo - // pendings.set(0); - } - } - - @Override - public boolean isClosed() { - return being_closed.get(); - } - - public AtomicBoolean getBeing_closed() { - return being_closed; - } - - public int getBuffer_size() { - return buffer_size; - } - - public SocketAddress getRemoteAddr() { - return remote_addr; - } - - public String name() { - return name; - } - - public void handleResponse() { - LOG.warn("Should be overload"); - } - - @Override - public TaskMessage recv(int flags) { - throw new UnsupportedOperationException( - "recvTask: Client connection should not receive any messages"); - } - - @Override - public void registerQueue(DisruptorQueue recvQueu) { - throw new UnsupportedOperationException( - "recvTask: Client connection should not receive any messages"); - } - - @Override - public void enqueue(TaskMessage message) { - throw new UnsupportedOperationException( - "recvTask: Client connection should not receive any messages"); - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/NettyClientAsync.java b/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/NettyClientAsync.java deleted file mode 100644 index 923402b8e..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/NettyClientAsync.java +++ /dev/null @@ -1,339 +0,0 @@ -package com.alibaba.jstorm.message.netty; - -import java.util.List; -import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; -import org.jboss.netty.channel.Channel; -import org.jboss.netty.channel.ChannelFactory; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import backtype.storm.Config; -import backtype.storm.messaging.TaskMessage; -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.utils.IntervalCheck; -import com.alibaba.jstorm.utils.JStormServerUtils; -import com.alibaba.jstorm.utils.JStormUtils; - -class NettyClientAsync extends NettyClient { - private static final Logger LOG = LoggerFactory - .getLogger(NettyClientAsync.class); - public static final String PREFIX = "Netty-Client-"; - - // when batch buffer size is more than BATCH_THREASHOLD_WARN - // it will block Drainer thread - protected long BATCH_THREASHOLD_WARN; - protected final boolean directlySend; - - protected AtomicBoolean flush_later; - protected int flushCheckInterval; - protected final boolean blockSend; - - boolean isDirectSend(Map conf) { - - if (JStormServerUtils.isOnePending(conf) == true) { - return true; - } - - return !ConfigExtension.isNettyTransferAsyncBatch(conf); - } - - boolean isBlockSend(Map storm_conf) { - if (ConfigExtension.isTopologyContainAcker(storm_conf) == false) { - return false; - } - - return ConfigExtension.isNettyASyncBlock(storm_conf); - } - - @SuppressWarnings("rawtypes") - NettyClientAsync(Map storm_conf, ChannelFactory factory, - ScheduledExecutorService scheduler, String host, int port, - ReconnectRunnable reconnector) { - super(storm_conf, factory, scheduler, host, port, reconnector); - - BATCH_THREASHOLD_WARN = ConfigExtension - .getNettyBufferThresholdSize(storm_conf); - - blockSend = isBlockSend(storm_conf); - - directlySend = isDirectSend(storm_conf); - - flush_later = new AtomicBoolean(false); - flushCheckInterval = Utils.getInt( - storm_conf.get(Config.STORM_NETTY_FLUSH_CHECK_INTERVAL_MS), 10); - - Runnable flusher = new Runnable() { - @Override - public void run() { - flush(); - } - }; - long initialDelay = Math.min(1000, max_sleep_ms * max_retries); - scheduler.scheduleWithFixedDelay(flusher, initialDelay, - flushCheckInterval, TimeUnit.MILLISECONDS); - - clientChannelFactory = factory; - - start(); - - LOG.info(this.toString()); - } - - /** - * Enqueue a task message to be sent to server - */ - @Override - public void send(List messages) { - // throw exception if the client is being closed - if (isClosed()) { - LOG.warn("Client is being closed, and does not take requests any more"); - return; - } - - sendTimer.start(); - try { - pushBatch(messages); - } catch (Exception e) { - throw new RuntimeException(e); - } finally { - sendTimer.stop(); - - } - } - - @Override - public void send(TaskMessage message) { - // throw exception if the client is being closed - if (isClosed()) { - LOG.warn("Client is being closed, and does not take requests any more"); - return; - } - - sendTimer.start(); - try { - pushBatch(message); - } catch (Exception e) { - throw new RuntimeException(e); - } finally { - sendTimer.stop(); - } - } - - void waitChannelReady(long cachedSize, long sleepMs) { - long begin = System.currentTimeMillis(); - boolean changeThreadhold = false; - IntervalCheck oneSecond = new IntervalCheck(); - IntervalCheck timeoutIntervalCheck = new IntervalCheck(); - timeoutIntervalCheck.setInterval(timeoutSecond); - while (isChannelReady() == null) { - - long now = System.currentTimeMillis(); - long delt = now - begin; - if (oneSecond.check() == true) { - LOG.warn( - "Target server {} is unavailable, pending {}, bufferSize {}, block sending {}ms", - name, pendings.get(), cachedSize, delt); - } - - if (timeoutIntervalCheck.check() == true) { - if (messageBatchRef.get() != null) { - LOG.warn( - "Target server {} is unavailable, wait too much time, throw timeout message", - name); - messageBatchRef.set(null); - } - setChannel(null); - LOG.warn("Reset channel as null"); - } - - reconnect(); - JStormUtils.sleepMs(sleepMs); - - if (delt > 2 * timeoutSecond * 1000L && changeThreadhold == false) { - if (channelRef.get() != null - && BATCH_THREASHOLD_WARN >= 2 * messageBatchSize) { - // it is just channel isn't writable; - BATCH_THREASHOLD_WARN = BATCH_THREASHOLD_WARN / 2; - LOG.info("Reduce BATCH_THREASHOLD_WARN to {}", - BATCH_THREASHOLD_WARN); - - changeThreadhold = true; - } - } - - if (isClosed()) { - LOG.info("Channel has been closed " + name()); - break; - } - } - } - - long getDelaySec(long cachedSize) { - long count = cachedSize / BATCH_THREASHOLD_WARN; - long sleepMs = (long)(Math.pow(2, count) * 10); - - if (sleepMs > 1000) { - sleepMs = 1000; - } - - return sleepMs; - } - - void handleFailedChannel(MessageBatch messageBatch) { - - messageBatchRef.set(messageBatch); - flush_later.set(true); - - long cachedSize = messageBatch.getEncoded_length(); - if (cachedSize > BATCH_THREASHOLD_WARN) { - - long sleepMs = getDelaySec(cachedSize); - - if (blockSend == false) { - LOG.warn( - "Target server {} is unavailable, pending {}, bufferSize {}, block sending {}ms", - name, pendings.get(), cachedSize, sleepMs); - - JStormUtils.sleepMs(sleepMs); - reconnect(); - } else { - waitChannelReady(cachedSize, sleepMs); - } - - } - return; - } - - void pushBatch(List messages) { - - if (messages.isEmpty()) { - return; - } - - MessageBatch messageBatch = messageBatchRef.getAndSet(null); - if (null == messageBatch) { - messageBatch = new MessageBatch(messageBatchSize); - } - - for (TaskMessage message : messages) { - if (TaskMessage.isEmpty(message)) { - continue; - } - - messageBatch.add(message); - - if (messageBatch.isFull()) { - Channel channel = isChannelReady(); - if (channel != null) { - flushRequest(channel, messageBatch); - - messageBatch = new MessageBatch(messageBatchSize); - } - - } - } - - Channel channel = isChannelReady(); - if (channel == null) { - handleFailedChannel(messageBatch); - return; - } else if (messageBatch.isEmpty() == false) { - flushRequest(channel, messageBatch); - } - - return; - } - - void pushBatch(TaskMessage message) { - - if (TaskMessage.isEmpty(message)) { - return; - } - - MessageBatch messageBatch = messageBatchRef.getAndSet(null); - if (null == messageBatch) { - messageBatch = new MessageBatch(messageBatchSize); - } - - messageBatch.add(message); - - Channel channel = isChannelReady(); - if (channel == null) { - handleFailedChannel(messageBatch); - return; - } - - if (messageBatch.isFull()) { - flushRequest(channel, messageBatch); - - return; - } - - if (directlySend) { - flushRequest(channel, messageBatch); - } else { - if(messageBatchRef.compareAndSet(null, messageBatch)) - flush_later.set(true); - else - LOG.error("MessageBatch will be lost. This should not happen."); - } - - return; - } - - void flush() { - if (isClosed() == true) { - return; - } - - if (flush_later.get() == false) { - return; - } - - Channel channel = isChannelReady(); - if (channel == null) { - return; - } - - flush_later.set(false); - MessageBatch toBeFlushed = messageBatchRef.getAndSet(null); - flushRequest(channel, toBeFlushed); - } - - Channel isChannelReady() { - Channel channel = channelRef.get(); - if (channel == null) { - return null; - } - - // improve performance skill check - if (channel.isWritable() == false) { - return null; - } - - if (blockSend == true && pendings.get() >= MAX_SEND_PENDING) { - return null; - } - return channel; - } - - @Override - public void handleResponse() { - // do nothing - return; - } - - @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/NettyClientSync.java b/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/NettyClientSync.java deleted file mode 100644 index 49b978255..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/NettyClientSync.java +++ /dev/null @@ -1,279 +0,0 @@ -package com.alibaba.jstorm.message.netty; - -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; -import org.jboss.netty.channel.Channel; -import org.jboss.netty.channel.ChannelFactory; -import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import backtype.storm.Config; -import backtype.storm.messaging.TaskMessage; -import backtype.storm.utils.DisruptorQueue; -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.metric.MetricDef; -import com.alibaba.jstorm.metric.Metrics; -import com.alibaba.jstorm.utils.JStormServerUtils; -import com.alibaba.jstorm.utils.JStormUtils; -import com.codahale.metrics.Gauge; -import com.lmax.disruptor.EventHandler; -import com.lmax.disruptor.WaitStrategy; -import com.lmax.disruptor.dsl.ProducerType; - -class NettyClientSync extends NettyClient implements EventHandler { - private static final Logger LOG = LoggerFactory - .getLogger(NettyClientSync.class); - - private ConcurrentLinkedQueue batchQueue; - private DisruptorQueue disruptorQueue; - private ExecutorService bossExecutor; - private ExecutorService workerExecutor; - - private AtomicLong emitTs = new AtomicLong(0); - - @SuppressWarnings("rawtypes") - NettyClientSync(Map storm_conf, ChannelFactory factory, - ScheduledExecutorService scheduler, String host, int port, - ReconnectRunnable reconnector) { - super(storm_conf, factory, scheduler, host, port, reconnector); - - batchQueue = new ConcurrentLinkedQueue(); - Metrics.register(address, MetricDef.NETTY_CLI_SYNC_BATCH_QUEUE, - new Gauge() { - @Override - public Integer getValue() { - return batchQueue.size(); - } - }, null, Metrics.MetricType.WORKER); - - WaitStrategy waitStrategy = (WaitStrategy) Utils - .newInstance((String) storm_conf - .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); - - disruptorQueue = DisruptorQueue.mkInstance(name, ProducerType.MULTI, - MAX_SEND_PENDING * 8, waitStrategy); - disruptorQueue.consumerStarted(); - - Metrics.registerQueue(address, MetricDef.NETTY_CLI_SYNC_DISR_QUEUE, disruptorQueue, - null, Metrics.MetricType.WORKER); - - Runnable trigger = new Runnable() { - @Override - public void run() { - trigger(); - } - }; - - scheduler.scheduleWithFixedDelay(trigger, 10, 1, TimeUnit.SECONDS); - - /** - * In sync mode, it can't directly use common factory, - * it will occur problem when client close and restart - */ - ThreadFactory bossFactory = new NettyRenameThreadFactory( - PREFIX + JStormServerUtils.getName(host, port) + "-boss"); - bossExecutor = Executors.newCachedThreadPool(bossFactory); - ThreadFactory workerFactory = new NettyRenameThreadFactory( - PREFIX + JStormServerUtils.getName(host, port) + "-worker"); - workerExecutor = Executors.newCachedThreadPool(workerFactory); - - clientChannelFactory = new NioClientSocketChannelFactory( - bossExecutor, - workerExecutor, 1); - - start(); - - LOG.info(this.toString()); - } - - /** - * Enqueue a task message to be sent to server - */ - @Override - public void send(List messages) { - for (TaskMessage msg : messages) { - disruptorQueue.publish(msg); - } - } - - @Override - public void send(TaskMessage message) { - disruptorQueue.publish(message); - } - - public void flushBatch(MessageBatch batch, Channel channel) { - emitTs.set(System.currentTimeMillis()); - if (batch == null) { - LOG.warn("Handle no data to {}, this shouldn't occur", name); - - } else if (channel == null || channel.isWritable() == false) { - LOG.warn("Channel occur exception, during batch messages {}", name); - batchQueue.offer(batch); - } else { - - flushRequest(channel, batch); - } - } - - /** - * Don't take care of competition - * - * @param blocked - */ - public void sendData() { - sendTimer.start(); - try { - MessageBatch batch = batchQueue.poll(); - if (batch == null) { - - disruptorQueue.consumeBatchWhenAvailable(this); - - batch = batchQueue.poll(); - } - - Channel channel = channelRef.get(); - flushBatch(batch, channel); - } catch (Throwable e) { - LOG.error("Occur e", e); - String err = name + " nettyclient occur unknow exception"; - JStormUtils.halt_process(-1, err); - } finally { - sendTimer.stop(); - } - } - - public void sendAllData() { - - sendTimer.start(); - try { - disruptorQueue.consumeBatch(this); - MessageBatch batch = batchQueue.poll(); - while (batch != null) { - Channel channel = channelRef.get(); - if (channel == null) { - LOG.info("No channel {} to flush all data", name); - return; - } else if (channel.isWritable() == false) { - LOG.info("Channel {} is no writable", name); - return; - } - flushBatch(batch, channel); - batch = batchQueue.poll(); - } - } catch (Throwable e) { - LOG.error("Occur e", e); - String err = name + " nettyclient occur unknow exception"; - JStormUtils.halt_process(-1, err); - } finally { - sendTimer.stop(); - } - } - - @Override - public void handleResponse() { - emitTs.set(0); - sendData(); - } - - @Override - public void onEvent(Object event, long sequence, boolean endOfBatch) - throws Exception { - if (event == null) { - return; - } - - TaskMessage message = (TaskMessage) event; - - MessageBatch messageBatch = messageBatchRef.getAndSet(null); - if (null == messageBatch) { - messageBatch = new MessageBatch(messageBatchSize); - } - - messageBatch.add(message); - - if (messageBatch.isFull()) { - batchQueue.offer(messageBatch); - } else if (endOfBatch == true) { - batchQueue.offer(messageBatch); - } else { - messageBatchRef.set(messageBatch); - } - } - - /** - * Handle lost message case - */ - void trigger() { - if (isClosed() == true) { - return; - } - - // if long time no receive NettyServer response - // it is likely lost message - long emitTime = emitTs.get(); - if (emitTime == 0) { - return; - } - - long now = System.currentTimeMillis(); - - long delt = now - emitTime; - if (delt < timeoutSecond * 100) { - return; - } - - Channel channel = channelRef.get(); - if (channel != null) { - LOG.info("Long time no response of {}, {}s", name, delt / 1000); - channel.write(ControlMessage.EOB_MESSAGE); - } - - } - - protected void shutdownPool() { - bossExecutor.shutdownNow(); - workerExecutor.shutdownNow(); - - try { - bossExecutor.awaitTermination(1, TimeUnit.SECONDS); - workerExecutor.awaitTermination(1, TimeUnit.SECONDS); - } catch (InterruptedException e) { - LOG.error("Error when shutting down client scheduler", e); - } - - clientChannelFactory.releaseExternalResources(); - } - - @Override - public void close() { - LOG.info( - "Begin to close connection to {} and flush all data, batchQueue {}, disruptor {}", - name, batchQueue.size(), disruptorQueue.population()); - sendAllData(); - disruptorQueue.haltWithInterrupt(); - Metrics.unregister(address, MetricDef.NETTY_CLI_SYNC_BATCH_QUEUE, null, Metrics.MetricType.WORKER); - Metrics.unregister(address, MetricDef.NETTY_CLI_SYNC_DISR_QUEUE, null, Metrics.MetricType.WORKER); - super.close(); - - shutdownPool(); - - } - - @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/NettyContext.java b/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/NettyContext.java deleted file mode 100644 index b81a6d9b4..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/NettyContext.java +++ /dev/null @@ -1,115 +0,0 @@ -package com.alibaba.jstorm.message.netty; - -import java.util.Map; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.TimeUnit; - -import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.alibaba.jstorm.callback.AsyncLoopRunnable; -import com.alibaba.jstorm.callback.AsyncLoopThread; -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.utils.JStormUtils; - -import backtype.storm.Config; -import backtype.storm.messaging.IConnection; -import backtype.storm.messaging.IContext; -import backtype.storm.utils.Utils; - -public class NettyContext implements IContext { - private final static Logger LOG = LoggerFactory - .getLogger(NettyContext.class); - @SuppressWarnings("rawtypes") - private Map storm_conf; - - private NioClientSocketChannelFactory clientChannelFactory; - - private ScheduledExecutorService clientScheduleService; - private final int MAX_CLIENT_SCHEDULER_THREAD_POOL_SIZE = 10; - - private ReconnectRunnable reconnector; - - private boolean isSyncMode = false; - - @SuppressWarnings("unused") - public NettyContext() { - } - - /** - * initialization per Storm configuration - */ - @SuppressWarnings("rawtypes") - public void prepare(Map storm_conf) { - this.storm_conf = storm_conf; - - int maxWorkers = Utils.getInt(storm_conf - .get(Config.STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS)); - ThreadFactory bossFactory = new NettyRenameThreadFactory(NettyClient.PREFIX + "boss"); - ThreadFactory workerFactory = new NettyRenameThreadFactory(NettyClient.PREFIX + "worker"); - - if (maxWorkers > 0) { - clientChannelFactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(bossFactory), - Executors.newCachedThreadPool(workerFactory), maxWorkers); - } else { - clientChannelFactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(bossFactory), - Executors.newCachedThreadPool(workerFactory)); - } - int otherWorkers = Utils.getInt(storm_conf.get(Config.TOPOLOGY_WORKERS), 1) - 1; - int poolSize = Math.min(Math.max(1, otherWorkers), MAX_CLIENT_SCHEDULER_THREAD_POOL_SIZE); - clientScheduleService = Executors.newScheduledThreadPool(poolSize, new NettyRenameThreadFactory("client-schedule-service")); - - reconnector = new ReconnectRunnable(); - new AsyncLoopThread(reconnector, true, Thread.MIN_PRIORITY, true); - - isSyncMode = ConfigExtension.isNettySyncMode(storm_conf); - } - - @Override - public IConnection bind(String topology_id, int port) { - IConnection retConnection = null; - try { - - retConnection = new NettyServer(storm_conf, port, isSyncMode); - } catch (Throwable e) { - LOG.error("Failed to instance NettyServer", e.getCause()); - JStormUtils.halt_process(-1, "Failed to bind " + port); - } - - return retConnection; - } - - @Override - public IConnection connect(String topology_id, String host, int port) { - if (isSyncMode == true) { - return new NettyClientSync(storm_conf, clientChannelFactory, clientScheduleService, host, port, reconnector); - }else { - return new NettyClientAsync(storm_conf, clientChannelFactory, clientScheduleService, host, port, reconnector); - } - } - - - /** - * terminate this context - */ - public void term() { - clientScheduleService.shutdown(); - // for (IConnection conn : connections) { - // conn.close(); - // } - try { - clientScheduleService.awaitTermination(10, TimeUnit.SECONDS); - } catch (InterruptedException e) { - LOG.error("Error when shutting down client scheduler", e); - } - // connections = null; - - clientChannelFactory.releaseExternalResources(); - - reconnector.shutdown(); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/NettyRenameThreadFactory.java b/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/NettyRenameThreadFactory.java deleted file mode 100644 index 03acd473e..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/NettyRenameThreadFactory.java +++ /dev/null @@ -1,35 +0,0 @@ -package com.alibaba.jstorm.message.netty; - -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.atomic.AtomicInteger; - -import org.jboss.netty.util.ThreadNameDeterminer; -import org.jboss.netty.util.ThreadRenamingRunnable; - -public class NettyRenameThreadFactory implements ThreadFactory { - - static { - //Rename Netty threads - ThreadRenamingRunnable.setThreadNameDeterminer(ThreadNameDeterminer.CURRENT); - } - - final ThreadGroup group; - final AtomicInteger index = new AtomicInteger(1); - final String name; - - NettyRenameThreadFactory(String name) { - SecurityManager s = System.getSecurityManager(); - group = (s != null)? s.getThreadGroup() : - Thread.currentThread().getThreadGroup(); - this.name = name; - } - - public Thread newThread(Runnable r) { - Thread t = new Thread(group, r, name + "-" + index.getAndIncrement(), 0); - if (t.isDaemon()) - t.setDaemon(false); - if (t.getPriority() != Thread.NORM_PRIORITY) - t.setPriority(Thread.NORM_PRIORITY); - return t; - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/NettyServer.java b/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/NettyServer.java deleted file mode 100644 index 93033ec7d..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/NettyServer.java +++ /dev/null @@ -1,199 +0,0 @@ -package com.alibaba.jstorm.message.netty; - -import java.net.InetSocketAddress; -import java.util.List; -import java.util.Map; -import java.util.concurrent.Executors; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; - -import org.jboss.netty.bootstrap.ServerBootstrap; -import org.jboss.netty.channel.Channel; -import org.jboss.netty.channel.ChannelFactory; -import org.jboss.netty.channel.group.ChannelGroup; -import org.jboss.netty.channel.group.DefaultChannelGroup; -import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.alibaba.jstorm.utils.JStormUtils; - -import backtype.storm.Config; -import backtype.storm.messaging.IConnection; -import backtype.storm.messaging.TaskMessage; -import backtype.storm.utils.DisruptorQueue; -import backtype.storm.utils.Utils; - -class NettyServer implements IConnection { - private static final Logger LOG = LoggerFactory - .getLogger(NettyServer.class); - @SuppressWarnings("rawtypes") - Map storm_conf; - int port; - DisruptorQueue recvQueue; - // private LinkedBlockingQueue message_queue; - volatile ChannelGroup allChannels = new DefaultChannelGroup("jstorm-server"); - final ChannelFactory factory; - final ServerBootstrap bootstrap; - - // ayncBatch is only one solution, so directly set it as true - private final boolean isSyncMode; - - @SuppressWarnings("rawtypes") - NettyServer(Map storm_conf, int port, boolean isSyncMode) { - this.storm_conf = storm_conf; - this.port = port; - this.isSyncMode = isSyncMode; - - // Configure the server. - int buffer_size = Utils.getInt(storm_conf - .get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE)); - int maxWorkers = Utils.getInt(storm_conf - .get(Config.STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS)); - - //asyncBatch = ConfigExtension.isNettyTransferAsyncBatch(storm_conf); - - ThreadFactory bossFactory = new NettyRenameThreadFactory("server" - + "-boss"); - ThreadFactory workerFactory = new NettyRenameThreadFactory("server" - + "-worker"); - if (maxWorkers > 0) { - factory = new NioServerSocketChannelFactory( - Executors.newCachedThreadPool(bossFactory), - Executors.newCachedThreadPool(workerFactory), maxWorkers); - } else { - factory = new NioServerSocketChannelFactory( - Executors.newCachedThreadPool(bossFactory), - Executors.newCachedThreadPool(workerFactory)); - } - - bootstrap = new ServerBootstrap(factory); - bootstrap.setOption("reuserAddress", true); - bootstrap.setOption("child.tcpNoDelay", true); - bootstrap.setOption("child.receiveBufferSize", buffer_size); - bootstrap.setOption("child.keepAlive", true); - - // Set up the pipeline factory. - bootstrap.setPipelineFactory(new StormServerPipelineFactory(this)); - - // Bind and start to accept incoming connections. - Channel channel = bootstrap.bind(new InetSocketAddress(port)); - allChannels.add(channel); - - LOG.info("Successfull bind {}, buffer_size:{}, maxWorkers:{}", port, - buffer_size, maxWorkers); - } - - @Override - public void registerQueue(DisruptorQueue recvQueu) { - this.recvQueue = recvQueu; - } - /** - * enqueue a received message - * - * @param message - * @throws InterruptedException - */ - public void enqueue(TaskMessage message) { - - recvQueue.publish(message); - - } - - /** - * fetch a message from message queue synchronously (flags != 1) or - * asynchronously (flags==1) - */ - public TaskMessage recv(int flags) { - try { - if ((flags & 0x01) == 0x01) { - return (TaskMessage) recvQueue.poll(); - // non-blocking - - } else { - return (TaskMessage) recvQueue.take(); - - } - - } catch (Exception e) { - LOG.warn("Occur unexception ", e); - return null; - } - - } - - /** - * register a newly created channel - * - * @param channel - */ - protected void addChannel(Channel channel) { - allChannels.add(channel); - } - - /** - * close a channel - * - * @param channel - */ - protected void closeChannel(Channel channel) { - channel.close().awaitUninterruptibly(); - allChannels.remove(channel); - } - - /** - * close all channels, and release resources - */ - public synchronized void close() { - LOG.info("Begin to shutdown NettyServer"); - if (allChannels != null) { - new Thread(new Runnable() { - - @Override - public void run() { - try { - // await(5, TimeUnit.SECONDS) - // sometimes allChannels.close() will block the exit thread - allChannels.close().await(1, TimeUnit.SECONDS); - LOG.info("Successfully close all channel"); - factory.releaseExternalResources(); - }catch(Exception e) { - - } - allChannels = null; - } - }).start(); - - JStormUtils.sleepMs(1 * 1000); - } - LOG.info("Successfully shutdown NettyServer"); - } - - @Override - public void send(List messages) { - throw new UnsupportedOperationException( - "Server connection should not send any messages"); - } - - @Override - public void send(TaskMessage message) { - throw new UnsupportedOperationException( - "Server connection should not send any messages"); - } - - @Override - public boolean isClosed() { - // TODO Auto-generated method stub - return false; - } - - public boolean isSyncMode() { - return isSyncMode; - } - - - - - -} 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 deleted file mode 100644 index d155aff13..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/ReconnectRunnable.java +++ /dev/null @@ -1,55 +0,0 @@ -package com.alibaba.jstorm.message.netty; - - -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingDeque; - -import org.apache.log4j.Logger; - -import com.alibaba.jstorm.callback.RunnableCallback; - -public class ReconnectRunnable extends RunnableCallback{ - private static final Logger LOG = Logger.getLogger(ReconnectRunnable.class); - - - private BlockingQueue queue = new LinkedBlockingDeque(); - public void pushEvent(NettyClient client) { - queue.offer(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 { - client = queue.take(); - } catch (InterruptedException e) { - continue; - } - if (client != null) { - client.doReconnect(); - } - - } - - LOG.info("Successfully shutdown reconnect thread"); - } - - @Override - public void shutdown() { - closed = true; - thread.interrupt(); - } - - @Override - public Object getResult() { - return -1; - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/StormClientHandler.java b/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/StormClientHandler.java deleted file mode 100644 index 7c6080b71..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/StormClientHandler.java +++ /dev/null @@ -1,87 +0,0 @@ -package com.alibaba.jstorm.message.netty; - -import java.net.ConnectException; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.jboss.netty.channel.Channel; -import org.jboss.netty.channel.ChannelHandlerContext; -import org.jboss.netty.channel.ChannelStateEvent; -import org.jboss.netty.channel.ExceptionEvent; -import org.jboss.netty.channel.MessageEvent; -import org.jboss.netty.channel.SimpleChannelUpstreamHandler; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class StormClientHandler extends SimpleChannelUpstreamHandler { - private static final Logger LOG = LoggerFactory - .getLogger(StormClientHandler.class); - private NettyClient client; - private AtomicBoolean being_closed; - - - StormClientHandler(NettyClient client) { - this.client = client; - being_closed = client.getBeing_closed(); - } - - /** - * Sometime when connect one bad channel which isn't writable, it will call this function - */ - @Override - public void channelConnected(ChannelHandlerContext ctx, - ChannelStateEvent event) { - // register the newly established channel - Channel channel = event.getChannel(); - LOG.info("connection established to :{}, local port:{}", - client.getRemoteAddr(), channel.getLocalAddress()); - - client.handleResponse(); - } - - @Override - public void messageReceived(ChannelHandlerContext ctx, MessageEvent event) { - client.handleResponse(); - - } - - /** - * - * @see org.jboss.netty.channel.SimpleChannelUpstreamHandler#exceptionCaught(org.jboss.netty.channel.ChannelHandlerContext, org.jboss.netty.channel.ExceptionEvent) - */ - @Override - public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent event) { - Throwable cause = event.getCause(); - if (being_closed.get() == false) { - if (!(cause instanceof ConnectException)) { - LOG.info("Connection failed:" + client.getRemoteAddr(), cause); - } - - client.exceptionChannel(event.getChannel()); - client.reconnect(); - } - } - - /** - * Attention please, - * @see org.jboss.netty.channel.SimpleChannelUpstreamHandler#channelDisconnected(org.jboss.netty.channel.ChannelHandlerContext, org.jboss.netty.channel.ChannelStateEvent) - */ - @Override - public void channelDisconnected(ChannelHandlerContext ctx, - ChannelStateEvent e) throws Exception { - LOG.info("Receive channelDisconnected to {}, channel = {}", - client.getRemoteAddr(), e.getChannel()); - // ctx.sendUpstream(e); - super.channelDisconnected(ctx, e); - - client.disconnectChannel(e.getChannel()); - } - - @Override - public void channelClosed(ChannelHandlerContext ctx, ChannelStateEvent e) - throws Exception { - LOG.info("Connection to {} has been closed, channel = {}", - client.getRemoteAddr(), e.getChannel()); - super.channelClosed(ctx, e); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/StormClientPipelineFactory.java b/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/StormClientPipelineFactory.java deleted file mode 100644 index 505eb319a..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/StormClientPipelineFactory.java +++ /dev/null @@ -1,27 +0,0 @@ -package com.alibaba.jstorm.message.netty; - -import org.jboss.netty.channel.ChannelPipeline; -import org.jboss.netty.channel.ChannelPipelineFactory; -import org.jboss.netty.channel.Channels; - -class StormClientPipelineFactory implements ChannelPipelineFactory { - private NettyClient client; - - StormClientPipelineFactory(NettyClient client) { - this.client = client; - } - - public ChannelPipeline getPipeline() throws Exception { - // Create a default pipeline implementation. - ChannelPipeline pipeline = Channels.pipeline(); - - // Decoder - pipeline.addLast("decoder", new MessageDecoder(false)); - // Encoder - pipeline.addLast("encoder", new MessageEncoder()); - // business logic. - pipeline.addLast("handler", new StormClientHandler(client)); - - return pipeline; - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/StormServerHandler.java b/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/StormServerHandler.java deleted file mode 100644 index b714227af..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/StormServerHandler.java +++ /dev/null @@ -1,99 +0,0 @@ -package com.alibaba.jstorm.message.netty; - -import java.util.List; -import java.util.Map; -import java.util.Iterator; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicInteger; - -import org.jboss.netty.channel.Channel; -import org.jboss.netty.channel.ChannelHandlerContext; -import org.jboss.netty.channel.ChannelStateEvent; -import org.jboss.netty.channel.ExceptionEvent; -import org.jboss.netty.channel.MessageEvent; -import org.jboss.netty.channel.SimpleChannelUpstreamHandler; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import backtype.storm.messaging.TaskMessage; - -class StormServerHandler extends SimpleChannelUpstreamHandler { - private static final Logger LOG = LoggerFactory - .getLogger(StormServerHandler.class); - private NettyServer server; - private Map failureCounters; - - StormServerHandler(NettyServer server) { - this.server = server; - failureCounters = new ConcurrentHashMap(); - } - - protected int getFailureCounter(Channel channel) { - Integer num = failureCounters.get(channel); - if (num == null) { - return 0; - } - - return num; - } - - protected void incFailureCounter(Channel channel) { - Integer num = failureCounters.get(channel); - if (num == null) { - num = Integer.valueOf(0); - } - num = num + 1; - - failureCounters.put(channel, num); - } - - protected void removeFailureCounter(Channel channel) { - failureCounters.remove(channel); - } - - @Override - public void channelConnected(ChannelHandlerContext ctx, ChannelStateEvent e) { - LOG.info("Connection established {}", e.getChannel().getRemoteAddress()); - server.addChannel(e.getChannel()); - } - - @Override - public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) { - - Object msg = e.getMessage(); - if (msg == null) - return; - - // end of batch? - if (msg == ControlMessage.EOB_MESSAGE) { - if (server.isSyncMode() == true) { - Channel channel = ctx.getChannel(); - // simplify the logic, just send OK_RESPONSE - channel.write(ControlMessage.OK_RESPONSE); - } - return; - } else if (msg instanceof ControlMessage) { - //LOG.debug("Receive ...{}", msg); - return; - } - - // enqueue the received message for processing - try { - server.enqueue((TaskMessage) msg); - } catch (Exception e1) { - LOG.warn("Failed to enqueue a request message" + e1.toString(), e); - // Channel channel = ctx.getChannel(); - // incFailureCounter(channel); - } - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) { - // removeFailureCounter(e.getChannel()); - if (e.getChannel() != null) { - LOG.info("Channel occur exception {}", e.getChannel().getRemoteAddress()); - } - - server.closeChannel(e.getChannel()); - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/StormServerPipelineFactory.java b/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/StormServerPipelineFactory.java deleted file mode 100644 index 08fc37bf8..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/netty/StormServerPipelineFactory.java +++ /dev/null @@ -1,27 +0,0 @@ -package com.alibaba.jstorm.message.netty; - -import org.jboss.netty.channel.ChannelPipeline; -import org.jboss.netty.channel.ChannelPipelineFactory; -import org.jboss.netty.channel.Channels; - -class StormServerPipelineFactory implements ChannelPipelineFactory { - private NettyServer server; - - StormServerPipelineFactory(NettyServer server) { - this.server = server; - } - - public ChannelPipeline getPipeline() throws Exception { - // Create a default pipeline implementation. - ChannelPipeline pipeline = Channels.pipeline(); - - // Decoder - pipeline.addLast("decoder", new MessageDecoder(true)); - // Encoder - pipeline.addLast("encoder", new MessageEncoder()); - // business logic. - pipeline.addLast("handler", new StormServerHandler(server)); - - return pipeline; - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/message/zeroMq/MQContext.java b/jstorm-server/src/main/java/com/alibaba/jstorm/message/zeroMq/MQContext.java deleted file mode 100644 index e24533d52..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/zeroMq/MQContext.java +++ /dev/null @@ -1,132 +0,0 @@ -package com.alibaba.jstorm.message.zeroMq; - -import java.util.Map; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.zeromq.ZMQ.Context; -import org.zeromq.ZMQ.Socket; - -import backtype.storm.Config; -import backtype.storm.messaging.IConnection; -import backtype.storm.messaging.IContext; - -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.StormConfig; -import com.alibaba.jstorm.utils.JStormUtils; - -/** - * zeroMQ context - * - * @author yannian/Longda/zhiyuan.ls - * - */ -public class MQContext implements IContext { - protected final static Logger LOG = LoggerFactory.getLogger(MQContext.class); - protected Map storm_conf; - protected int zmqThreads; - protected int linger_ms; - protected boolean ipc; - protected boolean virtportZmq = false; - protected int maxQueueMsg; - - private Context context; - - @Override - public void prepare(Map storm_conf) { - this.storm_conf = storm_conf; - zmqThreads = JStormUtils.parseInt(storm_conf - .get(Config.ZMQ_THREADS)); - linger_ms = JStormUtils.parseInt(storm_conf - .get(Config.ZMQ_LINGER_MILLIS)); - ipc = StormConfig.cluster_mode(storm_conf).equals("local"); - virtportZmq = JStormUtils.parseBoolean( - storm_conf.get(Config.STORM_LOCAL_MODE_ZMQ), false); - maxQueueMsg = JStormUtils.parseInt(storm_conf.get(Config.ZMQ_HWM), - ConfigExtension.DEFAULT_ZMQ_MAX_QUEUE_MSG); - init(); - - LOG.info("MQContext prepare done..."); - } - - protected void init() { - context = ZeroMq.context(zmqThreads); - } - - - @SuppressWarnings("unused") - protected MQContext() { - } - - - @Override - public IConnection bind(String topology_id, int port) { - - return zmq_bind(true, port); - - } - - protected IConnection zmq_bind(boolean distributeZmq, int port) { - String url = null; - if (distributeZmq) { - if (ipc) { - url = "ipc://" + port + ".ipc"; - } else { - url = "tcp://*:" + port; - } - } else { - // virtportZmq will be true - url = "inproc://" + port; - } - - Socket socket = ZeroMq.socket(context, ZeroMq.pull); - - ZeroMq.bind(socket, url); - ZeroMq.set_hwm(socket, maxQueueMsg); - - // ZeroMq.subscribe(socket); - - LOG.info("Create zmq receiver {}", url); - return new ZMQRecvConnection(socket); - } - - - @Override - public IConnection connect(String topology_id, String host, int port) { - return zmq_connect(true, host, port); - } - - protected IConnection zmq_connect(boolean distributeZmq, String host, int port) { - String url = null; - - if (distributeZmq) { - if (ipc) { - url = "ipc://" + port + ".ipc"; - } else { - url = "tcp://" + host + ":" + port; - } - } else { - // virtportZmq will be true - url = "inproc://" + port; - } - - Socket socket = ZeroMq.socket(context, ZeroMq.push); - socket = ZeroMq.set_linger(socket, linger_ms); - socket = ZeroMq.connect(socket, url); - ZeroMq.set_hwm(socket, maxQueueMsg); - - LOG.info("Create zmq sender {}", url); - return new ZMQSendConnection(socket, host, port); - } - - public void term() { - LOG.info("ZMQ context terminates "); - context.term(); - } - - -// public Context getContext() { -// return context; -// } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/message/zeroMq/PacketPair.java b/jstorm-server/src/main/java/com/alibaba/jstorm/message/zeroMq/PacketPair.java deleted file mode 100644 index 33ecdbe06..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/zeroMq/PacketPair.java +++ /dev/null @@ -1,73 +0,0 @@ -package com.alibaba.jstorm.message.zeroMq; - -import java.nio.ByteBuffer; - -import backtype.storm.serialization.KryoTupleDeserializer; - -/** - * virtualport send message - * - * @author yannian/Longda - * - */ -public class PacketPair { - private int port; - private byte[] message; - - public PacketPair(int port, byte[] message) { - this.port = port; - this.message = message; - } - - public int getPort() { - return port; - } - - public void setPort(int port) { - this.port = port; - } - - public byte[] getMessage() { - return message; - } - - public void setMessage(byte[] message) { - this.message = message; - } - - public static byte[] mk_packet(int virtual_port, byte[] message) { - ByteBuffer buff = ByteBuffer.allocate((Integer.SIZE / 8) - + message.length); - buff.putInt(virtual_port); - buff.put(message); - byte[] rtn = buff.array(); - return rtn; - } - - public static PacketPair parse_packet(byte[] packet) { - ByteBuffer buff = ByteBuffer.wrap(packet); - int port = buff.getInt(); - - /** - * @@@ Attention please, in order to reduce memory copy - * - * Here directly PacketPair.message use the packet buffer - * - * so need get rid of the target target taskid in - * KryoTupleDeserializer.deserialize - * - * - * The better design should tuple includes targetTaskId - */ - byte[] message = null; - if (KryoTupleDeserializer.USE_RAW_PACKET == true) { - message = packet; - } else { - message = new byte[buff.array().length - (Integer.SIZE / 8)]; - buff.get(message); - } - PacketPair pair = new PacketPair(port, message); - - return pair; - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/message/zeroMq/ZMQRecvConnection.java b/jstorm-server/src/main/java/com/alibaba/jstorm/message/zeroMq/ZMQRecvConnection.java deleted file mode 100644 index e5f3a6ef0..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/zeroMq/ZMQRecvConnection.java +++ /dev/null @@ -1,105 +0,0 @@ -package com.alibaba.jstorm.message.zeroMq; - -import java.util.List; - -import org.apache.log4j.Logger; -import org.zeromq.ZMQ.Socket; - -import com.alibaba.jstorm.callback.AsyncLoopThread; -import com.alibaba.jstorm.callback.RunnableCallback; - -import backtype.storm.messaging.IConnection; -import backtype.storm.messaging.TaskMessage; -import backtype.storm.serialization.KryoTupleDeserializer; -import backtype.storm.utils.DisruptorQueue; - -/** - * - * @author longda - * - */ -public class ZMQRecvConnection extends RunnableCallback implements IConnection { - private static final Logger LOG = Logger.getLogger(ZMQRecvConnection.class); - - private Socket socket; - private boolean closed = false; - private DisruptorQueue recvQueue; - - public ZMQRecvConnection(Socket _socket) { - socket = _socket; - - new AsyncLoopThread(this, true, - Thread.MAX_PRIORITY, true); - } - - @Override - public TaskMessage recv(int flags) { - byte[] data = ZeroMq.recv(socket, flags); - if (data == null || data.length <= 4) { - return null; - } - int port = KryoTupleDeserializer.deserializeTaskId(data); - return new TaskMessage(port, data); - } - - - @Override - public void close() { - socket.close(); - closed = true; - } - - @Override - public boolean isClosed() { - return closed; - } - - @Override - public void send(List messages) { - throw new UnsupportedOperationException( - "Server connection should not send any messages"); - } - @Override - public void send(TaskMessage message) { - throw new UnsupportedOperationException( - "Server connection should not send any messages"); - } - - @Override - public void registerQueue(DisruptorQueue recvQueu) { - this.recvQueue = recvQueu; - } - - @Override - public void enqueue(TaskMessage message) { - if (message != null ) { - recvQueue.publish(message); - } - - } - - @Override - public void run() { - LOG.info("Successfully start ZMQ Recv thread"); - - while(isClosed() == false) { - try { - TaskMessage message = recv(0); - enqueue(message); - }catch (Exception e) { - LOG.warn("ZMQ Recv thread receive error", e); - } - } - - LOG.info("Successfully shutdown ZMQ Recv thread"); - - } - - @Override - public Object getResult() { - LOG.info("Begin to shutdown ZMQ Recv thread"); - return -1; - } - - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/message/zeroMq/ZMQSendConnection.java b/jstorm-server/src/main/java/com/alibaba/jstorm/message/zeroMq/ZMQSendConnection.java deleted file mode 100644 index 0f99a6532..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/zeroMq/ZMQSendConnection.java +++ /dev/null @@ -1,92 +0,0 @@ -package com.alibaba.jstorm.message.zeroMq; - -import java.util.List; - -import org.zeromq.ZMQ.Socket; - -import backtype.storm.messaging.IConnection; -import backtype.storm.messaging.TaskMessage; -import backtype.storm.utils.DisruptorQueue; - -import com.alibaba.jstorm.metric.MetricDef; -import com.alibaba.jstorm.metric.JStormHistogram; -import com.alibaba.jstorm.metric.JStormTimer; -import com.alibaba.jstorm.metric.Metrics; -import com.alibaba.jstorm.utils.JStormServerUtils; - -/** - * - * @author longda - * - */ -public class ZMQSendConnection implements IConnection { - private org.zeromq.ZMQ.Socket socket; - private boolean closed = false; - private JStormTimer timer; - private JStormHistogram histogram; - private String prefix; - - public ZMQSendConnection(Socket _socket, String host, int port) { - socket = _socket; - prefix = JStormServerUtils.getName(host, port); - timer = Metrics.registerTimer(prefix, MetricDef.ZMQ_SEND_TIME, - null, Metrics.MetricType.WORKER); - histogram = Metrics.registerHistograms(prefix, MetricDef.ZMQ_SEND_MSG_SIZE, - null, Metrics.MetricType.WORKER); - } - - @Override - public void close() { - socket.close(); - closed = true; - } - - @Override - public boolean isClosed() { - return closed; - } - - @Override - public void registerQueue(DisruptorQueue recvQueu) { - throw new UnsupportedOperationException( - "recvTask() Client connection should not receive any messages"); - } - - @Override - public void enqueue(TaskMessage message) { - throw new UnsupportedOperationException( - "recvTask() Client connection should not receive any messages"); - } - - @Override - public void send(List messages) { - timer.start(); - - try { - for (TaskMessage message : messages) { - ZeroMq.send(socket, message.message()); - } - } finally { - timer.stop(); - histogram.update(messages.size()); - } - } - - @Override - public void send(TaskMessage message) { - timer.start(); - try { - ZeroMq.send(socket, message.message()); - } finally { - timer.stop(); - histogram.update(1); - } - } - - @Override - public TaskMessage recv(int flags) { - throw new UnsupportedOperationException( - "recvTask() Client connection should not receive any messages"); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/message/zeroMq/ZeroMq.java b/jstorm-server/src/main/java/com/alibaba/jstorm/message/zeroMq/ZeroMq.java deleted file mode 100644 index b910a3199..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/message/zeroMq/ZeroMq.java +++ /dev/null @@ -1,112 +0,0 @@ -package com.alibaba.jstorm.message.zeroMq; - -import org.apache.log4j.Logger; -import org.zeromq.ZMQ; -import org.zeromq.ZMQ.Context; -import org.zeromq.ZMQ.Socket; - -import com.alibaba.jstorm.utils.JStormUtils; - -/** - * Wrapper zeroMQ interface - * - * @author yannian - * - */ -public class ZeroMq { - private static Logger LOG = Logger.getLogger(ZeroMq.class); - - public static Context context(int threads) { - try { - return ZMQ.context(threads); - } catch (UnsatisfiedLinkError e) { - LOG.error("Failed to create zeroMQ context", e); - throw new RuntimeException(e); - } - } - - public static int sndmore = ZMQ.SNDMORE; - public static int req = ZMQ.REQ; - public static int rep = ZMQ.REP; - public static int xreq = ZMQ.XREQ; - public static int xrep = ZMQ.XREP; - public static int pub = ZMQ.PUB; - public static int sub = ZMQ.SUB; - public static int pair = ZMQ.PAIR; - public static int push = ZMQ.PUSH; - public static int pull = ZMQ.PULL; - - public static byte[] barr(Short v) { - return JStormUtils.barr(v); - } - - public static byte[] barr(Integer v) { - return JStormUtils.barr(v); - } - - public static Socket socket(Context context, int type) { - return context.socket(type); - } - - public static Socket set_linger(Socket socket, long linger_ms) { - socket.setLinger(linger_ms); - return socket; - } - - public static Socket set_hwm(Socket socket, long hwm) { - socket.setHWM(hwm); - return socket; - } - - public static Socket bind(Socket socket, String url) { - socket.bind(url); - return socket; - } - - public static Socket connect(Socket socket, String url) { - socket.connect(url); - return socket; - } - - public static Socket subscribe(Socket socket, byte[] topic) { - socket.subscribe(topic); - return socket; - } - - public static Socket subscribe(Socket socket) { - byte[] topic = {}; - return subscribe(socket, topic); - } - - public static Socket unsubscribe(Socket socket, byte[] topic) { - socket.unsubscribe(topic); - return socket; - } - - public static Socket unsubscribe(Socket socket) { - byte[] topic = {}; - return unsubscribe(socket, topic); - } - - public static Socket send(Socket socket, byte[] message, int flags) { - socket.send(message, flags); - return socket; - } - - public static Socket send(Socket socket, byte[] message) { - return send(socket, message, org.zeromq.ZMQ.NOBLOCK); - } - - public static byte[] recv(Socket socket, int flags) { - return socket.recv(flags); - } - - public static byte[] recv(Socket socket) { - return recv(socket, 0); - } - - public static boolean hasRecvMore(Socket socket) { - return socket.hasReceiveMore(); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/queue/disruptor/JstormEvent.java b/jstorm-server/src/main/java/com/alibaba/jstorm/queue/disruptor/JstormEvent.java deleted file mode 100644 index b3117b5c1..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/queue/disruptor/JstormEvent.java +++ /dev/null @@ -1,23 +0,0 @@ -package com.alibaba.jstorm.queue.disruptor; - -import com.lmax.disruptor.EventFactory; - -public class JstormEvent { - - private String msgId; - - public String getMsgId() { - return msgId; - } - - public void setMsgId(String msgId) { - this.msgId = msgId; - } - - public final static EventFactory EVENT_FACTORY = new EventFactory() { - public JstormEvent newInstance() { - return new JstormEvent(); - } - }; - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/queue/disruptor/JstormEventHandler.java b/jstorm-server/src/main/java/com/alibaba/jstorm/queue/disruptor/JstormEventHandler.java deleted file mode 100644 index 1446f2459..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/queue/disruptor/JstormEventHandler.java +++ /dev/null @@ -1,29 +0,0 @@ -package com.alibaba.jstorm.queue.disruptor; - -import org.apache.log4j.Logger; - -import com.lmax.disruptor.EventHandler; - -public class JstormEventHandler implements EventHandler { - - Logger logger = Logger.getLogger(JstormEventHandler.class); - - private int count; - - public JstormEventHandler(int count) { - this.count = count; - } - - @Override - public void onEvent(Object event, long sequence, boolean endOfBatch) - throws Exception { - long msgId = Long.parseLong(((JstormEvent) event).getMsgId()); - // if (msgId % size ==0) { - // logger.warn("consumer msgId=" + msgId + ", seq=" + sequence); - // } - if (msgId == count - 1) { - logger.warn("end..." + System.currentTimeMillis()); - } - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/queue/disruptor/JstormProducer.java b/jstorm-server/src/main/java/com/alibaba/jstorm/queue/disruptor/JstormProducer.java deleted file mode 100644 index 8cc7cb894..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/queue/disruptor/JstormProducer.java +++ /dev/null @@ -1,45 +0,0 @@ -package com.alibaba.jstorm.queue.disruptor; - -import org.apache.log4j.Logger; - -import com.lmax.disruptor.RingBuffer; - -public class JstormProducer implements Runnable { - - Logger logger = Logger.getLogger(JstormProducer.class); - - private RingBuffer ringBuffer; - private int size; - - public JstormProducer(RingBuffer ringBuffer, int size) { - this.ringBuffer = ringBuffer; - this.size = size; - } - - @Override - public void run() { - logger.warn("producer start..." + System.currentTimeMillis()); - - // while (true) { - // long seqId = ringBuffer.next(); - // - // ringBuffer.get(seqId).setMsgId(String.valueOf(seqId)); - // ringBuffer.publish(seqId); - // - // try { - // double random = Math.random(); - // Thread.sleep((long)(random * 1000)); - // } catch (InterruptedException e) { - // e.printStackTrace(); - // } - // } - - for (int i = 0; i < size; i++) { - long seqId = ringBuffer.next(); - - ringBuffer.get(seqId).setMsgId(String.valueOf(seqId)); - ringBuffer.publish(seqId); - } - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/CleanRunnable.java b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/CleanRunnable.java deleted file mode 100644 index 1788c7e91..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/CleanRunnable.java +++ /dev/null @@ -1,65 +0,0 @@ -package com.alibaba.jstorm.schedule; - -import java.io.File; - -import org.apache.log4j.Logger; - -import com.alibaba.jstorm.utils.OlderFileFilter; - -/** - * clean /nimbus/inbox jar every 600 seconds - * - * Default expire time is 3600 seconds - * - * @author lixin - * - */ -public class CleanRunnable implements Runnable { - - private static Logger log = Logger.getLogger(CleanRunnable.class); - - private String dir_location; - - private int seconds; - - public CleanRunnable(String dir_location, int inbox_jar_expiration_secs) { - this.dir_location = dir_location; - this.seconds = inbox_jar_expiration_secs; - } - - @Override - public void run() { - File inboxdir = new File(dir_location); - clean(inboxdir); - } - - private void clean(File file) { - // filter - OlderFileFilter filter = new OlderFileFilter(seconds); - - File[] files = file.listFiles(filter); - for (File f : files) { - if (f.isFile()) { - log.info("Cleaning inbox ... deleted: " + f.getName()); - try { - f.delete(); - } catch (Exception e) { - log.error("Cleaning inbox ... error deleting:" - + f.getName() + "," + e); - } - } else { - clean(f); - if (f.listFiles().length == 0) { - log.info("Cleaning inbox ... deleted: " + f.getName()); - try { - f.delete(); - } catch (Exception e) { - log.error("Cleaning inbox ... error deleting:" - + f.getName() + "," + e); - } - } - } - } - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/DelayEventRunnable.java b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/DelayEventRunnable.java deleted file mode 100644 index c3820b7c2..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/DelayEventRunnable.java +++ /dev/null @@ -1,25 +0,0 @@ -package com.alibaba.jstorm.schedule; - -import com.alibaba.jstorm.daemon.nimbus.NimbusData; -import com.alibaba.jstorm.daemon.nimbus.NimbusUtils; -import com.alibaba.jstorm.daemon.nimbus.StatusType; - -public class DelayEventRunnable implements Runnable { - - private NimbusData data; - private String topologyid; - private StatusType status; - - public DelayEventRunnable(NimbusData data, String topologyid, - StatusType status) { - this.data = data; - this.topologyid = topologyid; - this.status = status; - } - - @Override - public void run() { - NimbusUtils.transition(data, topologyid, false, status); - } - -} 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 deleted file mode 100644 index 34bfa69f4..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/FollowerRunnable.java +++ /dev/null @@ -1,269 +0,0 @@ -package com.alibaba.jstorm.schedule; - -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -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; - -import backtype.storm.Config; -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.Cluster; -import com.alibaba.jstorm.cluster.StormClusterState; -import com.alibaba.jstorm.cluster.StormConfig; -import com.alibaba.jstorm.daemon.nimbus.NimbusData; -import com.alibaba.jstorm.task.Assignment; -import com.alibaba.jstorm.utils.JStormServerUtils; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.NetWorkUtils; -import com.alibaba.jstorm.utils.PathUtils; - -public class FollowerRunnable implements Runnable { - - private static final Logger LOG = Logger.getLogger(FollowerRunnable.class); - - private NimbusData data; - - private int sleepTime; - - private volatile boolean state = true; - - private RunnableCallback callback; - - private final String hostPort; - - @SuppressWarnings("unchecked") - public FollowerRunnable(final NimbusData data, int sleepTime) { - this.data = data; - this.sleepTime = sleepTime; - if (!ConfigExtension.isNimbusUseIp(data.getConf())) { - this.hostPort = NetWorkUtils.hostname() - + ":" - + String.valueOf(Utils.getInt(data.getConf().get( - Config.NIMBUS_THRIFT_PORT))); - } else { - this.hostPort = NetWorkUtils.ip() - + ":" - + String.valueOf(Utils.getInt(data.getConf().get( - Config.NIMBUS_THRIFT_PORT))); - } - try { - this.tryToBeLeader(data.getConf()); - } catch (Exception e1) { - // TODO Auto-generated catch block - LOG.error("try to be leader error.", e1); - throw new RuntimeException(e1); - } - try { - data.getStormClusterState().register_nimbus_host(hostPort); - } catch (Exception e) { - LOG.error("register nimbus host fail!", e); - throw new RuntimeException(); - } - callback = new RunnableCallback() { - @Override - public void run() { - if (!data.isLeader()) - check(); - } - }; - } - - 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 (!zkClusterState.leader_existed()) { - this.tryToBeLeader(data.getConf()); - continue; - } - - 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(); - zkClusterState.update_follower_hb(hostPort, data.uptime()); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - continue; - } catch (Exception e) { - if (state) { - LOG.error("Unknow exception ", e); - } - } - } - LOG.info("Follower Thread has closed!"); - } - - public void clean() { - state = false; - } - - private synchronized void check() { - - StormClusterState clusterState = data.getStormClusterState(); - - try { - String master_stormdist_root = StormConfig.masterStormdistRoot(data - .getConf()); - - List code_ids = PathUtils - .read_dir_contents(master_stormdist_root); - - List assignments_ids = clusterState.assignments(callback); - - List done_ids = new ArrayList(); - - for (String id : code_ids) { - if (assignments_ids.contains(id)) { - done_ids.add(id); - } - } - - for (String id : done_ids) { - assignments_ids.remove(id); - code_ids.remove(id); - } - - for (String topologyId : code_ids) { - deleteLocalTopology(topologyId); - } - - for (String id : assignments_ids) { - Assignment assignment = clusterState.assignment_info(id, null); - downloadCodeFromMaster(assignment, id); - } - } catch (IOException e) { - // TODO Auto-generated catch block - LOG.error("Get stormdist dir error!", e); - return; - } catch (Exception e) { - // TODO Auto-generated catch block - LOG.error("Check error!", e); - return; - } - } - - private void deleteLocalTopology(String topologyId) throws IOException { - String dir_to_delete = StormConfig.masterStormdistRoot(data.getConf(), - topologyId); - try { - PathUtils.rmr(dir_to_delete); - LOG.info("delete:" + dir_to_delete + "successfully!"); - } catch (IOException e) { - // TODO Auto-generated catch block - LOG.error("delete:" + dir_to_delete + "fail!", e); - } - } - - private void downloadCodeFromMaster(Assignment assignment, String topologyId) - throws IOException, TException { - try { - String localRoot = StormConfig.masterStormdistRoot(data.getConf(), - topologyId); - String tmpDir = StormConfig.masterInbox(data.getConf()) + "/" - + UUID.randomUUID().toString(); - String masterCodeDir = assignment.getMasterCodeDir(); - JStormServerUtils.downloadCodeFromMaster(data.getConf(), tmpDir, - masterCodeDir, topologyId, false); - - FileUtils.moveDirectory(new File(tmpDir), new File(localRoot)); - } catch (TException e) { - // TODO Auto-generated catch block - LOG.error(e + " downloadStormCode failed " + "topologyId:" - + topologyId + "masterCodeDir:" - + assignment.getMasterCodeDir()); - throw e; - } - LOG.info("Finished downloading code for topology id " + topologyId - + " from " + assignment.getMasterCodeDir()); - } - - private void tryToBeLeader(final Map conf) throws Exception { - RunnableCallback masterCallback = new RunnableCallback() { - @Override - public void run() { - try { - tryToBeLeader(conf); - } catch (Exception e) { - LOG.error("To be master error", e); - JStormUtils.halt_process(30, - "Cant't to be master" + e.getMessage()); - } - } - }; - data.getStormClusterState().try_to_be_leader( - Cluster.MASTER_SUBTREE, hostPort, masterCallback); - } - - /** - * Check whether current node is master or not - * - * @throws Exception - */ - private void checkOwnMaster() throws Exception { - int retry_times = 10; - - StormClusterState zkClient = data.getStormClusterState(); - for (int i = 0; i < retry_times; i++, JStormUtils.sleepMs(sleepTime)) { - - if (zkClient.leader_existed() == false) { - continue; - } - - String zkHost = zkClient.get_leader_host(); - if (hostPort.equals(zkHost) == true) { - // current process own master - return; - } - LOG.warn("Current Nimbus has start thrift, but fail to own zk master :" - + zkHost); - } - - // current process doesn't own master - String err = "Current Nimubs fail to own nimbus_master, should halt process"; - LOG.error(err); - JStormUtils.halt_process(0, err); - - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/IToplogyScheduler.java b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/IToplogyScheduler.java deleted file mode 100644 index b5be4ff28..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/IToplogyScheduler.java +++ /dev/null @@ -1,14 +0,0 @@ -package com.alibaba.jstorm.schedule; - -import java.util.Map; -import java.util.Set; - -import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; -import com.alibaba.jstorm.utils.FailedAssignTopologyException; - -public interface IToplogyScheduler { - void prepare(Map conf); - - Set assignTasks(TopologyAssignContext contex) - throws FailedAssignTopologyException; -} 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 deleted file mode 100644 index 6fbec61fc..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/MonitorRunnable.java +++ /dev/null @@ -1,101 +0,0 @@ -package com.alibaba.jstorm.schedule; - -import java.util.Date; -import java.util.List; - -import org.apache.log4j.Logger; - -import com.alibaba.jstorm.cluster.StormClusterState; -import com.alibaba.jstorm.daemon.nimbus.NimbusData; -import com.alibaba.jstorm.daemon.nimbus.NimbusUtils; -import com.alibaba.jstorm.daemon.nimbus.StatusType; -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; - -/** - * - * Scan all task's heartbeat, if task isn't alive, DO - * NimbusUtils.transition(monitor) - * - * @author Longda - * - */ -public class MonitorRunnable implements Runnable { - private static Logger LOG = Logger.getLogger(MonitorRunnable.class); - - private NimbusData data; - - public MonitorRunnable(NimbusData data) { - this.data = data; - } - - /** - * @@@ Todo when one topology is being reassigned, the topology should be - * skip check - * @param data - */ - @Override - public void run() { - StormClusterState clusterState = data.getStormClusterState(); - - try { - // Attetion, need first check Assignments - List active_topologys = clusterState.assignments(null); - - if (active_topologys == null) { - LOG.info("Failed to get active topologies"); - return; - } - - for (String topologyid : active_topologys) { - - LOG.debug("Check tasks " + topologyid); - - // Attention, here don't check /ZK-dir/taskbeats/topologyid to - // get task ids - List taskIds = clusterState.task_ids(topologyid); - if (taskIds == null) { - LOG.info("Failed to get task ids of " + topologyid); - continue; - } - Assignment assignment = clusterState.assignment_info( - topologyid, null); - - boolean needReassign = false; - for (Integer task : taskIds) { - boolean isTaskDead = NimbusUtils.isTaskDead(data, - topologyid, task); - if (isTaskDead == true) { - LOG.info("Found " + topologyid + ",taskid:" + task - + " is dead"); - - ResourceWorkerSlot resource = null; - if (assignment != null) - resource = assignment.getWorkerByTaskId(task); - if (resource != null) { - Date now = new Date(); - String nowStr = TimeFormat.getSecond(now); - String errorInfo = "Task-" + task + " is dead on " - + resource.getHostname() + ":" - + resource.getPort() + ", " + nowStr; - LOG.info(errorInfo); - clusterState.report_task_error(topologyid, task, errorInfo); - } - needReassign = true; - } - } - if (needReassign == true) { - NimbusUtils.transition(data, topologyid, false, - StatusType.monitor); - } - } - - } catch (Exception e) { - // TODO Auto-generated catch block - LOG.error(e.getCause(), e); - } - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/TopologyAssignContext.java b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/TopologyAssignContext.java deleted file mode 100644 index 596efbdd1..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/TopologyAssignContext.java +++ /dev/null @@ -1,151 +0,0 @@ -package com.alibaba.jstorm.schedule; - -import java.util.Map; -import java.util.Set; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; - -import backtype.storm.generated.StormTopology; - -import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; -import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; -import com.alibaba.jstorm.task.Assignment; - -public class TopologyAssignContext { - - public static final int ASSIGN_TYPE_NEW = 0; // assign a new topology - - public static final int ASSIGN_TYPE_REBALANCE = 1; // rebalance a topology - - public static final int ASSIGN_TYPE_MONITOR = 2; // monitor a topology, some - // tasks are dead - - protected int assignType; - - protected StormTopology rawTopology; - - protected Map stormConf; - - // if assignType is ASSIGN_TYPE_NEW, oldAssignment is the Assignment last - // time - // otherwise it is the old assignment before assignment. - protected Assignment oldAssignment; - - protected Map cluster; - - protected Map taskToComponent; - - protected Set allTaskIds; // all tasks - protected Set deadTaskIds; // dead tasks - protected Set unstoppedTaskIds; // the task is alive, but his - // supervisor is dead - protected Set unstoppedWorkers; - - public TopologyAssignContext() { - - } - - public TopologyAssignContext(TopologyAssignContext copy) { - this.assignType = copy.getAssignType(); - this.rawTopology = copy.getRawTopology(); - this.stormConf = copy.getStormConf(); - this.oldAssignment = copy.getOldAssignment(); - this.cluster = copy.getCluster(); - this.taskToComponent = copy.getTaskToComponent(); - this.allTaskIds = copy.getAllTaskIds(); - this.deadTaskIds = copy.getDeadTaskIds(); - this.unstoppedTaskIds = copy.getUnstoppedTaskIds(); - } - - public int getAssignType() { - return assignType; - } - - public void setAssignType(int assignType) { - this.assignType = assignType; - } - - public StormTopology getRawTopology() { - return rawTopology; - } - - public void setRawTopology(StormTopology rawTopology) { - this.rawTopology = rawTopology; - } - - public Map getStormConf() { - return stormConf; - } - - public void setStormConf(Map stormConf) { - this.stormConf = stormConf; - } - - public Assignment getOldAssignment() { - return oldAssignment; - } - - public void setOldAssignment(Assignment oldAssignment) { - this.oldAssignment = oldAssignment; - } - - public Map getCluster() { - return cluster; - } - - public void setCluster(Map cluster) { - this.cluster = cluster; - } - - public Set getAllTaskIds() { - return allTaskIds; - } - - public void setAllTaskIds(Set allTaskIds) { - this.allTaskIds = allTaskIds; - } - - public Set getDeadTaskIds() { - return deadTaskIds; - } - - public void setDeadTaskIds(Set deadTaskIds) { - this.deadTaskIds = deadTaskIds; - } - - public Set getUnstoppedTaskIds() { - return unstoppedTaskIds; - } - - public void setUnstoppedTaskIds(Set unstoppedTaskIds) { - this.unstoppedTaskIds = unstoppedTaskIds; - } - - public Map getTaskToComponent() { - return taskToComponent; - } - - public void setTaskToComponent(Map taskToComponent) { - this.taskToComponent = taskToComponent; - } - - public static boolean isAssignTypeValid(int type) { - return (type == ASSIGN_TYPE_NEW) || (type == ASSIGN_TYPE_REBALANCE) - || (type == ASSIGN_TYPE_MONITOR); - } - - public Set getUnstoppedWorkers() { - return unstoppedWorkers; - } - - public void setUnstoppedWorkers(Set unstoppedWorkers) { - this.unstoppedWorkers = unstoppedWorkers; - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this, - ToStringStyle.SHORT_PREFIX_STYLE); - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/DefaultTopologyAssignContext.java b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/DefaultTopologyAssignContext.java deleted file mode 100644 index 20f579113..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/DefaultTopologyAssignContext.java +++ /dev/null @@ -1,203 +0,0 @@ -package com.alibaba.jstorm.schedule.default_assign; - -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; - -import backtype.storm.Config; -import backtype.storm.generated.Bolt; -import backtype.storm.generated.ComponentCommon; -import backtype.storm.generated.SpoutSpec; -import backtype.storm.generated.StateSpoutSpec; -import backtype.storm.generated.StormTopology; -import backtype.storm.utils.ThriftTopologyUtils; - -import com.alibaba.jstorm.cluster.Common; -import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; -import com.alibaba.jstorm.schedule.TopologyAssignContext; -import com.alibaba.jstorm.utils.FailedAssignTopologyException; -import com.alibaba.jstorm.utils.JStormUtils; - -public class DefaultTopologyAssignContext extends TopologyAssignContext { - - private final StormTopology sysTopology; - private final Map sidToHostname; - private final Map> hostToSid; - private final Set oldWorkers; - private final Map> componentTasks; - private final Set unstoppedWorkers = new HashSet(); - private final int totalWorkerNum; - private final int unstoppedWorkerNum; - - private int computeWorkerNum() { - Integer settingNum = JStormUtils.parseInt(stormConf - .get(Config.TOPOLOGY_WORKERS)); - - int hintSum = 0; - - Map components = ThriftTopologyUtils - .getComponents(sysTopology); - for (Entry entry : components.entrySet()) { - String componentName = entry.getKey(); - Object component = entry.getValue(); - - ComponentCommon common = null; - if (component instanceof Bolt) { - common = ((Bolt) component).get_common(); - } - if (component instanceof SpoutSpec) { - common = ((SpoutSpec) component).get_common(); - } - if (component instanceof StateSpoutSpec) { - common = ((StateSpoutSpec) component).get_common(); - } - - int hint = common.get_parallelism_hint(); - hintSum += hint; - } - - if (settingNum == null) { - return hintSum; - } else { - return Math.min(settingNum, hintSum); - } - } - - public int computeUnstoppedAssignments() { - for (Integer task : unstoppedTaskIds) { - // if unstoppedTasksIds isn't empty, it should be REASSIGN/Monitor - ResourceWorkerSlot worker = oldAssignment.getWorkerByTaskId(task); - unstoppedWorkers.add(worker); - } - - return unstoppedWorkers.size(); - } - - private void refineDeadTasks() { - Set rawDeadTasks = getDeadTaskIds(); - Set refineDeadTasks = new HashSet(); - refineDeadTasks.addAll(rawDeadTasks); - - Set unstoppedTasks = getUnstoppedTaskIds(); - - // if type is ASSIGN_NEW, rawDeadTasks is empty - // then the oldWorkerTasks should be existingAssignment - for (Integer task : rawDeadTasks) { - if (unstoppedTasks.contains(task)) { - continue; - } - for (ResourceWorkerSlot worker : oldWorkers) { - if (worker.getTasks().contains(task)) { - refineDeadTasks.addAll(worker.getTasks()); - - } - } - } - setDeadTaskIds(refineDeadTasks); - } - - /** - * @@@ Do we need just handle the case whose type is ASSIGN_TYPE_NEW? - * - * @return - */ - private Map generateSidToHost() { - Map sidToHostname = new HashMap(); - if (oldAssignment != null) { - sidToHostname.putAll(oldAssignment.getNodeHost()); - } - - for (Entry entry : cluster.entrySet()) { - String supervisorId = entry.getKey(); - SupervisorInfo supervisorInfo = entry.getValue(); - - sidToHostname.put(supervisorId, supervisorInfo.getHostName()); - - } - - return sidToHostname; - } - - public DefaultTopologyAssignContext(TopologyAssignContext context) { - super(context); - - try { - sysTopology = Common.system_topology(stormConf, rawTopology); - } catch (Exception e) { - throw new FailedAssignTopologyException( - "Failed to generate system topology"); - } - - sidToHostname = generateSidToHost(); - hostToSid = JStormUtils.reverse_map(sidToHostname); - - if (oldAssignment != null && oldAssignment.getWorkers() != null) { - oldWorkers = oldAssignment.getWorkers(); - } else { - oldWorkers = new HashSet(); - } - - refineDeadTasks(); - - componentTasks = JStormUtils.reverse_map(context.getTaskToComponent()); - - for (Entry> entry : componentTasks.entrySet()) { - List componentTaskList = entry.getValue(); - - Collections.sort(componentTaskList); - } - - totalWorkerNum = computeWorkerNum(); - - unstoppedWorkerNum = computeUnstoppedAssignments(); - } - - public StormTopology getSysTopology() { - return sysTopology; - } - - public Map getSidToHostname() { - return sidToHostname; - } - - public Map> getHostToSid() { - return hostToSid; - } - - public Map> getComponentTasks() { - return componentTasks; - } - - public int getTotalWorkerNum() { - return totalWorkerNum; - } - - public int getUnstoppedWorkerNum() { - return unstoppedWorkerNum; - } - - public Set getOldWorkers() { - return oldWorkers; - } - - public Set getUnstoppedWorkers() { - return unstoppedWorkers; - } - - @Override - public String toString() { - return (String) stormConf.get(Config.TOPOLOGY_NAME); - } - - public String toDetailString() { - return ToStringBuilder.reflectionToString(this, - ToStringStyle.SHORT_PREFIX_STYLE); - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/DefaultTopologyScheduler.java b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/DefaultTopologyScheduler.java deleted file mode 100644 index 23595102e..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/DefaultTopologyScheduler.java +++ /dev/null @@ -1,162 +0,0 @@ -package com.alibaba.jstorm.schedule.default_assign; - -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.log4j.Logger; - -import com.alibaba.jstorm.daemon.supervisor.SupervisorInfo; -import com.alibaba.jstorm.schedule.IToplogyScheduler; -import com.alibaba.jstorm.schedule.TopologyAssignContext; -import com.alibaba.jstorm.task.Assignment; -import com.alibaba.jstorm.utils.FailedAssignTopologyException; - -public class DefaultTopologyScheduler implements IToplogyScheduler { - private static final Logger LOG = Logger - .getLogger(DefaultTopologyScheduler.class); - - private Map nimbusConf; - - @Override - public void prepare(Map conf) { - nimbusConf = conf; - } - - /** - * @@@ Here maybe exist one problem, some dead slots have been free - * - * @param context - */ - protected void freeUsed(TopologyAssignContext context) { - Set canFree = new HashSet(); - canFree.addAll(context.getAllTaskIds()); - canFree.removeAll(context.getUnstoppedTaskIds()); - - Map cluster = context.getCluster(); - Assignment oldAssigns = context.getOldAssignment(); - for (Integer task : canFree) { - ResourceWorkerSlot worker = oldAssigns.getWorkerByTaskId(task); - if (worker == null) { - LOG.warn("When free rebalance resource, no ResourceAssignment of task " - + task); - continue; - } - - SupervisorInfo supervisorInfo = cluster.get(worker.getNodeId()); - if (supervisorInfo == null) { - continue; - } - supervisorInfo.getWorkerPorts().add(worker.getPort()); - } - } - - private Set getNeedAssignTasks(DefaultTopologyAssignContext context) { - Set needAssign = new HashSet(); - - int assignType = context.getAssignType(); - if (assignType == TopologyAssignContext.ASSIGN_TYPE_NEW) { - needAssign.addAll(context.getAllTaskIds()); - } else if (assignType == TopologyAssignContext.ASSIGN_TYPE_REBALANCE) { - needAssign.addAll(context.getAllTaskIds()); - needAssign.removeAll(context.getUnstoppedTaskIds()); - } else { - // monitor - needAssign.addAll(context.getDeadTaskIds()); - } - - return needAssign; - } - - /** - * Get the task Map which the task is alive and will be kept Only when type - * is ASSIGN_TYPE_MONITOR, it is valid - * - * @param defaultContext - * @param needAssigns - * @return - */ - public Set getKeepAssign( - DefaultTopologyAssignContext defaultContext, - Set needAssigns) { - - Set keepAssignIds = new HashSet(); - keepAssignIds.addAll(defaultContext.getAllTaskIds()); - keepAssignIds.removeAll(defaultContext.getUnstoppedTaskIds()); - keepAssignIds.removeAll(needAssigns); - Set keeps = new HashSet(); - if (keepAssignIds.isEmpty()) { - return keeps; - } - - Assignment oldAssignment = defaultContext.getOldAssignment(); - if (oldAssignment == null) { - return keeps; - } - keeps.addAll(defaultContext.getOldWorkers()); - for (ResourceWorkerSlot worker : defaultContext.getOldWorkers()) { - for (Integer task : worker.getTasks()) { - if (!keepAssignIds.contains(task)) { - keeps.remove(worker); - break; - } - } - } - return keeps; - } - - @Override - public Set assignTasks(TopologyAssignContext context) - throws FailedAssignTopologyException { - - int assignType = context.getAssignType(); - if (TopologyAssignContext.isAssignTypeValid(assignType) == false) { - throw new FailedAssignTopologyException("Invalide Assign Type " - + assignType); - } - - DefaultTopologyAssignContext defaultContext = new DefaultTopologyAssignContext( - context); - if (assignType == TopologyAssignContext.ASSIGN_TYPE_REBALANCE) { - freeUsed(defaultContext); - } - LOG.info("Dead tasks:" + defaultContext.getDeadTaskIds()); - LOG.info("Unstopped tasks:" + defaultContext.getUnstoppedTaskIds()); - - Set needAssignTasks = getNeedAssignTasks(defaultContext); - - Set keepAssigns = getKeepAssign(defaultContext, - needAssignTasks); - - // please use tree map to make task sequence - Set ret = new HashSet(); - ret.addAll(keepAssigns); - ret.addAll(defaultContext.getUnstoppedWorkers()); - - int allocWorkerNum = defaultContext.getTotalWorkerNum() - - defaultContext.getUnstoppedWorkerNum() - keepAssigns.size(); - - if (allocWorkerNum <= 0) { - LOG.warn("Don't need assign workers, all workers are fine " - + defaultContext.toDetailString()); - throw new FailedAssignTopologyException( - "Don't need assign worker, all workers are fine "); - } - - List newAssignList = WorkerMaker.getInstance() - .makeWorkers(defaultContext, needAssignTasks, allocWorkerNum); - TaskGanker ganker = new TaskGanker(defaultContext, needAssignTasks, - newAssignList); - Set newAssigns = new HashSet( - ganker.gankTask()); - ret.addAll(newAssigns); - - LOG.info("Keep Alive slots:" + keepAssigns); - LOG.info("Unstopped slots:" + defaultContext.getUnstoppedWorkers()); - LOG.info("New assign slots:" + newAssigns); - - return ret; - } - -} 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 deleted file mode 100644 index 6865bfa14..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/ResourceWorkerSlot.java +++ /dev/null @@ -1,139 +0,0 @@ -package com.alibaba.jstorm.schedule.default_assign; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; -import org.apache.log4j.Logger; - -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 { - - public static Logger LOG = Logger.getLogger(ResourceWorkerSlot.class); - private static final long serialVersionUID = 9138386287559932411L; - - private String hostname; - private long memSize; - private int cpu; - private Set tasks; - - private String jvm; - - public ResourceWorkerSlot() { - - } - - public ResourceWorkerSlot(String supervisorId, Integer port) { - super(supervisorId, port); - } - - public ResourceWorkerSlot(WorkerAssignment worker, - Map> componentToTask) { - super(worker.getNodeId(), worker.getPort()); - this.hostname = worker.getHostName(); - this.tasks = new HashSet(); - this.cpu = worker.getCpu(); - this.memSize = worker.getMem(); - this.jvm = worker.getJvm(); - for (Entry entry : worker.getComponentToNum() - .entrySet()) { - List tasks = componentToTask.get(entry.getKey()); - if (tasks == null || tasks.size() == 0) - continue; - int num = Math.min(tasks.size(), entry.getValue().intValue()); - List cTasks = new ArrayList(); - cTasks.addAll(tasks.subList(0, num)); - this.tasks.addAll(cTasks); - tasks.removeAll(cTasks); - } - } - - public String getHostname() { - return hostname; - } - - public void setHostname(String hostname) { - this.hostname = hostname; - } - - public Set getTasks() { - return tasks; - } - - public void setTasks(Set tasks) { - this.tasks = tasks; - } - - public String getJvm() { - return jvm; - } - - public void setJvm(String jvm) { - this.jvm = jvm; - } - - public long getMemSize() { - return memSize; - } - - public void setMemSize(long memSize) { - this.memSize = memSize; - } - - public int getCpu() { - return cpu; - } - - public void setCpu(int cpu) { - this.cpu = cpu; - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this, - ToStringStyle.SHORT_PREFIX_STYLE); - } - - public boolean compareToUserDefineWorker(WorkerAssignment worker, - Map taskToComponent) { - int cpu = worker.getCpu(); - if (cpu != 0 && this.cpu != cpu) - return false; - long mem = worker.getMem(); - if (mem != 0 && this.memSize != mem) - return false; - String jvm = worker.getJvm(); - if (jvm != null && !jvm.equals(this.jvm)) - return false; - String hostName = worker.getHostName(); - if (NetWorkUtils.equals(hostname, hostName) == false) - return false; - int port = worker.getPort(); - if (port != 0 && port != this.getPort()) - return false; - Map componentToNum = worker.getComponentToNum(); - Map myComponentToNum = new HashMap(); - for (Integer task : tasks) { - String component = taskToComponent.get(task); - Integer i = myComponentToNum.get(component); - if (i == null) { - i = 0; - } - myComponentToNum.put(component, ++i); - } - - return myComponentToNum.equals(componentToNum); - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/AbstractSelector.java b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/AbstractSelector.java deleted file mode 100644 index ebaa5bfb4..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/AbstractSelector.java +++ /dev/null @@ -1,54 +0,0 @@ -package com.alibaba.jstorm.schedule.default_assign.Selector; - -import java.util.ArrayList; -import java.util.Comparator; -import java.util.List; - -import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; -import com.alibaba.jstorm.schedule.default_assign.TaskGankerContext; - -public abstract class AbstractSelector implements Selector { - - protected final TaskGankerContext context; - - protected WorkerComparator workerComparator; - - protected WorkerComparator supervisorComparator; - - public AbstractSelector(TaskGankerContext context) { - this.context = context; - } - - protected List selectWorker( - List list, Comparator c) { - List result = new ArrayList(); - ResourceWorkerSlot best = null; - for (ResourceWorkerSlot worker : list) { - if (best == null) { - best = worker; - result.add(worker); - continue; - } - if (c.compare(best, worker) == 0) { - result.add(worker); - } else if (c.compare(best, worker) > 0) { - best = worker; - result.clear(); - result.add(best); - } - } - return result; - } - - @Override - public List select(List result, - String name) { - if (result.size() == 1) - return result; - result = this.selectWorker(result, workerComparator.get(name)); - if (result.size() == 1) - return result; - return this.selectWorker(result, supervisorComparator.get(name)); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/ComponentNumSelector.java b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/ComponentNumSelector.java deleted file mode 100644 index c1b851252..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/ComponentNumSelector.java +++ /dev/null @@ -1,37 +0,0 @@ -package com.alibaba.jstorm.schedule.default_assign.Selector; - -import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; -import com.alibaba.jstorm.schedule.default_assign.TaskGankerContext; - -public class ComponentNumSelector extends AbstractSelector { - - public ComponentNumSelector(final TaskGankerContext context) { - super(context); - // TODO Auto-generated constructor stub - this.workerComparator = new WorkerComparator() { - @Override - public int compare(ResourceWorkerSlot o1, ResourceWorkerSlot o2) { - // TODO Auto-generated method stub - int o1Num = context.getComponentNumOnWorker(o1, name); - int o2Num = context.getComponentNumOnWorker(o2, name); - if (o1Num == o2Num) - return 0; - return o1Num > o2Num ? 1 : -1; - } - }; - this.supervisorComparator = new WorkerComparator() { - @Override - public int compare(ResourceWorkerSlot o1, ResourceWorkerSlot o2) { - // TODO Auto-generated method stub - int o1Num = context.getComponentNumOnSupervisor(o1.getNodeId(), - name); - int o2Num = context.getComponentNumOnSupervisor(o2.getNodeId(), - name); - if (o1Num == o2Num) - return 0; - return o1Num > o2Num ? 1 : -1; - } - }; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/InputComponentNumSelector.java b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/InputComponentNumSelector.java deleted file mode 100644 index 2f118fedc..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/InputComponentNumSelector.java +++ /dev/null @@ -1,36 +0,0 @@ -package com.alibaba.jstorm.schedule.default_assign.Selector; - -import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; -import com.alibaba.jstorm.schedule.default_assign.TaskGankerContext; - -public class InputComponentNumSelector extends AbstractSelector { - - public InputComponentNumSelector(final TaskGankerContext context) { - super(context); - // TODO Auto-generated constructor stub - this.workerComparator = new WorkerComparator() { - @Override - public int compare(ResourceWorkerSlot o1, ResourceWorkerSlot o2) { - // TODO Auto-generated method stub - int o1Num = context.getInputComponentNumOnWorker(o1, name); - int o2Num = context.getInputComponentNumOnWorker(o2, name); - if (o1Num == o2Num) - return 0; - return o1Num > o2Num ? -1 : 1; - } - }; - this.supervisorComparator = new WorkerComparator() { - @Override - public int compare(ResourceWorkerSlot o1, ResourceWorkerSlot o2) { - // TODO Auto-generated method stub - int o1Num = context.getInputComponentNumOnSupervisor( - o1.getNodeId(), name); - int o2Num = context.getInputComponentNumOnSupervisor( - o2.getNodeId(), name); - if (o1Num == o2Num) - return 0; - return o1Num > o2Num ? -1 : 1; - } - }; - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/Selector.java b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/Selector.java deleted file mode 100644 index 31bba23c4..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/Selector.java +++ /dev/null @@ -1,10 +0,0 @@ -package com.alibaba.jstorm.schedule.default_assign.Selector; - -import java.util.List; - -import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; - -public interface Selector { - public List select(List result, - String name); -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/TotalTaskNumSelector.java b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/TotalTaskNumSelector.java deleted file mode 100644 index d2b7f84d6..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/TotalTaskNumSelector.java +++ /dev/null @@ -1,35 +0,0 @@ -package com.alibaba.jstorm.schedule.default_assign.Selector; - -import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; -import com.alibaba.jstorm.schedule.default_assign.TaskGankerContext; - -public class TotalTaskNumSelector extends AbstractSelector { - - public TotalTaskNumSelector(final TaskGankerContext context) { - super(context); - // TODO Auto-generated constructor stub - this.workerComparator = new WorkerComparator() { - @Override - public int compare(ResourceWorkerSlot o1, ResourceWorkerSlot o2) { - // TODO Auto-generated method stub - int o1Num = context.getTaskNumOnWorker(o1); - int o2Num = context.getTaskNumOnWorker(o2); - if (o1Num == o2Num) - return 0; - return o1Num > o2Num ? 1 : -1; - } - }; - this.supervisorComparator = new WorkerComparator() { - @Override - public int compare(ResourceWorkerSlot o1, ResourceWorkerSlot o2) { - // TODO Auto-generated method stub - int o1Num = context.getTaskNumOnSupervisor(o1.getNodeId()); - int o2Num = context.getTaskNumOnSupervisor(o2.getNodeId()); - if (o1Num == o2Num) - return 0; - return o1Num > o2Num ? 1 : -1; - } - }; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/WorkerComparator.java b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/WorkerComparator.java deleted file mode 100644 index 2aa8745d3..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/Selector/WorkerComparator.java +++ /dev/null @@ -1,15 +0,0 @@ -package com.alibaba.jstorm.schedule.default_assign.Selector; - -import java.util.Comparator; - -import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; - -public abstract class WorkerComparator implements Comparator { - - protected String name; - - public WorkerComparator get(String name) { - this.name = name; - return this; - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/TaskGanker.java b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/TaskGanker.java deleted file mode 100644 index da50c8e84..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/TaskGanker.java +++ /dev/null @@ -1,209 +0,0 @@ -package com.alibaba.jstorm.schedule.default_assign; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; - -import org.apache.log4j.Logger; - -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.Common; -import com.alibaba.jstorm.schedule.default_assign.Selector.ComponentNumSelector; -import com.alibaba.jstorm.schedule.default_assign.Selector.InputComponentNumSelector; -import com.alibaba.jstorm.schedule.default_assign.Selector.Selector; -import com.alibaba.jstorm.schedule.default_assign.Selector.TotalTaskNumSelector; -import com.alibaba.jstorm.utils.FailedAssignTopologyException; - -public class TaskGanker { - - public static Logger LOG = Logger.getLogger(TaskGanker.class); - - public static final String ACKER_NAME = "__acker"; - - private final TaskGankerContext taskContext; - - private List assignments = new ArrayList(); - - private int workerNum; - - private int baseNum; - - private int otherNum; - - private Set tasks; - - private DefaultTopologyAssignContext context; - - private Selector componentSelector; - - private Selector inputComponentSelector; - - private Selector totalTaskNumSelector; - - public TaskGanker(DefaultTopologyAssignContext context, Set tasks, - List workers) { - this.tasks = tasks; - this.context = context; - context.getRawTopology().get_spouts().keySet(); - this.taskContext = new TaskGankerContext( - this.buildSupervisorToWorker(workers), - Common.buildSpoutOutoputAndBoltInputMap(context)); - this.componentSelector = new ComponentNumSelector(taskContext); - this.inputComponentSelector = new InputComponentNumSelector(taskContext); - this.totalTaskNumSelector = new TotalTaskNumSelector(taskContext); - if (tasks.size() == 0) - return; - this.setTaskNum(); - } - - public List gankTask() { - if (tasks.size() == 0) - return assignments; - Set hadGanked = this.gankOnDifferentNodeTask(); - tasks.removeAll(hadGanked); - Set ackers = new HashSet(); - for (Integer task : tasks) { - String name = context.getTaskToComponent().get(task); - if (name.equals(TaskGanker.ACKER_NAME)) { - ackers.add(task); - continue; - } - this.beginGank(name, task); - } - for (Integer task : ackers) { - this.beginGank(TaskGanker.ACKER_NAME, task); - } - return assignments; - } - - private void beginGank(String name, Integer task) { - ResourceWorkerSlot worker = this.chooseWorker(name, - new ArrayList(taskContext - .getWorkerToTaskNum().keySet())); - this.pushTaskToWorker(task, name, worker); - } - - private Set gankOnDifferentNodeTask() { - Set result = new HashSet(); - for (Integer task : tasks) { - Map conf = Common.getComponentMap(context, task); - if (ConfigExtension.isTaskOnDifferentNode(conf)) - result.add(task); - } - for (Integer task : result) { - String name = context.getTaskToComponent().get(task); - ResourceWorkerSlot worker = this.chooseWorker(name, - this.getDifferentNodeTaskWokres(name)); - this.pushTaskToWorker(task, name, worker); - } - return result; - } - - private Map> buildSupervisorToWorker( - List workers) { - Map> supervisorToWorker = new HashMap>(); - for (ResourceWorkerSlot worker : workers) { - if (worker.getTasks() == null || worker.getTasks().size() == 0) { - List supervisor = supervisorToWorker - .get(worker.getNodeId()); - if (supervisor == null) { - supervisor = new ArrayList(); - supervisorToWorker.put(worker.getNodeId(), supervisor); - } - supervisor.add(worker); - } else { - assignments.add(worker); - } - } - this.workerNum = workers.size() - assignments.size(); - return supervisorToWorker; - } - - private ResourceWorkerSlot chooseWorker(String name, - List workers) { - List result = this.componentSelector.select( - workers, name); - result = this.totalTaskNumSelector.select(result, name); - if (name.equals(TaskGanker.ACKER_NAME)) - return result.iterator().next(); - return this.inputComponentSelector.select(result, name).iterator() - .next(); - } - - private void pushTaskToWorker(Integer task, String name, - ResourceWorkerSlot worker) { - Set tasks = worker.getTasks(); - if (tasks == null) { - tasks = new HashSet(); - worker.setTasks(tasks); - } - tasks.add(task); - int taskNum = taskContext.getWorkerToTaskNum().get(worker); - taskContext.getWorkerToTaskNum().put(worker, ++taskNum); - if (otherNum <= 0) { - if (taskNum == baseNum) { - taskContext.getWorkerToTaskNum().remove(worker); - assignments.add(worker); - } - } else { - if (taskNum == (baseNum + 1)) { - taskContext.getWorkerToTaskNum().remove(worker); - otherNum--; - assignments.add(worker); - } - if (otherNum <= 0) { - List needDelete = new ArrayList(); - for (Entry entry : taskContext - .getWorkerToTaskNum().entrySet()) { - if (entry.getValue() == baseNum) - needDelete.add(entry.getKey()); - } - for (ResourceWorkerSlot workerToDelete : needDelete) { - taskContext.getWorkerToTaskNum().remove(workerToDelete); - assignments.add(workerToDelete); - } - } - } - Map components = taskContext.getWorkerToComponentNum() - .get(worker); - if (components == null) { - components = new HashMap(); - taskContext.getWorkerToComponentNum().put(worker, components); - } - Integer componentNum = components.get(name); - if (componentNum == null) { - componentNum = 0; - } - components.put(name, ++componentNum); - } - - private void setTaskNum() { - this.baseNum = tasks.size() / workerNum; - this.otherNum = tasks.size() % workerNum; - for (Entry> entry : taskContext - .getSupervisorToWorker().entrySet()) { - for (ResourceWorkerSlot worker : entry.getValue()) { - taskContext.getWorkerToTaskNum().put(worker, 0); - } - } - } - - private List getDifferentNodeTaskWokres(String name) { - List workers = new ArrayList(); - workers.addAll(taskContext.getWorkerToTaskNum().keySet()); - for (Entry> entry : taskContext - .getSupervisorToWorker().entrySet()) { - if (taskContext.getComponentNumOnSupervisor(entry.getKey(), name) != 0) - workers.removeAll(entry.getValue()); - } - if (workers.size() == 0) - throw new FailedAssignTopologyException( - "there's no enough supervisor for making component: " - + name + " 's tasks on different node"); - return workers; - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/TaskGankerContext.java b/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/TaskGankerContext.java deleted file mode 100644 index 27c99a1e1..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/TaskGankerContext.java +++ /dev/null @@ -1,93 +0,0 @@ -package com.alibaba.jstorm.schedule.default_assign; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - -public class TaskGankerContext { - - private final Map> supervisorToWorker; - - private final Map> relationship; - - private final Map> workerToComponentNum = new HashMap>(); - - public Map getWorkerToTaskNum() { - return workerToTaskNum; - } - - private final Map workerToTaskNum = new HashMap(); - - public TaskGankerContext( - Map> supervisorToWorker, - Map> relationship) { - this.supervisorToWorker = supervisorToWorker; - this.relationship = relationship; - } - - public Map> getSupervisorToWorker() { - return supervisorToWorker; - } - - public Map> getWorkerToComponentNum() { - return workerToComponentNum; - } - - public Map> getRelationship() { - return relationship; - } - - public int getComponentNumOnSupervisor(String supervisor, String name) { - List workers = supervisorToWorker.get(supervisor); - if (workers == null) - return 0; - int result = 0; - for (ResourceWorkerSlot worker : workers) { - result = result + this.getComponentNumOnWorker(worker, name); - } - return result; - } - - public int getComponentNumOnWorker(ResourceWorkerSlot worker, String name) { - int result = 0; - Map componentNum = workerToComponentNum.get(worker); - if (componentNum != null && componentNum.get(name) != null) - result = componentNum.get(name); - return result; - } - - public int getTaskNumOnSupervisor(String supervisor) { - List workers = supervisorToWorker.get(supervisor); - if (workers == null) - return 0; - int result = 0; - for (ResourceWorkerSlot worker : workers) { - result = result + this.getTaskNumOnWorker(worker); - } - return result; - } - - public int getTaskNumOnWorker(ResourceWorkerSlot worker) { - return worker.getTasks() == null ? 0 : worker.getTasks().size(); - } - - public int getInputComponentNumOnSupervisor(String supervisor, String name) { - int result = 0; - List workers = supervisorToWorker.get(supervisor); - if (workers == null) - return 0; - for (ResourceWorkerSlot worker : workers) - result = result + this.getInputComponentNumOnWorker(worker, name); - return result; - } - - public int getInputComponentNumOnWorker(ResourceWorkerSlot worker, - String name) { - int result = 0; - for (String component : relationship.get(name)) - result = result + this.getComponentNumOnWorker(worker, component); - return result; - } - -} 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 deleted file mode 100644 index 03a044bd8..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/schedule/default_assign/WorkerMaker.java +++ /dev/null @@ -1,300 +0,0 @@ -package com.alibaba.jstorm.schedule.default_assign; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; - -import org.apache.commons.lang.math.NumberUtils; -import org.apache.log4j.Logger; - -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.NetWorkUtils; - -public class WorkerMaker { - - public static Logger LOG = Logger.getLogger(WorkerMaker.class); - - private static WorkerMaker instance; - - private WorkerMaker() { - - } - - public static WorkerMaker getInstance() { - if (instance == null) { - instance = new WorkerMaker(); - } - return instance; - } - - public List makeWorkers( - DefaultTopologyAssignContext context, Set needAssign, - int num) { - int workersNum = getWorkersNum(context, num); - if (workersNum == 0) { - throw new FailedAssignTopologyException("there's no enough worker"); - } - LOG.info("worker num is: " + workersNum); - List result = new ArrayList(); - // userdefine assignments - this.getRightWorkers( - context, - needAssign, - result, - workersNum, - getUserDefineWorkers(context, ConfigExtension - .getUserDefineAssignment(context.getStormConf()))); - // old assignments - if (ConfigExtension.isUseOldAssignment(context.getStormConf())) { - this.getRightWorkers(context, needAssign, result, workersNum, - context.getOldWorkers()); - } - int defaultWorkerNum = Math.min(workersNum - result.size(), - needAssign.size()); - LOG.info("Get workers from user define and old assignments: " + result); - LOG.info("Tasks: " + needAssign + " will be scheduled by default" - + " in " + defaultWorkerNum + " workers"); - for (int i = 0; i < defaultWorkerNum; i++) { - result.add(new ResourceWorkerSlot()); - } - 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 = ConfigExtension.getMemSizePerWorker(conf); - int defaultCpu = ConfigExtension.getCpuSlotPerWorker(conf); - for (ResourceWorkerSlot worker : result) { - if (worker.getMemSize() <= 0) - worker.setMemSize(defaultSize); - if (worker.getCpu() <= 0) - worker.setCpu(defaultCpu); - } - } - - private void putAllWorkerToSupervisor(List result, - List supervisors) { - for (ResourceWorkerSlot worker : result) { - if (worker.getNodeId() != null) - continue; - if (worker.getHostname() != null) { - for (SupervisorInfo supervisor : supervisors) { - if (NetWorkUtils.equals(supervisor.getHostName(), worker.getHostname()) - && supervisor.getWorkerPorts().size() > 0) { - this.putWorkerToSupervisor(supervisor, worker); - break; - } - } - } - } - supervisors = this.getCanUseSupervisors(supervisors); - Collections.sort(supervisors, new Comparator() { - - @Override - public int compare(SupervisorInfo o1, SupervisorInfo o2) { - // TODO Auto-generated method stub - return -NumberUtils.compare(o1.getWorkerPorts().size(), o2 - .getWorkerPorts().size()); - } - - }); - this.putWorkerToSupervisor(result, supervisors); - } - - private void putWorkerToSupervisor(SupervisorInfo supervisor, - ResourceWorkerSlot worker) { - int port = worker.getPort(); - if (!supervisor.getWorkerPorts().contains(worker.getPort())) { - port = supervisor.getWorkerPorts().iterator().next(); - } - worker.setPort(port); - supervisor.getWorkerPorts().remove(port); - worker.setNodeId(supervisor.getSupervisorId()); - } - - private void putWorkerToSupervisor(List result, - List supervisors) { - int key = 0; - for (ResourceWorkerSlot worker : result) { - if (supervisors.size() == 0) - return; - if (worker.getNodeId() != null) - continue; - if (key >= supervisors.size()) - key = 0; - SupervisorInfo supervisor = supervisors.get(key); - worker.setHostname(supervisor.getHostName()); - worker.setNodeId(supervisor.getSupervisorId()); - worker.setPort(supervisor.getWorkerPorts().iterator().next()); - supervisor.getWorkerPorts().remove(worker.getPort()); - if (supervisor.getWorkerPorts().size() == 0) - supervisors.remove(supervisor); - key++; - } - } - - private void getRightWorkers(DefaultTopologyAssignContext context, - Set needAssign, List result, - int workersNum, Collection workers) { - Set assigned = new HashSet(); - List users = new ArrayList(); - if (workers == null) - return; - for (ResourceWorkerSlot worker : workers) { - boolean right = true; - Set tasks = worker.getTasks(); - if (tasks == null) - continue; - for (Integer task : tasks) { - if (!needAssign.contains(task) || assigned.contains(task)) { - right = false; - break; - } - } - if (right) { - assigned.addAll(tasks); - users.add(worker); - } - } - if (users.size() + result.size() > workersNum) { - return; - } - - if (users.size() + result.size() == workersNum - && assigned.size() != needAssign.size()) { - return; - } - result.addAll(users); - needAssign.removeAll(assigned); - } - - 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(); - } - return Math.min(slotNum, workersNum); - } - - /** - * @param context - * @param workers - * @return - */ - private List getUserDefineWorkers( - DefaultTopologyAssignContext context, List workers) { - List ret = new ArrayList(); - if (workers == null) - return ret; - Map> componentToTask = (HashMap>) ((HashMap>) context - .getComponentTasks()).clone(); - if (context.getAssignType() != context.ASSIGN_TYPE_NEW) { - this.checkUserDefineWorkers(context, workers, - context.getTaskToComponent()); - } - for (WorkerAssignment worker : workers) { - ResourceWorkerSlot workerSlot = new ResourceWorkerSlot(worker, - componentToTask); - if (workerSlot.getTasks().size() != 0) { - ret.add(workerSlot); - } - } - return ret; - } - - private void checkUserDefineWorkers(DefaultTopologyAssignContext context, - List workers, Map taskToComponent) { - Set unstoppedWorkers = context - .getUnstoppedWorkers(); - List re = new ArrayList(); - for (WorkerAssignment worker : workers) { - for (ResourceWorkerSlot unstopped : unstoppedWorkers) { - if (unstopped - .compareToUserDefineWorker(worker, taskToComponent)) - re.add(worker); - } - } - workers.removeAll(re); - - } - - private List getCanUseSupervisors( - Map supervisors) { - List canUseSupervisors = new ArrayList(); - for (Entry entry : supervisors.entrySet()) { - SupervisorInfo supervisor = entry.getValue(); - if (supervisor.getWorkerPorts().size() > 0) - canUseSupervisors.add(entry.getValue()); - } - return canUseSupervisors; - } - - private List getCanUseSupervisors( - List supervisors) { - List canUseSupervisors = new ArrayList(); - for (SupervisorInfo supervisor : supervisors) { - if (supervisor.getWorkerPorts().size() > 0) - canUseSupervisors.add(supervisor); - } - return canUseSupervisors; - } - - private List getIsolationSupervisors( - DefaultTopologyAssignContext context) { - List isolationHosts = (List) context.getStormConf() - .get(Config.ISOLATION_SCHEDULER_MACHINES); - LOG.info("Isolation machines: " + isolationHosts); - if (isolationHosts == null) - return new ArrayList(); - List isolationSupervisors = new ArrayList(); - 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/stats/CommonStatsData.java b/jstorm-server/src/main/java/com/alibaba/jstorm/stats/CommonStatsData.java deleted file mode 100644 index eee9df2d2..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/CommonStatsData.java +++ /dev/null @@ -1,277 +0,0 @@ -package com.alibaba.jstorm.stats; - -import java.io.Serializable; -import java.util.HashMap; -import java.util.Map; -import java.util.Map.Entry; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; - -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.generated.TaskStats; - -import com.alibaba.jstorm.common.stats.StatBuckets; -import com.alibaba.jstorm.common.stats.StaticsType; - -/** - * Common stats data - * - * @author yannian/Longda - * - */ -public class CommonStatsData implements Serializable { - - private static final long serialVersionUID = -2811225938044543165L; - - protected Map> staticsMap; - protected int rate = StatFunction.NUM_STAT_BUCKETS; - - public static final long LATENCY_MS_RATIO = 1000; - - public static final Integer ALL_TIME_KEY = new Integer(0); - - public CommonStatsData() { - staticsMap = new HashMap>(); - - // > - HashMap emitted = new HashMap(); - HashMap sendTps = new HashMap(); - HashMap recvTps = new HashMap(); - HashMap acked = new HashMap(); - HashMap failed = new HashMap(); - HashMap processLatencies = new HashMap(); - - staticsMap.put(StaticsType.emitted, emitted); - staticsMap.put(StaticsType.send_tps, sendTps); - staticsMap.put(StaticsType.recv_tps, recvTps); - staticsMap.put(StaticsType.acked, acked); - staticsMap.put(StaticsType.failed, failed); - staticsMap.put(StaticsType.process_latencies, processLatencies); - - } - - public Map get(StaticsType type) { - return staticsMap.get(type); - } - - public void put(StaticsType type, Map value) { - staticsMap.put(type, value); - } - - public int getRate() { - return rate; - } - - public void setRate(int rate) { - this.rate = rate; - } - - @Override - public boolean equals(Object assignment) { - if ((assignment instanceof CommonStatsData) == false) { - return false; - } - - CommonStatsData otherData = (CommonStatsData) assignment; - - for (Entry> entry : staticsMap - .entrySet()) { - StaticsType type = entry.getKey(); - Map value = entry.getValue(); - - Map otherValue = otherData.get(type); - - if (value.equals(otherValue) == false) { - return false; - } - } - - return true; - } - - @Override - public int hashCode() { - int ret = 0; - for (Entry> entry : staticsMap - .entrySet()) { - StaticsType type = entry.getKey(); - Map value = entry.getValue(); - - ret += value.hashCode(); - } - return ret; - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this, - ToStringStyle.SHORT_PREFIX_STYLE); - } - - public Map> convertKey( - Map statics, K keySample, V valueSample) { - - Map> ret = new HashMap>(); - - for (Entry times : statics.entrySet()) { - - Map val = (Map) times.getValue(); - - Integer window = times.getKey(); - - String key = StatBuckets.parseTimeKey(window); - - ret.put(key, val); - } - - return ret; - } - - - /** - * Get emmitted statics - * - * @return > - */ - public Map> get_emitted() { - Map statics = staticsMap.get(StaticsType.emitted); - - return convertKey(statics, String.valueOf(""), Long.valueOf(0)); - } - - public Map> get_send_tps() { - Map statics = staticsMap.get(StaticsType.send_tps); - - return convertKey(statics, String.valueOf(""), Double.valueOf(0)); - } - - public Map> get_recv_tps() { - Map statics = staticsMap.get(StaticsType.recv_tps); - - GlobalStreamId streamIdSample = new GlobalStreamId("", ""); - return convertKey(statics, streamIdSample, Double.valueOf(0)); - } - - public Map> get_acked() { - Map statics = staticsMap.get(StaticsType.acked); - - GlobalStreamId streamIdSample = new GlobalStreamId("", ""); - return convertKey(statics, streamIdSample, Long.valueOf(0)); - } - - public Map> get_failed() { - Map statics = staticsMap.get(StaticsType.failed); - - GlobalStreamId streamIdSample = new GlobalStreamId("", ""); - return convertKey(statics, streamIdSample, Long.valueOf(0)); - } - - public Map> get_process_latencie() { - Map> ret = - new HashMap>(); - - Map statics = staticsMap - .get(StaticsType.process_latencies); - - GlobalStreamId streamIdSample = new GlobalStreamId("", ""); - - Map> raw = - convertKey(statics, streamIdSample, Double.valueOf(0)); - - for (Entry> windowEntry : raw.entrySet()) { - String windowStr = windowEntry.getKey(); - Map oldStreamMap = windowEntry.getValue(); - - Map newStreamMap = new HashMap(); - - for (Entry entry: oldStreamMap.entrySet()) { - GlobalStreamId key = entry.getKey(); - Double value = entry.getValue(); - - if (value == null) { - newStreamMap.put(key, Double.valueOf(0)); - }else { - newStreamMap.put(key, value/LATENCY_MS_RATIO); - } - - } - - ret.put(windowStr, newStreamMap); - } - - return ret; - } - - /* - * Get total statics of all-time - */ - public Long get_total_emitted() { - Long ret = new Long(0); - - Map allTimeEmitted = get_emitted().get(StatBuckets.ALL_WINDOW_STR); - for (Entry entry : allTimeEmitted.entrySet()) { - ret += entry.getValue(); - } - - return ret; - } - - public Double get_total_send_tps() { - Double ret = new Double(0); - - Map allTimeSendTps = get_send_tps().get(StatBuckets.ALL_WINDOW_STR); - for (Entry entry : allTimeSendTps.entrySet()) { - ret += entry.getValue(); - } - - return ret; - } - - public Double get_total_recv_tps() { - Double ret = new Double(0); - - Map allTimeRecvTps = get_recv_tps().get(StatBuckets.ALL_WINDOW_STR); - for (Entry entry : allTimeRecvTps.entrySet()) { - ret += entry.getValue(); - } - - return ret; - } - - public Long get_total_failed() { - Long ret = new Long(0); - - Map allTimeFailed = get_failed().get(StatBuckets.ALL_WINDOW_STR); - for (Entry entry : allTimeFailed.entrySet()) { - ret += entry.getValue(); - } - - return ret; - } - - public Double get_avg_latency() { - Double ret = new Double(0); - int i = 0; - - Map allAvglatency = get_process_latencie().get(StatBuckets.ALL_WINDOW_STR); - for (Entry entry : allAvglatency.entrySet()) { - ret += entry.getValue(); - i++; - } - return ret; - } - - public TaskStats getTaskStats() { - TaskStats taskStats = new TaskStats(); - - taskStats.set_emitted(get_emitted()); - taskStats.set_send_tps(get_send_tps()); - taskStats.set_recv_tps(get_recv_tps()); - taskStats.set_acked(get_acked()); - taskStats.set_failed(get_failed()); - taskStats.set_process_ms_avg(get_process_latencie()); - - return taskStats; - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/CommonStatsRolling.java b/jstorm-server/src/main/java/com/alibaba/jstorm/stats/CommonStatsRolling.java deleted file mode 100644 index 001afe97f..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/CommonStatsRolling.java +++ /dev/null @@ -1,332 +0,0 @@ -package com.alibaba.jstorm.stats; - -import java.util.HashMap; -import java.util.Map; -import java.util.Map.Entry; - -import backtype.storm.generated.GlobalStreamId; - -import com.alibaba.jstorm.cluster.Common; -import com.alibaba.jstorm.common.stats.StatBuckets; -import com.alibaba.jstorm.common.stats.StaticsType; -import com.alibaba.jstorm.stats.rolling.RollingWindowSet; -import com.alibaba.jstorm.utils.EventSampler; -import com.alibaba.jstorm.utils.Pair; -import com.alibaba.jstorm.utils.TimeUtils; - -/** - * both spout and bolt will use base statics - * - * @author Longda/yannian - * - */ -public class CommonStatsRolling { - - private static final long serialVersionUID = -2145444660360278001L; - - protected Map staticTypeMap = new HashMap(); - - // - protected Map emittedSamplers; - protected Map sendTpsSamplers; - - // > - protected Map> recvTpsSamplers; - - protected Map> processSamplers; - - // in order to improve performance, use two type of samplers - // protected Map> ackedSamplers; - // protected Map> failedSamplers; - protected Map> boltAckedSamplers; - protected Map> boltFailedSamplers; - protected Map spoutAckedSamplers; - protected Map spoutFailedSamplers; - - protected boolean enable = true; - - protected Integer rate; - - public CommonStatsRolling(Integer rate) { - RollingWindowSet emitted = StatFunction - .keyed_counter_rolling_window_set( - StatFunction.NUM_STAT_BUCKETS, StatBuckets.STAT_BUCKETS); - staticTypeMap.put(StaticsType.emitted, emitted); - - RollingWindowSet sendTps = StatFunction.keyed_avg_rolling_window_set( - StatFunction.NUM_STAT_BUCKETS, StatBuckets.STAT_BUCKETS); - staticTypeMap.put(StaticsType.send_tps, sendTps); - - RollingWindowSet recvTps = StatFunction.keyed_avg_rolling_window_set( - StatFunction.NUM_STAT_BUCKETS, StatBuckets.STAT_BUCKETS); - staticTypeMap.put(StaticsType.recv_tps, recvTps); - - RollingWindowSet acked = StatFunction.keyed_counter_rolling_window_set( - StatFunction.NUM_STAT_BUCKETS, StatBuckets.STAT_BUCKETS); - staticTypeMap.put(StaticsType.acked, acked); - - RollingWindowSet failed = StatFunction - .keyed_counter_rolling_window_set( - StatFunction.NUM_STAT_BUCKETS, StatBuckets.STAT_BUCKETS); - staticTypeMap.put(StaticsType.failed, failed); - - RollingWindowSet process_latencies = StatFunction - .keyed_avg_rolling_window_set(StatFunction.NUM_STAT_BUCKETS, - StatBuckets.STAT_BUCKETS); - - staticTypeMap.put(StaticsType.process_latencies, process_latencies); - - emittedSamplers = new HashMap(); - sendTpsSamplers = new HashMap(); - recvTpsSamplers = new HashMap>(); - boltAckedSamplers = new HashMap>(); - boltFailedSamplers = new HashMap>(); - spoutAckedSamplers = new HashMap(); - spoutFailedSamplers = new HashMap(); - processSamplers = new HashMap>(); - - this.rate = rate; - } - - /** - * update statics - * - * @param common - * @param path - * @param args - */ - public void update_task_stat(StaticsType type, Object... args) { - - RollingWindowSet statics = staticTypeMap.get(type); - if (statics != null) { - statics.update_rolling_window_set(args); - } - } - - public void send_tuple(String stream, int num_out_tasks) { - if (enable == false) { - return; - } - - if (num_out_tasks <= 0) { - return; - } - - EventSampler emittedSampler = emittedSamplers.get(stream); - if (emittedSampler == null) { - emittedSampler = new EventSampler(rate); - emittedSamplers.put(stream, emittedSampler); - } - - Integer times = emittedSampler.timesCheck(); - if (times != null) { - update_task_stat(StaticsType.emitted, stream, times * num_out_tasks); - } - - EventSampler sendTpsSampler = sendTpsSamplers.get(stream); - if (sendTpsSampler == null) { - sendTpsSampler = new EventSampler(rate); - sendTpsSamplers.put(stream, sendTpsSampler); - } - Integer send = sendTpsSampler.tpsCheck(); - - if (send != null) { - - update_task_stat(StaticsType.send_tps, stream, send * num_out_tasks); - - } - } - - public void recv_tuple(String component, String stream) { - if (enable == false) { - return; - } - - Map componentSamplers = recvTpsSamplers - .get(component); - if (componentSamplers == null) { - componentSamplers = new HashMap(); - recvTpsSamplers.put(component, componentSamplers); - } - - EventSampler sampler = componentSamplers.get(stream); - if (sampler == null) { - sampler = new EventSampler(rate); - componentSamplers.put(stream, sampler); - } - - Integer recv = sampler.tpsCheck(); - - if (recv != null) { - GlobalStreamId key = new GlobalStreamId(component, stream); - update_task_stat(StaticsType.recv_tps, key, recv); - } - } - - public void bolt_acked_tuple(String component, String stream, - Long latency_ms) { - - if (enable == false) { - return; - } - - if (latency_ms == null) { - return; - } - - Map componentSamplers = boltAckedSamplers - .get(component); - if (componentSamplers == null) { - componentSamplers = new HashMap(); - boltAckedSamplers.put(component, componentSamplers); - } - - EventSampler sampler = componentSamplers.get(stream); - if (sampler == null) { - sampler = new EventSampler(rate); - componentSamplers.put(stream, sampler); - } - - Pair pair = sampler.avgCheck(latency_ms * 1000); - if (pair == null) { - return; - } - - long avgLatency = (long)((double)pair.getSecond()); - GlobalStreamId key = new GlobalStreamId(component, stream); - update_task_stat(StaticsType.acked, key, pair.getFirst()); - update_task_stat(StaticsType.process_latencies, key, avgLatency); - } - - public void bolt_failed_tuple(String component, String stream) { - if (enable == false) { - return; - } - - Map componentSamplers = boltFailedSamplers - .get(component); - if (componentSamplers == null) { - componentSamplers = new HashMap(); - boltFailedSamplers.put(component, componentSamplers); - } - - EventSampler sampler = componentSamplers.get(stream); - if (sampler == null) { - sampler = new EventSampler(rate); - componentSamplers.put(stream, sampler); - } - - Integer times = sampler.timesCheck(); - if (times == null) { - return; - } - - GlobalStreamId key = new GlobalStreamId(component, stream); - update_task_stat(StaticsType.failed, key, times); - } - - public void spout_acked_tuple(String stream, long st) { - - if (enable == false) { - return; - } - - if (st == 0) { - return; - } - - EventSampler sampler = spoutAckedSamplers.get(stream); - if (sampler == null) { - sampler = new EventSampler(rate); - spoutAckedSamplers.put(stream, sampler); - } - - - - long latency_ms = TimeUtils.time_delta_ms(st); - Pair pair = sampler.avgCheck(latency_ms * 1000); - if (pair == null) { - return; - } - - long avgLatency = (long)((double)pair.getSecond()); - GlobalStreamId key = new GlobalStreamId(Common.ACKER_COMPONENT_ID, - stream); - update_task_stat(StaticsType.acked, key, pair.getFirst()); - update_task_stat(StaticsType.process_latencies, key, avgLatency); - } - - public void spout_failed_tuple(String stream) { - if (enable == false) { - return; - } - - EventSampler sampler = spoutFailedSamplers.get(stream); - if (sampler == null) { - sampler = new EventSampler(rate); - spoutFailedSamplers.put(stream, sampler); - } - - Integer times = sampler.timesCheck(); - if (times == null) { - return; - } - - GlobalStreamId key = new GlobalStreamId(Common.ACKER_COMPONENT_ID, - stream); - update_task_stat(StaticsType.failed, key, times); - } - - public CommonStatsData render_stats() { - - cleanup_stats(false); - - CommonStatsData ret = new CommonStatsData(); - - for (Entry entry : staticTypeMap - .entrySet()) { - StaticsType type = entry.getKey(); - RollingWindowSet rws = entry.getValue(); - - Map csData = rws.value_rolling_window_set(); - - ret.put(type, csData); - } - - return ret; - } - - protected void cleanup_common_stats() { - - for (Entry entry : staticTypeMap - .entrySet()) { - RollingWindowSet rws = entry.getValue(); - rws.cleanup_rolling_window_set(); - } - - } - - public void cleanup_stats(boolean skipcommon) { - if (skipcommon == false) { - cleanup_common_stats(); - } - - } - - public RollingWindowSet get(StaticsType type) { - return staticTypeMap.get(type); - } - - public void put(StaticsType type, RollingWindowSet statics) { - staticTypeMap.put(type, statics); - } - - public Integer getRate() { - return rate; - } - - public void setRate(Integer rate) { - this.rate = rate; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/StatFunction.java b/jstorm-server/src/main/java/com/alibaba/jstorm/stats/StatFunction.java deleted file mode 100644 index 4a02815fc..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/StatFunction.java +++ /dev/null @@ -1,129 +0,0 @@ -package com.alibaba.jstorm.stats; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.common.stats.StatBuckets; -import com.alibaba.jstorm.stats.incval.IncValExtractor; -import com.alibaba.jstorm.stats.incval.IncValMerger; -import com.alibaba.jstorm.stats.incval.IncValUpdater; -import com.alibaba.jstorm.stats.keyAvg.KeyAvgExtractor; -import com.alibaba.jstorm.stats.keyAvg.KeyAvgMerge; -import com.alibaba.jstorm.stats.keyAvg.KeyAvgUpdater; -import com.alibaba.jstorm.stats.rolling.RollingWindowSet; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.Pair; - -public class StatFunction { - - public static final Integer NUM_STAT_BUCKETS = StatBuckets.NUM_STAT_BUCKETS; - - /** - * create simple counter statics - * - * @param num_buckets - * @param bucket_sizes - * @return - */ - public static RollingWindowSet keyed_counter_rolling_window_set( - int num_buckets, Integer[] bucket_sizes) { - - RunnableCallback updater = new IncValUpdater(); - RunnableCallback merger = new IncValMerger(); - - RunnableCallback extractor = new IncValExtractor(); - return RollingWindowSet.rolling_window_set(updater, merger, extractor, - num_buckets, bucket_sizes); - } - - /** - * create averge statics - * - * @param num_buckets - * @param bucket_sizes - * @return - */ - public static RollingWindowSet keyed_avg_rolling_window_set( - int num_buckets, Integer[] bucket_sizes) { - RunnableCallback updater = new KeyAvgUpdater(); - - RunnableCallback merger = new KeyAvgMerge(); - - RunnableCallback extractor = new KeyAvgExtractor(); - - return RollingWindowSet.rolling_window_set(updater, merger, extractor, - num_buckets, bucket_sizes); - } - - public static void incr_val(Map map, Object key, Long amt) { - Long value = Long.valueOf(0); - if (map.containsKey(key)) { - value = map.get(key); - } - value = (Long) JStormUtils.add(value, amt); - map.put(key, value); - } - - public static void incr_val(Map map, Object key) { - incr_val(map, key, Long.valueOf(1)); - } - - public static synchronized Pair update_avg( - Pair curr, long val) { - curr.setFirst(curr.getFirst() + val); - curr.setSecond(curr.getSecond() + 1); - return curr; - } - - public static Pair merge_avg(List> avg) { - Pair rtn = new Pair(0l, 0l); - for (Pair p : avg) { - rtn.setFirst(rtn.getFirst() + p.getFirst()); - rtn.setSecond(rtn.getSecond() + p.getSecond()); - } - return rtn; - } - - public static double extract_avg(Pair p) { - if (p.getSecond() == 0) { - return 0d; - } - return (p.getFirst() * 1.0) / p.getSecond(); - } - - public static void update_keyed_avg(Map> map, - Object key, long val) { - Pair p = map.get(key); - if (p == null) { - p = new Pair(0l, 0l); - } - update_avg(p, val); - map.put(key, p); - } - - public static Pair merge_keyed_avg(List> avg) { - return merge_avg(avg); - } - - public static Map extract_key_avg( - Map> map) { - Map rtn = new HashMap(); - if (map != null) { - for (Entry> e : map.entrySet()) { - rtn.put(e.getKey(), extract_avg(e.getValue())); - } - } - return rtn; - } - - public static Map counter_extract(Map v) { - if (v == null) { - return new HashMap(); - } - return v; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/incval/IncValExtractor.java b/jstorm-server/src/main/java/com/alibaba/jstorm/stats/incval/IncValExtractor.java deleted file mode 100644 index ee379f696..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/incval/IncValExtractor.java +++ /dev/null @@ -1,22 +0,0 @@ -package com.alibaba.jstorm.stats.incval; - -import java.util.HashMap; -import java.util.Map; - -import com.alibaba.jstorm.callback.RunnableCallback; - -public class IncValExtractor extends RunnableCallback { - - @SuppressWarnings("unchecked") - @Override - public Object execute(T... args) { - Map v = null; - if (args != null && args.length > 0) { - v = (Map) args[0]; - } - if (v == null) { - v = new HashMap(); - } - return v; - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/incval/IncValMerger.java b/jstorm-server/src/main/java/com/alibaba/jstorm/stats/incval/IncValMerger.java deleted file mode 100644 index d835b20d5..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/incval/IncValMerger.java +++ /dev/null @@ -1,38 +0,0 @@ -package com.alibaba.jstorm.stats.incval; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.utils.JStormUtils; - -public class IncValMerger extends RunnableCallback { - /** - * Merget List> to Map - */ - @SuppressWarnings("unchecked") - @Override - public Object execute(T... args) { - Map result = null; - if (args != null && args.length > 0) { - - List> list = (List>) args[0]; - result = new HashMap(); - - for (Map each : list) { - - for (Entry e : each.entrySet()) { - Object key = e.getKey(); - Long val = e.getValue(); - if (result.containsKey(key)) { - val = (Long) JStormUtils.add(val, result.get(key)); - } - result.put(key, val); - } - } - } - return result; - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/incval/IncValUpdater.java b/jstorm-server/src/main/java/com/alibaba/jstorm/stats/incval/IncValUpdater.java deleted file mode 100644 index e0e8f441c..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/incval/IncValUpdater.java +++ /dev/null @@ -1,36 +0,0 @@ -package com.alibaba.jstorm.stats.incval; - -import java.util.HashMap; -import java.util.Map; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.stats.StatFunction; -import com.alibaba.jstorm.stats.rolling.UpdateParams; - -public class IncValUpdater extends RunnableCallback { - - @SuppressWarnings("unchecked") - @Override - public Object execute(T... args) { - Map curr = null; - if (args != null && args.length > 0) { - UpdateParams p = (UpdateParams) args[0]; - if (p.getCurr() != null) { - curr = (Map) p.getCurr(); - } else { - curr = new HashMap(); - } - Object[] incArgs = p.getArgs(); - - Long amt = 1l; - - if (incArgs.length > 1) { - amt = Long.parseLong(String.valueOf(incArgs[1])); - } - StatFunction.incr_val(curr, incArgs[0], amt); - - } - return curr; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/keyAvg/KeyAvgExtractor.java b/jstorm-server/src/main/java/com/alibaba/jstorm/stats/keyAvg/KeyAvgExtractor.java deleted file mode 100644 index eadd5b946..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/keyAvg/KeyAvgExtractor.java +++ /dev/null @@ -1,27 +0,0 @@ -package com.alibaba.jstorm.stats.keyAvg; - -import java.util.HashMap; -import java.util.Map; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.stats.StatFunction; -import com.alibaba.jstorm.utils.Pair; - -public class KeyAvgExtractor extends RunnableCallback { - - @SuppressWarnings("unchecked") - @Override - public Object execute(T... args) { - Map result = null; - if (args != null && args.length > 0) { - Map> v = (Map>) args[0]; - result = StatFunction.extract_key_avg(v); - } - - if (result == null) { - result = new HashMap(); - } - - return result; - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/keyAvg/KeyAvgMerge.java b/jstorm-server/src/main/java/com/alibaba/jstorm/stats/keyAvg/KeyAvgMerge.java deleted file mode 100644 index 3ac2c76e7..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/keyAvg/KeyAvgMerge.java +++ /dev/null @@ -1,43 +0,0 @@ -package com.alibaba.jstorm.stats.keyAvg; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.stats.StatFunction; -import com.alibaba.jstorm.utils.Pair; - -public class KeyAvgMerge extends RunnableCallback { - - @SuppressWarnings("unchecked") - @Override - public Object execute(T... args) { - List>> list = (List>>) args[0]; - - Map> result = new HashMap>(); - - Map>> trans = new HashMap>>(); - - for (Map> each : list) { - - for (Entry> e : each.entrySet()) { - - Object key = e.getKey(); - List> val = trans.get(key); - if (val == null) { - val = new ArrayList>(); - } - val.add(e.getValue()); - trans.put(key, val); - } - } - - for (Entry>> e : trans.entrySet()) { - result.put(e.getKey(), StatFunction.merge_keyed_avg(e.getValue())); - } - return result; - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/keyAvg/KeyAvgUpdater.java b/jstorm-server/src/main/java/com/alibaba/jstorm/stats/keyAvg/KeyAvgUpdater.java deleted file mode 100644 index 3da19937c..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/keyAvg/KeyAvgUpdater.java +++ /dev/null @@ -1,34 +0,0 @@ -package com.alibaba.jstorm.stats.keyAvg; - -import java.util.HashMap; -import java.util.Map; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.stats.StatFunction; -import com.alibaba.jstorm.stats.rolling.UpdateParams; -import com.alibaba.jstorm.utils.Pair; - -public class KeyAvgUpdater extends RunnableCallback { - - @SuppressWarnings("unchecked") - @Override - public Object execute(T... args) { - Map> curr = null; - if (args != null && args.length > 0) { - UpdateParams p = (UpdateParams) args[0]; - if (p.getCurr() != null) { - curr = (Map>) p.getCurr(); - } else { - curr = new HashMap>(); - } - Object[] keyAvgArgs = p.getArgs(); - - Long amt = 1l; - if (keyAvgArgs.length > 1) { - amt = Long.parseLong(String.valueOf(keyAvgArgs[1])); - } - StatFunction.update_keyed_avg(curr, keyAvgArgs[0], amt); - } - return curr; - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/rolling/RollingWindow.java b/jstorm-server/src/main/java/com/alibaba/jstorm/stats/rolling/RollingWindow.java deleted file mode 100644 index 049510d62..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/rolling/RollingWindow.java +++ /dev/null @@ -1,165 +0,0 @@ -package com.alibaba.jstorm.stats.rolling; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.utils.TimeUtils; - -public class RollingWindow { - private RunnableCallback updater; - private RunnableCallback merger; - private RunnableCallback extractor; - private Integer bucket_size_secs; - private Integer num_buckets; - // > - private Map buckets; - - public RunnableCallback getUpdater() { - return updater; - } - - public void setUpdater(RunnableCallback updater) { - this.updater = updater; - } - - public RunnableCallback getMerger() { - return merger; - } - - public void setMerger(RunnableCallback merger) { - this.merger = merger; - } - - public RunnableCallback getExtractor() { - return extractor; - } - - public void setExtractor(RunnableCallback extractor) { - this.extractor = extractor; - } - - public Integer getBucket_size_secs() { - return bucket_size_secs; - } - - public void setBucket_size_secs(Integer bucket_size_secs) { - this.bucket_size_secs = bucket_size_secs; - } - - public Integer getNum_buckets() { - return num_buckets; - } - - public void setNum_buckets(Integer num_buckets) { - this.num_buckets = num_buckets; - } - - public Map getBuckets() { - return buckets; - } - - public void setBuckets(Map buckets) { - this.buckets = buckets; - } - - public Integer curr_time_bucket(Integer time_secs) { - return (Integer) (bucket_size_secs * (time_secs / bucket_size_secs)); - } - - // num_buckets == StatFunction.NUM_STAT_BUCKETS - public static RollingWindow rolling_window(RunnableCallback updater, - RunnableCallback merger, RunnableCallback extractor, - Integer bucket_size_secs, Integer num_buckets) { - - RollingWindow rtn = new RollingWindow(); - rtn.setUpdater(updater); - rtn.setMerger(merger); - rtn.setExtractor(extractor); - rtn.setBucket_size_secs(bucket_size_secs); - rtn.setNum_buckets(num_buckets); - - rtn.setBuckets(new HashMap()); - - return rtn; - } - - /** - * - * @param time_secs - * @param args - * Object[0] -- key, Object[1] -- value - * @return - */ - public RollingWindow update_rolling_window(Integer time_secs, Object[] args) { - synchronized (this) { - Integer time_bucket = curr_time_bucket(time_secs); - Map curr = buckets.get(time_bucket); - - UpdateParams p = new UpdateParams(); - p.setArgs(args); - p.setCurr(curr); - curr = (Map) updater.execute((Object) p); - - buckets.put(time_bucket, curr); - - return this; - } - } - - /** - * - * @return - */ - public Object value_rolling_window() { - synchronized (this) { - // > buckets - List values = new ArrayList(); - for (Entry entry : buckets.entrySet()) { - values.add(entry.getValue()); - } - - // -- result - Object result = merger.execute(values); - return extractor.execute(result); - } - - } - - public RollingWindow cleanup_rolling_window(int cutoff) { - synchronized (this) { - List toremove = new ArrayList(); - for (Entry entry : buckets.entrySet()) { - Integer key = entry.getKey(); - if (key < cutoff) { - toremove.add(key); - } - } - - for (Integer i : toremove) { - buckets.remove(i); - } - return this; - } - } - - /** - * clean old data in the buckets before (now - bucket_size_secs * - * num_buckets) - * - * @param cutoff - * @return - */ - public RollingWindow cleanup_rolling_window() { - int cutoff = TimeUtils.current_time_secs() - rolling_window_size(); - return cleanup_rolling_window(cutoff); - } - - public int rolling_window_size() { - return bucket_size_secs * num_buckets; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/rolling/RollingWindowSet.java b/jstorm-server/src/main/java/com/alibaba/jstorm/stats/rolling/RollingWindowSet.java deleted file mode 100644 index 5ad27de95..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/rolling/RollingWindowSet.java +++ /dev/null @@ -1,124 +0,0 @@ -package com.alibaba.jstorm.stats.rolling; - -import java.util.HashMap; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.utils.TimeUtils; - -public class RollingWindowSet { - private RunnableCallback updater; - private RunnableCallback extractor; - private RollingWindow[] windows; - // all_time store the all_time result - private Object all_time; - - public RunnableCallback getUpdater() { - return updater; - } - - public void setUpdater(RunnableCallback updater) { - this.updater = updater; - } - - public RunnableCallback getExtractor() { - return extractor; - } - - public void setExtractor(RunnableCallback extractor) { - this.extractor = extractor; - } - - public RollingWindow[] getWindows() { - return windows; - } - - public void setWindows(RollingWindow[] windows) { - this.windows = windows; - } - - public Object getAll_time() { - return all_time; - } - - public void setAll_time(Object all_time) { - this.all_time = all_time; - } - - public static RollingWindowSet rolling_window_set(RunnableCallback updater, - RunnableCallback merger, RunnableCallback extractor, - Integer num_buckets, Integer[] bucket_size) { - - RollingWindowSet rtn = new RollingWindowSet(); - - rtn.setUpdater(updater); - - rtn.setExtractor(extractor); - - rtn.setWindows(new RollingWindow[bucket_size.length]); - int bSize = bucket_size.length; - for (int i = 0; i < bSize; i++) { - rtn.getWindows()[i] = RollingWindow.rolling_window(updater, merger, - extractor, bucket_size[i], num_buckets); - } - rtn.setAll_time(null); - return rtn; - } - - /** - * - * @param args - * Object[0] -- key, Object[1] -- value - */ - public void update_rolling_window_set(Object[] args) { - synchronized (this) { - int now = TimeUtils.current_time_secs(); - int winSize = windows.length; - for (int i = 0; i < winSize; i++) { - windows[i] = windows[i].update_rolling_window(now, args); - } - - UpdateParams p = new UpdateParams(); - p.setArgs(args); - p.setCurr(getAll_time()); - - setAll_time(updater.execute(p)); - } - } - - public RollingWindowSet cleanup_rolling_window_set() { - synchronized (this) { - for (int i = 0; i < windows.length; i++) { - windows[i] = windows[i].cleanup_rolling_window(); - } - return this; - } - } - - // Key -- 0 -- all time - /** - * Keys: 0 -- all time summary { 30 * 20, 540 * 20, 4320 * 20 } seconds - * summary - * - * @return - */ - public HashMap value_rolling_window_set() { - - HashMap rtn = new HashMap(); - - synchronized (this) { - int wSize = windows.length; - for (int i = 0; i < wSize; i++) { - int size = windows[i].rolling_window_size(); - // -- obj - Object obj = windows[i].value_rolling_window(); - - rtn.put(size, obj); - } - - Object result = extractor.execute(all_time); - - rtn.put(0, result); - return rtn; - } - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/rolling/UpdateParams.java b/jstorm-server/src/main/java/com/alibaba/jstorm/stats/rolling/UpdateParams.java deleted file mode 100644 index 943ae439b..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/stats/rolling/UpdateParams.java +++ /dev/null @@ -1,23 +0,0 @@ -package com.alibaba.jstorm.stats.rolling; - -public class UpdateParams { - private Object[] args; - private Object curr; - - public Object[] getArgs() { - return args; - } - - public void setArgs(Object[] args) { - this.args = args; - } - - public Object getCurr() { - return curr; - } - - public void setCurr(Object curr) { - this.curr = curr; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/Assignment.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/Assignment.java deleted file mode 100644 index 29d23a816..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/Assignment.java +++ /dev/null @@ -1,172 +0,0 @@ -package com.alibaba.jstorm.task; - -import java.io.Serializable; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; - -import com.alibaba.jstorm.schedule.default_assign.ResourceWorkerSlot; - -/** - * Assignment of one Toplogy, stored in /ZK-DIR/assignments/{topologyid} - * nodeHost {supervisorid: hostname} -- assigned supervisor Map - * taskStartTimeSecs: {taskid, taskStartSeconds} masterCodeDir: topology source - * code's dir in Nimbus taskToResource: {taskid, ResourceAssignment} - * - * @author Lixin/Longda - */ -public class Assignment implements Serializable { - - private static final long serialVersionUID = 6087667851333314069L; - - private final String masterCodeDir; - /** - * @@@ nodeHost store , this will waste some zk - * storage - */ - private final Map nodeHost; - private final Map taskStartTimeSecs; - private final Set workers; - - public Assignment(String masterCodeDir, Set workers, - Map nodeHost, - Map taskStartTimeSecs) { - this.workers = workers; - this.nodeHost = nodeHost; - this.taskStartTimeSecs = taskStartTimeSecs; - this.masterCodeDir = masterCodeDir; - } - - public Map getNodeHost() { - return nodeHost; - } - - public Map getTaskStartTimeSecs() { - return taskStartTimeSecs; - } - - public String getMasterCodeDir() { - return masterCodeDir; - } - - public Set getWorkers() { - return workers; - } - - /** - * find workers for every supervisorId (node) - * - * @param supervisorId - * @return Map - */ - public Map getTaskToNodePortbyNode( - String supervisorId) { - - Map result = new HashMap(); - for (ResourceWorkerSlot worker : workers) { - if (worker.getNodeId().equals(supervisorId)) { - result.put(worker.getPort(), worker); - } - } - return result; - } - - public Set getCurrentSuperviosrTasks(String supervisorId) { - Set Tasks = new HashSet(); - - for (ResourceWorkerSlot worker : workers) { - if (worker.getNodeId().equals(supervisorId)) - Tasks.addAll(worker.getTasks()); - } - - return Tasks; - } - - public Set getCurrentSuperviosrWorkers(String supervisorId) { - Set workerSet = new HashSet(); - - for (ResourceWorkerSlot worker : workers) { - if (worker.getNodeId().equals(supervisorId)) - workerSet.add(worker.getPort()); - } - - return workerSet; - } - - public Set getCurrentWorkerTasks(String supervisorId, int port) { - - for (ResourceWorkerSlot worker : workers) { - if (worker.getNodeId().equals(supervisorId) - && worker.getPort() == port) - return worker.getTasks(); - } - - return new HashSet(); - } - - public ResourceWorkerSlot getWorkerByTaskId(Integer taskId) { - for (ResourceWorkerSlot worker : workers) { - if (worker.getTasks().contains(taskId)) - return worker; - } - return null; - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result - + ((masterCodeDir == null) ? 0 : masterCodeDir.hashCode()); - result = prime * result - + ((nodeHost == null) ? 0 : nodeHost.hashCode()); - result = prime - * result - + ((taskStartTimeSecs == null) ? 0 : taskStartTimeSecs - .hashCode()); - result = prime * result + ((workers == null) ? 0 : workers.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - Assignment other = (Assignment) obj; - if (masterCodeDir == null) { - if (other.masterCodeDir != null) - return false; - } else if (!masterCodeDir.equals(other.masterCodeDir)) - return false; - if (nodeHost == null) { - if (other.nodeHost != null) - return false; - } else if (!nodeHost.equals(other.nodeHost)) - return false; - if (taskStartTimeSecs == null) { - if (other.taskStartTimeSecs != null) - return false; - } else if (!taskStartTimeSecs.equals(other.taskStartTimeSecs)) - return false; - if (workers == null) { - if (other.workers != null) - return false; - } else if (!workers.equals(other.workers)) - return false; - return true; - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this, - ToStringStyle.SHORT_PREFIX_STYLE); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/AssignmentBak.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/AssignmentBak.java deleted file mode 100644 index c7ea61350..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/AssignmentBak.java +++ /dev/null @@ -1,38 +0,0 @@ -package com.alibaba.jstorm.task; - -import java.io.Serializable; -import java.util.List; -import java.util.Map; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; - -public class AssignmentBak implements Serializable { - - /** */ - private static final long serialVersionUID = 7633746649144483965L; - - private final Map> componentTasks; - private final Assignment assignment; - - public AssignmentBak(Map> componentTasks, - Assignment assignment) { - super(); - this.componentTasks = componentTasks; - this.assignment = assignment; - } - - public Map> getComponentTasks() { - return componentTasks; - } - - public Assignment getAssignment() { - return assignment; - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this, - ToStringStyle.SHORT_PREFIX_STYLE); - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/LocalAssignment.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/LocalAssignment.java deleted file mode 100644 index a50888f8c..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/LocalAssignment.java +++ /dev/null @@ -1,130 +0,0 @@ -package com.alibaba.jstorm.task; - -import java.io.Serializable; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; - -/** - * Supervisor LocalAssignment - * - */ -public class LocalAssignment implements Serializable { - public static final long serialVersionUID = 4054639727225043554L; - private final String topologyId; - private final String topologyName; - private Set taskIds; - private long mem; - private int cpu; - private String jvm; - - public LocalAssignment(String topologyId, Set taskIds, - String topologyName, long mem, int cpu, String jvm) { - this.topologyId = topologyId; - this.taskIds = new HashSet(taskIds); - this.topologyName = topologyName; - this.mem = mem; - this.cpu = cpu; - this.jvm = jvm; - } - - public String getTopologyId() { - return topologyId; - } - - public Set getTaskIds() { - return taskIds; - } - - public void setTaskIds(Set taskIds) { - this.taskIds = new HashSet(taskIds); - } - - public String getTopologyName() { - return topologyName; - } - - public String getJvm() { - return jvm; - } - - public void setJvm(String jvm) { - this.jvm = jvm; - } - - public long getMem() { - return mem; - } - - public void setMem(long mem) { - this.mem = mem; - } - - public int getCpu() { - return cpu; - } - - public void setCpu(int cpu) { - this.cpu = cpu; - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + cpu; - result = prime * result + ((jvm == null) ? 0 : jvm.hashCode()); - result = prime * result + (int) (mem ^ (mem >>> 32)); - result = prime * result + ((taskIds == null) ? 0 : taskIds.hashCode()); - result = prime * result - + ((topologyId == null) ? 0 : topologyId.hashCode()); - result = prime * result - + ((topologyName == null) ? 0 : topologyName.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - LocalAssignment other = (LocalAssignment) obj; - if (cpu != other.cpu) - return false; - if (jvm == null) { - if (other.jvm != null) - return false; - } else if (!jvm.equals(other.jvm)) - return false; - if (mem != other.mem) - return false; - if (taskIds == null) { - if (other.taskIds != null) - return false; - } else if (!taskIds.equals(other.taskIds)) - return false; - if (topologyId == null) { - if (other.topologyId != null) - return false; - } else if (!topologyId.equals(other.topologyId)) - return false; - if (topologyName == null) { - if (other.topologyName != null) - return false; - } else if (!topologyName.equals(other.topologyName)) - return false; - return true; - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this, - ToStringStyle.SHORT_PREFIX_STYLE); - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/ShutdownableDameon.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/ShutdownableDameon.java deleted file mode 100644 index f0f07bfef..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/ShutdownableDameon.java +++ /dev/null @@ -1,10 +0,0 @@ -package com.alibaba.jstorm.task; - -import backtype.storm.daemon.Shutdownable; - -import com.alibaba.jstorm.cluster.DaemonCommon; - -public interface ShutdownableDameon extends Shutdownable, DaemonCommon, - Runnable { - -} 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 deleted file mode 100644 index f36cb28d9..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/Task.java +++ /dev/null @@ -1,292 +0,0 @@ -package com.alibaba.jstorm.task; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; -import backtype.storm.messaging.IContext; -import backtype.storm.serialization.KryoTupleSerializer; -import backtype.storm.spout.ISpout; -import backtype.storm.task.IBolt; -import backtype.storm.task.TopologyContext; -import backtype.storm.utils.DisruptorQueue; -import backtype.storm.utils.Utils; -import backtype.storm.utils.WorkerClassLoader; -import clojure.lang.Atom; - -import com.alibaba.jstorm.callback.AsyncLoopThread; -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.Common; -import com.alibaba.jstorm.cluster.StormClusterState; -import com.alibaba.jstorm.cluster.StormConfig; -import com.alibaba.jstorm.cluster.StormZkClusterState; -import com.alibaba.jstorm.daemon.worker.WorkerData; -import com.alibaba.jstorm.daemon.worker.WorkerHaltRunable; -import com.alibaba.jstorm.stats.CommonStatsRolling; -import com.alibaba.jstorm.task.comm.TaskSendTargets; -import com.alibaba.jstorm.task.comm.UnanchoredSend; -import com.alibaba.jstorm.task.error.ITaskReportErr; -import com.alibaba.jstorm.task.error.TaskReportError; -import com.alibaba.jstorm.task.error.TaskReportErrorAndDie; -import com.alibaba.jstorm.task.execute.BaseExecutors; -import com.alibaba.jstorm.task.execute.BoltExecutors; -import com.alibaba.jstorm.task.execute.spout.MultipleThreadSpoutExecutors; -import com.alibaba.jstorm.task.execute.spout.SingleThreadSpoutExecutors; -import com.alibaba.jstorm.task.execute.spout.SpoutExecutors; -import com.alibaba.jstorm.task.group.MkGrouper; -import com.alibaba.jstorm.task.heartbeat.TaskHeartbeatRunable; -import com.alibaba.jstorm.task.heartbeat.TaskStats; -import com.alibaba.jstorm.utils.JStormServerUtils; -import com.alibaba.jstorm.utils.JStormUtils; -import com.lmax.disruptor.WaitStrategy; -import com.lmax.disruptor.dsl.ProducerType; - -/** - * Task instance - * - * @author yannian/Longda - * - */ -public class Task { - - private final static Logger LOG = Logger.getLogger(Task.class); - - private Map stormConf; - - private TopologyContext topologyContext; - private TopologyContext userContext; - private String topologyid; - private IContext context; - - private TaskTransfer taskTransfer; - private Map innerTaskTransfer; - private Map deserializeQueues; - private WorkerHaltRunable workHalt; - - private Integer taskid; - private String componentid; - private volatile TaskStatus taskStatus; - private Atom openOrPrepareWasCalled; - // running time counter - private UptimeComputer uptime = new UptimeComputer(); - - private StormClusterState zkCluster; - private Object taskObj; - private CommonStatsRolling taskStats; - private WorkerData workerData; - private String componentType; //"spout" or "bolt" - - @SuppressWarnings("rawtypes") - public Task(WorkerData workerData, int taskId) throws Exception { - openOrPrepareWasCalled = new Atom(Boolean.valueOf(false)); - - this.workerData = workerData; - this.topologyContext = workerData.getContextMaker() - .makeTopologyContext(workerData.getSysTopology(), taskId, - openOrPrepareWasCalled); - this.userContext = workerData.getContextMaker().makeTopologyContext( - workerData.getRawTopology(), taskId, openOrPrepareWasCalled); - this.taskid = taskId; - this.componentid = topologyContext.getThisComponentId(); - - this.taskStatus = new TaskStatus(); - this.taskTransfer = getSendingTransfer(workerData); - this.innerTaskTransfer = workerData.getInnerTaskTransfer(); - this.deserializeQueues = workerData.getDeserializeQueues(); - this.topologyid = workerData.getTopologyId(); - this.context = workerData.getContext(); - this.workHalt = workerData.getWorkHalt(); - this.zkCluster = new StormZkClusterState(workerData.getZkClusterstate()); - - 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); - - LOG.info("Loading task " + componentid + ":" + taskid); - } - - private void setComponentType() { - if (taskObj instanceof IBolt) { - componentType = "bolt"; - } else if (taskObj instanceof ISpout) { - componentType = "spout"; - } - } - - private TaskSendTargets makeSendTargets() { - String component = topologyContext.getThisComponentId(); - - // get current task's output - // > - Map> streamComponentGrouper = Common - .outbound_components(topologyContext, workerData); - - Map task2Component = topologyContext - .getTaskToComponent(); - Map> component2Tasks = JStormUtils - .reverse_map(task2Component); - - return new TaskSendTargets(stormConf, component, - streamComponentGrouper, topologyContext, component2Tasks, - taskStats); - } - - private TaskTransfer getSendingTransfer(WorkerData workerData) { - - // sending tuple's serializer - KryoTupleSerializer serializer = new KryoTupleSerializer( - workerData.getStormConf(), topologyContext); - - String taskName = JStormServerUtils.getName(componentid, taskid); - // Task sending all tuples through this Object - return new TaskTransfer(taskName, serializer, taskStatus, workerData); - } - - public TaskSendTargets echoToSystemBolt() { - // send "startup" tuple to system bolt - List msg = new ArrayList(); - msg.add("startup"); - - // create task receive object - TaskSendTargets sendTargets = makeSendTargets(); - UnanchoredSend.send(topologyContext, sendTargets, taskTransfer, - Common.SYSTEM_STREAM_ID, msg); - - return sendTargets; - } - - public boolean isSingleThread(Map conf) { - boolean isOnePending = JStormServerUtils.isOnePending(conf); - if (isOnePending == true) { - return true; - } - - return ConfigExtension.isSpoutSingleThread(conf); - } - - public RunnableCallback mk_executors(DisruptorQueue deserializeQueue, - TaskSendTargets sendTargets, ITaskReportErr report_error) { - - if (taskObj instanceof IBolt) { - return new BoltExecutors((IBolt) taskObj, taskTransfer, - innerTaskTransfer, stormConf, deserializeQueue, sendTargets, - taskStatus, topologyContext, userContext, taskStats, - report_error); - } else if (taskObj instanceof ISpout) { - if (isSingleThread(stormConf) == true) { - return new SingleThreadSpoutExecutors((ISpout) taskObj, taskTransfer, - innerTaskTransfer, stormConf, deserializeQueue, sendTargets, - taskStatus, topologyContext, userContext, taskStats, - report_error); - }else { - return new MultipleThreadSpoutExecutors((ISpout) taskObj, taskTransfer, - innerTaskTransfer, stormConf, deserializeQueue, sendTargets, - taskStatus, topologyContext, userContext, taskStats, - report_error); - } - } - - return null; - } - - /** - * create executor to receive tuples and run bolt/spout execute function - * - * @param puller - * @param sendTargets - * @return - */ - private RunnableCallback mkExecutor(DisruptorQueue deserializeQueue, - TaskSendTargets sendTargets) { - // create report error callback, - // in fact it is storm_cluster.report-task-error - ITaskReportErr reportError = new TaskReportError(zkCluster, topologyid, - taskid); - - // report error and halt worker - TaskReportErrorAndDie reportErrorDie = new TaskReportErrorAndDie( - reportError, workHalt); - - return mk_executors(deserializeQueue, sendTargets, reportErrorDie); - } - - public DisruptorQueue registerDisruptorQueue() { - int queueSize = JStormUtils.parseInt( - stormConf.get(Config.TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE), 256); - - WaitStrategy waitStrategy = (WaitStrategy) Utils - .newInstance((String) stormConf - .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); - DisruptorQueue queue = DisruptorQueue.mkInstance("TaskDeserialize", ProducerType.SINGLE, - queueSize, waitStrategy); - - deserializeQueues.put(taskid, queue); - - return queue; - } - - public TaskShutdownDameon execute() throws Exception { - setComponentType(); - - DisruptorQueue deserializeQueue = registerDisruptorQueue(); - - TaskSendTargets sendTargets = echoToSystemBolt(); - - // create thread to get tuple from zeroMQ, - // and pass the tuple to bolt/spout - RunnableCallback baseExecutor = mkExecutor(deserializeQueue, sendTargets); - AsyncLoopThread executor_threads = new AsyncLoopThread(baseExecutor, - false, Thread.MAX_PRIORITY, true); - - List allThreads = new ArrayList(); - allThreads.add(executor_threads); - - TaskHeartbeatRunable.registerTaskStats(taskid, new TaskStats(componentType, taskStats)); - LOG.info("Finished loading task " + componentid + ":" + taskid); - - return getShutdown(allThreads, deserializeQueue, baseExecutor); - } - - public TaskShutdownDameon getShutdown(List allThreads, - DisruptorQueue deserializeQueue, RunnableCallback baseExecutor) { - - AsyncLoopThread ackerThread = null; - if (baseExecutor instanceof SpoutExecutors) { - ackerThread = ((SpoutExecutors) baseExecutor).getAckerRunnableThread(); - - if (ackerThread != null) { - allThreads.add(ackerThread); - } - } - AsyncLoopThread recvThread = ((BaseExecutors) baseExecutor).getDeserlizeThread(); - allThreads.add(recvThread); - - AsyncLoopThread serializeThread = taskTransfer.getSerializeThread(); - allThreads.add(serializeThread); - - TaskShutdownDameon shutdown = new TaskShutdownDameon(taskStatus, - topologyid, taskid, allThreads, zkCluster, taskObj); - - return shutdown; - } - - public static TaskShutdownDameon mk_task(WorkerData workerData, int taskId) - throws Exception { - - Task t = new Task(workerData, taskId); - - return t.execute(); - } - -} 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 deleted file mode 100644 index 533fb1d4f..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/TaskInfo.java +++ /dev/null @@ -1,58 +0,0 @@ -package com.alibaba.jstorm.task; - -import java.io.Serializable; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; - -/** - * /storm-zk-root/tasks/{topologyid}/{taskid} data - */ -public class TaskInfo implements Serializable { - - private static final long serialVersionUID = 5625165079055837777L; - private String componentId; - private String componentType; - - public TaskInfo(String componentId, String componentType) { - this.componentId = componentId; - this.componentType = componentType; - } - - public String getComponentId() { - return componentId; - } - - public void setComponentId(String componentId) { - this.componentId = componentId; - } - - public String getComponentType() { - return componentType; - } - - public void setComponentType(String componentType) { - this.componentType = componentType; - } - - @Override - public boolean equals(Object assignment) { - if (assignment instanceof TaskInfo - && ((TaskInfo) assignment).getComponentId().equals(getComponentId()) - && ((TaskInfo) assignment).getComponentType().equals(componentType)) { - return true; - } - return false; - } - - @Override - public int hashCode() { - return this.getComponentId().hashCode() + this.getComponentType().hashCode(); - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this, - ToStringStyle.SHORT_PREFIX_STYLE); - } -} 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 deleted file mode 100644 index a90ff46f1..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/TaskMetricInfo.java +++ /dev/null @@ -1,152 +0,0 @@ -package com.alibaba.jstorm.task; - -import java.io.Serializable; -import java.util.Map; -import java.util.HashMap; -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; -import com.codahale.metrics.Gauge; -import com.codahale.metrics.Timer; -import com.codahale.metrics.Counter; -import com.codahale.metrics.Histogram; -import com.codahale.metrics.Meter; -import com.codahale.metrics.Snapshot; -import com.alibaba.jstorm.metric.MetricDef; -import com.alibaba.jstorm.metric.MetricInfo; -import com.alibaba.jstorm.metric.Metrics.QueueGauge; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.task.execute.spout.TimerRatio; - -/** - * /storm-zk-root/Monitor/{topologyid}/{taskid} data - */ -public class TaskMetricInfo implements Serializable { - public static Logger LOG = Logger.getLogger(TaskMetricInfo.class); - - private static final long serialVersionUID = 7645367099257857979L; - private String taskId; - private String component; - - private Map gaugeData; - private Map counterData; - private Map meterData; - private Map timerData; - private Map histogramData; - - private static final double FULL_RATIO = 100.0; - - public static final String QEUEU_IS_FULL = "queue is full"; - - public TaskMetricInfo(String taskId, String component) { - this.taskId = taskId; - this.component = component; - - this.gaugeData = new HashMap(); - this.counterData = new HashMap(); - this.meterData = new HashMap(); - this.timerData = new HashMap(); - this.histogramData = new HashMap(); - } - - public String getTaskId() { - return taskId; - } - - public void setTaskId(String taskId) { - this.taskId = taskId; - } - - public String getComponent() { - return this.component; - } - - public void setComponent(String component) { - this.component = component; - } - - public Map getGaugeData() { - return gaugeData; - } - - public Map getCounterData() { - return counterData; - } - - public Map getMeterData() { - return meterData; - } - - public Map getTimerData() { - return timerData; - } - - public Map getHistogramData() { - return histogramData; - } - - public void updateMetricData(MetricInfo metricInfo) { - String name = metricInfo.getName(); - Metric metric = metricInfo.getMetric(); - if (metric instanceof QueueGauge) { - //covert to % - float queueRatio = (((QueueGauge) metric).getValue())*100; - double value = JStormUtils.formatDoubleDecPoint2((double)queueRatio); - gaugeData.put(name, value); - } else if ( (metric instanceof Gauge) || - (metric instanceof TimerRatio)) { - Double value = JStormUtils.convertToDouble(((Gauge) metric).getValue()); - if (value == null) { - LOG.warn("gauge value is null or unknow type."); - } else { - value = JStormUtils.formatDoubleDecPoint4(value); - gaugeData.put(name, value); - } - } else if (metric instanceof Timer) { - Snapshot snapshot = ((Timer) metric).getSnapshot(); - //Covert from ns to ms - Double value = JStormUtils.formatDoubleDecPoint4( - (snapshot.getMean())/1000000); - timerData.put(name, value); - } else if (metric instanceof Counter) { - Long value = ((Counter) metric).getCount(); - counterData.put(name, value.doubleValue()); - } else if (metric instanceof Meter) { - Double value = JStormUtils.formatDoubleDecPoint4( - ((Meter) metric).getMeanRate()); - meterData.put(name, value); - } else if (metric instanceof Histogram) { - Snapshot snapshot = ((Histogram) metric).getSnapshot(); - Double value = JStormUtils.formatDoubleDecPoint4( - snapshot.getMean()); - histogramData.put(name, value); - } else { - LOG.warn("Unknown metric type, name:" + name); - } - } - - public List anyQueueFull() { - List ret = new ArrayList(); - String taskInfo = component + "-" + taskId + ": "; - if (gaugeData.get(MetricDef.DESERIALIZE_QUEUE) == FULL_RATIO) { - ret.add(taskInfo + "deserialize-" + QEUEU_IS_FULL); - } else if (gaugeData.get(MetricDef.SERIALIZE_QUEUE) == FULL_RATIO) - { - ret.add(taskInfo + "serialize-" + QEUEU_IS_FULL); - } else if (gaugeData.get(MetricDef.EXECUTE_QUEUE) == FULL_RATIO) { - ret.add(taskInfo + "execute-" + QEUEU_IS_FULL); - } - 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/TaskShutdownDameon.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/TaskShutdownDameon.java deleted file mode 100644 index fa8bd49a0..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/TaskShutdownDameon.java +++ /dev/null @@ -1,151 +0,0 @@ -package com.alibaba.jstorm.task; - -import java.util.List; - -import org.apache.log4j.Logger; - -import backtype.storm.messaging.IConnection; -import backtype.storm.messaging.IContext; -import backtype.storm.spout.ISpout; -import backtype.storm.task.IBolt; -import backtype.storm.utils.WorkerClassLoader; - -import com.alibaba.jstorm.callback.AsyncLoopThread; -import com.alibaba.jstorm.cluster.StormClusterState; -import com.alibaba.jstorm.task.heartbeat.TaskHeartbeatRunable; -import com.alibaba.jstorm.utils.JStormUtils; - -/** - * shutdown one task - * - * @author yannian/Longda - * - */ -public class TaskShutdownDameon implements ShutdownableDameon { - private static Logger LOG = Logger.getLogger(TaskShutdownDameon.class); - - public static final byte QUIT_MSG = (byte) 0xff; - - private TaskStatus taskStatus; - private String topology_id; - private Integer task_id; - private List all_threads; - private StormClusterState zkCluster; - private Object task_obj; - private boolean isClosed = false; - - public TaskShutdownDameon(TaskStatus taskStatus, String topology_id, - Integer task_id, List all_threads, - StormClusterState zkCluster, Object task_obj) { - this.taskStatus = taskStatus; - this.topology_id = topology_id; - this.task_id = task_id; - this.all_threads = all_threads; - this.zkCluster = zkCluster; - this.task_obj = task_obj; - - } - - @Override - public void shutdown() { - synchronized (this) { - if (isClosed == true) { - return ; - } - isClosed = true; - } - - LOG.info("Begin to shut down task " + topology_id + ":" + task_id); - - // all thread will check the taskStatus - // once it has been set SHUTDOWN, it will quit - taskStatus.setStatus(TaskStatus.SHUTDOWN); - - // waiting 100ms for executor thread shutting it's own - try { - Thread.sleep(100); - } catch (InterruptedException e) { - } - - for (AsyncLoopThread thr : all_threads) { - LOG.info("Begin to shutdown " + thr.getThread().getName()); - thr.cleanup(); - JStormUtils.sleepMs(10); - thr.interrupt(); -// try { -// //thr.join(); -// thr.getThread().stop(new RuntimeException()); -// } catch (Throwable e) { -// } - LOG.info("Successfully shutdown " + thr.getThread().getName()); - } - - closeComponent(task_obj); - - try { - TaskHeartbeatRunable.unregisterTaskStats(task_id); - zkCluster.remove_task_heartbeat(topology_id, task_id); - zkCluster.disconnect(); - } catch (Exception e) { - // TODO Auto-generated catch block - LOG.info(e); - } - - LOG.info("Successfully shutdown task " + topology_id + ":" + task_id); - - } - - public void join() throws InterruptedException { - for (AsyncLoopThread t : all_threads) { - t.join(); - } - } - - private void closeComponent(Object _task_obj) { - if (_task_obj instanceof IBolt) { - ((IBolt) _task_obj).cleanup(); - } - - if (_task_obj instanceof ISpout) { - ((ISpout) _task_obj).close(); - } - } - - @Override - public boolean waiting() { - return taskStatus.isRun(); - } - - public void deactive() { - - if (task_obj instanceof ISpout) { - taskStatus.setStatus(TaskStatus.PAUSE); - WorkerClassLoader.switchThreadContext(); - - try { - ((ISpout) task_obj).deactivate(); - } finally { - WorkerClassLoader.restoreThreadContext(); - } - } - - } - - public void active() { - if (task_obj instanceof ISpout) { - taskStatus.setStatus(TaskStatus.RUN); - WorkerClassLoader.switchThreadContext(); - try { - ((ISpout) task_obj).activate(); - } finally { - WorkerClassLoader.restoreThreadContext(); - } - } - } - - @Override - public void run() { - // TODO Auto-generated method stub - shutdown(); - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/TaskStatus.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/TaskStatus.java deleted file mode 100644 index 22010098a..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/TaskStatus.java +++ /dev/null @@ -1,41 +0,0 @@ -package com.alibaba.jstorm.task; - -public class TaskStatus { - // task is alive, and it will run BaseExecutor's run - public static final byte RUN = 0; - // task is alive, but it won't run BaseExecutor's run - public static final byte PAUSE = 1; - // task is shutdown - public static final byte SHUTDOWN = 2; - - private volatile byte status = TaskStatus.RUN; - - public byte getStatus() { - return status; - } - - public void setStatus(byte status) { - this.status = status; - } - - public boolean isRun() { - return status == TaskStatus.RUN; - } - - public boolean isShutdown() { - return status == TaskStatus.SHUTDOWN; - } - - public boolean isPause() { - return status == TaskStatus.PAUSE; - } - - /** - * @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/TaskTransfer.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/TaskTransfer.java deleted file mode 100644 index 49c668b54..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/TaskTransfer.java +++ /dev/null @@ -1,145 +0,0 @@ -package com.alibaba.jstorm.task; - -import java.util.Map; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; -import backtype.storm.messaging.TaskMessage; -import backtype.storm.serialization.KryoTupleSerializer; -import backtype.storm.tuple.TupleExt; -import backtype.storm.utils.DisruptorQueue; -import backtype.storm.utils.Utils; -import backtype.storm.utils.WorkerClassLoader; - -import com.alibaba.jstorm.callback.AsyncLoopThread; -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.daemon.worker.WorkerData; -import com.alibaba.jstorm.metric.JStormTimer; -import com.alibaba.jstorm.metric.MetricDef; -import com.alibaba.jstorm.metric.Metrics; -import com.lmax.disruptor.EventHandler; -import com.lmax.disruptor.WaitStrategy; -import com.lmax.disruptor.dsl.ProducerType; - -/** - * Sending entrance - * - * Task sending all tuples through this Object - * - * Serialize the Tuple and put the serialized data to the sending queue - * - * @author yannian - * - */ -public class TaskTransfer { - - private static Logger LOG = Logger.getLogger(TaskTransfer.class); - - private Map storm_conf; - private DisruptorQueue transferQueue; - private KryoTupleSerializer serializer; - private Map innerTaskTransfer; - private DisruptorQueue serializeQueue; - private final AsyncLoopThread serializeThread; - private volatile TaskStatus taskStatus; - private String taskName; - private JStormTimer timer; - - public TaskTransfer(String taskName, - KryoTupleSerializer serializer, TaskStatus taskStatus, - WorkerData workerData) { - this.taskName = taskName; - this.serializer = serializer; - this.taskStatus = taskStatus; - this.storm_conf = workerData.getConf(); - this.transferQueue = workerData.getTransferQueue(); - this.innerTaskTransfer = workerData.getInnerTaskTransfer(); - - int queue_size = Utils.getInt(storm_conf - .get(Config.TOPOLOGY_EXECUTOR_SEND_BUFFER_SIZE)); - WaitStrategy waitStrategy = (WaitStrategy) Utils - .newInstance((String) storm_conf - .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); - this.serializeQueue = DisruptorQueue.mkInstance(taskName, ProducerType.MULTI, - queue_size, waitStrategy); - this.serializeQueue.consumerStarted(); - - String taskId = taskName.substring(taskName.indexOf(":") + 1); - Metrics.registerQueue(taskName, MetricDef.SERIALIZE_QUEUE, serializeQueue, taskId, Metrics.MetricType.TASK); - timer = Metrics.registerTimer(taskName, MetricDef.SERIALIZE_TIME, taskId, Metrics.MetricType.TASK); - - serializeThread = new AsyncLoopThread(new TransferRunnable()); - LOG.info("Successfully start TaskTransfer thread"); - - } - - public void transfer(TupleExt tuple) { - - int taskid = tuple.getTargetTaskId(); - - DisruptorQueue exeQueue = innerTaskTransfer.get(taskid); - if (exeQueue != null) { - exeQueue.publish(tuple); - } else { - serializeQueue.publish(tuple); - } - - } - - public AsyncLoopThread getSerializeThread() { - return serializeThread; - } - - class TransferRunnable extends RunnableCallback implements EventHandler { - - @Override - public String getThreadName() { - return taskName + "-" +TransferRunnable.class.getSimpleName(); - } - - @Override - public void run() { - - WorkerClassLoader.switchThreadContext(); - while (taskStatus.isShutdown() == false) { - serializeQueue.consumeBatchWhenAvailable(this); - - } - WorkerClassLoader.restoreThreadContext(); - } - - public Object getResult() { - if (taskStatus.isShutdown() == false) { - return 0; - } else { - return -1; - } - } - - @Override - public void onEvent(Object event, long sequence, boolean endOfBatch) - throws Exception { - - if (event == null) { - return; - } - - timer.start(); - - try { - TupleExt tuple = (TupleExt) event; - int taskid = tuple.getTargetTaskId(); - byte[] tupleMessage = serializer.serialize(tuple); - TaskMessage taskMessage = new TaskMessage(taskid, tupleMessage); - transferQueue.publish(taskMessage); - }finally { - timer.stop(); - } - - - } - - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/TkHbCacheTime.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/TkHbCacheTime.java deleted file mode 100644 index 63d31c619..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/TkHbCacheTime.java +++ /dev/null @@ -1,48 +0,0 @@ -package com.alibaba.jstorm.task; - -import com.alibaba.jstorm.task.heartbeat.TaskHeartbeat; -import com.alibaba.jstorm.utils.TimeUtils; - -/** - * TkHbCacheTime is describle taskheartcache (Map>>) - */ - -public class TkHbCacheTime { - private int nimbusTime; - private int taskReportedTime; - private int taskAssignedTime; - - public int getNimbusTime() { - return nimbusTime; - } - - public void setNimbusTime(int nimbusTime) { - this.nimbusTime = nimbusTime; - } - - public int getTaskReportedTime() { - return taskReportedTime; - } - - public void setTaskReportedTime(int taskReportedTime) { - this.taskReportedTime = taskReportedTime; - } - - public int getTaskAssignedTime() { - return taskAssignedTime; - } - - public void setTaskAssignedTime(int taskAssignedTime) { - this.taskAssignedTime = taskAssignedTime; - } - - public void update(TaskHeartbeat zkTaskHeartbeat) { - int nowSecs = TimeUtils.current_time_secs(); - this.nimbusTime = nowSecs; - this.taskReportedTime = zkTaskHeartbeat.getTimeSecs(); - this.taskAssignedTime = zkTaskHeartbeat.getTimeSecs() - - zkTaskHeartbeat.getUptimeSecs(); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/UptimeComputer.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/UptimeComputer.java deleted file mode 100644 index 96343abb9..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/UptimeComputer.java +++ /dev/null @@ -1,21 +0,0 @@ -package com.alibaba.jstorm.task; - -import com.alibaba.jstorm.utils.TimeUtils; - -/** - * Get how long task runs - * - * @author yannian - * - */ -public class UptimeComputer { - int start_time = 0; - - public UptimeComputer() { - start_time = TimeUtils.current_time_secs(); - } - - public synchronized int uptime() { - return TimeUtils.time_delta(start_time); - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/acker/AckObject.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/acker/AckObject.java deleted file mode 100644 index 30a924ac3..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/acker/AckObject.java +++ /dev/null @@ -1,19 +0,0 @@ -package com.alibaba.jstorm.task.acker; - -import com.alibaba.jstorm.utils.JStormUtils; - -public class AckObject { - public Long val = null; - public Integer spout_task = null; - public boolean failed = false; - - // val xor value - public void update_ack(Object value) { - synchronized (this) { - if (val == null) { - val = Long.valueOf(0); - } - val = JStormUtils.bit_xor(val, value); - } - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/acker/Acker.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/acker/Acker.java deleted file mode 100644 index a0bb4ea64..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/acker/Acker.java +++ /dev/null @@ -1,145 +0,0 @@ -package com.alibaba.jstorm.task.acker; - -import java.util.List; -import java.util.Map; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import backtype.storm.Config; -import backtype.storm.task.IBolt; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Tuple; - -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.RotatingMap; - -/** - * - * @author yannian/Longda - * - */ -public class Acker implements IBolt { - - private static final Logger LOG = LoggerFactory.getLogger(Acker.class); - - private static final long serialVersionUID = 4430906880683183091L; - - public static final String ACKER_COMPONENT_ID = "__acker"; - public static final String ACKER_INIT_STREAM_ID = "__ack_init"; - public static final String ACKER_ACK_STREAM_ID = "__ack_ack"; - public static final String ACKER_FAIL_STREAM_ID = "__ack_fail"; - - public static final int TIMEOUT_BUCKET_NUM = 3; - - private OutputCollector collector = null; - // private TimeCacheMap pending = null; - private RotatingMap pending = null; - private long lastRotate = System.currentTimeMillis(); - private long rotateTime; - - @Override - public void prepare(Map stormConf, TopologyContext context, - OutputCollector collector) { - this.collector = collector; - // pending = new TimeCacheMap(timeoutSec, - // TIMEOUT_BUCKET_NUM); - this.pending = new RotatingMap(TIMEOUT_BUCKET_NUM); - this.rotateTime = 1000L * JStormUtils.parseInt( - stormConf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS), 30)/ - (TIMEOUT_BUCKET_NUM - 1); - } - - @Override - public void execute(Tuple input) { - Object id = input.getValue(0); - - AckObject curr = pending.get(id); - - String stream_id = input.getSourceStreamId(); - - if (Acker.ACKER_INIT_STREAM_ID.equals(stream_id)) { - if (curr == null) { - curr = new AckObject(); - - curr.val = input.getLong(1); - curr.spout_task = input.getInteger(2); - - pending.put(id, curr); - } else { - // bolt's ack first come - curr.update_ack(input.getValue(1)); - curr.spout_task = input.getInteger(2); - } - - } else if (Acker.ACKER_ACK_STREAM_ID.equals(stream_id)) { - if (curr != null) { - curr.update_ack(input.getValue(1)); - - } else { - // two case - // one is timeout - // the other is bolt's ack first come - curr = new AckObject(); - - curr.val = Long.valueOf(input.getLong(1)); - - pending.put(id, curr); - - } - } else if (Acker.ACKER_FAIL_STREAM_ID.equals(stream_id)) { - if (curr == null) { - // do nothing - // already timeout, should go fail - return; - } - - curr.failed = true; - - } else { - LOG.info("Unknow source stream"); - return; - } - - Integer task = curr.spout_task; - - if (task != null) { - - if (curr.val == 0) { - pending.remove(id); - List values = JStormUtils.mk_list(id); - - collector.emitDirect(task, Acker.ACKER_ACK_STREAM_ID, values); - - } else { - - if (curr.failed) { - pending.remove(id); - List values = JStormUtils.mk_list(id); - collector.emitDirect(task, Acker.ACKER_FAIL_STREAM_ID, - values); - } - } - } else { - - } - - // add this operation to update acker's ACK statics - collector.ack(input); - - long now = System.currentTimeMillis(); - if (now - lastRotate > rotateTime) { - lastRotate = now; - Map tmp = pending.rotate(); - LOG.info("Acker's timeout item size:{}", tmp.size()); - } - - } - - @Override - public void cleanup() { - LOG.info("Successfully cleanup"); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/comm/TaskSendTargets.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/comm/TaskSendTargets.java deleted file mode 100644 index ae898117e..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/comm/TaskSendTargets.java +++ /dev/null @@ -1,133 +0,0 @@ -package com.alibaba.jstorm.task.comm; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import backtype.storm.Config; -import backtype.storm.task.TopologyContext; - -import com.alibaba.jstorm.stats.CommonStatsRolling; -import com.alibaba.jstorm.task.group.GrouperType; -import com.alibaba.jstorm.task.group.MkGrouper; -import com.alibaba.jstorm.utils.JStormUtils; - -/** - * - * tuple sending object, which get which task should tuple be send to, and - * update statics - * - * @author yannian/Longda - * - */ -public class TaskSendTargets { - private static Logger LOG = LoggerFactory.getLogger(TaskSendTargets.class); - - private Map stormConf; - // it is system TopologyContext - private TopologyContext topologyContext; - - // > - private Map> streamComponentgrouper; - // SpoutTaskStatsRolling or BoltTaskStatsRolling - private CommonStatsRolling taskStats; - - private Map> componentTasks; - private String componentId; - private int taskId; - private boolean isDebuging = false; - private String debugIdStr; - - public TaskSendTargets(Map _storm_conf, String _component, - Map> _stream_component_grouper, - TopologyContext _topology_context, - Map> _component_tasks, - CommonStatsRolling _task_stats) { - this.stormConf = _storm_conf; - this.componentId = _component; - this.streamComponentgrouper = _stream_component_grouper; - this.topologyContext = _topology_context; - this.componentTasks = _component_tasks; - this.taskStats = _task_stats; - - isDebuging = JStormUtils.parseBoolean( - stormConf.get(Config.TOPOLOGY_DEBUG), false); - - taskId = topologyContext.getThisTaskId(); - debugIdStr = " Emit from " + componentId + ":" + taskId + " "; - } - - // direct send tuple to special task - public java.util.List get(Integer out_task_id, String stream, - List tuple) { - - // in order to improve acker's speed, skip checking - // String target_component = - // topologyContext.getComponentId(out_task_id); - // Map component_prouping = streamComponentgrouper - // .get(stream); - // MkGrouper grouping = component_prouping.get(target_component); - // if (grouping != null && - // !GrouperType.direct.equals(grouping.gettype())) { - // throw new IllegalArgumentException( - // "Cannot emitDirect to a task expecting a regular grouping"); - // } - - if (isDebuging) { - LOG.info(debugIdStr + stream + " to " + out_task_id + ":" - + tuple.toString()); - } - - taskStats.send_tuple(stream, 1); - - java.util.List out_tasks = new ArrayList(); - out_tasks.add(out_task_id); - return out_tasks; - } - - // send tuple according to grouping - public java.util.List get(String stream, List tuple) { - java.util.List out_tasks = new ArrayList(); - - // get grouper, then get which task should tuple be sent to. - Map componentCrouping = streamComponentgrouper - .get(stream); - if (componentCrouping == null) { - // if the target component's parallelism is 0, don't need send to - // them - LOG.debug("Failed to get Grouper of " + stream + " in " - + debugIdStr); - return out_tasks; - } - - for (Entry ee : componentCrouping.entrySet()) { - String targetComponent = ee.getKey(); - MkGrouper g = ee.getValue(); - - if (GrouperType.direct.equals(g.gettype())) { - throw new IllegalArgumentException( - "Cannot do regular emit to direct stream"); - } - - out_tasks.addAll(g.grouper(tuple)); - - } - - if (isDebuging) { - - LOG.info(debugIdStr + stream + " to " + out_tasks + ":" - + tuple.toString()); - } - - int num_out_tasks = out_tasks.size(); - - taskStats.send_tuple(stream, num_out_tasks); - - return out_tasks; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/comm/TupleInfo.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/comm/TupleInfo.java deleted file mode 100644 index 9ed730a72..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/comm/TupleInfo.java +++ /dev/null @@ -1,56 +0,0 @@ -package com.alibaba.jstorm.task.comm; - -import java.io.Serializable; -import java.util.List; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; - -public class TupleInfo implements Serializable { - - private static final long serialVersionUID = -3348670497595864118L; - - private Object messageId; - private String stream; - private List values; - private long timestamp; - - public Object getMessageId() { - return messageId; - } - - public void setMessageId(Object messageId) { - this.messageId = messageId; - } - - public long getTimestamp() { - return timestamp; - } - - public void setTimestamp(long timestamp) { - this.timestamp = timestamp; - } - - public String getStream() { - return stream; - } - - public void setStream(String stream) { - this.stream = stream; - } - - public List getValues() { - return values; - } - - public void setValues(List values) { - this.values = values; - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this, - ToStringStyle.SHORT_PREFIX_STYLE); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/comm/UnanchoredSend.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/comm/UnanchoredSend.java deleted file mode 100644 index af920d0c0..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/comm/UnanchoredSend.java +++ /dev/null @@ -1,37 +0,0 @@ -package com.alibaba.jstorm.task.comm; - -import java.util.List; - -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.TupleImplExt; - -import com.alibaba.jstorm.task.TaskTransfer; - -/** - * Send init/ack/fail tuple to acker - * - * @author yannian - * - */ - -public class UnanchoredSend { - public static void send(TopologyContext topologyContext, - TaskSendTargets taskTargets, TaskTransfer transfer_fn, - String stream, List values) { - - java.util.List tasks = taskTargets.get(stream, values); - if (tasks.size() == 0) { - return; - } - - Integer taskId = topologyContext.getThisTaskId(); - - for (Integer task : tasks) { - TupleImplExt tup = new TupleImplExt(topologyContext, values, - taskId, stream); - tup.setTargetTaskId(task); - - transfer_fn.transfer(tup); - } - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/error/ITaskReportErr.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/error/ITaskReportErr.java deleted file mode 100644 index 9e3ea7c23..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/error/ITaskReportErr.java +++ /dev/null @@ -1,11 +0,0 @@ -package com.alibaba.jstorm.task.error; - -/** - * task report error interface - * - * @author yannian - * - */ -public interface ITaskReportErr { - public void report(Throwable error); -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/error/TaskError.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/error/TaskError.java deleted file mode 100644 index 1298ba2d3..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/error/TaskError.java +++ /dev/null @@ -1,38 +0,0 @@ -package com.alibaba.jstorm.task.error; - -import java.io.Serializable; - -/** - * Task error stored in Zk(/storm-zk-root/taskerrors/{topologyid}/{taskid}) - * - * @author yannian - * - */ -public class TaskError implements Serializable { - - private static final long serialVersionUID = 5028789764629555542L; - private String error; - private int timSecs; - - public TaskError(String error, int timSecs) { - this.error = error; - this.timSecs = timSecs; - } - - public String getError() { - return error; - } - - public void setError(String error) { - this.error = error; - } - - public int getTimSecs() { - return timSecs; - } - - public void setTimSecs(int timSecs) { - this.timSecs = timSecs; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/error/TaskErrorRunable.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/error/TaskErrorRunable.java deleted file mode 100644 index 7540d8e7c..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/error/TaskErrorRunable.java +++ /dev/null @@ -1,31 +0,0 @@ -package com.alibaba.jstorm.task.error; - -import com.alibaba.jstorm.callback.RunnableCallback; - -/** - * The callback will be called, when task occur error It just call - * TaskReportErrorAndDie - * - * @author yannian - * - */ -public class TaskErrorRunable extends RunnableCallback { - private TaskReportErrorAndDie report_error_and_die; - - public TaskErrorRunable(TaskReportErrorAndDie _report_error_and_die) { - this.report_error_and_die = _report_error_and_die; - } - - @Override - public Object execute(T... args) { - Exception e = null; - if (args != null && args.length > 0) { - e = (Exception) args[0]; - } - if (e != null) { - report_error_and_die.report(e); - } - return null; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/error/TaskReportError.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/error/TaskReportError.java deleted file mode 100644 index 0fb81282a..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/error/TaskReportError.java +++ /dev/null @@ -1,41 +0,0 @@ -package com.alibaba.jstorm.task.error; - -import org.apache.log4j.Logger; - -import com.alibaba.jstorm.cluster.StormClusterState; - -/** - * Task error report callback - * - * @author yannian - * - */ -public class TaskReportError implements ITaskReportErr { - private static Logger LOG = Logger.getLogger(TaskReportError.class); - private StormClusterState zkCluster; - private String topology_id; - private int task_id; - - public TaskReportError(StormClusterState _storm_cluster_state, - String _topology_id, int _task_id) { - this.zkCluster = _storm_cluster_state; - this.topology_id = _topology_id; - this.task_id = _task_id; - } - - @Override - public void report(Throwable error) { - - LOG.error("Report error to /ZK/taskerrors/" + topology_id + "/" + task_id - + "\n", error); - try { - zkCluster.report_task_error(topology_id, task_id, error); - } catch (Exception e) { - // TODO Auto-generated catch block - LOG.error("Failed update error to /ZK/taskerrors/" + topology_id + "/" - + task_id + "\n", e); - } - - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/error/TaskReportErrorAndDie.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/error/TaskReportErrorAndDie.java deleted file mode 100644 index 24f093a0f..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/error/TaskReportErrorAndDie.java +++ /dev/null @@ -1,26 +0,0 @@ -package com.alibaba.jstorm.task.error; - -import com.alibaba.jstorm.daemon.worker.WorkerHaltRunable; - -/** - * Task report error to ZK and halt the process - * - * @author yannian - * - */ -public class TaskReportErrorAndDie implements ITaskReportErr { - private ITaskReportErr reporterror; - private WorkerHaltRunable haltfn; - - public TaskReportErrorAndDie(ITaskReportErr _reporterror, - WorkerHaltRunable _haltfn) { - this.reporterror = _reporterror; - this.haltfn = _haltfn; - } - - @Override - public void report(Throwable error) { - this.reporterror.report(error); - this.haltfn.run(); - } -} 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 deleted file mode 100644 index 4690c45f7..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/BaseExecutors.java +++ /dev/null @@ -1,295 +0,0 @@ -package com.alibaba.jstorm.task.execute; - -import java.util.Map; - -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; -import backtype.storm.utils.DisruptorQueue; -import backtype.storm.utils.Utils; -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.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.message.zeroMq.IRecvConnection; -import com.alibaba.jstorm.stats.CommonStatsRolling; -import com.alibaba.jstorm.task.TaskStatus; -import com.alibaba.jstorm.task.TaskTransfer; -import com.alibaba.jstorm.task.error.ITaskReportErr; -import com.alibaba.jstorm.utils.JStormServerUtils; -import com.alibaba.jstorm.utils.JStormUtils; -import com.lmax.disruptor.EventHandler; -import com.lmax.disruptor.WaitStrategy; -import com.lmax.disruptor.dsl.ProducerType; - -/** - * Base executor share between spout and bolt - * - * - * @author Longda - * - */ -public class BaseExecutors extends RunnableCallback { - private static Logger LOG = Logger.getLogger(BaseExecutors.class); - - protected final String component_id; - protected final int taskId; - protected final boolean isDebugRecv; - protected final boolean isDebug; - protected final String idStr; - - protected Map storm_conf; - // ZMQConnection puller - protected DisruptorQueue deserializeQueue; - protected KryoTupleDeserializer deserializer; - protected AsyncLoopThread deserializeThread; - protected JStormTimer deserializeTimer; - - protected TopologyContext userTopologyCtx; - protected CommonStatsRolling task_stats; - - protected volatile TaskStatus taskStatus; - - protected int message_timeout_secs = 30; - - protected Throwable error = null; - - protected ITaskReportErr report_error; - - protected DisruptorQueue exeQueue; - protected Map innerTaskTransfer; - - - - // protected IntervalCheck intervalCheck = new IntervalCheck(); - - public BaseExecutors(TaskTransfer _transfer_fn, Map _storm_conf, - DisruptorQueue deserializeQueue, - Map innerTaskTransfer, - TopologyContext topology_context, TopologyContext _user_context, - CommonStatsRolling _task_stats, TaskStatus taskStatus, - ITaskReportErr _report_error) { - - this.storm_conf = _storm_conf; - this.deserializeQueue = deserializeQueue; - - this.userTopologyCtx = _user_context; - this.task_stats = _task_stats; - this.taskId = topology_context.getThisTaskId(); - this.innerTaskTransfer = innerTaskTransfer; - this.component_id = topology_context.getThisComponentId(); - this.idStr = JStormServerUtils.getName(component_id, taskId); - - this.taskStatus = taskStatus; - this.report_error = _report_error; - - this.deserializer = new KryoTupleDeserializer(storm_conf, - topology_context);// (KryoTupleDeserializer. - - this.isDebugRecv = ConfigExtension.isTopologyDebugRecvTuple(storm_conf); - this.isDebug = JStormUtils.parseBoolean( - storm_conf.get(Config.TOPOLOGY_DEBUG), false); - - message_timeout_secs = JStormUtils.parseInt( - storm_conf.get(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS), 30); - - int queue_size = Utils.getInt( - storm_conf.get(Config.TOPOLOGY_EXECUTOR_RECEIVE_BUFFER_SIZE), - 256); - WaitStrategy waitStrategy = (WaitStrategy) Utils - .newInstance((String) storm_conf - .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); - this.exeQueue = DisruptorQueue.mkInstance(idStr, ProducerType.MULTI, - queue_size, waitStrategy); - this.exeQueue.consumerStarted(); - - this.registerInnerTransfer(exeQueue); - - deserializeThread = new AsyncLoopThread(new DeserializeRunnable( - deserializeQueue, exeQueue)); - - 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 - public void run() { - // this function will be override by SpoutExecutor or BoltExecutor - throw new RuntimeException("Should implement this function"); - } - - @Override - public Object getResult() { - if (taskStatus.isRun()) { - return 0; - } else if (taskStatus.isPause()) { - return 0; - } else if (taskStatus.isShutdown()) { - this.shutdown(); - return -1; - } else { - LOG.info("Unknow TaskStatus, shutdown executing thread of " + idStr); - this.shutdown(); - return -1; - } - } - - @Override - public Exception error() { - if (error == null) { - return null; - } - - return new Exception(error); - } - - @Override - public void shutdown() { - LOG.info("Shutdown executing thread of " + idStr); - if (taskStatus.isShutdown() == false) { - LOG.info("Taskstatus isn't shutdown, but enter shutdown method, Occur exception"); - } - this.unregistorInnerTransfer(); - - } - - protected void registerInnerTransfer(DisruptorQueue disruptorQueue) { - LOG.info("Registor inner transfer for executor thread of " + idStr); - DisruptorQueue existInnerTransfer = innerTaskTransfer.get(taskId); - if (existInnerTransfer != null) { - LOG.info("Exist inner task transfer for executing thread of " - + idStr); - if (existInnerTransfer != disruptorQueue) { - throw new RuntimeException( - "Inner task transfer must be only one in executing thread of " - + idStr); - } - } - innerTaskTransfer.put(taskId, disruptorQueue); - } - - protected void unregistorInnerTransfer() { - LOG.info("Unregistor inner transfer for executor thread of " + idStr); - innerTaskTransfer.remove(taskId); - } - - public AsyncLoopThread getDeserlizeThread() { - return deserializeThread; - } - - class DeserializeRunnable extends RunnableCallback implements EventHandler { - - DisruptorQueue deserializeQueue; - DisruptorQueue exeQueue; - - DeserializeRunnable(DisruptorQueue deserializeQueue, - DisruptorQueue exeQueue) { - this.deserializeQueue = deserializeQueue; - this.exeQueue = exeQueue; - } - - @Override - public String getThreadName() { - return idStr + "-deserializer"; - } - - protected Tuple deserialize(byte[] ser_msg) { - deserializeTimer.start(); - try { - if (ser_msg == null) { - return null; - } - - if (ser_msg.length == 0) { - return null; - } else if (ser_msg.length == 1) { - byte newStatus = ser_msg[0]; - LOG.info("Change task status as " + newStatus); - taskStatus.setStatus(newStatus); - - return null; - } - - // ser_msg.length > 1 - Tuple tuple = deserializer.deserialize(ser_msg); - - if (isDebugRecv) { - - LOG.info(idStr + " receive " + tuple.toString()); - } - - // recv_tuple_queue.offer(tuple); - - return tuple; - - } catch (Throwable e) { - if (taskStatus.isShutdown() == false) { - LOG.error( - idStr + " recv thread error " - + JStormUtils.toPrintableString(ser_msg) - + "\n", e); - } - }finally { - deserializeTimer.stop(); - } - - return null; - } - - @Override - public void onEvent(Object event, long sequence, boolean endOfBatch) - throws Exception { - Tuple tuple = deserialize((byte[]) event); - - if (tuple != null) { - exeQueue.publish(tuple); - } - } - - @Override - public void run() { - WorkerClassLoader.switchThreadContext(); - - LOG.info("Successfully start recvThread of " + idStr); - - while (taskStatus.isShutdown() == false) { - try { - - deserializeQueue.consumeBatchWhenAvailable(this); - } catch (Throwable e) { - if (taskStatus.isShutdown() == false) { - LOG.error("Unknow exception ", e); - report_error.report(e); - } - } - - } - - WorkerClassLoader.restoreThreadContext(); - - LOG.info("Successfully shutdown recvThread of " + idStr); - } - - public Object getResult() { - LOG.info("Begin to shutdown recvThread of " + idStr); - return -1; - } - - } - -} 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 deleted file mode 100644 index d2e61dfc2..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/BoltCollector.java +++ /dev/null @@ -1,236 +0,0 @@ -package com.alibaba.jstorm.task.execute; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Random; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; -import backtype.storm.task.IOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.MessageId; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.TupleImplExt; - -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; -import com.alibaba.jstorm.task.acker.Acker; -import com.alibaba.jstorm.task.comm.TaskSendTargets; -import com.alibaba.jstorm.task.comm.UnanchoredSend; -import com.alibaba.jstorm.task.error.ITaskReportErr; -import com.alibaba.jstorm.utils.JStormServerUtils; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.RotatingMap; -import com.alibaba.jstorm.utils.TimeUtils; - -/** - * bolt output interface, do emit/ack/fail - * - * @author yannian/Longda - * - */ -public class BoltCollector implements IOutputCollector { - private static Logger LOG = Logger.getLogger(BoltCollector.class); - - private ITaskReportErr reportError; - private TaskSendTargets sendTargets; - private TaskTransfer taskTransfer; - private TopologyContext topologyContext; - private Integer task_id; - // private TimeCacheMap tuple_start_times; - private RotatingMap tuple_start_times; - private CommonStatsRolling task_stats; - // private TimeCacheMap pending_acks; - private RotatingMap pending_acks; - private long lastRotate = System.currentTimeMillis(); - private long rotateTime; - - private Map storm_conf; - private Integer ackerNum; - private JStormTimer timer; - private Random random; - - public BoltCollector(int message_timeout_secs, ITaskReportErr report_error, - TaskSendTargets _send_fn, Map _storm_conf, - TaskTransfer _transfer_fn, TopologyContext _topology_context, - Integer task_id, RotatingMap tuple_start_times, - CommonStatsRolling _task_stats) { - - this.rotateTime = 1000L * message_timeout_secs/(Acker.TIMEOUT_BUCKET_NUM - 1); - this.reportError = report_error; - this.sendTargets = _send_fn; - this.storm_conf = _storm_conf; - this.taskTransfer = _transfer_fn; - this.topologyContext = _topology_context; - this.task_id = task_id; - this.task_stats = _task_stats; - - this.pending_acks = new RotatingMap( - Acker.TIMEOUT_BUCKET_NUM); - // this.pending_acks = new TimeCacheMap(message_timeout_secs, - // Acker.TIMEOUT_BUCKET_NUM); - this.tuple_start_times = tuple_start_times; - - this.ackerNum = JStormUtils.parseInt(storm_conf - .get(Config.TOPOLOGY_ACKER_EXECUTORS)); - - String componentId = topologyContext.getThisComponentId(); - timer = Metrics.registerTimer(JStormServerUtils.getName(componentId, task_id), - MetricDef.EMIT_TIME, String.valueOf(task_id), Metrics.MetricType.TASK); - random = new Random(); - random.setSeed(System.currentTimeMillis()); - } - - @Override - public List emit(String streamId, Collection anchors, - List tuple) { - return boltEmit(streamId, anchors, tuple, null); - } - - @Override - public void emitDirect(int taskId, String streamId, - Collection anchors, List tuple) { - boltEmit(streamId, anchors, tuple, taskId); - } - - private List boltEmit(String out_stream_id, - Collection anchors, List values, Integer out_task_id) { - timer.start(); - try { - java.util.List out_tasks = null; - if (out_task_id != null) { - out_tasks = sendTargets.get(out_task_id, out_stream_id, values); - } else { - out_tasks = sendTargets.get(out_stream_id, values); - } - - for (Integer t : out_tasks) { - Map anchors_to_ids = new HashMap(); - if (anchors != null) { - for (Tuple a : anchors) { - //Long edge_id = MessageId.generateId(); - Long edge_id = MessageId.generateId(random); - long now = System.currentTimeMillis(); - if (now - lastRotate > rotateTime) { - pending_acks.rotate(); - lastRotate = now; - } - put_xor(pending_acks, a, edge_id); - for (Long root_id : a.getMessageId().getAnchorsToIds() - .keySet()) { - put_xor(anchors_to_ids, root_id, edge_id); - } - } - } - MessageId msgid = MessageId.makeId(anchors_to_ids); - TupleImplExt tupleExt = new TupleImplExt(topologyContext, - values, task_id, out_stream_id, msgid); - tupleExt.setTargetTaskId(t); - - taskTransfer.transfer(tupleExt); - - } - return out_tasks; - } catch (Exception e) { - LOG.error("bolt emit", e); - }finally { - timer.stop(); - } - return new ArrayList(); - } - - @Override - public void ack(Tuple input) { - - if (ackerNum > 0) { - - Long ack_val = Long.valueOf(0); - Object pend_val = pending_acks.remove(input); - if (pend_val != null) { - ack_val = (Long) (pend_val); - } - - for (Entry e : input.getMessageId().getAnchorsToIds() - .entrySet()) { - - UnanchoredSend.send( - topologyContext, - sendTargets, - taskTransfer, - Acker.ACKER_ACK_STREAM_ID, - JStormUtils.mk_list((Object) e.getKey(), - JStormUtils.bit_xor(e.getValue(), ack_val))); - } - } - - Long delta = tuple_time_delta(tuple_start_times, input); - if (delta != null) { - task_stats.bolt_acked_tuple(input.getSourceComponent(), - input.getSourceStreamId(), delta); - } - } - - @Override - public void fail(Tuple input) { - // if ackerNum == 0, we can just return - if (ackerNum > 0) { - pending_acks.remove(input); - for (Entry e : input.getMessageId().getAnchorsToIds() - .entrySet()) { - UnanchoredSend.send(topologyContext, sendTargets, taskTransfer, - Acker.ACKER_FAIL_STREAM_ID, - JStormUtils.mk_list((Object) e.getKey())); - } - } - - task_stats.bolt_failed_tuple(input.getSourceComponent(), - input.getSourceStreamId()); - - } - - @Override - public void reportError(Throwable error) { - reportError.report(error); - } - - // Utility functions, just used here - public static Long tuple_time_delta(RotatingMap start_times, - Tuple tuple) { - Long start_time = (Long) start_times.remove(tuple); - if (start_time != null) { - return TimeUtils.time_delta_ms(start_time); - } - return null; - } - - public static void put_xor(RotatingMap pending, Tuple key, - Long id) { - // synchronized (pending) { - Long curr = pending.get(key); - if (curr == null) { - curr = Long.valueOf(0); - } - pending.put(key, JStormUtils.bit_xor(curr, id)); - // } - } - - public static void put_xor(Map pending, Long key, Long id) { - // synchronized (pending) { - Long curr = pending.get(key); - if (curr == null) { - curr = Long.valueOf(0); - } - pending.put(key, JStormUtils.bit_xor(curr, id)); - // } - } - -} 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 deleted file mode 100644 index a316c6072..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/BoltExecutors.java +++ /dev/null @@ -1,210 +0,0 @@ -package com.alibaba.jstorm.task.execute; - -import java.util.Map; -import java.util.Map.Entry; - -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; -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Tuple; -import backtype.storm.utils.DisruptorQueue; -import backtype.storm.utils.WorkerClassLoader; - -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; -import com.alibaba.jstorm.task.TaskTransfer; -import com.alibaba.jstorm.task.acker.Acker; -import com.alibaba.jstorm.task.comm.TaskSendTargets; -import com.alibaba.jstorm.task.error.ITaskReportErr; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.RotatingMap; -import com.alibaba.jstorm.utils.TimeUtils; -import com.lmax.disruptor.EventHandler; - -/** - * - * BoltExecutor - * - * @author yannian/Longda - * - */ -public class BoltExecutors extends BaseExecutors implements EventHandler { - private static Logger LOG = Logger.getLogger(BoltExecutors.class); - - protected IBolt bolt; - - protected RotatingMap tuple_start_times; - - private int ackerNum = 0; - - // internal outputCollector is BoltCollector - private OutputCollector outputCollector; - - private JStormTimer boltExeTimer; - - public BoltExecutors(IBolt _bolt, TaskTransfer _transfer_fn, - Map innerTaskTransfer, Map storm_conf, - DisruptorQueue deserializeQueue, TaskSendTargets _send_fn, - TaskStatus taskStatus, TopologyContext sysTopologyCxt, - TopologyContext userTopologyCxt, CommonStatsRolling _task_stats, - ITaskReportErr _report_error) { - - super(_transfer_fn, storm_conf, deserializeQueue, innerTaskTransfer, - sysTopologyCxt, userTopologyCxt, _task_stats, taskStatus, - _report_error); - - this.bolt = _bolt; - - // create TimeCacheMap - - this.tuple_start_times = new RotatingMap( - Acker.TIMEOUT_BUCKET_NUM); - - this.ackerNum = JStormUtils.parseInt(storm_conf - .get(Config.TOPOLOGY_ACKER_EXECUTORS)); - - // don't use TimeoutQueue for recv_tuple_queue, - // then other place should check the queue size - // TimeCacheQueue.DefaultExpiredCallback logExpireCb = new - // TimeCacheQueue.DefaultExpiredCallback( - // idStr); - // this.recv_tuple_queue = new - // TimeCacheQueue(message_timeout_secs, - // TimeCacheQueue.DEFAULT_NUM_BUCKETS, logExpireCb); - - // create BoltCollector - IOutputCollector output_collector = new BoltCollector( - message_timeout_secs, _report_error, _send_fn, storm_conf, - _transfer_fn, sysTopologyCxt, taskId, tuple_start_times, - _task_stats); - - outputCollector = new OutputCollector(output_collector); - - boltExeTimer = Metrics.registerTimer(idStr, MetricDef.EXECUTE_TIME, - String.valueOf(taskId), Metrics.MetricType.TASK); - - 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(); - -// 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; - LOG.error("bolt prepare error ", e); - report_error.report(e); - } finally { - WorkerClassLoader.restoreThreadContext(); - } - - LOG.info("Successfully create BoltExecutors " + idStr); - - } - - @Override - public String getThreadName() { - return idStr + "-" + BoltExecutors.class.getSimpleName(); - } - - @Override - public void run() { - - WorkerClassLoader.switchThreadContext(); - while (taskStatus.isShutdown() == false) { - try { - - exeQueue.consumeBatchWhenAvailable(this); - - } catch (Throwable e) { - if (taskStatus.isShutdown() == false) { - LOG.error(idStr + " bolt exeutor error", e); - } - } - } - WorkerClassLoader.restoreThreadContext(); - - } - - @Override - public void onEvent(Object event, long sequence, boolean endOfBatch) - throws Exception { - - if (event == null) { - return; - } - - boltExeTimer.start(); - - try { - - if (event instanceof RotatingMapTrigger.Tick) { - // don't check the timetick name to improve performance - - Map timeoutMap = tuple_start_times.rotate(); - - if (ackerNum > 0) { - // only when acker is enable - for (Entry entry : timeoutMap.entrySet()) { - Tuple input = entry.getKey(); - task_stats.bolt_failed_tuple( - input.getSourceComponent(), - input.getSourceStreamId()); - } - } - - return; - } - - Tuple tuple = (Tuple) event; - - task_stats.recv_tuple(tuple.getSourceComponent(), - tuple.getSourceStreamId()); - - tuple_start_times.put(tuple, System.currentTimeMillis()); - - try { - bolt.execute(tuple); - } catch (Throwable e) { - error = e; - LOG.error("bolt execute error ", e); - report_error.report(e); - } - - if (ackerNum == 0) { - // only when acker is disable - // get tuple process latency - Long start_time = (Long) tuple_start_times.remove(tuple); - if (start_time != null) { - Long delta = TimeUtils.time_delta_ms(start_time); - task_stats.bolt_acked_tuple(tuple.getSourceComponent(), - tuple.getSourceStreamId(), delta); - } - } - } finally { - boltExeTimer.stop(); - } - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/AckSpoutMsg.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/AckSpoutMsg.java deleted file mode 100644 index 1864cffaf..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/AckSpoutMsg.java +++ /dev/null @@ -1,60 +0,0 @@ -package com.alibaba.jstorm.task.execute.spout; - -import java.util.List; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import backtype.storm.spout.ISpout; - -import com.alibaba.jstorm.client.spout.IAckValueSpout; -import com.alibaba.jstorm.stats.CommonStatsRolling; -import com.alibaba.jstorm.task.comm.TupleInfo; - -/** - * The action after spout receive one ack tuple - * - * @author yannian/Longda - * - */ -public class AckSpoutMsg implements IAckMsg { - private static Logger LOG = LoggerFactory.getLogger(AckSpoutMsg.class); - - private ISpout spout; - private Object msgId; - private String stream; - private long timeStamp; - private List values; - private CommonStatsRolling task_stats; - private boolean isDebug = false; - - public AckSpoutMsg(ISpout _spout, TupleInfo tupleInfo, - CommonStatsRolling _task_stats, boolean _isDebug) { - - this.task_stats = _task_stats; - - this.spout = _spout; - this.isDebug = _isDebug; - - this.msgId = tupleInfo.getMessageId(); - this.stream = tupleInfo.getStream(); - this.timeStamp = tupleInfo.getTimestamp(); - this.values = tupleInfo.getValues(); - } - - public void run() { - if (isDebug) { - LOG.info("Acking message {}", msgId); - } - - if (spout instanceof IAckValueSpout) { - IAckValueSpout ackValueSpout = (IAckValueSpout) spout; - ackValueSpout.ack(msgId, values); - } else { - spout.ack(msgId); - } - - task_stats.spout_acked_tuple(stream, timeStamp); - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/FailSpoutMsg.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/FailSpoutMsg.java deleted file mode 100644 index 4ecd1d18b..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/FailSpoutMsg.java +++ /dev/null @@ -1,55 +0,0 @@ -package com.alibaba.jstorm.task.execute.spout; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import backtype.storm.spout.ISpout; -import backtype.storm.utils.WorkerClassLoader; - -import com.alibaba.jstorm.client.spout.IFailValueSpout; -import com.alibaba.jstorm.stats.CommonStatsRolling; -import com.alibaba.jstorm.task.comm.TupleInfo; - -/** - * Do the action after spout receive one failed tuple or sending tuple timeout - * - * @author yannian/Longda - * - */ -public class FailSpoutMsg implements IAckMsg { - private static Logger LOG = LoggerFactory.getLogger(FailSpoutMsg.class); - private Object id; - private ISpout spout; - private TupleInfo tupleInfo; - private CommonStatsRolling task_stats; - private boolean isDebug = false; - - public FailSpoutMsg(Object id, ISpout _spout, TupleInfo _tupleInfo, - CommonStatsRolling _task_stats, boolean _isDebug) { - this.id = id; - this.spout = _spout; - this.tupleInfo = _tupleInfo; - this.task_stats = _task_stats; - this.isDebug = _isDebug; - } - - public void run() { - - Object msg_id = tupleInfo.getMessageId(); - - if (spout instanceof IFailValueSpout) { - IFailValueSpout enhanceSpout = (IFailValueSpout) spout; - enhanceSpout.fail(msg_id, tupleInfo.getValues()); - } else { - spout.fail(msg_id); - } - - task_stats.spout_failed_tuple(tupleInfo.getStream()); - - if (isDebug) { - LOG.info("Failed message rootId: {}, messageId:{} : {}", id, - msg_id, tupleInfo.getValues().toString()); - } - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/IAckMsg.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/IAckMsg.java deleted file mode 100644 index cdf0d2e49..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/IAckMsg.java +++ /dev/null @@ -1,5 +0,0 @@ -package com.alibaba.jstorm.task.execute.spout; - -public interface IAckMsg extends Runnable { - -} 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 deleted file mode 100644 index 2835c699b..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/MultipleThreadSpoutExecutors.java +++ /dev/null @@ -1,116 +0,0 @@ -package com.alibaba.jstorm.task.execute.spout; - -import java.util.Map; - -import org.apache.log4j.Logger; - -import backtype.storm.task.TopologyContext; -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.metric.MetricDef; -import com.alibaba.jstorm.metric.Metrics; -import com.alibaba.jstorm.stats.CommonStatsRolling; -import com.alibaba.jstorm.task.TaskStatus; -import com.alibaba.jstorm.task.TaskTransfer; -import com.alibaba.jstorm.task.acker.Acker; -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.RotatingMap; -import com.codahale.metrics.Gauge; - -/** - * spout executor - * - * All spout actions will be done here - * - * @author yannian/Longda - * - */ -public class MultipleThreadSpoutExecutors extends SpoutExecutors { - private static Logger LOG = Logger - .getLogger(MultipleThreadSpoutExecutors.class); - - public MultipleThreadSpoutExecutors(backtype.storm.spout.ISpout _spout, - TaskTransfer _transfer_fn, - Map innerTaskTransfer, Map _storm_conf, - DisruptorQueue deserializeQueue, TaskSendTargets sendTargets, - TaskStatus taskStatus, TopologyContext topology_context, - TopologyContext _user_context, CommonStatsRolling _task_stats, - ITaskReportErr _report_error) { - super(_spout, _transfer_fn, innerTaskTransfer, _storm_conf, - deserializeQueue, sendTargets, taskStatus, topology_context, - _user_context, _task_stats, _report_error); - - ackerRunnableThread = new AsyncLoopThread(new AckerRunnable()); - pending = new RotatingMap(Acker.TIMEOUT_BUCKET_NUM, null, false); - Metrics.register(idStr, MetricDef.PENDING_MAP, new Gauge() { - - @Override - public Integer getValue() { - return pending.size(); - } - - }, String.valueOf(taskId), Metrics.MetricType.TASK); - - super.prepare(sendTargets, _transfer_fn, topology_context); - } - - @Override - public String getThreadName() { - return idStr + "-" +MultipleThreadSpoutExecutors.class.getSimpleName(); - } - - @Override - public void run() { - - WorkerClassLoader.switchThreadContext(); - try { - - super.nextTuple(); - }finally { - WorkerClassLoader.restoreThreadContext(); - } - - - } - - class AckerRunnable extends RunnableCallback { - - @Override - public String getThreadName() { - return idStr + "-" +AckerRunnable.class.getSimpleName(); - } - - @Override - public void run() { - LOG.info("Successfully start Spout's acker thread " + idStr); - - while (MultipleThreadSpoutExecutors.this.taskStatus.isShutdown() == false) { - WorkerClassLoader.switchThreadContext(); - try { - exeQueue.consumeBatchWhenAvailable(MultipleThreadSpoutExecutors.this); - } catch (Exception e) { - if (taskStatus.isShutdown() == false) { - LOG.error("Actor occur unknow exception ", e); - } - }finally { - WorkerClassLoader.restoreThreadContext(); - } - - } - - LOG.info("Successfully shutdown Spout's acker thread " + idStr); - } - - public Object getResult() { - LOG.info("Begin to shutdown Spout's acker thread " + idStr); - return -1; - } - - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/SingleThreadSpoutExecutors.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/SingleThreadSpoutExecutors.java deleted file mode 100644 index 5248fa3de..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/SingleThreadSpoutExecutors.java +++ /dev/null @@ -1,91 +0,0 @@ -package com.alibaba.jstorm.task.execute.spout; - -import java.util.Map; - -import org.apache.log4j.Logger; - -import backtype.storm.task.TopologyContext; -import backtype.storm.utils.DisruptorQueue; -import backtype.storm.utils.WorkerClassLoader; - -import com.alibaba.jstorm.metric.MetricDef; -import com.alibaba.jstorm.metric.Metrics; -import com.alibaba.jstorm.stats.CommonStatsRolling; -import com.alibaba.jstorm.task.TaskStatus; -import com.alibaba.jstorm.task.TaskTransfer; -import com.alibaba.jstorm.task.acker.Acker; -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.RotatingMap; -import com.codahale.metrics.Gauge; - -/** - * spout executor - * - * All spout actions will be done here - * - * @author yannian/Longda - * - */ -public class SingleThreadSpoutExecutors extends SpoutExecutors { - private static Logger LOG = Logger - .getLogger(SingleThreadSpoutExecutors.class); - - public SingleThreadSpoutExecutors(backtype.storm.spout.ISpout _spout, - TaskTransfer _transfer_fn, - Map innerTaskTransfer, Map _storm_conf, - DisruptorQueue deserializeQueue, TaskSendTargets sendTargets, - TaskStatus taskStatus, TopologyContext topology_context, - TopologyContext _user_context, CommonStatsRolling _task_stats, - ITaskReportErr _report_error) { - super(_spout, _transfer_fn, innerTaskTransfer, _storm_conf, - deserializeQueue, sendTargets, taskStatus, topology_context, - _user_context, _task_stats, _report_error); - - // sending Tuple's TimeCacheMap - pending = new RotatingMap(Acker.TIMEOUT_BUCKET_NUM, null, true); - Metrics.register(idStr, MetricDef.PENDING_MAP, new Gauge() { - - @Override - public Integer getValue() { - return pending.size(); - } - - }, String.valueOf(taskId), Metrics.MetricType.TASK); - - super.prepare(sendTargets, _transfer_fn, topology_context); - } - - @Override - public String getThreadName() { - return idStr + "-" +SingleThreadSpoutExecutors.class.getSimpleName(); - } - - @Override - public void run() { - WorkerClassLoader.switchThreadContext(); - try { - - executeEvent(); - - super.nextTuple(); - } finally { - WorkerClassLoader.restoreThreadContext(); - } - - } - - private void executeEvent() { - try { - exeQueue.consumeBatch(this); - - } catch (Exception e) { - if (taskStatus.isShutdown() == false) { - LOG.error("Actor occur unknow exception ", e); - } - } - - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/SpoutCollector.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/SpoutCollector.java deleted file mode 100644 index f70d0eea9..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/SpoutCollector.java +++ /dev/null @@ -1,194 +0,0 @@ -package com.alibaba.jstorm.task.execute.spout; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Random; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; -import backtype.storm.spout.ISpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.MessageId; -import backtype.storm.tuple.TupleImplExt; -import backtype.storm.utils.DisruptorQueue; - -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.TaskTransfer; -import com.alibaba.jstorm.task.acker.Acker; -import com.alibaba.jstorm.task.comm.TaskSendTargets; -import com.alibaba.jstorm.task.comm.TupleInfo; -import com.alibaba.jstorm.task.comm.UnanchoredSend; -import com.alibaba.jstorm.task.error.ITaskReportErr; -import com.alibaba.jstorm.utils.JStormServerUtils; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.TimeOutMap; - -/** - * spout collector, sending tuple through this Object - * - * @author yannian/Longda - * - */ -public class SpoutCollector implements ISpoutOutputCollector { - private static Logger LOG = Logger.getLogger(SpoutCollector.class); - - private TaskSendTargets sendTargets; - private Map storm_conf; - private TaskTransfer transfer_fn; - // private TimeCacheMap pending; - private TimeOutMap pending; - // topology_context is system topology context - private TopologyContext topology_context; - - private DisruptorQueue disruptorAckerQueue; - private CommonStatsRolling task_stats; - private backtype.storm.spout.ISpout spout; - private ITaskReportErr report_error; - - private Integer task_id; - private Integer ackerNum; - private boolean isDebug = false; - - private JStormTimer emitTotalTimer; - Random random; - - public SpoutCollector(Integer task_id, backtype.storm.spout.ISpout spout, - CommonStatsRolling task_stats, TaskSendTargets sendTargets, - Map _storm_conf, TaskTransfer _transfer_fn, - TimeOutMap pending, - TopologyContext topology_context, - DisruptorQueue disruptorAckerQueue, ITaskReportErr _report_error) { - this.sendTargets = sendTargets; - this.storm_conf = _storm_conf; - this.transfer_fn = _transfer_fn; - this.pending = pending; - this.topology_context = topology_context; - - this.disruptorAckerQueue = disruptorAckerQueue; - - this.task_stats = task_stats; - this.spout = spout; - this.task_id = task_id; - this.report_error = _report_error; - - ackerNum = JStormUtils.parseInt(storm_conf - .get(Config.TOPOLOGY_ACKER_EXECUTORS)); - isDebug = JStormUtils.parseBoolean( - storm_conf.get(Config.TOPOLOGY_DEBUG), false); - - random = new Random(); - random.setSeed(System.currentTimeMillis()); - - String componentId = topology_context.getThisComponentId(); - emitTotalTimer = Metrics.registerTimer(JStormServerUtils.getName(componentId, task_id), - MetricDef.EMIT_TIME, String.valueOf(task_id), Metrics.MetricType.TASK); - } - - @Override - public List emit(String streamId, List tuple, - Object messageId) { - return sendSpoutMsg(streamId, tuple, messageId, null); - } - - @Override - public void emitDirect(int taskId, String streamId, List tuple, - Object messageId) { - sendSpoutMsg(streamId, tuple, messageId, taskId); - } - - private List sendSpoutMsg(String out_stream_id, - List values, Object message_id, Integer out_task_id) { - - emitTotalTimer.start(); - - try { - java.util.List out_tasks = null; - if (out_task_id != null) { - out_tasks = sendTargets.get(out_task_id, out_stream_id, values); - } else { - out_tasks = sendTargets.get(out_stream_id, values); - } - - if (out_tasks.size() == 0) { - // don't need send tuple to other task - return out_tasks; - } - List ackSeq = new ArrayList(); - Boolean needAck = (message_id != null) && (ackerNum > 0); - - //This change storm logic - // Storm can't make sure root_id is unique - // storm's logic is root_id = MessageId.generateId(random); - // when duplicate root_id, it will miss call ack/fail - Long root_id = MessageId.generateId(random); - if (needAck) { - while(pending.containsKey(root_id) == true) { - root_id = MessageId.generateId(random); - } - } - for (Integer t : out_tasks) { - MessageId msgid; - if (needAck) { - //Long as = MessageId.generateId(); - Long as = MessageId.generateId(random); - msgid = MessageId.makeRootId(root_id, as); - ackSeq.add(as); - } else { - msgid = MessageId.makeUnanchored(); - } - - TupleImplExt tp = new TupleImplExt(topology_context, values, - task_id, out_stream_id, msgid); - tp.setTargetTaskId(t); - transfer_fn.transfer(tp); - - } - - if (needAck) { - - TupleInfo info = new TupleInfo(); - info.setStream(out_stream_id); - info.setValues(values); - info.setMessageId(message_id); - info.setTimestamp(System.currentTimeMillis()); - - pending.putHead(root_id, info); - - List ackerTuple = JStormUtils.mk_list((Object) root_id, - JStormUtils.bit_xor_vals(ackSeq), task_id); - - UnanchoredSend.send(topology_context, sendTargets, transfer_fn, - Acker.ACKER_INIT_STREAM_ID, ackerTuple); - - } else if (message_id != null) { - TupleInfo info = new TupleInfo(); - info.setStream(out_stream_id); - info.setValues(values); - info.setMessageId(message_id); - info.setTimestamp(0); - - AckSpoutMsg ack = new AckSpoutMsg(spout, info, task_stats, - isDebug); - ack.run(); - - } - - return out_tasks; - } finally { - emitTotalTimer.stop(); - } - - } - - @Override - public void reportError(Throwable error) { - // TODO Auto-generated method stub - report_error.report(error); - } - -} 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 deleted file mode 100644 index 854f43e3d..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/SpoutExecutors.java +++ /dev/null @@ -1,260 +0,0 @@ -package com.alibaba.jstorm.task.execute.spout; - -import java.util.Map; - -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.client.ConfigExtension; -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; -import com.alibaba.jstorm.task.TaskTransfer; -import com.alibaba.jstorm.task.acker.Acker; -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.task.execute.BaseExecutors; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.RotatingMap; -import com.lmax.disruptor.EventHandler; - -/** - * spout executor - * - * All spout actions will be done here - * - * @author yannian/Longda - * - */ -public class SpoutExecutors extends BaseExecutors implements EventHandler { - private static Logger LOG = Logger.getLogger(SpoutExecutors.class); - - protected final Integer max_spout_pending; - - protected backtype.storm.spout.ISpout spout; - protected RotatingMap pending; - - protected ISpoutOutputCollector output_collector; - - protected boolean firstTime = true; - - protected JStormTimer nextTupleTimer; - protected JStormTimer ackerTimer; - protected TimerRatio emptyCpuCounter; - - protected AsyncLoopThread ackerRunnableThread; - - protected boolean isSpoutFullSleep; - - public SpoutExecutors(backtype.storm.spout.ISpout _spout, - TaskTransfer _transfer_fn, - Map innerTaskTransfer, Map _storm_conf, - DisruptorQueue _puller, TaskSendTargets sendTargets, - TaskStatus taskStatus, TopologyContext topology_context, - TopologyContext _user_context, CommonStatsRolling _task_stats, - ITaskReportErr _report_error) { - super(_transfer_fn, _storm_conf, _puller, innerTaskTransfer, - topology_context, _user_context, _task_stats, taskStatus, - _report_error); - - this.spout = _spout; - - this.max_spout_pending = JStormUtils.parseInt(storm_conf - .get(Config.TOPOLOGY_MAX_SPOUT_PENDING)); - - this.nextTupleTimer = Metrics.registerTimer(idStr, MetricDef.EXECUTE_TIME, - String.valueOf(taskId), Metrics.MetricType.TASK); - this.ackerTimer = Metrics.registerTimer(idStr, MetricDef.ACKER_TIME, - String.valueOf(taskId), Metrics.MetricType.TASK); - this.emptyCpuCounter = new TimerRatio(); - Metrics.register(idStr, MetricDef.EMPTY_CPU_RATIO, emptyCpuCounter, - String.valueOf(taskId), Metrics.MetricType.TASK); - - isSpoutFullSleep = ConfigExtension.isSpoutPendFullSleep(storm_conf); - LOG.info("isSpoutFullSleep:" + isSpoutFullSleep); - - } - - - public void prepare(TaskSendTargets sendTargets, TaskTransfer transferFn, - TopologyContext topologyContext) { - - - // collector, in fact it call send_spout_msg - this.output_collector = new SpoutCollector(taskId, spout, task_stats, - sendTargets, storm_conf, transferFn, pending, topologyContext, - exeQueue, report_error); - - try { - WorkerClassLoader.switchThreadContext(); - this.spout.open(storm_conf, userTopologyCtx, - new SpoutOutputCollector(output_collector)); - } catch (Throwable e) { - error = e; - LOG.error("spout open error ", e); - report_error.report(e); - } finally { - WorkerClassLoader.restoreThreadContext(); - } - - LOG.info("Successfully create SpoutExecutors " + idStr); - - } - - 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; - LOG.info(idStr + " is ready "); - } - - if (taskStatus.isRun() == false) { - JStormUtils.sleepMs(1); - return; - } - - // if don't need ack, pending map will be always empty - if (max_spout_pending == null || pending.size() < max_spout_pending) { - emptyCpuCounter.stop(); - - nextTupleTimer.start(); - try { - spout.nextTuple(); - } catch (Throwable e) { - error = e; - LOG.error("spout execute error ", e); - report_error.report(e); - } finally { - nextTupleTimer.stop(); - } - - return; - } else { - if (isSpoutFullSleep) { - JStormUtils.sleepMs(1); - } - emptyCpuCounter.start(); - // just return, no sleep - } - } - - @Override - public void run() { - - throw new RuntimeException("Should implement this function"); - } - - /** - * Handle acker message - * - * @see com.lmax.disruptor.EventHandler#onEvent(java.lang.Object, long, - * boolean) - */ - @Override - public void onEvent(Object event, long sequence, boolean endOfBatch) - throws Exception { - ackerTimer.start(); - try { - - if (event == null) { - return; - } - - Runnable runnable = null; - if (event instanceof Tuple) { - - Tuple tuple = (Tuple) event; - Object id = tuple.getValue(0); - Object obj = pending.remove((Long) id); - - if (obj == null ) { - if (isDebug) { - LOG.info("Pending map no entry:" + id ); - } - return; - } - - TupleInfo tupleInfo = (TupleInfo) obj; - - String stream_id = tuple.getSourceStreamId(); - - if (stream_id.equals(Acker.ACKER_ACK_STREAM_ID)) { - - runnable = new AckSpoutMsg(spout, tupleInfo, task_stats, - isDebug); - } else if (stream_id.equals(Acker.ACKER_FAIL_STREAM_ID)) { - runnable = new FailSpoutMsg(id, spout, tupleInfo, task_stats, - isDebug); - } else { - LOG.warn("Receive one unknow source Tuple " + idStr); - return; - } - - task_stats.recv_tuple(tuple.getSourceComponent(), - tuple.getSourceStreamId()); - - } else if (event instanceof RotatingMapTrigger.Tick) { - - Map timeoutMap = pending.rotate(); - for (java.util.Map.Entry entry : timeoutMap - .entrySet()) { - TupleInfo tupleInfo = entry.getValue(); - FailSpoutMsg fail = new FailSpoutMsg(entry.getKey(), spout, - (TupleInfo) tupleInfo, task_stats, isDebug); - fail.run(); - } - return; - } else if (event instanceof IAckMsg) { - - runnable = (Runnable) event; - } else if (event instanceof Runnable) { - - runnable = (Runnable) event; - } else { - - LOG.warn("Receive one unknow event " + idStr); - return; - } - - runnable.run(); - - } catch (Throwable e) { - if (taskStatus.isShutdown() == false) { - LOG.info("Unknow excpetion ", e); - report_error.report(e); - } - } finally { - ackerTimer.stop(); - } - } - - public AsyncLoopThread getAckerRunnableThread() { - return ackerRunnableThread; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/SpoutTimeoutCallBack.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/SpoutTimeoutCallBack.java deleted file mode 100644 index 24a24372f..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/SpoutTimeoutCallBack.java +++ /dev/null @@ -1,54 +0,0 @@ -package com.alibaba.jstorm.task.execute.spout; - -import java.util.Map; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; -import backtype.storm.utils.DisruptorQueue; - -import com.alibaba.jstorm.stats.CommonStatsRolling; -import com.alibaba.jstorm.task.comm.TupleInfo; -import com.alibaba.jstorm.utils.ExpiredCallback; -import com.alibaba.jstorm.utils.JStormUtils; - -public class SpoutTimeoutCallBack implements - ExpiredCallback { - private static Logger LOG = Logger.getLogger(SpoutTimeoutCallBack.class); - - private DisruptorQueue disruptorEventQueue; - private backtype.storm.spout.ISpout spout; - private Map storm_conf; - private CommonStatsRolling task_stats; - private boolean isDebug; - - public SpoutTimeoutCallBack(DisruptorQueue disruptorEventQueue, - backtype.storm.spout.ISpout _spout, Map _storm_conf, - CommonStatsRolling stat) { - this.storm_conf = _storm_conf; - this.disruptorEventQueue = disruptorEventQueue; - this.spout = _spout; - this.task_stats = stat; - this.isDebug = JStormUtils.parseBoolean( - storm_conf.get(Config.TOPOLOGY_DEBUG), false); - } - - /** - * pending.put(root_id, JStormUtils.mk_list(message_id, TupleInfo, ms)); - */ - @Override - public void expire(K key, V val) { - if (val == null) { - return; - } - try { - TupleInfo tupleInfo = (TupleInfo) val; - FailSpoutMsg fail = new FailSpoutMsg(key, spout, (TupleInfo) tupleInfo, - task_stats, isDebug); - - disruptorEventQueue.publish(fail); - } catch (Exception e) { - LOG.error("expire error", e); - } - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/TimerRatio.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/TimerRatio.java deleted file mode 100644 index a14451151..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/execute/spout/TimerRatio.java +++ /dev/null @@ -1,58 +0,0 @@ -package com.alibaba.jstorm.task.execute.spout; - -import java.util.concurrent.atomic.AtomicLong; - -import com.codahale.metrics.Gauge; - -/** - * - * @author zhongyan.feng - * @version $Id: - */ -public class TimerRatio implements Gauge { - - private long lastUpdateTime = 0; - private long sum = 0; - private long lastGaugeTime; - - public void init() { - lastGaugeTime = System.currentTimeMillis(); - } - - public synchronized void start() { - if (lastUpdateTime == 0) { - lastUpdateTime = System.currentTimeMillis(); - } - } - - public synchronized void stop() { - if (lastUpdateTime != 0) { - long now = System.currentTimeMillis(); - long cost = now - lastUpdateTime; - lastUpdateTime = 0; - sum += cost; - } - - } - - @Override - public Double getValue() { - synchronized (this) { - stop(); - - long now = System.currentTimeMillis(); - long cost = now - lastGaugeTime; - if (cost == 0) { - return 1.0; - } - - lastGaugeTime = now; - double ratio = ((double) sum) / cost; - sum = 0; - return ratio; - - } - - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/GrouperType.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/GrouperType.java deleted file mode 100644 index c10abbfbb..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/GrouperType.java +++ /dev/null @@ -1,11 +0,0 @@ -package com.alibaba.jstorm.task.group; - -/** - * Grouping type - * - * @author yannian - * - */ -public enum GrouperType { - global, fields, all, shuffle, none, custom_obj, custom_serialized, direct, local_or_shuffle, localFirst -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkCustomGrouper.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkCustomGrouper.java deleted file mode 100644 index eaee79dc3..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkCustomGrouper.java +++ /dev/null @@ -1,34 +0,0 @@ -package com.alibaba.jstorm.task.group; - -import java.util.List; - -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.grouping.CustomStreamGrouping; -import backtype.storm.task.TopologyContext; -import backtype.storm.task.WorkerTopologyContext; -import backtype.storm.tuple.Fields; - -/** - * user defined grouping method - * - * @author Longda/yannian - * - */ -public class MkCustomGrouper { - private CustomStreamGrouping grouping; - - private int myTaskId; - - public MkCustomGrouper(TopologyContext context, - CustomStreamGrouping _grouping, GlobalStreamId stream, - List targetTask, int myTaskId) { - this.myTaskId = myTaskId; - this.grouping = _grouping; - this.grouping.prepare(context, stream, targetTask); - - } - - public List grouper(List values) { - return this.grouping.chooseTasks(myTaskId, values); - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkFieldsGrouper.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkFieldsGrouper.java deleted file mode 100644 index 0f9e73386..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkFieldsGrouper.java +++ /dev/null @@ -1,43 +0,0 @@ -package com.alibaba.jstorm.task.group; - -import java.util.Iterator; -import java.util.List; - -import backtype.storm.tuple.Fields; - -import com.alibaba.jstorm.utils.JStormUtils; - -/** - * field grouping - * - * @author yannian - * - */ -public class MkFieldsGrouper { - private Fields out_fields; - private Fields group_fields; - private List out_tasks; - - public MkFieldsGrouper(Fields _out_fields, Fields _group_fields, - List _out_tasks) { - - for (Iterator it = _group_fields.iterator(); it.hasNext();) { - String groupField = it.next(); - - // if groupField isn't in _out_fields, it would throw Exception - _out_fields.fieldIndex(groupField); - } - - this.out_fields = _out_fields; - this.group_fields = _group_fields; - this.out_tasks = _out_tasks; - - } - - public List grouper(List values) { - int hashcode = this.out_fields.select(this.group_fields, values) - .hashCode(); - int group = Math.abs(hashcode % this.out_tasks.size()); - return JStormUtils.mk_list(out_tasks.get(group)); - } -} 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 deleted file mode 100644 index 15905c4a3..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkGrouper.java +++ /dev/null @@ -1,183 +0,0 @@ -package com.alibaba.jstorm.task.group; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Random; - -import org.apache.log4j.Logger; - -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.generated.Grouping; -import backtype.storm.generated.JavaObject; -import backtype.storm.grouping.CustomStreamGrouping; -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Fields; -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.daemon.worker.WorkerData; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.RandomRange; -import com.alibaba.jstorm.utils.Thrift; - -/** - * Grouper, get which task should be send to for one tuple - * - * @author yannian - * - */ -public class MkGrouper { - private static final Logger LOG = Logger.getLogger(MkGrouper.class); - - private TopologyContext topology_context; - // this component output fields - private Fields out_fields; - private Grouping thrift_grouping; - private Grouping._Fields fields; - private GrouperType grouptype; - private List out_tasks; - private List local_tasks; - private String streamId; - - // grouping method - private RandomRange randomrange; - private Random random; - private MkShuffer shuffer; - private MkCustomGrouper custom_grouper; - private MkFieldsGrouper fields_grouper; - private MkLocalShuffer local_shuffer_grouper; - private MkLocalFirst localFirst; - - - public MkGrouper(TopologyContext _topology_context, Fields _out_fields, - Grouping _thrift_grouping, List _outTasks, - String streamId, WorkerData workerData) { - this.topology_context = _topology_context; - this.out_fields = _out_fields; - this.thrift_grouping = _thrift_grouping; - this.streamId = streamId; - - this.out_tasks = new ArrayList(); - this.out_tasks.addAll(_outTasks); - Collections.sort(this.out_tasks); - - this.local_tasks = _topology_context.getThisWorkerTasks(); - this.fields = Thrift.groupingType(thrift_grouping); - this.grouptype = this.parseGroupType(workerData); - - String id = _topology_context.getThisTaskId() + ":" + streamId; - LOG.info(id + " grouptype is " + grouptype); - - } - - public GrouperType gettype() { - return grouptype; - } - - private GrouperType parseGroupType(WorkerData workerData) { - - GrouperType grouperType = null; - - if (Grouping._Fields.FIELDS.equals(fields)) { - - if (Thrift.isGlobalGrouping(thrift_grouping)) { - - // global grouping, just send tuple to first task - grouperType = GrouperType.global; - } else { - - List fields_group = Thrift - .fieldGrouping(thrift_grouping); - Fields fields = new Fields(fields_group); - - fields_grouper = new MkFieldsGrouper(out_fields, fields, - out_tasks); - - // hashcode by fields - grouperType = GrouperType.fields; - } - - } else if (Grouping._Fields.ALL.equals(fields)) { - // send to every task - grouperType = GrouperType.all; - } else if (Grouping._Fields.SHUFFLE.equals(fields)) { - grouperType = GrouperType.shuffle; - shuffer = new MkShuffer(out_tasks, workerData); - } else if (Grouping._Fields.NONE.equals(fields)) { - // random send one task - this.random = new Random(); - grouperType = GrouperType.none; - } else if (Grouping._Fields.CUSTOM_OBJECT.equals(fields)) { - // user custom grouping by JavaObject - JavaObject jobj = thrift_grouping.get_custom_object(); - CustomStreamGrouping g = Thrift.instantiateJavaObject(jobj); - int myTaskId = topology_context.getThisTaskId(); - String componentId = topology_context.getComponentId(myTaskId); - GlobalStreamId stream = new GlobalStreamId(componentId, streamId); - custom_grouper = new MkCustomGrouper(topology_context, g, stream, - out_tasks, myTaskId); - grouperType = GrouperType.custom_obj; - } else if (Grouping._Fields.CUSTOM_SERIALIZED.equals(fields)) { - // user custom group by serialized Object - byte[] obj = thrift_grouping.get_custom_serialized(); - CustomStreamGrouping g = (CustomStreamGrouping) Utils - .deserialize(obj); - int myTaskId = topology_context.getThisTaskId(); - String componentId = topology_context.getComponentId(myTaskId); - GlobalStreamId stream = new GlobalStreamId(componentId, streamId); - custom_grouper = new MkCustomGrouper(topology_context, g, stream, - out_tasks, myTaskId); - grouperType = GrouperType.custom_serialized; - } else if (Grouping._Fields.DIRECT.equals(fields)) { - // directly send to a special task - 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, workerData); - }else if (Grouping._Fields.LOCAL_FIRST.equals(fields)) { - grouperType = GrouperType.localFirst; - localFirst = new MkLocalFirst(local_tasks, out_tasks, workerData); - } - - - return grouperType; - } - - /** - * get which task should tuple be sent to - * - * @param values - * @return - */ - public List grouper(List values) { - if (GrouperType.global.equals(grouptype)) { - // send to task which taskId is 0 - return JStormUtils.mk_list(out_tasks.get(0)); - } else if (GrouperType.fields.equals(grouptype)) { - // field grouping - return fields_grouper.grouper(values); - } else if (GrouperType.all.equals(grouptype)) { - // send to every task - return out_tasks; - } else if (GrouperType.shuffle.equals(grouptype)) { - // random, but the random is different from none - 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)); - } else if (GrouperType.custom_obj.equals(grouptype)) { - return custom_grouper.grouper(values); - } else if (GrouperType.custom_serialized.equals(grouptype)) { - return custom_grouper.grouper(values); - } else if (GrouperType.local_or_shuffle.equals(grouptype)) { - return local_shuffer_grouper.grouper(values); - } else if (GrouperType.localFirst.equals(grouptype)) { - return localFirst.grouper(values); - }else { - LOG.warn("Unsupportted group type"); - } - - return new ArrayList(); - } - -} 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 deleted file mode 100644 index b0a9d11b0..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkLocalFirst.java +++ /dev/null @@ -1,132 +0,0 @@ -package com.alibaba.jstorm.task.group; - -import java.util.ArrayList; -import java.util.List; -import java.util.Set; - -import org.apache.log4j.Logger; - -import com.alibaba.jstorm.daemon.worker.WorkerData; -import com.alibaba.jstorm.utils.IntervalCheck; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.RandomRange; - -/** - * - * - * @author zhongyan.feng - * @version - */ -public class MkLocalFirst extends Shuffer { - private static final Logger LOG = Logger.getLogger(MkLocalFirst.class); - - private List allOutTasks; - private List outTasks; - private RandomRange randomrange; - private boolean isLocalWorker; - private WorkerData workerData; - private IntervalCheck intervalCheck; - private Set lastLocalNodeTasks; - - public MkLocalFirst(List workerTasks, List allOutTasks, - WorkerData workerData) { - super(workerData); - - intervalCheck = new IntervalCheck(); - intervalCheck.setInterval(60); - - this.allOutTasks = allOutTasks; - this.workerData = workerData; - - List localWorkerOutTasks = new ArrayList(); - - for (Integer outTask : allOutTasks) { - if (workerTasks.contains(outTask)) { - localWorkerOutTasks.add(outTask); - } - } - - if (localWorkerOutTasks.size() != 0) { - isLocalWorker = true; - this.outTasks = localWorkerOutTasks; - randomrange = new RandomRange(outTasks.size()); - - return; - } else { - this.isLocalWorker = false; - this.outTasks = allOutTasks; - randomrange = new RandomRange(outTasks.size()); - refreshLocalNodeTasks(); - return; - } - - } - - /** - * Don't need to take care of multiple thread, - * One task one thread - */ - private void refreshLocalNodeTasks() { - Set localNodeTasks = workerData.getLocalNodeTasks(); - - if (localNodeTasks == null) { - this.outTasks = allOutTasks; - randomrange = new RandomRange(outTasks.size()); - return; - } - - if (localNodeTasks.equals(lastLocalNodeTasks)) { - // no local task changed - return; - } - LOG.info("Old localNodeTasks:" + lastLocalNodeTasks + ", new:" + localNodeTasks); - lastLocalNodeTasks = localNodeTasks; - - List localNodeOutTasks = new ArrayList(); - - for (Integer outTask : allOutTasks) { - if (localNodeTasks.contains(outTask)) { - localNodeOutTasks.add(outTask); - } - } - - if (localNodeOutTasks.isEmpty() == false) { - this.outTasks = localNodeOutTasks; - } else { - this.outTasks = allOutTasks; - } - - randomrange = new RandomRange(outTasks.size()); - } - - 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)); - - } - - public List grouper(List values) { - if (isLocalWorker == true) { - return innerGroup(values); - } - - if (intervalCheck.check()) { - refreshLocalNodeTasks(); - } - - return innerGroup(values); - } - - /** - * @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/MkLocalShuffer.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkLocalShuffer.java deleted file mode 100644 index f8d36ad35..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkLocalShuffer.java +++ /dev/null @@ -1,55 +0,0 @@ -package com.alibaba.jstorm.task.group; - -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 extends Shuffer{ - - private List outTasks; - private RandomRange randomrange; - private boolean isLocal; - - public MkLocalShuffer(List workerTasks, List allOutTasks, WorkerData workerData) { - super(workerData); - List localOutTasks = new ArrayList(); - - for (Integer outTask : allOutTasks) { - if (workerTasks.contains(outTask)) { - localOutTasks.add(outTask); - } - } - - if (localOutTasks.size() != 0) { - this.outTasks = localOutTasks; - isLocal = true; - } else { - this.outTasks = new ArrayList() ; - this.outTasks.addAll(allOutTasks); - isLocal = false; - } - - 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/MkShuffer.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkShuffer.java deleted file mode 100644 index f1806fc1f..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/MkShuffer.java +++ /dev/null @@ -1,38 +0,0 @@ -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 deleted file mode 100644 index 1b22d1ab0..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/group/Shuffer.java +++ /dev/null @@ -1,31 +0,0 @@ -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/TaskHeartbeat.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/heartbeat/TaskHeartbeat.java deleted file mode 100644 index e9189e110..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/heartbeat/TaskHeartbeat.java +++ /dev/null @@ -1,85 +0,0 @@ -package com.alibaba.jstorm.task.heartbeat; - -import java.io.Serializable; - -import org.apache.commons.lang.builder.ToStringBuilder; -import org.apache.commons.lang.builder.ToStringStyle; - -import com.alibaba.jstorm.stats.CommonStatsData; - -/** - * Task heartbeat, this Object will be updated to ZK timely - * - * @author yannian - * - */ -public class TaskHeartbeat implements Serializable { - - private static final long serialVersionUID = -6369195955255963810L; - private Integer timeSecs; - private Integer uptimeSecs; - private CommonStatsData stats; // BoltTaskStats or - // SpoutTaskStats - private String componentType; - - public TaskHeartbeat(int timeSecs, int uptimeSecs, CommonStatsData stats, String componentType) { - this.timeSecs = timeSecs; - this.uptimeSecs = uptimeSecs; - this.stats = stats; - this.componentType = componentType; - } - - public int getTimeSecs() { - return timeSecs; - } - - @Override - public String toString() { - return ToStringBuilder.reflectionToString(this, - ToStringStyle.SHORT_PREFIX_STYLE); - } - - public void setTimeSecs(int timeSecs) { - this.timeSecs = timeSecs; - } - - public int getUptimeSecs() { - return uptimeSecs; - } - - public void setUptimeSecs(int uptimeSecs) { - this.uptimeSecs = uptimeSecs; - } - - public CommonStatsData getStats() { - return stats; - } - - public void setStats(CommonStatsData stats) { - this.stats = stats; - } - - public void setComponentType(String componentType) { - this.componentType = componentType; - } - - public String getComponentType() { - return componentType; - } - - @Override - public boolean equals(Object hb) { - if (hb instanceof TaskHeartbeat - && ((TaskHeartbeat) hb).timeSecs.equals(timeSecs) - && ((TaskHeartbeat) hb).uptimeSecs.equals(uptimeSecs) - && ((TaskHeartbeat) hb).stats.equals(stats)) { - return true; - } - return false; - } - - @Override - public int hashCode() { - return timeSecs.hashCode() + uptimeSecs.hashCode() + stats.hashCode(); - } -} 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 deleted file mode 100644 index f035c3e92..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/heartbeat/TaskHeartbeatRunable.java +++ /dev/null @@ -1,147 +0,0 @@ -package com.alibaba.jstorm.task.heartbeat; - -import java.util.HashMap; -import java.util.Map; -import java.util.Map.Entry; -import java.util.concurrent.LinkedBlockingDeque; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.cluster.StormClusterState; -import com.alibaba.jstorm.daemon.worker.WorkerData; -import com.alibaba.jstorm.stats.CommonStatsRolling; -import com.alibaba.jstorm.task.TaskStatus; -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 - * - * @author yannian - * - */ -public class TaskHeartbeatRunable extends RunnableCallback { - private static final Logger LOG = Logger - .getLogger(TaskHeartbeatRunable.class); - - private StormClusterState zkCluster; - private String topology_id; - private UptimeComputer uptime; - private Map storm_conf; - private Integer frequence; - - private AtomicBoolean active; - - private static Map taskStatsMap = - new HashMap(); - private static LinkedBlockingDeque eventQueue = new - LinkedBlockingDeque(); - - public static void registerTaskStats(int taskId, TaskStats taskStats) { - Event event = new Event(Event.REGISTER_TYPE, taskId, taskStats); - eventQueue.offer(event); - } - - public static void unregisterTaskStats(int taskId) { - Event event = new Event(Event.UNREGISTER_TYPE, taskId, null); - eventQueue.offer(event); - } - - public TaskHeartbeatRunable(WorkerData workerData) { - - - this.zkCluster = workerData.getZkCluster(); - this.topology_id = workerData.getTopologyId(); - this.uptime = new UptimeComputer();; - this.storm_conf = workerData.getStormConf(); - this.active = workerData.getActive(); - - String key = Config.TASK_HEARTBEAT_FREQUENCY_SECS; - Object time = storm_conf.get(key); - frequence = JStormUtils.parseInt(time, 10); - - } - - @Override - public void run() { - Event event = eventQueue.poll(); - while(event != null) { - if (event.getType() == Event.REGISTER_TYPE) { - taskStatsMap.put(event.getTaskId(), event.getTaskStats()); - }else { - taskStatsMap.remove(event.getTaskId()); - } - - event = eventQueue.poll(); - } - - Integer currtime = TimeUtils.current_time_secs(); - - for (Entry entry : taskStatsMap.entrySet()) { - Integer taskId = entry.getKey(); - CommonStatsRolling taskStats = entry.getValue().getTaskStat(); - - String idStr = " " + topology_id + ":" + taskId + " "; - - try { - TaskHeartbeat hb = new TaskHeartbeat(currtime, uptime.uptime(), - taskStats.render_stats(), entry.getValue().getComponentType()); - zkCluster.task_heartbeat(topology_id, taskId, hb); - } catch (Exception e) { - // TODO Auto-generated catch block - String errMsg = "Failed to update heartbeat to ZK " + idStr - + "\n"; - LOG.error(errMsg, e); - continue; - } - } - - LOG.info("update all task hearbeat ts " + currtime + "," + taskStatsMap.keySet()); - } - - @Override - public Object getResult() { - if (active.get() == true) { - return frequence; - - } else { - LOG.info("Successfully shutdown Task's headbeat thread"); - return -1; - } - } - - private static class Event { - public static final int REGISTER_TYPE = 0; - public static final int UNREGISTER_TYPE = 1; - private final int type; - private final int taskId; - private final TaskStats taskStats; - - public Event(int type, int taskId, TaskStats taskStats) { - this.type = type; - this.taskId = taskId; - this.taskStats = taskStats; - } - - public int getType() { - return type; - } - - public int getTaskId() { - return taskId; - } - - public TaskStats getTaskStats() { - return taskStats; - } - - - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/task/heartbeat/TaskStats.java b/jstorm-server/src/main/java/com/alibaba/jstorm/task/heartbeat/TaskStats.java deleted file mode 100644 index c8efb7462..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/task/heartbeat/TaskStats.java +++ /dev/null @@ -1,21 +0,0 @@ -package com.alibaba.jstorm.task.heartbeat; - -import com.alibaba.jstorm.stats.CommonStatsRolling; - -public class TaskStats { - private CommonStatsRolling taskStats; - private String componentType; - - public TaskStats(String componentType, CommonStatsRolling taskStats) { - this.componentType = componentType; - this.taskStats = taskStats; - } - - public CommonStatsRolling getTaskStat() { - return taskStats; - } - - public String getComponentType() { - return componentType; - } -} \ No newline at end of file diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/utils/DisruptorRunable.java b/jstorm-server/src/main/java/com/alibaba/jstorm/utils/DisruptorRunable.java deleted file mode 100644 index 185b1ff90..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/utils/DisruptorRunable.java +++ /dev/null @@ -1,89 +0,0 @@ -package com.alibaba.jstorm.utils; - -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.log4j.Logger; - -import backtype.storm.utils.DisruptorQueue; - -import com.alibaba.jstorm.callback.RunnableCallback; -import com.alibaba.jstorm.metric.JStormTimer; -import com.alibaba.jstorm.metric.Metrics; -import com.codahale.metrics.Timer; -import com.lmax.disruptor.EventHandler; - -//import com.alibaba.jstorm.message.zeroMq.ISendConnection; - -/** - * - * Disruptor Consumer thread - * - * @author yannian - * - */ -public abstract class DisruptorRunable extends RunnableCallback implements EventHandler { - private final static Logger LOG = Logger.getLogger(DisruptorRunable.class); - - protected DisruptorQueue queue; - protected String idStr; - protected AtomicBoolean active; - protected JStormTimer timer; - - - public DisruptorRunable(DisruptorQueue queue, JStormTimer timer, String idStr, - AtomicBoolean active) { - this.queue = queue; - this.timer = timer; - this.idStr = idStr; - this.active = active; - } - - public abstract void handleEvent(Object event, boolean endOfBatch) throws Exception; - /** - * This function need to be implements - * @see com.lmax.disruptor.EventHandler#onEvent(java.lang.Object, long, boolean) - */ - @Override - public void onEvent(Object event, long sequence, boolean endOfBatch) - throws Exception{ - if (event == null) { - return ; - } - timer.start(); - try { - handleEvent(event, endOfBatch); - }finally { - timer.stop(); - } - } - - @Override - public void run() { - LOG.info("Successfully start thread " + idStr); - queue.consumerStarted(); - - while (active.get()) { - try { - - queue.consumeBatchWhenAvailable(this); - - } catch (Exception e) { - if (active.get() == true) { - LOG.error("DrainerRunable send error", e); - throw new RuntimeException(e); - } - } - } - - LOG.info("Successfully exit thread " + idStr); - } - - @Override - public Object getResult() { - if (active.get()) - return 0; - else - return -1; - } - -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/utils/FailedAssignTopologyException.java b/jstorm-server/src/main/java/com/alibaba/jstorm/utils/FailedAssignTopologyException.java deleted file mode 100644 index b2d08996d..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/utils/FailedAssignTopologyException.java +++ /dev/null @@ -1,22 +0,0 @@ -package com.alibaba.jstorm.utils; - -public class FailedAssignTopologyException extends RuntimeException { - - private static final long serialVersionUID = 6921061096898899476L; - - public FailedAssignTopologyException() { - super(); - } - - public FailedAssignTopologyException(String msg) { - super(msg); - } - - public FailedAssignTopologyException(String msg, Throwable cause) { - super(msg, cause); - } - - public FailedAssignTopologyException(Throwable cause) { - super(cause); - } -} diff --git a/jstorm-server/src/main/java/com/alibaba/jstorm/utils/JStormServerUtils.java b/jstorm-server/src/main/java/com/alibaba/jstorm/utils/JStormServerUtils.java deleted file mode 100644 index 6e30ac8a3..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/utils/JStormServerUtils.java +++ /dev/null @@ -1,132 +0,0 @@ -package com.alibaba.jstorm.utils; - -import java.io.File; -import java.io.IOException; -import java.util.List; -import java.util.Map; - -import org.apache.commons.io.FileUtils; -import org.apache.log4j.Logger; -import org.apache.thrift7.TException; - -import backtype.storm.Config; -import backtype.storm.GenericOptionsParser; -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.StormConfig; - -/** - * storm utils - * - * - * @author yannian/Longda/Xin.Zhou/Xin.Li - * - */ -public class JStormServerUtils { - - private static final Logger LOG = Logger.getLogger(JStormServerUtils.class); - - public static void downloadCodeFromMaster(Map conf, String localRoot, - String masterCodeDir, String topologyId, boolean isSupervisor) - throws IOException, TException { - FileUtils.forceMkdir(new File(localRoot)); - FileUtils.forceMkdir(new File(StormConfig.stormlib_path(localRoot))); - - String localStormjarPath = StormConfig.stormjar_path(localRoot); - String masterStormjarPath = StormConfig.stormjar_path(masterCodeDir); - Utils.downloadFromMaster(conf, masterStormjarPath, localStormjarPath); - - String localStormcodePath = StormConfig.stormcode_path(localRoot); - String masterStormcodePath = StormConfig.stormcode_path(masterCodeDir); - Utils.downloadFromMaster(conf, masterStormcodePath, localStormcodePath); - - String localStormConfPath = StormConfig.stormconf_path(localRoot); - String masterStormConfPath = StormConfig.stormconf_path(masterCodeDir); - Utils.downloadFromMaster(conf, masterStormConfPath, localStormConfPath); - - Map stormConf = (Map) StormConfig.readLocalObject(topologyId, - localStormConfPath); - - if (stormConf == null) - throw new IOException("Get topology conf error: " + topologyId); - - List libs = (List) stormConf - .get(GenericOptionsParser.TOPOLOGY_LIB_NAME); - if (libs == null) - return; - for (String libName : libs) { - String localStromLibPath = StormConfig.stormlib_path(localRoot, - libName); - String masterStormLibPath = StormConfig.stormlib_path( - masterCodeDir, libName); - Utils.downloadFromMaster(conf, masterStormLibPath, - localStromLibPath); - } - } - - public static void createPid(String dir) throws Exception { - File file = new File(dir); - - if (file.exists() == false) { - file.mkdirs(); - } else if (file.isDirectory() == false) { - throw new RuntimeException("pid dir:" + dir + " isn't directory"); - } - - String[] existPids = file.list(); - - // touch pid before - String pid = JStormUtils.process_pid(); - String pidPath = dir + File.separator + pid; - PathUtils.touch(pidPath); - LOG.info("Successfully touch pid " + pidPath); - - for (String existPid : existPids) { - try { - JStormUtils.kill(Integer.valueOf(existPid)); - PathUtils.rmpath(dir + File.separator + existPid); - } catch (Exception e) { - LOG.warn(e.getMessage(), e); - } - } - - } - - public static void startTaobaoJvmMonitor() { - // JmonitorBootstrap bootstrap = JmonitorBootstrap.getInstance(); - // bootstrap.start(); - } - - public static boolean isOnePending(Map conf) { - Object pending = conf.get(Config.TOPOLOGY_MAX_SPOUT_PENDING); - if (pending == null) { - return false; - } - - int pendingNum = JStormUtils.parseInt(pending); - if (pendingNum == 1) { - return true; - }else { - return false; - } - } - - public static String getName(String componentId, int taskId) { - return componentId + ":" + taskId; - } - - public static String getHostName(Map conf) { - String hostName = ConfigExtension.getSupervisorHost(conf); - if (hostName == null) { - hostName = NetWorkUtils.hostname(); - } - - if (ConfigExtension.isSupervisorUseIp(conf)) { - hostName = NetWorkUtils.ip(); - } - - return hostName; - } - -}; 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 deleted file mode 100644 index 485b37ed0..000000000 --- a/jstorm-server/src/main/java/com/alibaba/jstorm/utils/Thrift.java +++ /dev/null @@ -1,484 +0,0 @@ -package com.alibaba.jstorm.utils; - -import java.lang.reflect.Constructor; -import java.nio.ByteBuffer; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.log4j.Logger; - -import backtype.storm.generated.Bolt; -import backtype.storm.generated.ComponentCommon; -import backtype.storm.generated.ComponentObject; -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.generated.Grouping; -import backtype.storm.generated.JavaObject; -import backtype.storm.generated.JavaObjectArg; -import backtype.storm.generated.NullStruct; -import backtype.storm.generated.StormTopology; -import backtype.storm.generated.StormTopology._Fields; -import backtype.storm.generated.StreamInfo; -import backtype.storm.generated.TopologyInitialStatus; -import backtype.storm.grouping.CustomStreamGrouping; -import backtype.storm.task.IBolt; -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.cluster.StormStatus; -import com.alibaba.jstorm.daemon.nimbus.StatusType; - -/** - * Thrift utils - * - * 2012-03-28 - * - * @author yannian - * - */ -public class Thrift { - private static Logger LOG = Logger.getLogger(Thrift.class); - - public static StormStatus topologyInitialStatusToStormStatus( - TopologyInitialStatus tStatus) { - if (tStatus.equals(TopologyInitialStatus.ACTIVE)) { - return new StormStatus(StatusType.active); - } else { - return new StormStatus(StatusType.inactive); - } - } - - public static CustomStreamGrouping instantiateJavaObject(JavaObject obj) { - - List args = obj.get_args_list(); - Class[] paraTypes = new Class[args.size()]; - Object[] paraValues = new Object[args.size()]; - for (int i = 0; i < args.size(); i++) { - JavaObjectArg arg = args.get(i); - paraValues[i] = arg.getFieldValue(); - - if (arg.getSetField().equals(JavaObjectArg._Fields.INT_ARG)) { - paraTypes[i] = Integer.class; - } else if (arg.getSetField().equals(JavaObjectArg._Fields.LONG_ARG)) { - paraTypes[i] = Long.class; - } else if (arg.getSetField().equals( - JavaObjectArg._Fields.STRING_ARG)) { - paraTypes[i] = String.class; - } else if (arg.getSetField().equals(JavaObjectArg._Fields.BOOL_ARG)) { - paraTypes[i] = Boolean.class; - } else if (arg.getSetField().equals( - JavaObjectArg._Fields.BINARY_ARG)) { - paraTypes[i] = ByteBuffer.class; - } else if (arg.getSetField().equals( - JavaObjectArg._Fields.DOUBLE_ARG)) { - paraTypes[i] = Double.class; - } else { - paraTypes[i] = Object.class; - } - } - - try { - Class clas = Class.forName(obj.get_full_class_name()); - Constructor cons = clas.getConstructor(paraTypes); - return (CustomStreamGrouping) cons.newInstance(paraValues); - } catch (Exception e) { - LOG.error("instantiate_java_object fail", e); - } - - return null; - - } - - public static Grouping._Fields groupingType(Grouping grouping) { - return grouping.getSetField(); - } - - public static List fieldGrouping(Grouping grouping) { - if (!Grouping._Fields.FIELDS.equals(groupingType(grouping))) { - throw new IllegalArgumentException( - "Tried to get grouping fields from non fields grouping"); - } - - return grouping.get_fields(); - } - - public static boolean isGlobalGrouping(Grouping grouping) { - if (Grouping._Fields.FIELDS.equals(groupingType(grouping))) { - return fieldGrouping(grouping).isEmpty(); - } - - return false; - } - - public static int parallelismHint(ComponentCommon component_common) { - int phint = component_common.get_parallelism_hint(); - if (!component_common.is_set_parallelism_hint()) { - phint = 1; - } - return phint; - } - - public static StreamInfo directOutputFields(List fields) { - return new StreamInfo(fields, true); - } - - public static StreamInfo outputFields(List fields) { - return new StreamInfo(fields, false); - } - - public static Grouping mkFieldsGrouping(List fields) { - return Grouping.fields(fields); - } - - public static Grouping mkDirectGrouping() { - return Grouping.direct(new NullStruct()); - } - - private static ComponentCommon mkComponentcommon( - Map inputs, - HashMap output_spec, Integer parallelism_hint) { - ComponentCommon ret = new ComponentCommon(inputs, output_spec); - if (parallelism_hint != null) { - ret.set_parallelism_hint(parallelism_hint); - } - return ret; - } - - public static Bolt mkBolt(Map inputs, - IBolt bolt, HashMap output, Integer p) { - ComponentCommon common = mkComponentcommon(inputs, output, p); - byte[] boltSer = Utils.serialize(bolt); - ComponentObject component = ComponentObject.serialized_java(boltSer); - return new Bolt(component, common); - } - - public static StormTopology._Fields[] STORM_TOPOLOGY_FIELDS = null; - public static StormTopology._Fields[] SPOUT_FIELDS = { - StormTopology._Fields.SPOUTS, StormTopology._Fields.STATE_SPOUTS }; - static { - Set<_Fields> keys = StormTopology.metaDataMap.keySet(); - STORM_TOPOLOGY_FIELDS = new StormTopology._Fields[keys.size()]; - keys.toArray(STORM_TOPOLOGY_FIELDS); - } - - // - // private static HashMap mk_acker_inputs2( - // Map> inputs) { - // HashMap rtn = new HashMap(); - // for (Entry> e : inputs.entrySet()) { - // Object stream_id = e.getKey(); - // GlobalStreamId SID = null; - // if (stream_id instanceof Object[]) { - // String[] l = (String[]) stream_id; - // SID = new GlobalStreamId(l[0], l[1]); - // } else if (stream_id instanceof List) { - // List l = (List) stream_id; - // SID = new GlobalStreamId(l.get(0), l.get(1)); - // } else { - // SID = new GlobalStreamId((String) stream_id, - // Utils.DEFAULT_STREAM_ID); - // } - // - // rtn.put(SID, mk_fields_grouping(e.getValue())); - // } - // - // return rtn; - // } - // - // public static Bolt mk_ackerBolt(Map> inputs, - // IBolt bolt, HashMap> output, Integer p) { - // HashMap commonInputs=mk_inputs(inputs); - // ComponentCommon common = mk_component_common(commonInputs,output, p); - // return new Bolt(ComponentObject.serialized_java(Utils.serialize(bolt)), - // common); - // } - - // public static Map - // mk_output_StreamInfo(HashMap> output_spec) { - // Map> o_spec = output_spec; - // - // Map rtn = new HashMap(); - // - // for (Entry> e : o_spec.entrySet()) { - // List val = e.getValue(); - // rtn.put(e.getKey(), new StreamInfo((List) val, false)); - // - // } - // - // return rtn; - // } - - // public static Map - // mk_output_StreamInfo2(HashMap output_spec) { - // return output_spec; - // } - - // private static ComponentCommon mk_component_common( - // Map inputs, HashMap> - // output_spec, - // Integer parallelism_hint) { - // ComponentCommon ret = new ComponentCommon(inputs, - // mk_output_StreamInfo(output_spec)); - // if (parallelism_hint != null) { - // ret.set_parallelism_hint(parallelism_hint); - // } - // - // return ret; - // } - - // public static Object[] nimbus_client_and_conn(String host, Integer port) - // throws TTransportException { - // TFramedTransport transport = new TFramedTransport(new TSocket(host, - // port)); - // TBinaryProtocol prot = new TBinaryProtocol(transport); - // Nimbus.Client client = new Nimbus.Client(prot); - // transport.open(); - // Object[] rtn = { client, transport }; - // return rtn; - // } - // public static SpoutSpec mk_spout_spec(Object spout, Object outputs, - // Integer p) { - // Map inputs = new HashMap(); - // return new SpoutSpec(ComponentObject.serialized_java(Utils - // .serialize(spout)), mk_plain_component_common(inputs, outputs, - // p)); - // } - - // public static SpoutSpec mk_spout_spec(Object spout, Object outputs) { - // return mk_spout_spec(spout, outputs, null); - // } - - // public static Grouping mk_shuffle_grouping() { - // return Grouping.shuffle(new NullStruct()); - // } - - // public static Grouping mk_global_grouping() { - // return mk_fields_grouping(new ArrayList()); - // } - - // public static Grouping mk_all_grouping() { - // return Grouping.all(new NullStruct()); - // } - - // public static Grouping mk_none_grouping() { - // return Grouping.none(new NullStruct()); - // } - - // public static Object deserialized_component_object(ComponentObject obj) { - // if (obj.getSetField().equals(ComponentObject._Fields.SERIALIZED_JAVA)) { - // throw new RuntimeException( - // "Cannot deserialize non-java-serialized object"); - // } - // - // return Utils.deserialize(obj.get_serialized_java()); - // } - - // public static ComponentObject serialize_component_object(Object obj) { - // return ComponentObject.serialized_java(Utils.serialize(obj)); - // } - - // public static Grouping mk_grouping(Object grouping_spec) { - // if (grouping_spec == null) { - // return mk_none_grouping(); - // } - // - // if (grouping_spec instanceof Grouping) { - // return (Grouping) grouping_spec; - // } - // - // if (grouping_spec instanceof CustomStreamGrouping) { - // return Grouping.custom_serialized(Utils.serialize(grouping_spec)); - // } - // - // if (grouping_spec instanceof JavaObject) { - // return Grouping.custom_object((JavaObject) grouping_spec); - // } - // - // if (grouping_spec instanceof List) { - // return mk_fields_grouping((List) grouping_spec); - // } - // - // // - // if (GroupingConstants.shuffle.equals(grouping_spec)) { - // return mk_shuffle_grouping(); - // } - // - // if (GroupingConstants.none.equals(grouping_spec)) { - // return mk_none_grouping(); - // } - // - // if (GroupingConstants.all.equals(grouping_spec)) { - // return mk_all_grouping(); - // } - // - // if ("global".equals(grouping_spec)) { - // return mk_global_grouping(); - // } - // - // if (GroupingConstants.direct.equals(grouping_spec)) { - // return mk_direct_grouping(); - // } - // - // throw new IllegalArgumentException(grouping_spec - // + " is not a valid grouping"); - // } - - // public static Bolt mk_bolt_spec(Map> inputs, - // IBolt bolt, Object output) { - // return mk_bolt_spec(inputs, bolt, output, null); - // } - - // - // public class BoltSpecObj { - // public IRichBolt obj; - // public Map conf; - // public Map> inputs; - // public Integer p; - // } - // public static BoltSpecObj mk_bolt_spec(Map> inputs, - // IRichBolt bolt, Object... args) { - // Integer parallelism_hint = null; - // Integer p = null; - // Map conf = null; - // if (args.length >= 1) { - // parallelism_hint = (Integer) args[0]; - // } - // if (args.length >= 2) { - // p = (Integer) args[1]; - // } - // if (args.length >= 3) { - // conf = (Map) args[2]; - // } - // - // return mk_bolt_spec(inputs, bolt, parallelism_hint, p, conf); - // } - // public class SpoutSpecObj { - // public IRichSpout obj; - // public Map conf; - // public Integer p; - // } - // public static SpoutSpecObj mk_spout_spec(Map> - // inputs, - // IRichBolt bolt, Object... args) { - // Integer parallelism_hint = null; - // Integer p = null; - // Map conf = null; - // if (args.length >= 1) { - // parallelism_hint = (Integer) args[0]; - // } - // if (args.length >= 2) { - // p = (Integer) args[1]; - // } - // if (args.length >= 3) { - // conf = (Map) args[2]; - // } - // - // return mk_spout_spec(inputs, bolt, parallelism_hint, p, conf); - // } - // - // public static SpoutSpecObj mk_spout_spec(IRichSpout bolt, - // Integer parallelism_hint, Integer p, Map conf) { - // if (p != null) { - // parallelism_hint = p; - // } - // SpoutSpecObj rtn = new SpoutSpecObj(); - // rtn.p = parallelism_hint; - // rtn.obj = bolt; - // rtn.conf = conf; - // return rtn; - // } - // - // public static void add_inputs(BoltDeclarer declarer, - // Map> inputs) { - // HashMap getinputs = mk_inputs(inputs); - // for (Entry e : getinputs.entrySet()) { - // declarer.grouping(e.getKey(), e.getValue()); - // } - // } - - // public static StormTopology mk_topology( - // HashMap spout_map, - // HashMap bolt_map) { - // TopologyBuilder builder = new TopologyBuilder(); - // for (Entry e : spout_map.entrySet()) { - // String id = e.getKey(); - // SpoutSpecObj spout_obj = e.getValue(); - // SpoutDeclarer dec = builder - // .setSpout(id, spout_obj.obj, spout_obj.p); - // if (spout_obj.conf != null) { - // dec.addConfigurations(spout_obj.conf); - // } - // } - // - // for (Entry e : bolt_map.entrySet()) { - // String id = e.getKey(); - // BoltSpecObj bolt = e.getValue(); - // BoltDeclarer dec = builder.setBolt(id, bolt.obj, bolt.p); - // if (bolt.conf != null) { - // dec.addConfigurations(bolt.conf); - // } - // add_inputs(dec, bolt.inputs); - // } - // - // return builder.createTopology(); - // } - - // public static BoltSpecObj mk_bolt_spec(Map> inputs, - // IRichBolt bolt, Integer parallelism_hint, Integer p, Map conf) { - // if (p != null) { - // parallelism_hint = p; - // } - // BoltSpecObj rtn = new BoltSpecObj(); - // rtn.inputs = inputs; - // rtn.p = parallelism_hint; - // rtn.obj = bolt; - // rtn.conf = conf; - // return rtn; - // } - - // public static BoltSpecObj mk_shellbolt_spec( - // Map> inputs, String command, String script, - // Object output_spec, Object... args) { - // Integer parallelism_hint = null; - // Integer p = null; - // Map conf = null; - // if (args.length >= 1) { - // parallelism_hint = (Integer) args[0]; - // } - // if (args.length >= 2) { - // p = (Integer) args[1]; - // } - // if (args.length >= 3) { - // conf = (Map) args[2]; - // } - // - // return mk_shellbolt_spec(inputs, command, script, output_spec, - // parallelism_hint, p, conf); - // } - - // public static BoltSpecObj mk_shellbolt_spec( - // Map> inputs, String command, String script, - // Object output_spec, Integer parallelism_hint, Integer p, Map conf) { - // return mk_bolt_spec(inputs, new RichShellBolt(script, command, - // mk_output_spec(output_spec)), parallelism_hint, p, conf); - // } - - // public static Bolt mk_spout_spec(Map inputs, - // IComponent bolt, Object output) { - // return mk_spout_spec(inputs, bolt, output, null); - // } - - // public static Bolt mk_spout_spec(Map inputs, - // IComponent bolt, Object output, Integer p) { - // ComponentCommon common = mk_plain_component_common(inputs, output, p); - // - // return new Bolt(ComponentObject.serialized_java(Utils.serialize(bolt)), - // common); - // } - - // public static String COORD_STREAM = Constants.COORDINATED_STREAM_ID; - -} diff --git a/jstorm-server/src/main/resources/defaults.yaml b/jstorm-server/src/main/resources/defaults.yaml deleted file mode 100644 index 29b653a05..000000000 --- a/jstorm-server/src/main/resources/defaults.yaml +++ /dev/null @@ -1,281 +0,0 @@ -########### These all have default values as shown -########### Additional configuration goes into storm.yaml - -java.library.path: "/usr/local/lib:/opt/local/lib:/usr/lib" - -### storm.* configs are general configurations -# the local dir is where jars are kept -storm.local.dir: "jstorm-local" -storm.zookeeper.servers: - - "localhost" -storm.zookeeper.port: 2181 -storm.zookeeper.root: "/jstorm" -storm.zookeeper.session.timeout: 20000 -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: 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 -nimbus.monitor.freq.secs: 10 -nimbus.cleanup.inbox.freq.secs: 600 -nimbus.inbox.jar.expiration.secs: 3600 -nimbus.task.launch.secs: 240 -nimbus.reassign: true -nimbus.file.copy.expiration.secs: 600 -nimbus.topology.validator: "backtype.storm.nimbus.DefaultTopologyValidator" -nimbus.classpath: "" -nimbus.use.ip: true -nimbus.credential.renewers.freq.secs: 600 - -### ui.* configs are for the master -ui.port: 8080 -ui.childopts: " -Xms1g -Xmx1g -Xmn256m -XX:PermSize=96m -XX:+UseConcMarkSweepGC -XX:+UseCMSInitiatingOccupancyOnly -XX:CMSInitiatingOccupancyFraction=70 -XX:CMSFullGCsBeforeCompaction=5 -XX:+HeapDumpOnOutOfMemoryError -XX:+UseCMSCompactAtFullCollection -XX:CMSMaxAbortablePrecleanTime=5000 " - -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 -transactional.zookeeper.port: null -# -# -# ##### These may optionally be filled in: -# -## Map of tokens to a serialization class. tokens less than 32 are reserved by storm. -## Tokens are written on the wire to identify the field. -# topology.serializations: -# - "org.mycompany.MyObjectSerialization" -# - "org.mycompany.MyOtherObjectSerialization" -## Locations of the drpc servers -drpc.servers: - - "localhost" - -### 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 - -# 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 -supervisor.worker.timeout.secs: 120 -#how frequently the supervisor checks on the status of the processes it's monitoring and restarts if necessary -supervisor.monitor.frequency.secs: 10 -#how frequently the supervisor heartbeats to the cluster state (for nimbus) -supervisor.heartbeat.frequency.secs: 60 -supervisor.enable: true -#if set null, it will be get by system -supervisor.hostname: null -# use ip -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=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 -# if worker.redirect.output.file is null, then it will be $LOG.out -# please use absolute path -worker.redirect.output.file: null -# when supervisor is shutdown, automatically shutdown worker -worker.stop.without.supervisor: false -worker.memory.size: 2147483648 - -task.heartbeat.frequency.secs: 10 -task.refresh.poll.secs: 10 -# how long task do cleanup -task.cleanup.timeout.sec: 10 - -zmq.threads: 1 -zmq.linger.millis: 5000 -zmq.hwm: 0 - -# Netty thread num: 0 means no limit -storm.messaging.netty.server_worker_threads: 1 -storm.messaging.netty.client_worker_threads: 1 -storm.messaging.netty.buffer_size: 5242880 #5MB buffer -storm.messaging.netty.max_retries: 30 -storm.messaging.netty.max_wait_ms: 1000 -storm.messaging.netty.min_wait_ms: 100 -storm.messaging.netty.disruptor: true -# If async and batch is used in netty transfer, netty will batch message -storm.messaging.netty.transfer.async.batch: true -# If the Netty messaging layer is busy(netty internal buffer not writable), the Netty client will try to batch message as more as possible up to the size of storm.messaging.netty.transfer.batch.size bytes, otherwise it will try to flush message as soon as possible to reduce latency. -storm.messaging.netty.transfer.batch.size: 262144 -# We check with this interval that whether the Netty channel is writable and try to write pending messages if it is. -storm.messaging.netty.flush.check.interval.ms: 10 -# when netty connection is broken, -# when buffer size is more than storm.messaging.netty.buffer.threshold -# it will slow down the netty sending speed -storm.messaging.netty.buffer.threshold: 8388608 -storm.messaging.netty.max.pending: 16 -## send message with sync or async mode -storm.messaging.netty.sync.mode: false -## when netty is in sync mode and client channel is unavailable, -## it will block sending until channel is ready -storm.messaging.netty.async.block: true - -### topology.* configs are for specific executing storms -topology.enable.message.timeouts: true -topology.debug: false -topology.optimize: true -topology.workers: 1 -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 -# to change the component's parallelism when executing restart command -#topology.spout.parallelism: -# { -# "spoutName" : Num -# } -#topology.bolt.parallelism: null -# { -# "BoltName_1" : Num, -# "BoltName_2" : Num -# } -topology.max.spout.pending: null -topology.state.synchronization.timeout.secs: 60 -topology.stats.sample.rate: 0.05 -topology.builtin.metrics.bucket.size.secs: 60 -topology.fall.back.on.java.serialization: true -topology.worker.childopts: null -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: 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" -topology.spout.wait.strategy: "backtype.storm.spout.SleepSpoutWaitStrategy" -topology.sleep.spout.wait.strategy.time.ms: 1 -topology.error.throttle.interval.secs: 10 -topology.max.error.report.per.interval: 5 -topology.kryo.factory: "backtype.storm.serialization.DefaultKryoFactory" -topology.tuple.serializer: "backtype.storm.serialization.types.ListDelegateSerializer" -topology.trident.batch.emit.interval.millis: 500 - -# jstorm metrics monitor configuration -topology.performance.metrics: true -topology.alimonitor.metrics.post: false -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 - -# enable supervisor use cgroup to make resource isolation -# Before enable it, you should make sure: -# 1. Linux version (>= 2.6.18) -# 2. Have installed cgroup (check the file's existence:/proc/cgroups) -# 3. You should start your supervisor on root -# You can get more about cgroup: -# http://t.cn/8s7nexU -# -# For cgroup root dir, the full path is "/cgroup/cpu + root_dir". -# The /cgroup/cpu part is fixed. The root dir is configurable, -# which should be consistent with the part configured in /etc/cgconfig.conf -supervisor.enable.cgroup: false -supervisor.cgroup.rootdir: "jstorm" - -dev.zookeeper.path: "/tmp/dev-storm-zookeeper" - -#if this configuration has been set, -# the spout or bolt will log all received tuples -# topology.debug just for logging all sent tuples -topology.debug.recv.tuple: false - -#Usually, spout finish preparation before bolt, -#so spout need wait several seconds so that bolt finish preparation -# the default setting is 30 seconds -spout.delay.run: 30 - -#Force spout use single thread -spout.single.thread: false - -#When spout pending number is full, spout nextTuple will do sleep operation -spout.pending.full.sleep: true - -# container setting means jstorm is running under other system, such as hadoop-yarn/Ali-Apsara -# For example, in Ali-Apsara, Fuxi start c++ container process, -# the C++ container fork process to start nimbus or supervisor -container.heartbeat.timeout.seconds: 240 -container.heartbeat.frequence: 10 - - -# enable java sandbox or not -java.sandbox.enable: false. - -# logview port -nimbus.deamon.logview.port: 7621 -supervisor.deamon.logview.port: 7622 - -# logview encoding -supervisor.deamon.logview.encoding: "utf-8" - -### when disruptor queue is full, sleep until queue isn't full -### the default disruptor will query disruptor until queue isn't full -### this will cost much cpu -disruptor.use.sleep: true diff --git a/jstorm-server/src/test/java/com/alibaba/jstorm/message/context/ContextTest.java b/jstorm-server/src/test/java/com/alibaba/jstorm/message/context/ContextTest.java deleted file mode 100644 index 021ea7303..000000000 --- a/jstorm-server/src/test/java/com/alibaba/jstorm/message/context/ContextTest.java +++ /dev/null @@ -1,43 +0,0 @@ -package com.alibaba.jstorm.message.context; - -import java.lang.reflect.Constructor; -import java.util.HashMap; -import java.util.Map; - -import junit.framework.Assert; - -import org.junit.Test; - -import com.google.common.collect.Maps; - -import backtype.storm.Config; -import backtype.storm.messaging.IContext; -import backtype.storm.messaging.TransportFactory; - -public class ContextTest { - - @Test - public void test_zmq() throws Exception { - String klassName = "com.alibaba.jstorm.message.zeroMq.MQContext"; - Class klass = Class.forName(klassName); - Constructor constructor = klass.getDeclaredConstructor(); - constructor.setAccessible(true); - IContext context = (IContext) constructor.newInstance(); - Assert.assertNotNull(context); - } - - @Test - public void test_netty() { - Map storm_conf = Maps.newHashMap(); - storm_conf.put(Config.STORM_MESSAGING_TRANSPORT, - "com.alibaba.jstorm.message.netty.NettyContext"); - storm_conf.put(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE, 1024); - storm_conf.put(Config.STORM_MESSAGING_NETTY_MAX_RETRIES, 10); - storm_conf.put(Config.STORM_MESSAGING_NETTY_MIN_SLEEP_MS, 1000); - storm_conf.put(Config.STORM_MESSAGING_NETTY_MAX_SLEEP_MS, 5000); - storm_conf.put(Config.STORM_MESSAGING_NETTY_SERVER_WORKER_THREADS, 1); - storm_conf.put(Config.STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS, 1); - IContext context = TransportFactory.makeContext(storm_conf); - Assert.assertNotNull(context); - } -} diff --git a/jstorm-server/src/test/java/com/alibaba/jstorm/message/netty/NettyUnitTest.java b/jstorm-server/src/test/java/com/alibaba/jstorm/message/netty/NettyUnitTest.java deleted file mode 100644 index 4d8429756..000000000 --- a/jstorm-server/src/test/java/com/alibaba/jstorm/message/netty/NettyUnitTest.java +++ /dev/null @@ -1,781 +0,0 @@ -package com.alibaba.jstorm.message.netty; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; - -import junit.framework.Assert; - -import org.apache.log4j.Logger; -import org.junit.Test; - -import backtype.storm.Config; -import backtype.storm.messaging.IConnection; -import backtype.storm.messaging.IContext; -import backtype.storm.messaging.TaskMessage; -import backtype.storm.messaging.TransportFactory; -import backtype.storm.utils.DisruptorQueue; -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.utils.JStormUtils; -import com.lmax.disruptor.WaitStrategy; -import com.lmax.disruptor.dsl.ProducerType; - -public class NettyUnitTest { - - private static final Logger LOG = Logger.getLogger(NettyUnitTest.class); - - private static int port = 6700; - private static int task = 1; - private static Lock lock = new ReentrantLock(); - private static Condition clientClose = lock.newCondition(); - private static Condition contextClose = lock.newCondition(); - - - private static Map storm_conf = new HashMap(); - private static IContext context = null; - static { - storm_conf = Utils.readDefaultConfig(); - boolean syncMode = false; - if (syncMode) { - DisruptorQueue.setLimited(true); - ConfigExtension.setNettyMaxSendPending(storm_conf, 1); - ConfigExtension.setNettySyncMode(storm_conf, true); - } else { - ConfigExtension.setNettySyncMode(storm_conf, false); - ConfigExtension.setNettyASyncBlock(storm_conf, false); - } - - //Check whether context can be reused or not - context = TransportFactory.makeContext(storm_conf); - } - - @Test - public void test_small_message() { - System.out.println("!!!!!!!!Start test_small_message !!!!!!!!!!!"); - String req_msg = "Aloha is the most Hawaiian word."; - - IConnection server = null; - IConnection client = null; - - server = context.bind(null, port); - - WaitStrategy waitStrategy = (WaitStrategy) Utils - .newInstance((String) storm_conf - .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); - DisruptorQueue recvQueue = DisruptorQueue.mkInstance( - "NettyUnitTest", ProducerType.SINGLE, 1024, waitStrategy); - server.registerQueue(recvQueue); - - client = context.connect(null, "localhost", port); - - List list = new ArrayList(); - TaskMessage message = new TaskMessage(task, req_msg.getBytes()); - list.add(message); - - client.send(message); - - TaskMessage recv = server.recv(0); - Assert.assertEquals(req_msg, new String(recv.message())); - - System.out.println("!!!!!!!!!!!!!!!!!!Test one time!!!!!!!!!!!!!!!!!"); - - server.close(); - client.close(); - - - System.out.println("!!!!!!!!!!!!End test_small_message!!!!!!!!!!!!!"); - } - - - - public String setupLargMsg() { - StringBuilder sb = new StringBuilder(); - for (int i = 0; i < Short.MAX_VALUE * 64; i++) { - sb.append("Aloha is the most Hawaiian word."); - } - - return sb.toString(); - } - - @Test - public void test_large_msg() { - System.out.println("!!!!!!!!!!start larget message test!!!!!!!!"); - String req_msg = setupLargMsg(); - System.out.println("!!!!Finish batch data, size:" + req_msg.length() - + "!!!!"); - - IConnection server = null; - IConnection client = null; - - server = context.bind(null, port); - - WaitStrategy waitStrategy = (WaitStrategy) Utils - .newInstance((String) storm_conf - .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); - DisruptorQueue recvQueue = DisruptorQueue.mkInstance( - "NettyUnitTest", ProducerType.SINGLE, 1024, waitStrategy); - server.registerQueue(recvQueue); - - client = context.connect(null, "localhost", port); - - List list = new ArrayList(); - TaskMessage message = new TaskMessage(task, req_msg.getBytes()); - list.add(message); - - LOG.info("Client send data"); - client.send(message); - - TaskMessage recv = server.recv(0); - Assert.assertEquals(req_msg, new String(recv.message())); - - client.close(); - server.close(); - System.out.println("!!!!!!!!!!End larget message test!!!!!!!!"); - } - - @Test - public void test_server_delay() throws InterruptedException { - System.out.println("!!!!!!!!!!Start delay message test!!!!!!!!"); - String req_msg = setupLargMsg(); - - - IConnection server = null; - IConnection client = null; - - server = context.bind(null, port); - - WaitStrategy waitStrategy = (WaitStrategy) Utils - .newInstance((String) storm_conf - .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); - DisruptorQueue recvQueue = DisruptorQueue.mkInstance( - "NettyUnitTest", ProducerType.SINGLE, 1024, waitStrategy); - server.registerQueue(recvQueue); - - client = context.connect(null, "localhost", port); - - List list = new ArrayList(); - TaskMessage message = new TaskMessage(task, req_msg.getBytes()); - list.add(message); - - LOG.info("Client send data"); - client.send(message); - Thread.sleep(1000); - - TaskMessage recv = server.recv(0); - Assert.assertEquals(req_msg, new String(recv.message())); - - server.close(); - client.close(); - System.out.println("!!!!!!!!!!End delay message test!!!!!!!!"); - } - - @Test - public void test_first_client() throws InterruptedException { - System.out.println("!!!!!!!!Start test_first_client !!!!!!!!!!!"); - final String req_msg = setupLargMsg(); - - final IContext context = TransportFactory.makeContext(storm_conf); - - new Thread(new Runnable() { - - @Override - public void run() { - - lock.lock(); - IConnection client = context.connect(null, "localhost", port); - - List list = new ArrayList(); - TaskMessage message = new TaskMessage(task, req_msg.getBytes()); - list.add(message); - - client.send(message); - System.out.println("!!Client has sent data"); - JStormUtils.sleepMs(1000); - - try { - clientClose.await(); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - client.close(); - contextClose.signal(); - lock.unlock(); - - } - }).start(); - - - IConnection server = null; - - JStormUtils.sleepMs(1000); - System.out.println("!!server begin start!!!!!"); - - server = context.bind(null, port); - - WaitStrategy waitStrategy = (WaitStrategy) Utils - .newInstance((String) storm_conf - .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); - DisruptorQueue recvQueue = DisruptorQueue.mkInstance( - "NettyUnitTest", ProducerType.SINGLE, 1024, waitStrategy); - server.registerQueue(recvQueue); - - TaskMessage recv = server.recv(0); - Assert.assertEquals(req_msg, new String(recv.message())); - - lock.lock(); - clientClose.signal(); - server.close(); - contextClose.await(); - context.term(); - lock.unlock(); - - System.out.println("!!!!!!!!!!!!End test_first_client!!!!!!!!!!!!!"); - } - - @Test - public void test_batch() throws InterruptedException { - System.out.println("!!!!!!!!!!Start batch message test!!!!!!!!"); - final int base = 100000; - - final IContext context = TransportFactory.makeContext(storm_conf); - final IConnection server = context.bind(null, port); - - - WaitStrategy waitStrategy = (WaitStrategy) Utils - .newInstance((String) storm_conf - .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); - DisruptorQueue recvQueue = DisruptorQueue.mkInstance( - "NettyUnitTest", ProducerType.SINGLE, 1024, waitStrategy); - server.registerQueue(recvQueue); - - - new Thread(new Runnable() { - - public void send() { - final IConnection client = context.connect(null, "localhost", port); - - List list = new ArrayList(); - - for (int i = 1; i < Short.MAX_VALUE; i++) { - - String req_msg = String.valueOf(i + base); - - TaskMessage message = new TaskMessage(i, req_msg.getBytes()); - list.add(message); - - } - - client.send(list); - - System.out.println("Finish Send "); - JStormUtils.sleepMs(1000); - - try { - clientClose.await(); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - client.close(); - contextClose.signal(); - - } - - @Override - public void run() { - lock.lock(); - try{ - send(); - }finally { - lock.unlock(); - } - } - }).start(); - - for (int i = 1; i < Short.MAX_VALUE; i++) { - TaskMessage message = server.recv(0); - - Assert.assertEquals(String.valueOf(i + base), - new String(message.message())); - - if (i % 1000 == 0) { - System.out.println("Receive " + message.task()); - } - } - - System.out.println("Finish Receive "); - - lock.lock(); - clientClose.signal(); - server.close(); - contextClose.await(); - context.term(); - lock.unlock(); - System.out.println("!!!!!!!!!!End batch message test!!!!!!!!"); - } - - @Test - public void test_slow_receive() throws InterruptedException { - System.out - .println("!!!!!!!!!!Start test_slow_receive message test!!!!!!!!"); - final int base = 100000; - - final IContext context = TransportFactory.makeContext(storm_conf); - final IConnection server = context.bind(null, port); - - WaitStrategy waitStrategy = (WaitStrategy) Utils - .newInstance((String) storm_conf - .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); - DisruptorQueue recvQueue = DisruptorQueue.mkInstance( - "NettyJunitTest", ProducerType.SINGLE, 4, waitStrategy); - server.registerQueue(recvQueue); - - - new Thread(new Runnable() { - - @Override - public void run() { - lock.lock(); - - IConnection client = null; - - client = context.connect(null, "localhost", port); - - List list = new ArrayList(); - - for (int i = 1; i < Short.MAX_VALUE; i++) { - - String req_msg = String.valueOf(i + base); - - TaskMessage message = new TaskMessage(i, req_msg.getBytes()); - list.add(message); - - if (i % 1000 == 0) { - System.out.println("send " + i); - client.send(list); - list = new ArrayList(); - } - - } - - client.send(list); - - System.out.println("Finish Send "); - JStormUtils.sleepMs(1000); - - try { - clientClose.await(); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - client.close(); - contextClose.signal(); - lock.unlock(); - } - }).start(); - - for (int i = 1; i < Short.MAX_VALUE; i++) { - TaskMessage message = server.recv(0); - JStormUtils.sleepMs(1); - - Assert.assertEquals(String.valueOf(i + base), - new String(message.message())); - - if (i % 1000 == 0) { - System.out.println("Receive " + message.task()); - } - } - - System.out.println("Finish Receive "); - - lock.lock(); - clientClose.signal(); - server.close(); - contextClose.await(); - context.term(); - lock.unlock(); - System.out - .println("!!!!!!!!!!End test_slow_receive message test!!!!!!!!"); - } - - @Test - public void test_slow_receive_big() throws InterruptedException { - System.out - .println("!!!!!!!!!!Start test_slow_receive_big message test!!!!!!!!"); - final int base = 100; - final String req_msg = setupLargMsg(); - - final IContext context = TransportFactory.makeContext(storm_conf); - final IConnection server = context.bind(null, port); - - - - WaitStrategy waitStrategy = (WaitStrategy) Utils - .newInstance((String) storm_conf - .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); - DisruptorQueue recvQueue = DisruptorQueue.mkInstance( - "NettyJunitTest", ProducerType.SINGLE, 4, waitStrategy); - server.registerQueue(recvQueue); - - new Thread(new Runnable() { - - @Override - public void run() { - final IConnection client = context.connect(null, "localhost", port); - - lock.lock(); - for (int i = 1; i < base; i++) { - - TaskMessage message = new TaskMessage(i, req_msg.getBytes()); - System.out.println("send " + i); - client.send(message); - - } - - System.out.println("Finish Send "); - JStormUtils.sleepMs(1000); - - try { - clientClose.await(); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - client.close(); - contextClose.signal(); - lock.unlock(); - - } - }).start(); - - for (int i = 1; i < base; i++) { - TaskMessage message = server.recv(0); - JStormUtils.sleepMs(100); - - Assert.assertEquals(req_msg, new String(message.message())); - System.out.println("receive " + message.task()); - - } - - System.out.println("Finish Receive "); - - lock.lock(); - clientClose.signal(); - server.close(); - contextClose.await(); - context.term(); - lock.unlock(); - System.out - .println("!!!!!!!!!!End test_slow_receive_big message test!!!!!!!!"); - } - - @Test - public void test_client_reboot() throws InterruptedException { - System.out.println("!!!!!!!!!!Start client reboot test!!!!!!!!"); - final String req_msg = setupLargMsg(); - - final IContext context = TransportFactory.makeContext(storm_conf); - - new Thread(new Runnable() { - - @Override - public void run() { - - - IConnection client = context.connect(null, "localhost", port); - - lock.lock(); - - List list = new ArrayList(); - TaskMessage message = new TaskMessage(task, req_msg.getBytes()); - list.add(message); - - client.send(message); - - System.out.println("Send first"); - - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - } - client.close(); - - IConnection client2 = context.connect(null, "localhost", port); - System.out.println("!!!!!!! restart client !!!!!!!!!!"); - - client2.send(message); - System.out.println("Send second"); - JStormUtils.sleepMs(1000); - - - try { - clientClose.await(); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - client2.close(); - contextClose.signal(); - lock.unlock(); - } - }).start(); - - IConnection server = null; - - server = context.bind(null, port); - - WaitStrategy waitStrategy = (WaitStrategy) Utils - .newInstance((String) storm_conf - .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); - DisruptorQueue recvQueue = DisruptorQueue.mkInstance( - "NettyUnitTest", ProducerType.SINGLE, 1024, waitStrategy); - server.registerQueue(recvQueue); - - TaskMessage recv = server.recv(0); - System.out.println("Sever receive first"); - Assert.assertEquals(req_msg, new String(recv.message())); - - Thread.sleep(1000); - - TaskMessage recv2 = server.recv(0); - System.out.println("Sever receive second"); - Assert.assertEquals(req_msg, new String(recv2.message())); - - lock.lock(); - clientClose.signal(); - server.close(); - contextClose.await(); - context.term(); - lock.unlock(); - System.out.println("!!!!!!!!!!End client reboot test!!!!!!!!"); - } - - @Test - public void test_server_reboot() throws InterruptedException { - System.out.println("!!!!!!!!!!Start server reboot test!!!!!!!!"); - final String req_msg = setupLargMsg(); - - final IContext context = TransportFactory.makeContext(storm_conf); - IConnection server = null; - - - new Thread(new Runnable() { - - @Override - public void run() { - final IConnection client = context.connect(null, "localhost", port); - - lock.lock(); - - List list = new ArrayList(); - TaskMessage message = new TaskMessage(task, req_msg.getBytes()); - list.add(message); - - client.send(message); - - System.out.println("Send first"); - - JStormUtils.sleepMs(10000); - - System.out.println("Begin to Send second"); - client.send(message); - System.out.println("Send second"); - - JStormUtils.sleepMs(15000); - client.send(message); - System.out.println("Send third time"); - - try { - clientClose.await(); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - client.close(); - contextClose.signal(); - lock.unlock(); - - } - }).start(); - - server = context.bind(null, port); - - WaitStrategy waitStrategy = (WaitStrategy) Utils - .newInstance((String) storm_conf - .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); - DisruptorQueue recvQueue = DisruptorQueue.mkInstance( - "NettyUnitTest", ProducerType.SINGLE, 1024, waitStrategy); - server.registerQueue(recvQueue); - - TaskMessage recv = server.recv(0); - System.out.println("Receive first"); - Assert.assertEquals(req_msg, new String(recv.message())); - - server.close(); - - System.out.println("!!shutdow server and sleep 30s, please wait!!"); - Thread.sleep(30000); - - IConnection server2 = context.bind(null, port); - server2.registerQueue(recvQueue); - System.out.println("!!!!!!!!!!!!!!!!!!!! restart server !!!!!!!!!!!"); - - TaskMessage recv2 = server2.recv(0); - Assert.assertEquals(req_msg, new String(recv2.message())); - - lock.lock(); - clientClose.signal(); - server2.close(); - contextClose.await(); - context.term(); - lock.unlock(); - System.out.println("!!!!!!!!!!End server reboot test!!!!!!!!"); - } - - /** - * Due to there is only one client to one server in one jvm - * It can't do this test - * - * @throws InterruptedException - */ - public void test_multiple_client() throws InterruptedException { - System.out.println("!!!!!!!!Start test_multiple_client !!!!!!!!!!!"); - final String req_msg = setupLargMsg(); - - final int clientNum = 3; - final AtomicLong received = new AtomicLong(clientNum); - - for (int i = 0; i < clientNum; i++ ) { - - new Thread(new Runnable() { - - @Override - public void run() { - - IConnection client = context.connect(null, "localhost", port); - - List list = new ArrayList(); - TaskMessage message = new TaskMessage(task, req_msg.getBytes()); - list.add(message); - - client.send(message); - System.out.println("!!Client has sent data"); - - while(received.get() != 0) { - JStormUtils.sleepMs(1000); - } - - - client.close(); - - } - }).start(); - } - - - IConnection server = null; - - JStormUtils.sleepMs(1000); - System.out.println("!!server begin start!!!!!"); - - server = context.bind(null, port); - - WaitStrategy waitStrategy = (WaitStrategy) Utils - .newInstance((String) storm_conf - .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); - DisruptorQueue recvQueue = DisruptorQueue.mkInstance( - "NettyUnitTest", ProducerType.SINGLE, 1024, waitStrategy); - server.registerQueue(recvQueue); - - for (int i = 0; i < clientNum; i++) { - TaskMessage recv = server.recv(0); - Assert.assertEquals(req_msg, new String(recv.message())); - received.decrementAndGet(); - } - - server.close(); - - - System.out.println("!!!!!!!!!!!!End test_multiple_client!!!!!!!!!!!!!"); - } - - - @Test - public void test_multiple_server() throws InterruptedException { - System.out.println("!!!!!!!!Start test_multiple_server !!!!!!!!!!!"); - final String req_msg = setupLargMsg(); - - final int clientNum = 3; - final AtomicLong received = new AtomicLong(clientNum); - - for (int i = 0; i < clientNum; i++ ) { - final int realPort = port + i; - - new Thread(new Runnable() { - - @Override - public void run() { - - IConnection server = null; - - JStormUtils.sleepMs(1000); - System.out.println("!!server begin start!!!!!"); - - server = context.bind(null, realPort); - - WaitStrategy waitStrategy = (WaitStrategy) Utils - .newInstance((String) storm_conf - .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); - DisruptorQueue recvQueue = DisruptorQueue.mkInstance( - "NettyUnitTest", ProducerType.SINGLE, 1024, waitStrategy); - server.registerQueue(recvQueue); - - TaskMessage recv = server.recv(0); - Assert.assertEquals(req_msg, new String(recv.message())); - received.decrementAndGet(); - System.out.println("!!server received !!!!!" + realPort); - - server.close(); - - } - }).start(); - } - - List list = new ArrayList(); - TaskMessage message = new TaskMessage(task, req_msg.getBytes()); - list.add(message); - - List clients = new ArrayList(); - - for (int i = 0; i < clientNum; i++ ) { - final int realPort = port + i; - - IConnection client = context.connect(null, "localhost", realPort); - clients.add(client); - - client.send(message); - System.out.println("!!Client has sent data to " + realPort); - } - - - - while(received.get() != 0) { - JStormUtils.sleepMs(1000); - } - - for (int i = 0; i < clientNum; i++) { - clients.get(i).close(); - } - - System.out.println("!!!!!!!!!!!!End test_multiple_server!!!!!!!!!!!!!"); - } -} 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 deleted file mode 100644 index 98a378d04..000000000 --- a/jstorm-server/src/test/java/com/alibaba/jstorm/message/zmq/ZmqUnitTest.java +++ /dev/null @@ -1,299 +0,0 @@ -package com.alibaba.jstorm.message.zmq; - -import java.util.HashMap; -import java.util.Map; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; - -public class ZmqUnitTest { - - private static final Logger LOG = Logger.getLogger(ZmqUnitTest.class); - - private static int port = 6700; - private static int task = 1; - - private static String context_class_name = "com.alibaba.jstorm.message.zeroMq.MQContext"; - - private static Map storm_conf = new HashMap(); - static { - storm_conf.put(Config.STORM_MESSAGING_TRANSPORT, context_class_name); - } - - /** - * This is only can be test under linux - */ - - - -// @Test -// public void test_basic() { -// System.out -// .println("!!!!!!!!!!!!!!!!!Start basic test!!!!!!!!!!!!!!!!!"); -// String req_msg = "Aloha is the most Hawaiian word."; -// -// IContext context = TransportFactory.makeContext(storm_conf); -// IConnection server = null; -// IConnection client = null; -// -// server = context.bind(null, port); -// -// WaitStrategy waitStrategy = (WaitStrategy) Utils -// .newInstance((String) storm_conf -// .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); -// DisruptorQueue recvQueue = DisruptorQueue.mkInstance( -// new SingleThreadedClaimStrategy(1024), waitStrategy); -// server.registerQueue(recvQueue); -// -// client = context.connect(null, "localhost", port); -// -// List list = new ArrayList(); -// TaskMessage message = new TaskMessage(task, req_msg.getBytes()); -// list.add(message); -// -// client.send(list); -// -// TaskMessage recv = server.recv(0); -// Assert.assertEquals(req_msg, new String(recv.message())); -// -// System.out.println("!!!!!!!!!!!!!!!!!!Test one time!!!!!!!!!!!!!!!!!"); -// -// server.close(); -// client.close(); -// context.term(); -// -// System.out -// .println("!!!!!!!!!!!!!!!!!!!!End basic test!!!!!!!!!!!!!!!!!!!"); -// } -// -// public String setupLargMsg() { -// StringBuilder sb = new StringBuilder(); -// for (int i = 0; i < Short.MAX_VALUE * 10; i++) { -// sb.append("Aloha is the most Hawaiian word.").append(i); -// } -// -// return sb.toString(); -// } -// -// @Test -// public void test_large_msg() { -// System.out.println("!!!!!!!!!!start larget message test!!!!!!!!"); -// String req_msg = setupLargMsg(); -// System.out.println("!!!!Finish batch data, size:" + req_msg.length() -// + "!!!!"); -// -// IContext context = TransportFactory.makeContext(storm_conf); -// IConnection server = null; -// IConnection client = null; -// -// server = context.bind(null, port); -// -// WaitStrategy waitStrategy = (WaitStrategy) Utils -// .newInstance((String) storm_conf -// .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); -// DisruptorQueue recvQueue = DisruptorQueue.mkInstance( -// new SingleThreadedClaimStrategy(1024), waitStrategy); -// server.registerQueue(recvQueue); -// -// client = context.connect(null, "localhost", port); -// -// List list = new ArrayList(); -// TaskMessage message = new TaskMessage(task, req_msg.getBytes()); -// list.add(message); -// -// client.send(list); -// -// TaskMessage recv = server.recv(0); -// Assert.assertEquals(req_msg, new String(recv.message())); -// -// server.close(); -// client.close(); -// context.term(); -// System.out.println("!!!!!!!!!!End larget message test!!!!!!!!"); -// } -// -// @Test -// public void test_server_delay() throws InterruptedException { -// System.out.println("!!!!!!!!!!Start delay message test!!!!!!!!"); -// String req_msg = setupLargMsg(); -// -// IContext context = TransportFactory.makeContext(storm_conf); -// IConnection server = null; -// IConnection client = null; -// -// server = context.bind(null, port); -// -// WaitStrategy waitStrategy = (WaitStrategy) Utils -// .newInstance((String) storm_conf -// .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); -// DisruptorQueue recvQueue = DisruptorQueue.mkInstance( -// new SingleThreadedClaimStrategy(1024), waitStrategy); -// server.registerQueue(recvQueue); -// -// client = context.connect(null, "localhost", port); -// -// List list = new ArrayList(); -// TaskMessage message = new TaskMessage(task, req_msg.getBytes()); -// list.add(message); -// -// client.send(list); -// Thread.sleep(1000); -// -// TaskMessage recv = server.recv(0); -// Assert.assertEquals(req_msg, new String(recv.message())); -// -// server.close(); -// client.close(); -// context.term(); -// System.out.println("!!!!!!!!!!End delay message test!!!!!!!!"); -// } -// -// @Test -// public void test_batch() { -// System.out.println("!!!!!!!!!!Start batch message test!!!!!!!!"); -// -// IContext context = TransportFactory.makeContext(storm_conf); -// final IConnection server = context.bind(null, port); -// IConnection client = null; -// -// WaitStrategy waitStrategy = (WaitStrategy) Utils -// .newInstance((String) storm_conf -// .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); -// DisruptorQueue recvQueue = DisruptorQueue.mkInstance( -// new SingleThreadedClaimStrategy(1024), waitStrategy); -// server.registerQueue(recvQueue); -// -// client = context.connect(null, "localhost", port); -// -// final int base = 100000; -// -// List list = new ArrayList(); -// -// client.send(list); -// for (int i = 1; i < Short.MAX_VALUE; i++) { -// -// String req_msg = String.valueOf(i + base); -// -// TaskMessage message = new TaskMessage(task, req_msg.getBytes()); -// list.add(message); -// -// } -// -// client.send(list); -// -// System.out.println("Finish Send "); -// -// for (int i = 1; i < Short.MAX_VALUE; i++) { -// TaskMessage message = server.recv(0); -// -// Assert.assertEquals(String.valueOf(i + base), -// new String(message.message())); -// -// if (i % 1000 == 0) { -// System.out.println("Receive " + i); -// } -// } -// -// System.out.println("Finish Receive "); -// -// client.close(); -// server.close(); -// context.term(); -// System.out.println("!!!!!!!!!!End batch message test!!!!!!!!"); -// } -// -// @Test -// public void test_client_reboot() throws InterruptedException { -// System.out.println("!!!!!!!!!!Start client reboot test!!!!!!!!"); -// String req_msg = setupLargMsg(); -// -// IContext context = TransportFactory.makeContext(storm_conf); -// IConnection server = null; -// IConnection client = null; -// -// server = context.bind(null, port); -// -// WaitStrategy waitStrategy = (WaitStrategy) Utils -// .newInstance((String) storm_conf -// .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); -// DisruptorQueue recvQueue = DisruptorQueue.mkInstance( -// new SingleThreadedClaimStrategy(1024), waitStrategy); -// server.registerQueue(recvQueue); -// -// client = context.connect(null, "localhost", port); -// -// List list = new ArrayList(); -// TaskMessage message = new TaskMessage(task, req_msg.getBytes()); -// list.add(message); -// -// client.send(list); -// -// TaskMessage recv = server.recv(0); -// Assert.assertEquals(req_msg, new String(recv.message())); -// -// client.close(); -// IConnection client2 = context.connect(null, "localhost", port); -// System.out.println("!!!!!!! restart client !!!!!!!!!!"); -// -// client2.send(list); -// Thread.sleep(1000); -// -// TaskMessage recv2 = server.recv(0); -// Assert.assertEquals(req_msg, new String(recv2.message())); -// -// client2.close(); -// server.close(); -// context.term(); -// System.out.println("!!!!!!!!!!End client reboot test!!!!!!!!"); -// } -// -// @Test -// public void test_server_reboot() throws InterruptedException { -// System.out.println("!!!!!!!!!!Start server reboot test!!!!!!!!"); -// String req_msg = setupLargMsg(); -// -// IContext context = TransportFactory.makeContext(storm_conf); -// IConnection server = null; -// IConnection client = null; -// -// server = context.bind(null, port); -// -// WaitStrategy waitStrategy = (WaitStrategy) Utils -// .newInstance((String) storm_conf -// .get(Config.TOPOLOGY_DISRUPTOR_WAIT_STRATEGY)); -// DisruptorQueue recvQueue = DisruptorQueue.mkInstance( -// new SingleThreadedClaimStrategy(1024), waitStrategy); -// server.registerQueue(recvQueue); -// -// client = context.connect(null, "localhost", port); -// -// List list = new ArrayList(); -// TaskMessage message = new TaskMessage(task, req_msg.getBytes()); -// list.add(message); -// -// client.send(list); -// -// TaskMessage recv = server.recv(0); -// Assert.assertEquals(req_msg, new String(recv.message())); -// -// server.close(); -// -// client.send(list); -// System.out.println("!!!!!!!! shutdow server and sleep 30s !!!!!"); -// Thread.sleep(30000); -// -// IConnection server2 = context.bind(null, port); -// server2.registerQueue(recvQueue); -// System.out.println("!!!!!!!!!!!!!!!!!!!! restart server !!!!!!!!!!!"); -// -// TaskMessage recv2 = server2.recv(0); -// Assert.assertEquals(req_msg, new String(recv2.message())); -// -// client.close(); -// server2.close(); -// context.term(); -// System.out.println("!!!!!!!!!!End server reboot test!!!!!!!!"); -// } - -} 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 deleted file mode 100644 index 5647bb260..000000000 --- a/jstorm-server/src/test/java/com/alibaba/jstorm/superivosr/deamon/HttpserverTest.java +++ /dev/null @@ -1,25 +0,0 @@ -package com.alibaba.jstorm.superivosr.deamon; - -import java.util.HashMap; - -import org.junit.Test; - -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.daemon.supervisor.Httpserver; -import com.google.common.collect.Maps; - -public class HttpserverTest { - - @Test - public void test_httpserver() { - int port = ConfigExtension.getSupervisorDeamonHttpserverPort(Maps.newHashMap()); - Httpserver httpserver = new Httpserver(port, new HashMap()); - httpserver.start(); - System.out.println("start...."); - - httpserver.shutdown(); - } - - - -} 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 deleted file mode 100644 index 0c95b7999..000000000 --- a/jstorm-server/src/test/java/com/alibaba/jstorm/topology/SingleJoinBolt.java +++ /dev/null @@ -1,128 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package 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 deleted file mode 100644 index f85a7bbcc..000000000 --- a/jstorm-server/src/test/java/com/alibaba/jstorm/topology/SingleJoinTest.java +++ /dev/null @@ -1,79 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package 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 deleted file mode 100644 index 75524cd5e..000000000 --- a/jstorm-server/src/test/java/com/alibaba/jstorm/topology/TransactionalWordsTest.java +++ /dev/null @@ -1,274 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package 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-server/src/test/java/com/alibaba/jstorm/utils/JStormUtilsTest.java b/jstorm-server/src/test/java/com/alibaba/jstorm/utils/JStormUtilsTest.java deleted file mode 100644 index 68f830864..000000000 --- a/jstorm-server/src/test/java/com/alibaba/jstorm/utils/JStormUtilsTest.java +++ /dev/null @@ -1,158 +0,0 @@ -package com.alibaba.jstorm.utils; - -import java.io.BufferedOutputStream; -import java.io.DataOutputStream; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; - -import org.apache.log4j.Logger; - - -public class JStormUtilsTest { - private static final Logger LOG = Logger.getLogger(JStormUtilsTest.class); - - private static final int RESTART_TIMES = 1000; - - private static String cmd = ""; - - private static String pidDir = "/tmp/pids"; - - private static int killedTimes = 0; - private static int forkTimes = 0; - - - public static void testRestartProcess() { - final int intervalSec = Integer.valueOf("10"); - new Thread(new Runnable () { - - @Override - public void run() { - LOG.info("Begin to start fork thread"); - Map environment = new HashMap(); - - // TODO Auto-generated method stub - while(forkTimes < RESTART_TIMES) { - - try { - JStormUtils.launch_process(cmd + " " + forkTimes, environment, true); - } catch (IOException e) { - // TODO Auto-generated catch block - LOG.error("Failed to fork process " + cmd + forkTimes, e); - continue; - } - - LOG.info("Successfully launch " + forkTimes); - - JStormUtils.sleepMs(1000); - - forkTimes++; - - - } - LOG.info("Successfully shutdown fok thread"); - } - - }).start(); - - new Thread(new Runnable(){ - - @Override - public void run() { - LOG.info("Begin to start killing thread"); - - File file = new File(pidDir); - // TODO Auto-generated method stub - while(killedTimes < RESTART_TIMES) { - File[] pids = file.listFiles(); - if (pids == null) { - JStormUtils.sleepMs(100); - continue; - } - - for (File pidFile : pids) { - String pid = pidFile.getName(); - - JStormUtils.ensure_process_killed(Integer.valueOf(pid)); - - killedTimes++; - - pidFile.delete(); - } - - - JStormUtils.sleepMs(100); - - - } - LOG.info("Successfully shutdown killing thread"); - } - - }).start(); - - - while(killedTimes < RESTART_TIMES) { - JStormUtils.sleepMs(100); - } - } - - public static void fillData() { - Map map = new HashMap(); - - for (long l = 0; l < 1000000; l++) { - map.put(l, String.valueOf(l)); - } - } - - public static void testJar(String id) { - try { - - PathUtils.local_mkdirs(pidDir); - } catch (IOException e) { - LOG.error("Failed to rmr " + pidDir, e); - } - - fillData(); - LOG.info("Finish load data"); - - String pid = JStormUtils.process_pid(); - - String pidFile = pidDir + File.separator + pid; - - - try { - PathUtils.touch(pidFile); - } catch (IOException e) { - // TODO Auto-generated catch block - LOG.error("Failed to touch " + pidFile, e); - } - try{ - - DataOutputStream raf = new DataOutputStream(new BufferedOutputStream( - new FileOutputStream(new File(pidFile), true))); - - raf.writeBytes(pid); - }catch(Exception e) { - LOG.error("",e); - } - - while(true) { - JStormUtils.sleepMs(1000); - LOG.info(id + " is living"); - } - - - } - - - public static void main(String[] args) { - if (args.length == 0) { - testRestartProcess(); - }else { - testJar(args[0]); - } - } - -} diff --git a/jstorm-server/src/test/resources/log4j.properties b/jstorm-server/src/test/resources/log4j.properties deleted file mode 100644 index 9fd11e9eb..000000000 --- a/jstorm-server/src/test/resources/log4j.properties +++ /dev/null @@ -1,47 +0,0 @@ -#This file should be deleted when deployed to server (workaround to leiningen classpath putting dev resources on path) -#This file is needed for tests - - -log4j.rootLogger=INFO,D -#log4j.rootLogger=INFO,stdout - -### output to console ### -### In order to avoid dead lock, redirect supervisor out/err to /dev/null -### Stdout logger can't be used until manually start worker -log4j.appender.stdout = org.apache.log4j.ConsoleAppender -log4j.appender.stdout.Target = System.out -log4j.appender.stdout.layout = org.apache.log4j.PatternLayout -log4j.appender.stdout.layout.ConversionPattern = [%p %d{yyyy-MM-dd HH:mm:ss} %c{1}:%L %t] %m%n - -### output to file ### -log4j.appender.D = org.apache.log4j.RollingFileAppender -log4j.appender.D.File = test.log -log4j.appender.D.Append = true -log4j.appender.D.Threshold = INFO -log4j.appender.D.MaxFileSize=1GB -log4j.appender.D.MaxBackupIndex=5 -log4j.appender.D.layout = org.apache.log4j.PatternLayout -log4j.appender.D.layout.ConversionPattern = [%p %d{yyyy-MM-dd HH:mm:ss} %c{1}:%L %t] %m%n - - - -### jstorm metrics ### -log4j.logger.com.alibaba.jstorm.daemon.worker.metrics= INFO, M -log4j.additivity.com.alibaba.jstorm.daemon.worker.metrics=false -log4j.logger.com.alibaba.jstorm.task.heartbeat= INFO, M -log4j.additivity.com.alibaba.jstorm.task.heartbeat=false -log4j.logger.com.alibaba.jstorm.daemon.worker.hearbeat= INFO, M -log4j.additivity.com.alibaba.jstorm.daemon.worker.hearbeat=false - -log4j.appender.M = org.apache.log4j.RollingFileAppender -log4j.appender.M.File = test.log.metrics -log4j.appender.M.Append = true -log4j.appender.M.Threshold = INFO -log4j.appender.M.MaxFileSize=100MB -log4j.appender.M.MaxBackupIndex=5 -log4j.appender.M.layout = org.apache.log4j.PatternLayout -log4j.appender.M.layout.ConversionPattern = [%p %d{yyyy-MM-dd HH:mm:ss} %c{1}:%L %t] %m%n - - -##################jmonitor appender ########################## -#log4j.appender.jmonitor=com.alibaba.alimonitor.jmonitor.plugin.log4j.JMonitorLog4jAppender \ No newline at end of file diff --git a/jstorm-ui/.classpath b/jstorm-ui/.classpath new file mode 100755 index 000000000..0f4a4c27e --- /dev/null +++ b/jstorm-ui/.classpath @@ -0,0 +1,32 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/jstorm-ui/.gitignore b/jstorm-ui/.gitignore new file mode 100755 index 000000000..b83d22266 --- /dev/null +++ b/jstorm-ui/.gitignore @@ -0,0 +1 @@ +/target/ diff --git a/jstorm-ui/.project b/jstorm-ui/.project new file mode 100755 index 000000000..af405e726 --- /dev/null +++ b/jstorm-ui/.project @@ -0,0 +1,42 @@ + + + jstorm-ui + + + + + + org.eclipse.wst.jsdt.core.javascriptValidator + + + + + org.eclipse.jdt.core.javabuilder + + + + + org.eclipse.wst.common.project.facet.core.builder + + + + + org.eclipse.wst.validation.validationbuilder + + + + + org.eclipse.m2e.core.maven2Builder + + + + + + org.eclipse.jem.workbench.JavaEMFNature + org.eclipse.wst.common.modulecore.ModuleCoreNature + org.eclipse.jdt.core.javanature + org.eclipse.m2e.core.maven2Nature + org.eclipse.wst.common.project.facet.core.nature + org.eclipse.wst.jsdt.core.jsNature + + diff --git a/jstorm-ui/.settings/.jsdtscope b/jstorm-ui/.settings/.jsdtscope new file mode 100755 index 000000000..585c967a9 --- /dev/null +++ b/jstorm-ui/.settings/.jsdtscope @@ -0,0 +1,13 @@ + + + + + + + + + + + + + diff --git a/jstorm-ui/.settings/org.eclipse.core.resources.prefs b/jstorm-ui/.settings/org.eclipse.core.resources.prefs new file mode 100755 index 000000000..cf6931b95 --- /dev/null +++ b/jstorm-ui/.settings/org.eclipse.core.resources.prefs @@ -0,0 +1,4 @@ +eclipse.preferences.version=1 +encoding//src/main/java=UTF-8 +encoding//src/main/resources=UTF-8 +encoding/=UTF-8 diff --git a/jstorm-ui/.settings/org.eclipse.jdt.core.prefs b/jstorm-ui/.settings/org.eclipse.jdt.core.prefs new file mode 100755 index 000000000..62a317c86 --- /dev/null +++ b/jstorm-ui/.settings/org.eclipse.jdt.core.prefs @@ -0,0 +1,8 @@ +eclipse.preferences.version=1 +org.eclipse.jdt.core.compiler.codegen.inlineJsrBytecode=enabled +org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.6 +org.eclipse.jdt.core.compiler.compliance=1.6 +org.eclipse.jdt.core.compiler.problem.assertIdentifier=error +org.eclipse.jdt.core.compiler.problem.enumIdentifier=error +org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning +org.eclipse.jdt.core.compiler.source=1.6 diff --git a/jstorm-ui/.settings/org.eclipse.m2e.core.prefs b/jstorm-ui/.settings/org.eclipse.m2e.core.prefs new file mode 100755 index 000000000..14b697b7b --- /dev/null +++ b/jstorm-ui/.settings/org.eclipse.m2e.core.prefs @@ -0,0 +1,4 @@ +activeProfiles= +eclipse.preferences.version=1 +resolveWorkspaceProjects=true +version=1 diff --git a/jstorm-ui/.settings/org.eclipse.wst.common.component b/jstorm-ui/.settings/org.eclipse.wst.common.component new file mode 100755 index 000000000..c6036b536 --- /dev/null +++ b/jstorm-ui/.settings/org.eclipse.wst.common.component @@ -0,0 +1,13 @@ + + + + + + + + uses + + + + + diff --git a/jstorm-ui/.settings/org.eclipse.wst.common.project.facet.core.prefs.xml b/jstorm-ui/.settings/org.eclipse.wst.common.project.facet.core.prefs.xml new file mode 100755 index 000000000..1ac9c2993 --- /dev/null +++ b/jstorm-ui/.settings/org.eclipse.wst.common.project.facet.core.prefs.xml @@ -0,0 +1,12 @@ + + + + + + + + + + + + diff --git a/jstorm-ui/.settings/org.eclipse.wst.common.project.facet.core.xml b/jstorm-ui/.settings/org.eclipse.wst.common.project.facet.core.xml new file mode 100755 index 000000000..a3dd227a7 --- /dev/null +++ b/jstorm-ui/.settings/org.eclipse.wst.common.project.facet.core.xml @@ -0,0 +1,9 @@ + + + + + + + + + diff --git a/jstorm-ui/.settings/org.eclipse.wst.jsdt.ui.superType.container b/jstorm-ui/.settings/org.eclipse.wst.jsdt.ui.superType.container new file mode 100755 index 000000000..3bd5d0a48 --- /dev/null +++ b/jstorm-ui/.settings/org.eclipse.wst.jsdt.ui.superType.container @@ -0,0 +1 @@ +org.eclipse.wst.jsdt.launching.baseBrowserLibrary \ No newline at end of file diff --git a/jstorm-ui/.settings/org.eclipse.wst.jsdt.ui.superType.name b/jstorm-ui/.settings/org.eclipse.wst.jsdt.ui.superType.name new file mode 100755 index 000000000..05bd71b6e --- /dev/null +++ b/jstorm-ui/.settings/org.eclipse.wst.jsdt.ui.superType.name @@ -0,0 +1 @@ +Window \ No newline at end of file diff --git a/jstorm-ui/.settings/org.eclipse.wst.validation.prefs b/jstorm-ui/.settings/org.eclipse.wst.validation.prefs new file mode 100755 index 000000000..6f1cba68d --- /dev/null +++ b/jstorm-ui/.settings/org.eclipse.wst.validation.prefs @@ -0,0 +1,2 @@ +disabled=06target +eclipse.preferences.version=1 diff --git a/jstorm-ui/nb-configuration.xml b/jstorm-ui/nb-configuration.xml old mode 100644 new mode 100755 diff --git a/jstorm-ui/pom.xml b/jstorm-ui/pom.xml old mode 100644 new mode 100755 index 1c4a00429..8c13332d0 --- a/jstorm-ui/pom.xml +++ b/jstorm-ui/pom.xml @@ -1,126 +1,120 @@ - - 4.0.0 - - com.alibaba.jstorm - jstorm-all - 0.9.6.3 - .. - - - jstorm-ui - jstorm web ui modules - jstorm web ui modules - war - - - ${project.build.directory}/endorsed - UTF-8 - Tomcat - - - - - com.alibaba.jstorm - jstorm-client - ${project.version} - - - com.alibaba.jstorm - jstorm-client-extension - ${project.version} - - - com.sun.faces - jsf-api - 2.1.7 - - - com.sun.faces - jsf-impl - 2.1.7 - - - javax.servlet - jstl - 1.2 - - - taglibs - standard - 1.1.2 - - - org.primefaces - primefaces - 3.2 - - - org.primefaces.themes - cupertino - 1.0.3 - - - - javax - javaee-web-api - 6.0 - provided - - - - org.apache.httpcomponents - httpclient - 4.3.2 - - - - - - - org.apache.maven.plugins - maven-compiler-plugin - 2.3.2 - - 1.6 - 1.6 - - ${endorsed.dir} - - - - - org.apache.maven.plugins - maven-war-plugin - 2.1.1 - - false - - - - org.apache.maven.plugins - maven-dependency-plugin - 2.4 - - - - - - - http://download.java.net/maven/2/ - jsf20 - default - Repository for library Library[jsf20] - - - http://repository.primefaces.org/ - primefaces - default - Repository for library Library[primefaces] - - - + + 4.0.0 + + com.alibaba.jstorm + jstorm-all + 2.0.4-SNAPSHOT + .. + + + jstorm-ui + jstorm web ui modules + jstorm web ui modules + war + + + ${project.build.directory}/endorsed + UTF-8 + Tomcat + + + + + com.alibaba.jstorm + jstorm-core + ${project.version} + + + com.sun.faces + jsf-api + 2.1.7 + + + com.sun.faces + jsf-impl + 2.1.7 + + + javax.servlet + jstl + 1.2 + + + taglibs + standard + 1.1.2 + + + org.primefaces + primefaces + 3.2 + + + org.primefaces.themes + cupertino + 1.0.3 + + + javax + javaee-web-api + 6.0 + provided + + + + org.apache.httpcomponents + httpclient + 4.3.3 + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 2.3.2 + + 1.6 + 1.6 + + ${endorsed.dir} + + + + + org.apache.maven.plugins + maven-war-plugin + 2.1.1 + + false + + + + org.apache.maven.plugins + maven-dependency-plugin + 2.4 + + + + + + + http://download.java.net/maven/2/ + jsf20 + default + Repository for library Library[jsf20] + + + http://repository.primefaces.org/ + primefaces + default + Repository for library Library[primefaces] + + + diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/DescendComparator.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/DescendComparator.java old mode 100644 new mode 100755 index 015b3df8f..d084834b6 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/DescendComparator.java +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/DescendComparator.java @@ -1,25 +1,42 @@ -package com.alibaba.jstorm.ui; - -import java.util.Comparator; - -public class DescendComparator implements Comparator { - - @Override - public int compare(Object o1, Object o2) { - - if (o1 instanceof Double && o2 instanceof Double) { - Double i1 = (Double) o1; - Double i2 = (Double) o2; - return -i1.compareTo(i2); - } else if (o1 instanceof Integer && o2 instanceof Integer) { - Integer i1 = (Integer) o1; - Integer i2 = (Integer) o2; - return -i1.compareTo(i2); - } else { - Double i1 = Double.valueOf(String.valueOf(o1)); - Double i2 = Double.valueOf(String.valueOf(o2)); - - return -i1.compareTo(i2); - } - } -} +/** + * 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.ui; + +import java.util.Comparator; + +public class DescendComparator implements Comparator { + + @Override + public int compare(Object o1, Object o2) { + + if (o1 instanceof Double && o2 instanceof Double) { + Double i1 = (Double) o1; + Double i2 = (Double) o2; + return -i1.compareTo(i2); + } else if (o1 instanceof Integer && o2 instanceof Integer) { + Integer i1 = (Integer) o1; + Integer i2 = (Integer) o2; + return -i1.compareTo(i2); + } else { + Double i1 = Double.valueOf(String.valueOf(o1)); + Double i2 = Double.valueOf(String.valueOf(o2)); + + return -i1.compareTo(i2); + } + } +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/NimbusClientManager.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/NimbusClientManager.java new file mode 100755 index 000000000..9757ac82b --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/NimbusClientManager.java @@ -0,0 +1,113 @@ +/** + * 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.ui; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.util.Map; + +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.yaml.snakeyaml.Yaml; + +import backtype.storm.Config; +import backtype.storm.utils.NimbusClient; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.utils.ExpiredCallback; +import com.alibaba.jstorm.utils.TimeCacheMap; + +public class NimbusClientManager { + private static Logger LOG = LoggerFactory + .getLogger(NimbusClientManager.class); + + public static String DEFAULT = "default"; + + protected static TimeCacheMap clientManager; + static { + + clientManager = + new TimeCacheMap(3600, + new ExpiredCallback() { + + @Override + public void expire(String key, NimbusClient val) { + // TODO Auto-generated method stub + LOG.info("Close connection of " + key); + val.close(); + } + + }); + } + + public static NimbusClient getNimbusClient(String clusterName) + throws Exception { + + Map conf = UIUtils.readUiConfig(); + + if (DEFAULT.equals(clusterName)) { + // do nothing + } else if (StringUtils.isBlank(clusterName) == false) { + UIUtils.getClusterInfoByName(conf, clusterName); + } + return NimbusClient.getConfiguredClient(conf); + } + + public static NimbusClient getNimbusClient(Map parameterMap) + throws Exception { + + String clusterName = parameterMap.get(UIDef.CLUSTER); + if (StringUtils.isBlank(clusterName) == true) { + clusterName = DEFAULT; + } + + NimbusClient client = clientManager.get(clusterName); + if (client != null) { + try { + client.getClient().getVersion(); + }catch(Exception e) { + LOG.info("Nimbus has been restarted, it begin to reconnect"); + client = null; + } + } + + if (client != null) { + return client; + } + + client = getNimbusClient(clusterName); + + clientManager.put(clusterName, client); + + return client; + + } + + public static void removeClient(Map parameterMap) { + String clusterName = parameterMap.get(UIDef.CLUSTER); + if (StringUtils.isBlank(clusterName) == true) { + clusterName = DEFAULT; + } + NimbusClient client = (NimbusClient)clientManager.remove(clusterName); + if (client != null) { + client.close(); + } + } +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/UIDef.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/UIDef.java new file mode 100755 index 000000000..d595161f5 --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/UIDef.java @@ -0,0 +1,114 @@ +/** + * 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.ui; + +import java.util.HashMap; +import java.util.Map; + +import com.alibaba.jstorm.ui.model.PageGenerator; +import com.alibaba.jstorm.ui.model.pages.ClusterPage; +import com.alibaba.jstorm.ui.model.pages.ComponentPage; +import com.alibaba.jstorm.ui.model.pages.ConfPage; +import com.alibaba.jstorm.ui.model.pages.JStackPage; +import com.alibaba.jstorm.ui.model.pages.ListLogsPage; +import com.alibaba.jstorm.ui.model.pages.NettyPage; +import com.alibaba.jstorm.ui.model.pages.SupervisorPage; +import com.alibaba.jstorm.ui.model.pages.TopologyPage; + +public class UIDef { + public static String LINK_CLUSTER_INFO = ""; + + public static final String LINK_TABLE_PAGE = "table.jsf"; + public static final String LINK_WINDOW_TABLE = "windowtable.jsf"; + public static final String LINK_LOG = "log.jsf"; + + public static final String PAGE_TYPE = "pageType"; + + public static final String PAGE_TYPE_CLUSTER = "cluster"; + public static final String PAGE_TYPE_SUPERVISOR = "supervisor"; + public static final String PAGE_TYPE_TOPOLOGY = "topology"; + public static final String PAGE_TYPE_COMPONENT = "component"; + public static final String PAGE_TYPE_CONF = "conf"; + public static final String PAGE_TYPE_LOG = "log"; + public static final String PAGE_TYPE_LISTLOG = "listLog"; + public static final String PAGE_TYPE_JSTACK = "jstack"; + public static final String PAGE_TYPE_NETTY = "netty"; + + public static final String CONF_TYPE = "confType"; + public static final String CONF_TYPE_NIMBUS = "nimbus"; + public static final String CONF_TYPE_SUPERVISOR = "supervisor"; + public static final String CONF_TYPE_TOPOLOGY = "topology"; + + public static final String HEADER_SUPERVISOR = "Supervisor Num"; + public static final String HEADER_TOTAL_PORT = "Total Ports"; + public static final String HEADER_USED_PORT = "Used Ports"; + public static final String HEADER_FREE_PORT = "Free Ports"; + public static final String HEADER_VERSION = "Version"; + public static final String HEADER_CONF = "Configurations"; + public static final String HEADER_LOGS = "Logs"; + public static final String HEADER_LOG = "Log"; + public static final String HEADER_UPTIME = "Uptime"; + public static final String HEADER_TOPOLOGY_NAME = "Topology Name"; + public static final String HEADER_TOPOLOGY_ID = "Topology ID"; + public static final String HEADER_TOPOLOGY_METRICS = "Topology Metrics"; + public static final String HEADER_STATUS = "Status"; + public static final String HEADER_WORKER_NUM = "Worker Num"; + public static final String HEADER_TASK_NUM = "Task Num"; + public static final String HEADER_TASK_LIST = "Task List"; + public static final String HEADER_TASK_ID = "Task Id"; + public static final String HEADER_COMPONENT_NAME = "Component Name"; + public static final String HEADER_COMPONENT_TYPE = "Component Type"; + public static final String HEADER_ERROR = "Errors"; + public static final String HEADER_STREAM_ID = "Stream Id"; + public static final String HEADER_NETWORKER_CONNECTION = "NetworkConnection"; + + + public static final String CLUSTER = "cluster"; + public static final String HOST = "host"; + public static final String IP = "IP"; + public static final String PORT = "port"; + public static final String TOPOLOGY = "topology"; + public static final String TOPOLOGY_NAME = "topologyName"; + public static final String SUPERVISOR = "supervisor"; + public static final String DIR = "dir"; + public static final String WOKER = "worker"; + public static final String COMPONENT = "component"; + public static final String JSTACK = "jstack"; + public static final String METRICS = "metrics"; + public static final String WINDOW = "window"; + public static final String BOLT = "bolt"; + public static final String SPOUT = "spout"; + public static final String LOG_NAME = "logName"; + public static final String LOG_SERVER_PORT = "logServerPort"; + public static final String POS = "pos"; + + public static final Map pageGeneratos = + new HashMap(); + static { + pageGeneratos.put(PAGE_TYPE_CLUSTER, new ClusterPage()); + pageGeneratos.put(PAGE_TYPE_SUPERVISOR, new SupervisorPage()); + pageGeneratos.put(PAGE_TYPE_TOPOLOGY, new TopologyPage()); + pageGeneratos.put(PAGE_TYPE_COMPONENT, new ComponentPage()); + pageGeneratos.put(PAGE_TYPE_CONF, new ConfPage()); + pageGeneratos.put(PAGE_TYPE_JSTACK, new JStackPage()); + pageGeneratos.put(PAGE_TYPE_LISTLOG, new ListLogsPage()); + pageGeneratos.put(PAGE_TYPE_NETTY, new NettyPage()); + + } + +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/UIMetrics.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/UIMetrics.java new file mode 100755 index 000000000..11e6f1bbc --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/UIMetrics.java @@ -0,0 +1,62 @@ +/** + * 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.ui; + +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.metric.SimpleJStormMetric; +import com.alibaba.jstorm.utils.JStormUtils; + +public class UIMetrics extends SimpleJStormMetric { + + private static final long serialVersionUID = -5726788610247404713L; + private static final Logger LOG = LoggerFactory.getLogger(UIMetrics.class); + + protected static UIMetrics instance = null; + + + public static UIMetrics mkInstance() { + synchronized (UIMetrics.class) { + if (instance == null) { + instance = new UIMetrics(); + LOG.info("Make instance"); + } + + return instance; + } + } + + protected UIMetrics() { + super(); + } + + static { + instance = mkInstance(); + UIUtils.scheduExec.scheduleAtFixedRate(instance, 60, 60, TimeUnit.SECONDS); + LOG.info("Start scheduler"); + } + + public static void main(String[] args) { + UIMetrics test = mkInstance(); + test.run(); + JStormUtils.sleepMs(100 * 1000); + } +} 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 old mode 100644 new mode 100755 index 11615376d..96b8672da --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/UIUtils.java +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/UIUtils.java @@ -1,578 +1,549 @@ -package com.alibaba.jstorm.ui; - -import java.io.File; -import java.io.FileInputStream; -import java.io.FileNotFoundException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.TreeMap; - -import javax.faces.context.FacesContext; - -import org.apache.commons.lang.StringUtils; -import org.apache.thrift7.protocol.TBinaryProtocol; -import org.apache.thrift7.transport.TFramedTransport; -import org.apache.thrift7.transport.TSocket; -import org.apache.thrift7.transport.TTransportException; -import org.yaml.snakeyaml.Yaml; - -import backtype.storm.Config; -import backtype.storm.generated.Bolt; -import backtype.storm.generated.ClusterSummary; -import backtype.storm.generated.ErrorInfo; -import backtype.storm.generated.Nimbus.Client; -import backtype.storm.generated.SpoutSpec; -import backtype.storm.generated.StormTopology; -import backtype.storm.generated.SupervisorSummary; -import backtype.storm.generated.TaskStats; -import backtype.storm.generated.TaskSummary; -import backtype.storm.generated.TopologyInfo; -import backtype.storm.generated.TopologySummary; -import backtype.storm.generated.WorkerSummary; -import backtype.storm.utils.NimbusClient; -import backtype.storm.utils.Utils; - -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.common.stats.StatBuckets; -import com.alibaba.jstorm.common.stats.StaticsType; -import com.alibaba.jstorm.ui.model.ClusterSumm; -import com.alibaba.jstorm.ui.model.ComponentTask; -import com.alibaba.jstorm.ui.model.Components; -import com.alibaba.jstorm.ui.model.SupervisorSumm; -import com.alibaba.jstorm.ui.model.TopologySumm; -import com.alibaba.jstorm.utils.JStormUtils; -import com.alibaba.jstorm.utils.NetWorkUtils; - -public class UIUtils { - - private static int maxErrortime = 1800; - private static int maxErrornum = 200; - - public static String getWindow(FacesContext ctx) { - String window = null; - if (ctx.getExternalContext().getRequestParameterMap().get("window") != null) { - window = (String) ctx.getExternalContext().getRequestParameterMap() - .get("window"); - } - - return StatBuckets.getTimeKey(window); - } - - public static List nimbusClientandConn(String host, Integer port) - throws TTransportException { - TSocket ts = new TSocket(host, port); - TFramedTransport tt = new TFramedTransport(ts); - TBinaryProtocol prot = new TBinaryProtocol(tt); - Client nc = new Client(prot); - ts.open(); - List l = new ArrayList(); - l.add(nc); - l.add(tt); - return l; - } - - public static final String SPOUT_STR = "spout"; - public static final String BOLT_STR = "bolt"; - - public static String componentType(StormTopology topology, String id) { - Map bolts = topology.get_bolts(); - Map spouts = topology.get_spouts(); - String type = ""; - if (bolts.containsKey(id)) { - type = BOLT_STR; - } else if (spouts.containsKey(id)) { - type = SPOUT_STR; - } - return type; - } - - private List addPairs(ArrayList p1, ArrayList p2) { - if (p1 == null || p2 == null) { - return null; - } else { - int p1Size = p1.size(); - int p2Size = p2.size(); - if (p1Size != p2Size) { - return null; - } else { - List rtn = new ArrayList(); - for (int i = 0; i < p1Size; i++) { - rtn.set(i, p1.get(i) + p2.get(i)); - } - return rtn; - } - } - - } - - public static void addTask(Map> taskMap, - TaskSummary t, String componentId) { - - List taskList = taskMap.get(componentId); - - if (taskList == null) { - taskList = new ArrayList(); - taskMap.put(componentId, taskList); - } - - taskList.add(t); - } - - /** - * Merge stream value - * - * @param rawMap - * Map> rawMap - * @param keySample - * @param zeroValue - * 0 Value - * @return Map - */ - public static Map mergeStream( - Map> rawMap, V zeroValue) { - Map ret = new HashMap(); - - for (Entry> rawEntry : rawMap.entrySet()) { - String window = rawEntry.getKey(); - Map streamMap = rawEntry.getValue(); - - V retValue = zeroValue; - - if (streamMap != null) { - for (Entry streamEntry : streamMap.entrySet()) { - K entry = streamEntry.getKey(); - V counter = streamEntry.getValue(); - - retValue = (V) JStormUtils.add(retValue, counter); - } - } - - ret.put(window, retValue); - } - - return ret; - } - - public static Map> mergeStream( - List taskSummaries, String window) { - - Map> ret = new HashMap>(); - - List emitted = new ArrayList(); - List sendTps = new ArrayList(); - List recvTps = new ArrayList(); - List acked = new ArrayList(); - List failed = new ArrayList(); - List process = new ArrayList(); - - ret.put(StaticsType.emitted, emitted); - ret.put(StaticsType.send_tps, sendTps); - ret.put(StaticsType.recv_tps, recvTps); - ret.put(StaticsType.acked, acked); - ret.put(StaticsType.failed, failed); - ret.put(StaticsType.process_latencies, process); - - for (TaskSummary taskSummary : taskSummaries) { - TaskStats taskStats = taskSummary.get_stats(); - - if (taskStats == null) { - continue; - } - - Map emittedMap = mergeStream(taskStats.get_emitted(), - Long.valueOf(0)); - emitted.add(emittedMap.get(window)); - - Map rendTpsMap = mergeStream( - taskStats.get_send_tps(), Double.valueOf(0)); - sendTps.add(rendTpsMap.get(window)); - - Map recvTpsMap = mergeStream( - taskStats.get_recv_tps(), Double.valueOf(0)); - recvTps.add(recvTpsMap.get(window)); - - Map ackedMap = mergeStream(taskStats.get_acked(), - Long.valueOf(0)); - acked.add(ackedMap.get(window)); - - Map failedMap = mergeStream(taskStats.get_failed(), - Long.valueOf(0)); - failed.add(failedMap.get(window)); - - Map processMap = mergeStream( - taskStats.get_process_ms_avg(), Double.valueOf(0)); - process.add(processMap.get(window)); - } - - return ret; - - } - - public static Map mergeTasks( - List taskSummaries, String window) { - Map ret = new HashMap(); - - Map> mergedStreamTasks = mergeStream( - taskSummaries, window); - for (Entry> entry : mergedStreamTasks - .entrySet()) { - StaticsType type = entry.getKey(); - List valueList = entry.getValue(); - - Object valueSum = JStormUtils.mergeList(valueList); - - ret.put(type, valueSum); - } - - // special handle process - Object value = ret.get(StaticsType.process_latencies); - if (value != null && taskSummaries.size() > 0) { - Double process = (Double) value; - process = process / taskSummaries.size(); - ret.put(StaticsType.process_latencies, process); - } - - return ret; - } - - public static Components getComponent(List taskSummaries, - String componentId, String type, String window) { - - Map staticsType = UIUtils.mergeTasks( - taskSummaries, window); - - Components component = new Components(); - component.setType(type); - component.setComponetId(componentId); - component.setParallelism(String.valueOf(taskSummaries.size())); - component.setValues(staticsType); - - return component; - } - - /** - * Convert thrift TaskSummary to UI bean ComponentTask - * - * @param summ - * @return - */ - public static ComponentTask getComponentTask(TaskSummary task, - String topologyid) { - - ComponentTask componentTask = new ComponentTask(); - - componentTask.setComponentid(task.get_component_id()); - componentTask.setTaskid(String.valueOf(task.get_task_id())); - componentTask.setHost(task.get_host()); - componentTask.setPort(String.valueOf(task.get_port())); - - - componentTask.setStatus(task.get_status()); - if (componentTask.getStatus() == null) { - // This is for old jstorm version - componentTask.setStatus(ConfigExtension.TASK_STATUS_ACTIVE); - } - - if (componentTask.getStatus().equals(ConfigExtension.TASK_STATUS_ACTIVE)) { - componentTask.setUptime(StatBuckets.prettyUptimeStr(task - .get_uptime_secs())); - componentTask.setLastErr(UIUtils.getTaskError(task.get_errors())); - } - - componentTask.setIp(NetWorkUtils.host2Ip(task.get_host())); - - componentTask.setTopologyid(topologyid); - - return componentTask; - } - - public static List getTaskList( - List taskSummaries, String componentId) { - List ret = new ArrayList(); - - for (TaskSummary task : taskSummaries) { - if (componentId.equals(task.get_component_id())) { - ret.add(task); - } - } - - return ret; - } - - public static String mostRecentError(List summarys) { - String rtn = ""; - int summarysSzie = 0; - if (summarys != null) { - summarysSzie = summarys.size(); - } - for (int i = 0; i < summarysSzie; i++) { - List einfos = summarys.get(i).get_errors(); - rtn += getTaskError(einfos); - } - - return rtn; - } - - public static String getTaskError(List errList) { - if (errList == null) { - return ""; - } - - List errors = new ArrayList(); - - for (ErrorInfo einfo : errList) { - long current = System.currentTimeMillis() / 1000; - - //shorten the most recent time for "queue is full" error - int maxTime = JStormUtils.MIN_30; - if (einfo.get_error().indexOf("queue is full") != -1) - maxTime = JStormUtils.MIN_1*3; - else if (einfo.get_error().indexOf("is dead on") != -1) - maxTime = JStormUtils.DAY_1*3; - - if (current - einfo.get_error_time_secs() < maxTime) { - errors.add(einfo.get_error()); - } - } - - String rtn = ""; - int size = 0; - for (String e : errors) { - if (size >= maxErrornum) { - break; - } - rtn += e + ";"; - size++; - } - return rtn; - } - - /** - * Convert thrift TopologySummary to UI bean TopologySumm - * - * @param ts - * @return - */ - public static List topologySummary(List ts) { - - List tsumm = new ArrayList(); - if (ts != null) { - for (TopologySummary t : ts) { - - TopologySumm topologySumm = new TopologySumm(); - topologySumm.setTopologyId(t.get_id()); - topologySumm.setTopologyName(t.get_name()); - - topologySumm.setStatus(t.get_status()); - topologySumm.setUptime(StatBuckets.prettyUptimeStr(t - .get_uptime_secs())); - - topologySumm.setNumWorkers(String.valueOf(t.get_num_workers())); - topologySumm.setNumTasks(String.valueOf(t.get_num_tasks())); - - topologySumm.setErrorInfo(t.get_error_info()); - tsumm.add(topologySumm); - } - } - return tsumm; - } - - /** - * Convert thrift TopologyInfo to UI bean TopologySumm - * - * @param summ - * @return - */ - public static List topologySummary(TopologyInfo t) { - - List workers = t.get_workers(); - int taskNum = 0; - int memNum = 0; - for (WorkerSummary worker : workers) { - taskNum += worker.get_tasks_size(); - } - - List tsumm = new ArrayList(); - - // TopologySumm ts = new TopologySumm(summ.get_name(), summ.get_id(), - // summ.get_status(), StatBuckets.prettyUptimeStr(summ - // .get_uptime_secs()), String.valueOf(workers.size()), - // String.valueOf(taskSize), summ.get_uptime_secs()); - - TopologySumm topologySumm = new TopologySumm(); - topologySumm.setTopologyId(t.get_id()); - topologySumm.setTopologyName(t.get_name()); - topologySumm.setStatus(t.get_status()); - topologySumm - .setUptime(StatBuckets.prettyUptimeStr(t.get_uptime_secs())); - - topologySumm.setNumWorkers(String.valueOf(workers.size())); - topologySumm.setNumTasks(String.valueOf(taskNum)); - - tsumm.add(topologySumm); - return tsumm; - } - - /** - * Connvert thrift ClusterSummary to UI bean ClusterSumm - * - * @param summ - * @return - */ - public static List clusterSummary(ClusterSummary summ, - NimbusClient client, Map conf) throws Exception { - // "Supervisors" "Used slots" "Free slots" "Total slots" "Running task" - List sups = summ.get_supervisors(); - int supSize = 0; - - int totalMemSlots = 0; - int useMemSlots = 0; - int freeMemSlots = 0; - - int totalPortSlots = 0; - int usePortSlots = 0; - int freePortSlots = 0; - - if (sups != null) { - supSize = sups.size(); - for (SupervisorSummary ss : sups) { - - totalPortSlots += ss.get_num_workers(); - usePortSlots += ss.get_num_used_workers(); - } - - freeMemSlots = totalMemSlots - useMemSlots; - freePortSlots = totalPortSlots - usePortSlots; - } - - // "Running tasks" - int totalTasks = 0; - List topos = summ.get_topologies(); - if (topos != null) { - int topoSize = topos.size(); - for (int j = 0; j < topoSize; j++) { - totalTasks += topos.get(j).get_num_tasks(); - } - - } - - String nimbustime = StatBuckets.prettyUptimeStr(summ - .get_nimbus_uptime_secs()); - - List clusumms = new ArrayList(); - - ClusterSumm clusterSumm = new ClusterSumm(); - String master = client.getMasterHost(); - - if (master.contains(":")) { - 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); - clusterSumm.setNimbusLogPort(String.valueOf(port)); - clusterSumm.setNimbusUptime(nimbustime); - clusterSumm.setSupervisorNum(String.valueOf(supSize)); - clusterSumm.setRunningTaskNum(String.valueOf(totalTasks)); - - clusterSumm.setTotalPortSlotNum(String.valueOf(totalPortSlots)); - clusterSumm.setUsedPortSlotNum(String.valueOf(usePortSlots)); - clusterSumm.setFreePortSlotNum(String.valueOf(freePortSlots)); - - clusterSumm.setVersion(summ.get_version()); - - clusumms.add(clusterSumm); - return clusumms; - } - - /** - * Convert thrift SupervisorSummary to UI bean SupervisorSumm - * - * @param ss - * @return - */ - public static List supervisorSummary( - List ss) { - // uptime host slots usedslots - - List ssumm = new ArrayList(); - - if (ss == null) { - ss = new ArrayList(); - } - - for (SupervisorSummary s : ss) { - SupervisorSumm ssum = new SupervisorSumm(s); - - ssumm.add(ssum); - } - - return ssumm; - } - - public static Map readUiConfig() { - Map ret = Utils.readStormConfig(); - String curDir = System.getProperty("user.home"); - String confPath = curDir + File.separator + ".jstorm" + File.separator - + "storm.yaml"; - File file = new File(confPath); - if (file.exists()) { - - FileInputStream fileStream; - try { - fileStream = new FileInputStream(file); - Yaml yaml = new Yaml(); - - Map clientConf = (Map) yaml.load(fileStream); - - if (clientConf != null) { - ret.putAll(clientConf); - } - } catch (FileNotFoundException e) { - // TODO Auto-generated catch block - } - - } - if (ret.containsKey(Config.NIMBUS_HOST) == false) { - ret.put(Config.NIMBUS_HOST, "localhost"); - - } - return ret; - } - - public static double getDoubleValue(Double value) { - double ret = (value != null ? value.doubleValue() : 0.0); - return ret; - } - - public static void getClusterInfoByName(Map conf, String clusterName) { - List uiClusters = ConfigExtension.getUiClusters(conf); - Map cluster = ConfigExtension.getUiClusterInfo( - uiClusters, clusterName); - - conf.put(Config.STORM_ZOOKEEPER_ROOT, - ConfigExtension.getUiClusterZkRoot(cluster)); - conf.put(Config.STORM_ZOOKEEPER_SERVERS, - ConfigExtension.getUiClusterZkServers(cluster)); - conf.put(Config.STORM_ZOOKEEPER_PORT, - ConfigExtension.getUiClusterZkPort(cluster)); - } - - public static NimbusClient getNimbusClient(Map conf, String clusterName) throws Exception{ - if(StringUtils.isBlank(clusterName) == false) { - getClusterInfoByName(conf, clusterName); - } - return NimbusClient.getConfiguredClient(conf); - } - - public static void main(String[] args) { - } -} +/** + * 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.ui; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; + +import org.apache.commons.lang.StringUtils; +import org.yaml.snakeyaml.Yaml; + +import backtype.storm.Config; +import backtype.storm.generated.ComponentSummary; +import backtype.storm.generated.ErrorInfo; +import backtype.storm.generated.MetricInfo; +import backtype.storm.generated.MetricWindow; +import backtype.storm.generated.TopologyInfo; +import backtype.storm.generated.TopologySummary; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.common.metric.MetricRegistry; +import com.alibaba.jstorm.common.metric.window.StatBuckets; +import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.ui.model.ColumnData; +import com.alibaba.jstorm.ui.model.LinkData; +import com.alibaba.jstorm.ui.model.PageGenerator; +import com.alibaba.jstorm.ui.model.PageIndex; +import com.alibaba.jstorm.ui.model.PageGenerator.Output; +import com.alibaba.jstorm.ui.model.TableData; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.NetWorkUtils; + +public class UIUtils { + public static ScheduledExecutorService scheduExec = Executors.newScheduledThreadPool(4); + + public static Long ONE_TABLE_PAGE_SIZE = null; + + public static Map readUiConfig() { + Map ret = Utils.readStormConfig(); + String curDir = System.getProperty("user.home"); + String confPath = + curDir + File.separator + ".jstorm" + File.separator + + "storm.yaml"; + File file = new File(confPath); + if (file.exists()) { + + FileInputStream fileStream; + try { + fileStream = new FileInputStream(file); + Yaml yaml = new Yaml(); + + Map clientConf = (Map) yaml.load(fileStream); + + if (clientConf != null) { + ret.putAll(clientConf); + } + + if (ONE_TABLE_PAGE_SIZE == null) { + ONE_TABLE_PAGE_SIZE = ConfigExtension.getUiOneTablePageSize(clientConf); + } + } catch (FileNotFoundException e) { + // TODO Auto-generated catch block + } + + } + if (ret.containsKey(Config.NIMBUS_HOST) == false) { + ret.put(Config.NIMBUS_HOST, "localhost"); + + } + return ret; + } + + public static String getHostIp(String ipPort) { + + if (ipPort.contains(":")) { + String firstPart = ipPort.substring(0, ipPort.indexOf(":")); + String lastPart = ipPort.substring(ipPort.indexOf(":")); + + return NetWorkUtils.ip2Host(firstPart) + lastPart; + } else { + return NetWorkUtils.ip2Host(ipPort); + } + + } + + public static String getHost(String ipPort) { + if (ipPort.contains(":")) { + String firstPart = ipPort.substring(0, ipPort.indexOf(":")); + + return NetWorkUtils.ip2Host(firstPart); + } else { + return NetWorkUtils.ip2Host(ipPort); + } + } + + public static Set getKeys(Collection metrics) { + Set ret = new TreeSet(); + for (MetricInfo metric : metrics) { + Set oneKeys = metric.get_baseMetric().keySet(); + ret.addAll(oneKeys); + } + + return ret; + } + + public static String getValue(MetricInfo metric, String key, Integer window) { + MetricWindow metricWindow = metric.get_baseMetric().get(key); + if (metricWindow == null) { + return String.valueOf("000.000"); + } + + Double value = metricWindow.get_metricWindow().get(window); + if (value == null) { + return String.valueOf("000.000"); + } + + return JStormUtils.formatSimpleDouble(value); + } + + public static void checkKey(List ret, Set temp, String key) { + if (temp.contains(key)) { + ret.add(key); + temp.remove(key); + } + return; + } + + public static List getSortedKeys(Set keys) { + List ret = new ArrayList(); + + Set temp = new TreeSet(); + temp.addAll(keys); + + checkKey(ret, temp, MetricDef.CPU_USED_RATIO); + checkKey(ret, temp, MetricDef.MEMORY_USED); + checkKey(ret, temp, MetricDef.SEND_TPS); + checkKey(ret, temp, MetricDef.RECV_TPS); + + checkKey(ret, temp, MetricDef.EMMITTED_NUM); + checkKey(ret, temp, MetricDef.ACKED_NUM); + checkKey(ret, temp, MetricDef.FAILED_NUM); + checkKey(ret, temp, MetricDef.PROCESS_LATENCY); + + ret.addAll(temp); + return ret; + } + + public static TableData getMetricTable(MetricInfo metricInfo, Integer window) { + TableData table = new TableData(); + List headers = table.getHeaders(); + List> lines = table.getLines(); + // table.setName(UIDef.TOPOLOGY.toUpperCase()); + + Map baseMetric = metricInfo.get_baseMetric(); + List keys = getSortedKeys(baseMetric.keySet()); + + headers.addAll(keys); + Map line = + getMetricLine(metricInfo, headers, window); + lines.add(line); + + return table; + } + + public static Map getMetricLine(MetricInfo metricInfo, + List headers, Integer window) { + Map line = new HashMap(); + + for (String key : headers) { + String value = getValue(metricInfo, key, window); + + ColumnData columnData = new ColumnData(); + columnData.addText(value); + + line.put(key, columnData); + } + + return line; + } + + public static void complementingTable(TableData table) { + List headers = table.getHeaders(); + List> lines = table.getLines(); + + for (Map line : lines) { + for (String header : headers) { + ColumnData item = line.get(header); + if (item == null) { + item = new ColumnData(); + item.addText("000.000"); + line.put(header, item); + } + } + } + + return; + } + + public static TableData errorTable(String erro) { + TableData table = new TableData(); + List headers = table.getHeaders(); + List> lines = table.getLines(); + + headers.add("Error"); + + Map line = new HashMap(); + lines.add(line); + + ColumnData columnData = new ColumnData(); + columnData.addText(erro); + line.put("Error", columnData); + + return table; + } + + public static PageGenerator.Output getErrorInfo(Exception e) { + String errMsg = JStormUtils.getErrorInfo(e); + Output ret = new Output(); + ret.tables = new ArrayList(); + ret.rawData = + "!!!!!!!!!!!!!!!! Occur Exception:" + e.getMessage() + + " !!!!!!!!!\r\n\r\nPlease refresh once again\r\n" + errMsg; + + return ret; + } + + public static double getDoubleValue(Double value) { + double ret = (value != null ? value.doubleValue() : 0.0); + return ret; + } + + public static void getClusterInfoByName(Map conf, String clusterName) { + List uiClusters = ConfigExtension.getUiClusters(conf); + Map cluster = ConfigExtension.getUiClusterInfo(uiClusters, clusterName); + + conf.put(Config.STORM_ZOOKEEPER_ROOT, + ConfigExtension.getUiClusterZkRoot(cluster)); + conf.put(Config.STORM_ZOOKEEPER_SERVERS, + ConfigExtension.getUiClusterZkServers(cluster)); + conf.put(Config.STORM_ZOOKEEPER_PORT, + ConfigExtension.getUiClusterZkPort(cluster)); + } + + public static TableData getTopologyTable( + List topologySummaries, + Map paramMap) { + TableData table = new TableData(); + List headers = table.getHeaders(); + List> lines = table.getLines(); + table.setName(UIDef.TOPOLOGY.toUpperCase()); + + headers.add(UIDef.HEADER_TOPOLOGY_NAME); + headers.add(UIDef.HEADER_TOPOLOGY_ID); + headers.add(UIDef.HEADER_STATUS); + headers.add(UIDef.HEADER_UPTIME); + headers.add(UIDef.HEADER_WORKER_NUM); + headers.add(UIDef.HEADER_TASK_NUM); + headers.add(UIDef.HEADER_CONF); + headers.add(UIDef.HEADER_ERROR); + + if (topologySummaries == null) { + return table; + } + + for (TopologySummary topologySummary : topologySummaries) { + Map line = new HashMap(); + lines.add(line); + + ColumnData topologyNameColumn = new ColumnData(); + LinkData linkData = new LinkData(); + topologyNameColumn.addLinkData(linkData); + line.put(UIDef.HEADER_TOPOLOGY_NAME, topologyNameColumn); + + linkData.setUrl(UIDef.LINK_WINDOW_TABLE); + linkData.setText(topologySummary.get_name()); + linkData.addParam(UIDef.CLUSTER, paramMap.get(UIDef.CLUSTER)); + linkData.addParam(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_TOPOLOGY); + linkData.addParam(UIDef.TOPOLOGY, topologySummary.get_id()); + + ColumnData topologyIdColumn = new ColumnData(); + topologyIdColumn.addText(topologySummary.get_id()); + line.put(UIDef.HEADER_TOPOLOGY_ID, topologyIdColumn); + + ColumnData statusColumn = new ColumnData(); + statusColumn.addText(topologySummary.get_status()); + line.put(UIDef.HEADER_STATUS, statusColumn); + + ColumnData uptimeColumn = new ColumnData(); + int uptime = topologySummary.get_uptime_secs(); + uptimeColumn.addText(StatBuckets.prettyUptimeStr(uptime)); + line.put(UIDef.HEADER_UPTIME, uptimeColumn); + + ColumnData workerNumColumn = new ColumnData(); + workerNumColumn.addText(String.valueOf(topologySummary + .get_num_workers())); + line.put(UIDef.HEADER_WORKER_NUM, workerNumColumn); + + ColumnData taskNumColumn = new ColumnData(); + taskNumColumn.addText(String.valueOf(topologySummary + .get_num_tasks())); + line.put(UIDef.HEADER_TASK_NUM, taskNumColumn); + + ColumnData confColumn = new ColumnData(); + LinkData confLink = new LinkData(); + confColumn.addLinkData(confLink); + line.put(UIDef.HEADER_CONF, confColumn); + + confLink.setUrl(UIDef.LINK_TABLE_PAGE); + confLink.setText(UIDef.HEADER_CONF.toLowerCase()); + confLink.addParam(UIDef.CLUSTER, paramMap.get(UIDef.CLUSTER)); + confLink.addParam(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_CONF); + confLink.addParam(UIDef.CONF_TYPE, UIDef.CONF_TYPE_TOPOLOGY); + confLink.addParam(UIDef.TOPOLOGY, topologySummary.get_id()); + + ColumnData errorColumn = new ColumnData(); + String errorInfo = String.valueOf(topologySummary.get_error_info()); + errorColumn.addText(errorInfo); + line.put(UIDef.HEADER_ERROR, errorColumn); + } + + return table; + } + + public static TableData getComponentTable(TopologyInfo topologyInfo, + List componentSummaries, + Map componentMetrics, + Map paramMap, Integer window) { + TableData table = new TableData(); + List headers = table.getHeaders(); + List> lines = table.getLines(); + + List keys = + UIUtils.getSortedKeys(UIUtils.getKeys(componentMetrics.values())); + + headers.add(UIDef.HEADER_COMPONENT_NAME); + headers.add(UIDef.HEADER_TASK_NUM); + headers.addAll(keys); + headers.add(UIDef.HEADER_ERROR); + + for (ComponentSummary componentSummary : componentSummaries) { + Map line = new HashMap(); + lines.add(line); + + String name = componentSummary.get_name(); + + ColumnData nameColumnData = new ColumnData(); + LinkData linkData = new LinkData(); + nameColumnData.addLinkData(linkData); + line.put(UIDef.HEADER_COMPONENT_NAME, nameColumnData); + + linkData.setUrl(UIDef.LINK_WINDOW_TABLE); + linkData.setText(name); + linkData.addParam(UIDef.CLUSTER, paramMap.get(UIDef.CLUSTER)); + linkData.addParam(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_COMPONENT); + linkData.addParam(UIDef.TOPOLOGY, topologyInfo.get_topology() + .get_id()); + linkData.addParam(UIDef.COMPONENT, name); + + ColumnData taskNumColumn = new ColumnData(); + taskNumColumn.addText(String.valueOf(componentSummary + .get_parallel())); + line.put(UIDef.HEADER_TASK_NUM, taskNumColumn); + + ColumnData errColumn = new ColumnData(); + line.put(UIDef.HEADER_ERROR, errColumn); + List errs = componentSummary.get_errors(); + if (errs == null) { + errColumn.addText(""); + + } else { + for (ErrorInfo err : errs) { + errColumn.addText(err.get_error() + "\r\n"); + } + + } + + MetricInfo metric = componentMetrics.get(name); + if (metric == null) { + continue; + } + + for (String key : keys) { + String value = UIUtils.getValue(metric, key, window); + ColumnData columnData = new ColumnData(); + columnData.addText(value); + line.put(key, columnData); + + } + + } + + return table; + } + + public static ColumnData getConnectionColumnData(Map paramMap, String connectionName) { + ColumnData columnData = new ColumnData(); + LinkData linkData = new LinkData(); + columnData.addLinkData(linkData); + + linkData.setUrl(UIDef.LINK_WINDOW_TABLE); + linkData.setText(connectionName); + Map linkDataParam = new HashMap(); + linkData.setParamMap(linkDataParam); + linkDataParam.putAll(paramMap); + linkDataParam.remove(UIDef.POS); + + + try { + int pos = connectionName.indexOf(":"); + if (pos > 0) { + String source = connectionName.substring(0, pos); + linkDataParam.put(UIDef.HOST, source); + } + + } catch (Exception e) { + } + + return columnData; + } + + public static TableData getNettyMetricsTable(Map metrics, + Integer window, Map paramMap) { + + TableData nettyTable = new TableData(); + nettyTable.setName(MetricDef.NETTY + " Metrics " + StatBuckets.getShowTimeStr(window)); + + List nettyHeaders = nettyTable.getHeaders(); + nettyHeaders.add(UIDef.HEADER_NETWORKER_CONNECTION); + nettyHeaders.addAll(getKeys(metrics.values())); + + List> nettyLines = nettyTable.getLines(); + for (Entry entry : metrics.entrySet()) { + String connectionName = entry.getKey(); + MetricInfo metricInfo = entry.getValue(); + + Map line = getMetricLine(metricInfo, nettyHeaders, window); + + ColumnData columnData = getConnectionColumnData(paramMap, connectionName); + line.put(UIDef.HEADER_NETWORKER_CONNECTION, columnData); + + nettyLines.add(line); + } + + return nettyTable; + } + + public static List getWorkerMetricsTable(Map metrics, + Integer window, Map paramMap) { + List ret = new ArrayList(); + TableData table = new TableData(); + ret.add(table); + List headers = table.getHeaders(); + List> lines = table.getLines(); + table.setName("Worker " + UIDef.METRICS); + + List keys = getSortedKeys(UIUtils.getKeys(metrics.values())); + headers.add(UIDef.PORT); + headers.add(MetricDef.NETTY); + headers.addAll(keys); + + TreeMap tmpMap = new TreeMap(); + tmpMap.putAll(metrics); + Map showMap = new TreeMap(); + + long pos = JStormUtils.parseLong(paramMap.get(UIDef.POS), 0); + long index = 0; + for (Entry entry : tmpMap.entrySet()) { + if (index < pos) { + index ++; + }else if (pos <= index && index < pos + UIUtils.ONE_TABLE_PAGE_SIZE) { + showMap.put(entry.getKey(), entry.getValue()); + index++; + }else { + break; + } + } + + for (Entry entry : showMap.entrySet()) { + Map line = new HashMap(); + lines.add(line); + + String slot = entry.getKey(); + MetricInfo metric = entry.getValue(); + + ColumnData slotColumn = new ColumnData(); + slotColumn.addText(slot); + line.put(UIDef.PORT, slotColumn); + + ColumnData nettyColumn = new ColumnData(); + line.put(MetricDef.NETTY, nettyColumn); + + if (StringUtils.isBlank(paramMap.get(UIDef.TOPOLOGY) ) ) { + nettyColumn.addText(MetricDef.NETTY); + }else { + LinkData linkData = new LinkData(); + nettyColumn.addLinkData(linkData); + + linkData.setUrl(UIDef.LINK_WINDOW_TABLE); + linkData.setText(MetricDef.NETTY); + linkData.addParam(UIDef.CLUSTER, paramMap.get(UIDef.CLUSTER)); + linkData.addParam(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_NETTY); + linkData.addParam(UIDef.TOPOLOGY, paramMap.get(UIDef.TOPOLOGY)); + } + + + for (String key : keys) { + String value = UIUtils.getValue(metric, key, window); + ColumnData valueColumn = new ColumnData(); + valueColumn.addText(value); + line.put(key, valueColumn); + } + } + + return ret; + } + + + + public static void main(String[] args) { + } +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ClusterInfo.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ClusterInfo.java deleted file mode 100644 index b1201e04d..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ClusterInfo.java +++ /dev/null @@ -1,45 +0,0 @@ -package com.alibaba.jstorm.ui.model; - -import java.util.List; -import java.io.Serializable; - -public class ClusterInfo implements Serializable { - private static final long serialVersionUID = -7966384220162644896L; - - private String clusterName; - private String zkRoot; - private Integer zkPort; - private List zkServers; - - public String getClusterName() { - return clusterName; - } - - public void setClusterName(String name) { - this.clusterName = name; - } - - public String getZkRoot() { - return zkRoot; - } - - public void setZkRoot(String zkRoot) { - this.zkRoot = zkRoot; - } - - public Integer getZkPort() { - return zkPort; - } - - public void setZkPort(Integer zkPort) { - this.zkPort = zkPort; - } - - public List getZkServers() { - return zkServers; - } - - public void setZkServers(List zkServers) { - this.zkServers = zkServers; - } -} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ClusterSumm.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ClusterSumm.java deleted file mode 100644 index 49c50e790..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ClusterSumm.java +++ /dev/null @@ -1,104 +0,0 @@ -package com.alibaba.jstorm.ui.model; - -/** - * mainpage:ClusterSummary - * @author xin.zhou/zhiyuan.ls - * - */ -import java.io.Serializable; - -public class ClusterSumm implements Serializable { - private static final long serialVersionUID = -7936384220562644886L; - - private String nimbusIp; - private String nimbusLogPort; - private String nimbusHostname; - private String nimbusUptime; - private String supervisorNum; - private String runningTaskNum; - private String totalPortSlotNum; - private String usedPortSlotNum; - private String freePortSlotNum; - private String version; - - public String getNimbusIp() { - return nimbusIp; - } - - public void setNimbusIp(String nimbusIp) { - this.nimbusIp = nimbusIp; - } - - public String getNimbusLogPort() { - return nimbusLogPort; - } - - public void setNimbusLogPort(String nimbusLogPort) { - this.nimbusLogPort = nimbusLogPort; - } - - public String getNimbusHostname() { - return nimbusHostname; - } - - public void setNimbusHostname(String nimbusHostname) { - this.nimbusHostname = nimbusHostname; - } - - public String getNimbusUptime() { - return nimbusUptime; - } - - public void setNimbusUptime(String nimbusUptime) { - this.nimbusUptime = nimbusUptime; - } - - public String getSupervisorNum() { - return supervisorNum; - } - - public void setSupervisorNum(String supervisorNum) { - this.supervisorNum = supervisorNum; - } - - public String getRunningTaskNum() { - return runningTaskNum; - } - - public void setRunningTaskNum(String runningTaskNum) { - this.runningTaskNum = runningTaskNum; - } - - public String getTotalPortSlotNum() { - return totalPortSlotNum; - } - - public void setTotalPortSlotNum(String totalPortSlotNum) { - this.totalPortSlotNum = totalPortSlotNum; - } - - public String getUsedPortSlotNum() { - return usedPortSlotNum; - } - - public void setUsedPortSlotNum(String usedPortSlotNum) { - this.usedPortSlotNum = usedPortSlotNum; - } - - public String getFreePortSlotNum() { - return freePortSlotNum; - } - - public void setFreePortSlotNum(String freePortSlotNum) { - this.freePortSlotNum = freePortSlotNum; - } - - public String getVersion() { - return version; - } - - public void setVersion(String version) { - this.version = version; - } - -} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ColumnData.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ColumnData.java new file mode 100755 index 000000000..659ea56b0 --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ColumnData.java @@ -0,0 +1,59 @@ +package com.alibaba.jstorm.ui.model; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; + +import com.alibaba.jstorm.utils.JStormUtils; + +public class ColumnData implements Serializable { + List texts = new ArrayList(); + List links = new ArrayList(); + + public List getTexts() { + return texts; + } + + public void setTexts(List texts) { + this.texts = texts; + } + + public List getLinks() { + return links; + } + + public void setLinks(List links) { + this.links = links; + } + + public void addText(String text) { + try { + Double value = Double.valueOf(text); + + if(Math.abs(value - Math.round(value)) < 0.001){ + texts.add(text); + }else { + texts.add(JStormUtils.formatSimpleDouble(value)); + } + + + }catch(Exception e) { + texts.add(text); + } + + } + + public void addLinkData(LinkData linkData) { + links.add(linkData); + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } + +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ComponentInput.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ComponentInput.java deleted file mode 100644 index 4d45e83a9..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ComponentInput.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * To change this template, choose Tools | Templates - * and open the template in the editor. - */ -package com.alibaba.jstorm.ui.model; - -import java.io.Serializable; - -import backtype.storm.generated.GlobalStreamId; - -import com.alibaba.jstorm.utils.JStormUtils; - -/** - * - * @author xin.zhou/Longda - */ -public class ComponentInput implements Serializable { - - private static final long serialVersionUID = -1415620236852453926L; - - private String componentId; - private String stream; - private String recvTps; - private String process; - private String acked; - private String failed; - - public String getComponentId() { - return componentId; - } - - public void setComponentId(String componentId) { - this.componentId = componentId; - } - - public String getStream() { - return stream; - } - - public void setStream(String stream) { - this.stream = stream; - } - - public String getRecvTps() { - return recvTps; - } - - public void setRecvTps(String recvTps) { - this.recvTps = recvTps; - } - - public String getProcess() { - return process; - } - - public void setProcess(String process) { - this.process = process; - } - - public String getAcked() { - return acked; - } - - public void setAcked(String acked) { - this.acked = acked; - } - - public String getFailed() { - return failed; - } - - public void setFailed(String failed) { - this.failed = failed; - } - - public void setValues(GlobalStreamId gstreamId, Double recvTps, - Double process, Long acked, Long failed) { - this.componentId = gstreamId.get_componentId(); - this.stream = gstreamId.get_streamId(); - this.recvTps = JStormUtils.formatValue(recvTps); - this.acked = JStormUtils.formatValue(acked); - this.failed = JStormUtils.formatValue(failed); - this.process = JStormUtils.formatValue(process); - - } - -} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ComponentOutput.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ComponentOutput.java deleted file mode 100644 index 525d6e965..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ComponentOutput.java +++ /dev/null @@ -1,51 +0,0 @@ -package com.alibaba.jstorm.ui.model; - -/** - * componentpage:ComponentOutput - * - * @author xin.zhou - * - */ -import java.io.Serializable; - -import com.alibaba.jstorm.utils.JStormUtils; - -public class ComponentOutput implements Serializable { - - private static final long serialVersionUID = 5607257248459397567L; - - private String stream; - private String emitted; - private String sendTps; - - public String getStream() { - return stream; - } - - public void setStream(String stream) { - this.stream = stream; - } - - public String getEmitted() { - return emitted; - } - - public void setEmitted(String emitted) { - this.emitted = emitted; - } - - public String getSendTps() { - return sendTps; - } - - public void setSendTps(String sendTps) { - this.sendTps = sendTps; - } - - public void setValues(String stream, Long emitted, Double sendTps) { - this.stream = stream; - this.emitted = JStormUtils.formatValue(emitted); - this.sendTps = JStormUtils.formatValue(sendTps); - } - -} \ No newline at end of file diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ComponentStats.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ComponentStats.java deleted file mode 100644 index 7e25fb1ec..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ComponentStats.java +++ /dev/null @@ -1,85 +0,0 @@ -package com.alibaba.jstorm.ui.model; - -/** - * componentpage:ComponentSummary - * - * @author xin.zhou - * - */ -import java.io.Serializable; -import java.util.Map; - -import com.alibaba.jstorm.common.stats.StaticsType; -import com.alibaba.jstorm.utils.JStormUtils; - -public class ComponentStats implements Serializable { - - private static final long serialVersionUID = 2183846733736949858L; - private String emitted; - private String sendTps; - private String recvTps; - private String acked; - private String failed; - private String process; - - public String getEmitted() { - return emitted; - } - - public void setEmitted(String emitted) { - this.emitted = emitted; - } - - public String getSendTps() { - return sendTps; - } - - public void setSendTps(String sendTps) { - this.sendTps = sendTps; - } - - public String getRecvTps() { - return recvTps; - } - - public void setRecvTps(String recvTps) { - this.recvTps = recvTps; - } - - public String getAcked() { - return acked; - } - - public void setAcked(String acked) { - this.acked = acked; - } - - public String getFailed() { - return failed; - } - - public void setFailed(String failed) { - this.failed = failed; - } - - public String getProcess() { - return process; - } - - public void setProcess(String process) { - this.process = process; - } - - public void setValues(Map staticsType) { - emitted = JStormUtils.formatValue(staticsType.get(StaticsType.emitted)); - sendTps = JStormUtils - .formatValue(staticsType.get(StaticsType.send_tps)); - recvTps = JStormUtils - .formatValue(staticsType.get(StaticsType.recv_tps)); - acked = JStormUtils.formatValue(staticsType.get(StaticsType.acked)); - failed = JStormUtils.formatValue(staticsType.get(StaticsType.failed)); - process = JStormUtils.formatValue(staticsType - .get(StaticsType.process_latencies)); - } - -} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ComponentSummary.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ComponentSummary.java deleted file mode 100644 index 3cc31c1f7..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ComponentSummary.java +++ /dev/null @@ -1,45 +0,0 @@ -package com.alibaba.jstorm.ui.model; - -/** - * componentpage:ComponentSummary - * - * @author xin.zhou - * - */ -import java.io.Serializable; - -public class ComponentSummary implements Serializable { - - private static final long serialVersionUID = 681219575043845569L; - private String componentId; - private String topologyname; - private String parallelism; - - public ComponentSummary() { - } - - public String getComponentId() { - return componentId; - } - - public void setComponentId(String componentId) { - this.componentId = componentId; - } - - public String getTopologyname() { - return topologyname; - } - - public void setTopologyname(String topologyname) { - this.topologyname = topologyname; - } - - public String getParallelism() { - return parallelism; - } - - public void setParallelism(String parallelism) { - this.parallelism = parallelism; - } - -} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ComponentTask.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ComponentTask.java deleted file mode 100644 index 59bc8a3ea..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ComponentTask.java +++ /dev/null @@ -1,96 +0,0 @@ -package com.alibaba.jstorm.ui.model; - -/** - * componentpage:Task - * - * @author xin.zhou - * - */ -import java.io.Serializable; - -public class ComponentTask extends ComponentStats implements Serializable { - - private static final long serialVersionUID = -8501129148725843924L; - - private String topologyid; - private String componentid; - private String taskid; - private String ip; - private String host; - private String port; - private String uptime; - private String lastErr; - private String status; - - public String getTopologyid() { - return topologyid; - } - - public void setTopologyid(String topologyid) { - this.topologyid = topologyid; - } - - public String getComponentid() { - return componentid; - } - - public void setComponentid(String componentid) { - this.componentid = componentid; - } - - public String getTaskid() { - return taskid; - } - - public void setTaskid(String taskid) { - this.taskid = taskid; - } - - public String getIp() { - return ip; - } - - public void setIp(String ip) { - this.ip = ip; - } - - public String getHost() { - return host; - } - - public void setHost(String host) { - this.host = host; - } - - public String getPort() { - return port; - } - - public void setPort(String port) { - this.port = port; - } - - public String getUptime() { - return uptime; - } - - public void setUptime(String uptime) { - this.uptime = uptime; - } - - public String getLastErr() { - return lastErr; - } - - public void setLastErr(String lastErr) { - this.lastErr = lastErr; - } - - public String getStatus() { - return status; - } - - public void setStatus(String status) { - this.status = status; - } -} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/Components.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/Components.java deleted file mode 100644 index 1e85ff773..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/Components.java +++ /dev/null @@ -1,52 +0,0 @@ -package com.alibaba.jstorm.ui.model; - -/** - * topologypage:Spouts/Bolts - * - * @author xin.zhou - * - */ -import java.io.Serializable; - -public class Components extends ComponentStats implements Serializable { - - private static final long serialVersionUID = -5697993689701474L; - - private String type; // spout/bolt - private String componetId; - private String parallelism; - private String lastError; - - public String getType() { - return type; - } - - public void setType(String type) { - this.type = type; - } - - public String getComponetId() { - return componetId; - } - - public void setComponetId(String componetId) { - this.componetId = componetId; - } - - public String getParallelism() { - return parallelism; - } - - public void setParallelism(String parallelism) { - this.parallelism = parallelism; - } - - public String getLastError() { - return lastError; - } - - public void setLastError(String lastError) { - this.lastError = lastError; - } - -} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ErrorSummary.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ErrorSummary.java deleted file mode 100644 index 18dce445e..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/ErrorSummary.java +++ /dev/null @@ -1,37 +0,0 @@ -package com.alibaba.jstorm.ui.model; - -/** - * taskpage:ErrorSummary - * - * @author xin.zhou - * - */ -import java.io.Serializable; - -public class ErrorSummary implements Serializable { - - private static final long serialVersionUID = -4762717099406678507L; - private String time; - private String error; - - public ErrorSummary(String time, String error) { - this.time = time; - this.error = error; - } - - public String getTime() { - return time; - } - - public void setTime(String time) { - this.time = time; - } - - public String getError() { - return error; - } - - public void setError(String error) { - this.error = error; - } -} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/LinkData.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/LinkData.java new file mode 100755 index 000000000..04ddd1447 --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/LinkData.java @@ -0,0 +1,65 @@ +/** + * 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.ui.model; + +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; + +public class LinkData implements Serializable { + private String url; + private String text; + private Map paramMap = new HashMap(); + + public String getUrl() { + return url; + } + + public void setUrl(String url) { + this.url = url; + } + + public String getText() { + return text; + } + + public void setText(String text) { + this.text = text; + } + + public Map getParamMap() { + return paramMap; + } + + public void setParamMap(Map paramMap) { + this.paramMap = paramMap; + } + + public void addParam(String key, String value) { + this.paramMap.put(key, value); + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/LogPageIndex.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/LogPageIndex.java deleted file mode 100644 index d5cbf86af..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/LogPageIndex.java +++ /dev/null @@ -1,28 +0,0 @@ -package com.alibaba.jstorm.ui.model; - -import java.io.Serializable; - -public class LogPageIndex implements Serializable { - /** - * - */ - private static final long serialVersionUID = -6305581906533640556L; - - - private String index; - private String pos; - public String getIndex() { - return index; - } - public void setIndex(String index) { - this.index = index; - } - public String getPos() { - return pos; - } - public void setPos(String pos) { - this.pos = pos; - } - - -} 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 deleted file mode 100644 index 596fc7dac..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/NimbusSlave.java +++ /dev/null @@ -1,67 +0,0 @@ -package com.alibaba.jstorm.ui.model; - -import java.io.Serializable; - -import org.apache.commons.lang.StringUtils; - -import com.alibaba.jstorm.utils.NetWorkUtils; - -public class NimbusSlave implements Serializable { - - private static final long serialVersionUID = 2134152872653314400L; - - private String ip; - - private String logviewPort; - - private String hostname; - - private String uptime; - - public NimbusSlave(String hostname, String uptime, int port) { - - 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; - this.logviewPort = String.valueOf(port); - } - - public String getIp() { - return ip; - } - - public void setIp(String ip) { - this.ip = ip; - } - - public String getHostname() { - return hostname; - } - public void setHostname(String hostname) { - this.hostname = hostname; - } - public String getUptime() { - return uptime; - } - public void setUptime(String uptime) { - this.uptime = uptime; - } - - public String getLogviewPort() { - return logviewPort; - } - - public void setLogviewPort(String logviewPort) { - this.logviewPort = logviewPort; - } - - -} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/PageGenerator.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/PageGenerator.java new file mode 100755 index 000000000..cf56e46e0 --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/PageGenerator.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.alibaba.jstorm.ui.model; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public interface PageGenerator { + public static class Output { + public List tables; + public List pages = new ArrayList(); + public String rawData = ""; + } + + Output generate(Map paramMap); +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/PageIndex.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/PageIndex.java new file mode 100755 index 000000000..950aef2cb --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/PageIndex.java @@ -0,0 +1,151 @@ +/** + * 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.ui.model; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.ui.UIDef; +import com.alibaba.jstorm.utils.JStormUtils; + +public class PageIndex implements Serializable { + + private static final long serialVersionUID = -6305581906533640556L; + + public String status; + public LinkData linkData; + + public String getStatus() { + return status; + } + + public void setStatus(String status) { + this.status = status; + } + + public LinkData getLinkData() { + return linkData; + } + + public void setLinkData(LinkData linkData) { + this.linkData = linkData; + } + + public static class Event { + public long totalSize; + public long pos; + public long pageSize; + public String url; + public Map paramMap; + } + + + /** + * @param index + * @param pageSize + */ + public static void insertPage(long index, String text, + boolean isActive, + Event event, List ret) { + long pos = index * event.pageSize; + + PageIndex page = new PageIndex(); + + LinkData linkData = new LinkData(); + page.linkData = linkData; + + linkData.setUrl(event.url); + linkData.setText(text); + Map paramMap = new HashMap(); + paramMap.putAll(event.paramMap); + paramMap.put(UIDef.POS, String.valueOf(pos)); + linkData.setParamMap(paramMap); + +// if (isDisable == true) { +// page.status = "disabled"; +// } else + if (isActive == true) { + page.status = "active"; + } + + ret.add(page); + } + + public static List generatePageIndex(long totalSize, + long pageSize, String url, + Map paramMap) { + PageIndex.Event event = new PageIndex.Event(); + event.totalSize = totalSize; + event.pos = JStormUtils.parseLong(paramMap.get(UIDef.POS), 0); + event.pageSize = pageSize; + event.url = url; + event.paramMap = paramMap; + + return generatePageIndex(event); + } + + public static List generatePageIndex(Event event) { + List ret = new ArrayList(); + + long pageNum = (event.totalSize + event.pageSize - 1) / event.pageSize; + long currentPageIndex = event.pos/event.pageSize; + + if (pageNum <= 10) { + for (long i = pageNum - 1; i >= 0; i--) { + insertPage(i, String.valueOf(i), + i == currentPageIndex, event, ret); + } + return ret; + } + + if (pageNum - currentPageIndex < 5) { + for (long i = pageNum - 1; i >= currentPageIndex; i--) { + insertPage(i, String.valueOf(i), + i == currentPageIndex, event, ret); + } + } else { + insertPage(pageNum - 1, "End", + pageNum - 1 == currentPageIndex, event, ret); + insertPage(currentPageIndex + 4, "...", false, event, ret); + for (long i = currentPageIndex + 3; i >= currentPageIndex; i--) { + insertPage(i, String.valueOf(i), + i == currentPageIndex, event, ret); + } + } + + if (currentPageIndex < 5) { + for (long i = currentPageIndex - 1; i > 0; i--) { + insertPage(i, String.valueOf(i), + i == currentPageIndex, event, ret); + } + } else { + for (long i = currentPageIndex - 1; i >= currentPageIndex - 3; i--) { + insertPage(i, String.valueOf(i), + i == currentPageIndex, event, ret); + } + insertPage(currentPageIndex - 4, "...", false, event, ret); + insertPage(0, "Begin", 0 == currentPageIndex, event, ret); + } + + return ret; + } +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/SpoutOutput.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/SpoutOutput.java deleted file mode 100644 index e1b38043d..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/SpoutOutput.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * To change this template, choose Tools | Templates - * and open the template in the editor. - */ -package com.alibaba.jstorm.ui.model; - -import java.io.Serializable; - -import backtype.storm.generated.GlobalStreamId; - -import com.alibaba.jstorm.utils.JStormUtils; - -/** - * - * @author xin.zhou/Longda - */ -public class SpoutOutput implements Serializable { - - /** - * - */ - private static final long serialVersionUID = -5631649054937247856L; - - private String stream; - private String emitted; - private String sendTps; - private String process; - private String acked; - private String failed; - - public String getEmitted() { - return emitted; - } - - public void setEmitted(String emitted) { - this.emitted = emitted; - } - - public String getSendTps() { - return sendTps; - } - - public void setSendTps(String sendTps) { - this.sendTps = sendTps; - } - - public String getStream() { - return stream; - } - - public void setStream(String stream) { - this.stream = stream; - } - - public String getProcess() { - return process; - } - - public void setProcess(String process) { - this.process = process; - } - - public String getAcked() { - return acked; - } - - public void setAcked(String acked) { - this.acked = acked; - } - - public String getFailed() { - return failed; - } - - public void setFailed(String failed) { - this.failed = failed; - } - - public void setValues(String stream, Long emitted, Double sendTps, - Double process, Long acked, Long failed) { - this.stream = stream; - this.emitted = JStormUtils.formatValue(emitted); - this.sendTps = JStormUtils.formatValue(sendTps); - this.acked = JStormUtils.formatValue(acked); - this.failed = JStormUtils.formatValue(failed); - this.process = JStormUtils.formatValue(process); - - } - -} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/SupervisorSumm.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/SupervisorSumm.java deleted file mode 100644 index 6ad9ba63a..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/SupervisorSumm.java +++ /dev/null @@ -1,78 +0,0 @@ -package com.alibaba.jstorm.ui.model; - -import java.io.Serializable; - -import backtype.storm.generated.SupervisorSummary; - -import com.alibaba.jstorm.common.stats.StatBuckets; -import com.alibaba.jstorm.utils.NetWorkUtils; - -/** - * mainpage:SupervisorSummary - * - * @author xin.zhou - * - */ -public class SupervisorSumm implements Serializable { - - private static final long serialVersionUID = -5631649054937247850L; - private String ip; - - private String host; - private String uptime; - private String totalPort; - private String usedPort; - - public SupervisorSumm() { - } - - public SupervisorSumm(SupervisorSummary s) { - this.host = NetWorkUtils.ip2Host(s.get_host()); - this.ip = NetWorkUtils.host2Ip(s.get_host()); - this.uptime = StatBuckets.prettyUptimeStr(s.get_uptime_secs()); - - this.totalPort = String.valueOf(s.get_num_workers()); - this.usedPort = String.valueOf(s.get_num_used_workers()); - } - - public String getIp() { - return ip; - } - - public void setIp(String ip) { - this.ip = ip; - } - - public String getHost() { - return host; - } - - public void setHost(String host) { - this.host = host; - } - - public String getUptime() { - return uptime; - } - - public void setUptime(String uptime) { - this.uptime = uptime; - } - - public String getTotalPort() { - return totalPort; - } - - public void setTotalPort(String totalPort) { - this.totalPort = totalPort; - } - - public String getUsedPort() { - return usedPort; - } - - public void setUsedPort(String usedPort) { - this.usedPort = usedPort; - } - -} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/TableData.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/TableData.java new file mode 100755 index 000000000..2134defd1 --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/TableData.java @@ -0,0 +1,41 @@ +package com.alibaba.jstorm.ui.model; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; + +public class TableData implements Serializable { + private static final long serialVersionUID = 1L; + + protected String name; + protected List headers = new ArrayList(); + protected List> lines = + new ArrayList>(); + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public List getHeaders() { + return headers; + } + + public List> getLines() { + return lines; + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this, + ToStringStyle.SHORT_PREFIX_STYLE); + } + +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/TaskMetrics.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/TaskMetrics.java deleted file mode 100644 index 98e837f01..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/TaskMetrics.java +++ /dev/null @@ -1,151 +0,0 @@ -package com.alibaba.jstorm.ui.model; - -import java.io.Serializable; - -import com.alibaba.jstorm.metric.MetricDef; -import com.alibaba.jstorm.ui.UIUtils; - -import backtype.storm.generated.*; - -public class TaskMetrics implements Serializable{ - /** */ - private static final long serialVersionUID = -113082870281288187L; - - int taskId; - String componentId; - double deserializeQueue; - double deserializeTime; - double executorQueue; - double executorTime; - double serializeQueue; - double serializeTime; - double ackerTime; - double emptyCpuRatio; - double pendingNum; - double emitTime; - - public TaskMetrics() { - } - - public int getTaskId() { - return taskId; - } - - public void setTaskId(int taskId) { - this.taskId = taskId; - } - - public String getComponentId() { - return componentId; - } - - public void setComponentId(String componentId) { - this.componentId = componentId; - } - - public double getDeserializeQueue() { - return deserializeQueue; - } - - public void setDeserializeQueue(double value) { - this.deserializeQueue = value; - } - - public double getDeserializeTime() { - return deserializeTime; - } - - public void setDeserializeTime(double value) { - this.deserializeTime = value; - } - - public double getExecutorQueue() { - return executorQueue; - } - - public void setExecutorQueue(double value) { - this.executorQueue = value; - } - - public double getExecutorTime() { - return executorTime; - } - - public void setExecutorTime(double value) { - this.executorTime = value; - } - - public double getSerializeQueue() { - return serializeQueue; - } - - public void setSerializeQueue(double value) { - this.serializeQueue = value; - } - - public double getSerializeTime() { - return serializeTime; - } - - public void setSerializeTime(double value) { - this.serializeTime = value; - } - - public double getAckerTime() { - return ackerTime; - } - - public void setAckerTime(double value) { - this.ackerTime = value; - } - - public double getEmptyCpuRatio() { - return emptyCpuRatio; - } - - public void setEmptyCpuRatio(double value) { - this.emptyCpuRatio = value; - } - - public double getPendingNum() { - return pendingNum; - } - - public void setPendingNum(double value) { - this.pendingNum = value; - } - - public double getEmitTime() { - return emitTime; - } - - public void setEmitTime(double value) { - this.emitTime = value; - } - - public void updateTaskMetricData(TaskMetricData metricData) { - taskId = metricData.get_task_id(); - componentId = metricData.get_component_id(); - deserializeQueue = UIUtils.getDoubleValue( - metricData.get_gauge().get(MetricDef.DESERIALIZE_QUEUE)); - deserializeTime = UIUtils.getDoubleValue( - metricData.get_timer().get(MetricDef.DESERIALIZE_TIME)); - executorQueue = UIUtils.getDoubleValue( - metricData.get_gauge().get(MetricDef.EXECUTE_QUEUE)); - executorTime = UIUtils.getDoubleValue( - metricData.get_timer().get(MetricDef.EXECUTE_TIME)); - serializeQueue = UIUtils.getDoubleValue( - metricData.get_gauge().get(MetricDef.SERIALIZE_QUEUE)); - serializeTime = UIUtils.getDoubleValue( - metricData.get_timer().get(MetricDef.SERIALIZE_TIME)); - ackerTime = UIUtils.getDoubleValue( - metricData.get_timer().get(MetricDef.ACKER_TIME)); - emitTime = UIUtils.getDoubleValue( - metricData.get_timer().get(MetricDef.EMIT_TIME)); - emptyCpuRatio = UIUtils.getDoubleValue( - metricData.get_gauge().get(MetricDef.EMPTY_CPU_RATIO)); - pendingNum = UIUtils.getDoubleValue( - metricData.get_gauge().get(MetricDef.PENDING_MAP)); - } - -} \ No newline at end of file diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/TaskSumm.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/TaskSumm.java deleted file mode 100644 index 7754806d4..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/TaskSumm.java +++ /dev/null @@ -1,84 +0,0 @@ -package com.alibaba.jstorm.ui.model; - -/** - * taskpage:TaskSummary - * - * @author xin.zhou - * - */ -public class TaskSumm { - - private String taskid; - private String host; - private String port; - private String topologyname; - private String componentId; - private String uptime; - - public TaskSumm(String taskid, String host, String port, - String topologyname, String componentId, String uptime) { - this.taskid = taskid; - this.host = host; - this.port = port; - this.topologyname = topologyname; - this.componentId = componentId; - this.uptime = uptime; - - } - - public TaskSumm(String taskid, String host, String port, String uptime) { - this.taskid = taskid; - this.host = host; - this.port = port; - this.uptime = uptime; - - } - - public String getTaskid() { - return taskid; - } - - public void setTaskid(String taskid) { - this.taskid = taskid; - } - - public String getHost() { - return host; - } - - public void setHost(String host) { - this.host = host; - } - - public String getPort() { - return port; - } - - public void setPort(String port) { - this.port = port; - } - - public String getTopologyname() { - return topologyname; - } - - public void setTopologyname(String topologyname) { - this.topologyname = topologyname; - } - - public String getComponentId() { - return componentId; - } - - public void setComponentId(String componentId) { - this.componentId = componentId; - } - - public String getUptime() { - return uptime; - } - - public void setUptime(String uptime) { - this.uptime = uptime; - } -} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/TopologySumm.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/TopologySumm.java deleted file mode 100644 index 6b57c18d0..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/TopologySumm.java +++ /dev/null @@ -1,78 +0,0 @@ -package com.alibaba.jstorm.ui.model; - -import java.io.Serializable; - -/** - * mainpage:TopologySummary - * - * @author xin.zhou - * - */ -public class TopologySumm implements Serializable { - - private static final long serialVersionUID = 189495975527682322L; - private String topologyName; - private String topologyId; - private String status; - private String uptime; - private String numWorkers; - private String numTasks; - private String errorInfo; - - public String getTopologyName() { - return topologyName; - } - - public void setTopologyName(String topologyName) { - this.topologyName = topologyName; - } - - public String getTopologyId() { - return topologyId; - } - - public void setTopologyId(String topologyId) { - this.topologyId = topologyId; - } - - public String getStatus() { - return status; - } - - public void setStatus(String status) { - this.status = status; - } - - public String getUptime() { - return uptime; - } - - public void setUptime(String uptime) { - this.uptime = uptime; - } - - public String getNumWorkers() { - return numWorkers; - } - - public void setNumWorkers(String numWorkers) { - this.numWorkers = numWorkers; - } - - public String getNumTasks() { - return numTasks; - } - - public void setNumTasks(String numTasks) { - this.numTasks = numTasks; - } - - public String getErrorInfo() { - return this.errorInfo; - } - - public void setErrorInfo(String errorInfo) { - this.errorInfo = errorInfo; - } - -} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/WinComponentStats.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/WinComponentStats.java deleted file mode 100644 index 984695d68..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/WinComponentStats.java +++ /dev/null @@ -1,24 +0,0 @@ -package com.alibaba.jstorm.ui.model; - -/** - * topologypage:TopologyStats - * - * @author xin.zhou - * - */ -import java.io.Serializable; - -public class WinComponentStats extends ComponentStats implements Serializable { - - private static final long serialVersionUID = 4819784268512595428L; - private String window; - - public String getWindow() { - return window; - } - - public void setWindow(String window) { - this.window = window; - } - -} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/WorkerMetrics.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/WorkerMetrics.java deleted file mode 100644 index 4c4e98c20..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/WorkerMetrics.java +++ /dev/null @@ -1,183 +0,0 @@ -package com.alibaba.jstorm.ui.model; - -import java.io.Serializable; - -import org.apache.log4j.Logger; - -import backtype.storm.generated.WorkerMetricData; - -import com.alibaba.jstorm.metric.MetricDef; -import com.alibaba.jstorm.ui.UIUtils; -import com.alibaba.jstorm.utils.JStormUtils; - -public class WorkerMetrics implements Serializable{ - /** */ - private static final long serialVersionUID = -5610156437643520186L; - - private static final Logger LOG = Logger.getLogger(WorkerMetrics.class); - - String hostName; - int port; - double usedCpu; - double usedMem; - double nettyservDecodeTime; - double nettyservMsgTransTime; - double dispatchTime; - double dispatchQueue; - double batchTupleTime; - double batchTupleQueue; - double nettycliSendTime; - double nettycliBatchSize; - double nettycliSendPending; - double nettycliSyncBatchQueue; - double nettycliSyncDisrpQueue; - - public WorkerMetrics() { - } - - public String getHostName() { - return hostName; - } - - public void setHostName(String hostName) { - this.hostName = hostName; - } - - public int getPort() { - return port; - } - - public void setPort(int port) { - this.port = port; - } - - public double getNettyservDecodeTime() { - return nettyservDecodeTime; - } - - public void setNettyservDecodeTime(double value) { - this.nettyservDecodeTime = value; - } - - public double getNettyservMsgTransTime() { - return nettyservMsgTransTime; - } - - public void setNettyservMsgTransTime(double value) { - this.nettyservMsgTransTime = value; - } - - public double getDispatchTime() { - return dispatchTime; - } - - public void setDispatchTime(double value) { - this.dispatchTime = value; - } - - public double getDispatchQueue() { - return dispatchQueue; - } - - public void setDispatchQueue(double value) { - this.dispatchQueue = value; - } - - public double getBatchTupleTime() { - return batchTupleTime; - } - - public void setBatchTupleTime(double value) { - this.batchTupleTime = value; - } - - public double getBatchTupleQueue() { - return batchTupleQueue; - } - - public void setBatchTupleQueue(double value) { - this.batchTupleQueue = value; - } - - public double getNettycliSendTime() { - return nettycliSendTime; - } - - public void setNettycliSendTime(double value) { - this.nettycliSendTime = value; - } - - public double getNettycliBatchSize() { - return nettycliBatchSize; - } - - public void setNettycliBatchSize(double value) { - this.nettycliBatchSize = value; - } - - public double getNettycliSendPending() { - return nettycliSendPending; - } - - public void setNettycliSendPending(double value) { - this.nettycliSendPending = value; - } - - public double getNettycliSyncBatchQueue() { - return nettycliSyncBatchQueue; - } - - public void setNettycliSyncBatchQueue(double value) { - this.nettycliSyncBatchQueue = value; - } - - public double getNettycliSyncDisrpQueue() { - return nettycliSyncDisrpQueue; - } - - public void setNettycliSyncDisrpQueue(double value) { - this.nettycliSyncDisrpQueue = value; - } - - public double getUsedCpu() { - return usedCpu; - } - - public double getusedMem() { - return usedMem; - } - - public void updateWorkerMetricData(WorkerMetricData metricData) { - hostName = metricData.get_hostname(); - port = metricData.get_port(); - - usedCpu = UIUtils.getDoubleValue( - metricData.get_gauge().get(MetricDef.CPU_USED_RATIO)); - usedMem = UIUtils.getDoubleValue( - metricData.get_gauge().get(MetricDef.MEMORY_USED)); - usedMem = JStormUtils.formatDoubleDecPoint2(usedMem/(1024*1204)); - - batchTupleQueue = UIUtils.getDoubleValue( - metricData.get_gauge().get(MetricDef.BATCH_TUPLE_QUEUE)); - batchTupleTime = UIUtils.getDoubleValue( - metricData.get_timer().get(MetricDef.BATCH_TUPLE_TIME)); - dispatchQueue = UIUtils.getDoubleValue( - metricData.get_gauge().get(MetricDef.DISPATCH_QUEUE)); - dispatchTime = UIUtils.getDoubleValue( - metricData.get_timer().get(MetricDef.DISPATCH_TIME)); - nettycliBatchSize = UIUtils.getDoubleValue( - metricData.get_histogram().get(MetricDef.NETTY_CLI_BATCH_SIZE)); - nettycliSendTime = UIUtils.getDoubleValue( - metricData.get_timer().get(MetricDef.NETTY_CLI_SEND_TIME)); - nettyservDecodeTime = UIUtils.getDoubleValue( - metricData.get_timer().get(MetricDef.NETTY_SERV_DECODE_TIME)); - nettyservMsgTransTime = UIUtils.getDoubleValue( - metricData.get_histogram().get(MetricDef.NETWORK_MSG_TRANS_TIME)); - nettycliSendPending = UIUtils.getDoubleValue( - metricData.get_gauge().get(MetricDef.NETTY_CLI_SEND_PENDING)); - nettycliSyncBatchQueue = UIUtils.getDoubleValue( - metricData.get_gauge().get(MetricDef.NETTY_CLI_SYNC_BATCH_QUEUE)); - nettycliSyncDisrpQueue = UIUtils.getDoubleValue( - metricData.get_gauge().get(MetricDef.NETTY_CLI_SYNC_DISR_QUEUE)); - } -} \ No newline at end of file diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/WorkerSumm.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/WorkerSumm.java deleted file mode 100644 index c8b2f750c..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/WorkerSumm.java +++ /dev/null @@ -1,114 +0,0 @@ -package com.alibaba.jstorm.ui.model; - -import java.io.Serializable; -import java.util.List; - -import backtype.storm.generated.TaskSummary; -import backtype.storm.generated.WorkerSummary; - -import com.alibaba.jstorm.common.stats.StatBuckets; - -/** - * mainpage:SupervisorSummary - * - * @author longda - * - */ -public class WorkerSumm implements Serializable { - - private static final long serialVersionUID = -5631649054937247856L; - private String port; - private String topology; - private String uptime; - private String tasks; - private String components; - private String cpuNum; - private String memNum; - private String disks; - private List taskSummList; - - public WorkerSumm() { - } - - public WorkerSumm(WorkerSummary workerSummary) { - this.port = String.valueOf(workerSummary.get_port()); - this.topology = workerSummary.get_topology(); - - StringBuilder taskSB = new StringBuilder(); - StringBuilder componentSB = new StringBuilder(); - boolean isFirst = true; - - int minUptime = 0; - taskSummList = workerSummary.get_tasks(); - for (TaskSummary taskSummary : taskSummList) { - if (isFirst == false) { - taskSB.append(','); - componentSB.append(','); - } else { - minUptime = taskSummary.get_uptime_secs(); - } - - taskSB.append(taskSummary.get_task_id()); - componentSB.append(taskSummary.get_component_id()); - - if (minUptime < taskSummary.get_uptime_secs()) { - minUptime = taskSummary.get_uptime_secs(); - } - - isFirst = false; - } - - this.uptime = StatBuckets.prettyUptimeStr(minUptime); - this.tasks = taskSB.toString(); - this.components = componentSB.toString(); - - } - - public String getPort() { - return port; - } - - public void setPort(String port) { - this.port = port; - } - - public String getTopology() { - return topology; - } - - public void setTopology(String topology) { - this.topology = topology; - } - - public String getUptime() { - return uptime; - } - - public void setUptime(String uptime) { - this.uptime = uptime; - } - - public String getTasks() { - return tasks; - } - - public void setTasks(String tasks) { - this.tasks = tasks; - } - - public String getComponents() { - return components; - } - - public void setComponents(String components) { - this.components = components; - } - - public List gettaskSummList() { - return taskSummList; - } - - public void settaskSummList(List taskSummList) { - this.taskSummList = taskSummList; - } -} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/BoltPage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/BoltPage.java deleted file mode 100644 index 98ab7020b..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/BoltPage.java +++ /dev/null @@ -1,391 +0,0 @@ -/* - * To change this template, choose Tools | Templates - * and open the template in the editor. - */ -package com.alibaba.jstorm.ui.model.data; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.HashMap; -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.log4j.Logger; -import org.apache.thrift7.TException; - -import backtype.storm.Config; -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.generated.NotAliveException; -import backtype.storm.generated.StormTopology; -import backtype.storm.generated.TaskMetricData; -import backtype.storm.generated.TaskStats; -import backtype.storm.generated.TaskSummary; -import backtype.storm.generated.TopologyInfo; -import backtype.storm.generated.TopologyMetricInfo; -import backtype.storm.utils.NimbusClient; - -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.common.stats.StatBuckets; -import com.alibaba.jstorm.common.stats.StaticsType; -import com.alibaba.jstorm.ui.UIUtils; -import com.alibaba.jstorm.ui.model.ComponentInput; -import com.alibaba.jstorm.ui.model.ComponentOutput; -import com.alibaba.jstorm.ui.model.ComponentSummary; -import com.alibaba.jstorm.ui.model.ComponentTask; -import com.alibaba.jstorm.ui.model.WinComponentStats; -import com.alibaba.jstorm.ui.model.TaskMetrics; -import com.alibaba.jstorm.utils.JStormUtils; - -/** - * - * @author xin.zhou - */ -@ManagedBean(name = "boltpage") -@ViewScoped -public class BoltPage implements Serializable { - - private static final long serialVersionUID = 2629472722725558979L; - - private static final Logger LOG = Logger.getLogger(BoltPage.class); - - private String clusterName = null; - private String topologyid = null; - private String window = null; - private String componentid = null; - private String topologyName = null; - private List coms = null; - private List comstats = null; - private List coos = null; - private List cois = null; - private List cts = null; - private List taskmetrics = null; - - public BoltPage() throws TException, NotAliveException { - FacesContext ctx = FacesContext.getCurrentInstance(); - if (ctx.getExternalContext().getRequestParameterMap().get("clusterName") != null) { - clusterName = (String) ctx.getExternalContext() - .getRequestParameterMap().get("clusterName"); - } - - if (ctx.getExternalContext().getRequestParameterMap().get("topologyid") != null) { - topologyid = (String) ctx.getExternalContext() - .getRequestParameterMap().get("topologyid"); - } - - if (ctx.getExternalContext().getRequestParameterMap() - .get("componentid") != null) { - componentid = (String) ctx.getExternalContext() - .getRequestParameterMap().get("componentid"); - } - - window = UIUtils.getWindow(ctx); - - if (topologyid == null) { - throw new NotAliveException("Input topologyId is null "); - } - - init(); - } - - public BoltPage(String clusterName, String topologyId, String componentId, String window) - throws TException, NotAliveException { - this.clusterName = clusterName; - this.topologyid = topologyId; - this.componentid = componentId; - this.window = window; - - init(); - } - - private List getComponentSummaries(TopologyInfo summ, - List ts) { - List ret = new ArrayList(); - - ComponentSummary cs = new ComponentSummary(); - - cs.setComponentId(componentid); - cs.setTopologyname(summ.get_name()); - cs.setParallelism(String.valueOf(ts.size())); - - ret.add(cs); - - return ret; - } - - private List getComponentTasks(List taskList, - String window) { - List ret = new ArrayList(); - - for (TaskSummary task : taskList) { - ComponentTask componentTask = UIUtils.getComponentTask(task, topologyid); - - if (componentTask.getStatus().equals(ConfigExtension.TASK_STATUS_ACTIVE)) { - TaskStats taskStats = task.get_stats(); - - Map emitted = UIUtils.mergeStream( - taskStats.get_emitted(), Long.valueOf(0)); - Map sendTps = UIUtils.mergeStream( - taskStats.get_send_tps(), Double.valueOf(0)); - Map recvTps = UIUtils.mergeStream( - taskStats.get_recv_tps(), Double.valueOf(0)); - Map acked = UIUtils.mergeStream( - taskStats.get_acked(), Long.valueOf(0)); - Map failed = UIUtils.mergeStream( - taskStats.get_failed(), Long.valueOf(0)); - Map process = UIUtils.mergeStream( - taskStats.get_process_ms_avg(), Double.valueOf(0)); - - componentTask.setEmitted(JStormUtils.formatValue(emitted - .get(window))); - componentTask.setSendTps(JStormUtils.formatValue(sendTps - .get(window))); - componentTask.setRecvTps(JStormUtils.formatValue(recvTps - .get(window))); - componentTask.setAcked(JStormUtils.formatValue(acked.get(window))); - componentTask - .setFailed(JStormUtils.formatValue(failed.get(window))); - componentTask.setProcess(JStormUtils.formatValue(process - .get(window))); - } - - ret.add(componentTask); - } - - return ret; - } - - private List getWinComponentStats( - List taskList, String window) { - List ret = new ArrayList(); - - Map staticsMap = UIUtils.mergeTasks(taskList, - window); - - WinComponentStats winComponentStats = new WinComponentStats(); - - winComponentStats.setWindow(window); - winComponentStats.setValues(staticsMap); - - ret.add(winComponentStats); - - return ret; - } - - private void getInputOutputSummary(List taskSummaries, - String window) { - coos = new ArrayList(); - cois = new ArrayList(); - - List> emittedList = new ArrayList>(); - List> sendTpsList = new ArrayList>(); - List> recvTpsList = new ArrayList>(); - List> ackedList = new ArrayList>(); - List> failedList = new ArrayList>(); - List> processList = new ArrayList>(); - - for (TaskSummary taskSummary : taskSummaries) { - if (taskSummary.get_status() == null) { - // this is for old JStorm version - taskSummary.set_status(ConfigExtension.TASK_STATUS_ACTIVE); - } - - if (taskSummary.get_status().equals(ConfigExtension.TASK_STATUS_ACTIVE) == false) - continue; - - TaskStats taskStats = taskSummary.get_stats(); - - emittedList.add(taskStats.get_emitted().get(window)); - sendTpsList.add(taskStats.get_send_tps().get(window)); - recvTpsList.add(taskStats.get_recv_tps().get(window)); - ackedList.add(taskStats.get_acked().get(window)); - failedList.add(taskStats.get_failed().get(window)); - processList.add(taskStats.get_process_ms_avg().get(window)); - - } - - Map emitted = JStormUtils.mergeMapList(emittedList); - Map sendTps = JStormUtils.mergeMapList(sendTpsList); - Map recvTps = JStormUtils - .mergeMapList(recvTpsList); - Map acked = JStormUtils.mergeMapList(ackedList); - Map failed = JStormUtils.mergeMapList(failedList); - - Map processSum = JStormUtils - .mergeMapList(processList); - - Map process = new HashMap(); - for (Entry entry : processSum.entrySet()) { - GlobalStreamId streamId = entry.getKey(); - Double value = entry.getValue(); - - if (value != null && processList.size() > 0) { - process.put(streamId, value/processList.size()); - } - } - - for (Entry emittedEntry : emitted.entrySet()) { - String outputStreamId = emittedEntry.getKey(); - Long emittedValue = emittedEntry.getValue(); - Double sendTpsValue = sendTps.get(outputStreamId); - - ComponentOutput co = new ComponentOutput(); - - co.setValues(outputStreamId, emittedValue, sendTpsValue); - - coos.add(co); - } - - for (Entry entry : process.entrySet()) { - GlobalStreamId streamId = entry.getKey(); - Double processValue = entry.getValue(); - - Double recvTpsValue = recvTps.get(streamId); - Long ackedValue = acked.get(streamId); - Long failedValue = failed.get(streamId); - - ComponentInput ci = new ComponentInput(); - ci.setValues(streamId, recvTpsValue, processValue, ackedValue, - failedValue); - - cois.add(ci); - - } - - return; - - } - public List getTaskMetricsList(List totalTskMetrList) { - if (totalTskMetrList == null) return null; - List ret = new ArrayList(); - LOG.debug("get task metrics list: component ID: " + this.componentid); - for (TaskMetricData taskMetricData : totalTskMetrList) { - if ((taskMetricData.get_component_id()).equals(this.componentid)) { - TaskMetrics taskMetircs = new TaskMetrics(); - taskMetircs.updateTaskMetricData(taskMetricData); - ret.add(taskMetircs); - } - } - return ret; - } - - @SuppressWarnings("rawtypes") - private void init() throws TException, NotAliveException { - - NimbusClient client = null; - - try { - Map conf = UIUtils.readUiConfig(); - client = UIUtils.getNimbusClient(conf, clusterName); - - TopologyInfo summ = client.getClient().getTopologyInfo(topologyid); - StormTopology topology = client.getClient().getTopology(topologyid); - TopologyMetricInfo topologyMetricInfo = client.getClient().getTopologyMetric(topologyid); - - String type = UIUtils.componentType(topology, componentid); - - List ts = UIUtils.getTaskList(summ.get_tasks(), - componentid); - - topologyName = summ.get_name(); - - coms = getComponentSummaries(summ, ts); - - cts = getComponentTasks(ts, window); - - comstats = getWinComponentStats(ts, window); - - getInputOutputSummary(ts, window); - List totoaltaskmetrics = topologyMetricInfo.get_task_metric_list(); - taskmetrics = getTaskMetricsList(totoaltaskmetrics); - - } catch (TException e) { - LOG.error(e.getCause(), e); - throw e; - } catch (NotAliveException e) { - LOG.error(e.getCause(), e); - throw e; - } catch (Exception e) { - LOG.error(e.getCause(), e); - throw new TException(e); - } finally { - if (client != null) { - client.close(); - } - } - - } - - public List getComstats() { - return comstats; - } - - public void setComstats(List comstats) { - this.comstats = comstats; - } - - public List getCoos() { - return coos; - } - - public void setCoos(List coos) { - this.coos = coos; - } - - public List getCois() { - return cois; - } - - public void setCois(List cois) { - this.cois = cois; - } - - public List getCts() { - return cts; - } - - public void setCts(List cts) { - this.cts = cts; - } - - public List getComs() { - return coms; - } - - public void setComs(List coms) { - this.coms = coms; - } - - public List gettaskmetrics() { - return this.taskmetrics; - } - - public void settaskmetrics(List taskmetrs) { - this.taskmetrics = taskmetrs; - } - - public String getTopologyName() { - return topologyName; - } - - public void setTopologyName(String topologyName) { - this.topologyName = topologyName; - } - - public static void main(String[] args) { - try { - BoltPage instance = new BoltPage("/jstorm", "sequence_test-3-1363789458", - "Total", StatBuckets.ALL_WINDOW_STR); - } catch (TException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } catch (NotAliveException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - } -} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/ClusterPage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/ClusterPage.java deleted file mode 100644 index e7b3f7bf8..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/ClusterPage.java +++ /dev/null @@ -1,101 +0,0 @@ -package com.alibaba.jstorm.ui.model.data; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; -import java.util.ArrayList; -import java.util.Map; -import java.util.Map.Entry; - -import javax.faces.bean.ManagedBean; -import javax.faces.bean.ViewScoped; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; -import backtype.storm.generated.ClusterSummary; -import backtype.storm.utils.NimbusClient; - -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.ClusterState; -import com.alibaba.jstorm.common.stats.StatBuckets; -import com.alibaba.jstorm.ui.UIUtils; -import com.alibaba.jstorm.ui.model.ClusterInfo; -import com.alibaba.jstorm.utils.NetWorkUtils; -import com.alibaba.jstorm.zk.ZkTool; -import com.google.common.collect.Lists; - -/** - * - * @author xin.zhou/Longda - */ -@ManagedBean(name = "clusterpage") -@ViewScoped -public class ClusterPage implements Serializable { - - private static final long serialVersionUID = -6103468603521876731L; - - private static final Logger LOG = Logger.getLogger(ClusterPage.class); - - public static String SINGLE_CLUSTER = "single"; - public static String MULTI_CLUSTER = "multi"; - - private List uiClusters = null; - private List clusterInfos = null; - private String clusterType; - - public ClusterPage() throws Exception { - init(); - } - - @SuppressWarnings("rawtypes") - private void init() throws Exception { - - try { - LOG.info("ClusterPage init..."); - Map conf = UIUtils.readUiConfig(); - uiClusters = ConfigExtension.getUiClusters(conf); - - if (uiClusters != null) { - clusterType = MULTI_CLUSTER; - clusterInfos = new ArrayList(); - for (Map cluster : uiClusters) { - LOG.debug("Get ui cluster config infor, " + cluster); - ClusterInfo clusterInfo = new ClusterInfo(); - clusterInfo.setClusterName(ConfigExtension.getUiClusterName(cluster)); - clusterInfo.setZkPort(ConfigExtension.getUiClusterZkPort(cluster)); - clusterInfo.setZkRoot(ConfigExtension.getUiClusterZkRoot(cluster)); - clusterInfo.setZkServers(ConfigExtension.getUiClusterZkServers(cluster)); - clusterInfos.add(clusterInfo); - } - } else { - clusterType = SINGLE_CLUSTER; - } - - } catch (Exception e) { - LOG.error("Failed to get cluster information:", e); - throw e; - } finally { - } - } - - public List getClusterInfo() { - return clusterInfos; - } - - public String getClusterType() { - return clusterType; - } - - public static void main(String[] args) { - try { - ClusterPage c = new ClusterPage(); - System.out.println(c.getClusterInfo()); - } catch (Exception e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - - } - -} 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 deleted file mode 100644 index bb427aa8f..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/ConfPage.java +++ /dev/null @@ -1,180 +0,0 @@ -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 deleted file mode 100644 index d6650874f..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/JStackPage.java +++ /dev/null @@ -1,154 +0,0 @@ -package com.alibaba.jstorm.ui.model.data; - -import java.io.Serializable; -import java.util.Map; - -import javax.faces.bean.ManagedBean; -import javax.faces.bean.ViewScoped; -import javax.faces.context.FacesContext; - -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 com.alibaba.jstorm.client.ConfigExtension; -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") -@ViewScoped -public class JStackPage implements Serializable { - - private static final long serialVersionUID = 4326599394273506083L; - - private static final Logger LOG = Logger.getLogger(JStackPage.class); - - /** - * proxy url, which call the log service on the task node. - */ - private static final String PROXY_URL = "http://%s:%s/logview?%s=%s&%s=%s"; - - /** - * jstack data - */ - private String data = ""; - - private int workerPort; - - /** - * Http server port - */ - private String host; - private int port; - - private Map conf; - - public JStackPage() 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); - } - - if (ctx.getExternalContext().getRequestParameterMap() - .get(HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_WORKER_PORT) != null) { - workerPort = JStormUtils.parseInt(ctx.getExternalContext() - .getRequestParameterMap() - .get(HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_WORKER_PORT)); - } - - init(); - } - - private void init() throws Exception { - - try { - conf = UIUtils.readUiConfig(); - - if (port == 0) { - port = ConfigExtension.getSupervisorDeamonHttpserverPort(conf); - } - - // proxy call - queryLog(conf); - - } catch (Exception e) { - LOG.error(e.getCause(), e); - throw e; - } - } - - /** - * proxy query log for the specified task. - * - * @param task - * the specified task - */ - private void queryLog(Map conf) { - // PROXY_URL = "http://%s:%s/logview?%s=%s&%s=%s"; - String baseUrl = String - .format(PROXY_URL, NetWorkUtils.host2Ip(host), port, - HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD, - HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD_JSTACK, - HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_WORKER_PORT, - workerPort); - 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); - - setData(EntityUtils.toString(response.getEntity())); - - } catch (Exception e) { - setData(e.getMessage()); - LOG.error(e.getCause(), e); - } - } - - public String getData() { - return data; - } - - public void setData(String data) { - this.data = data; - } - - public int getworkerPort() { - return workerPort; - } - - public void setworkerPort(int workerPort) { - this.workerPort = workerPort; - } - - public int getPort() { - return port; - } - - public void setPort(int port) { - this.port = port; - } - - public String getHost() { - return host; - } - - public void setHost(String host) { - this.host = host; - } - -} 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 deleted file mode 100644 index f7c370355..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/ListLogPage.java +++ /dev/null @@ -1,237 +0,0 @@ -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 javax.faces.bean.ManagedBean; -import javax.faces.bean.ViewScoped; -import javax.faces.context.FacesContext; - -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 com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.ui.UIUtils; -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.
- * implement view the specified task log through proxy way. current support - * - * - * @author longda - * @version 1.0.0 <2014-04-20 21:23> - * @since JDK1.6 - */ -@ManagedBean(name = "listlogpage") -@ViewScoped -public class ListLogPage implements Serializable { - - private static final long serialVersionUID = 4326599394273506085L; - - private static final Logger LOG = Logger.getLogger(LogPage.class); - - /** - * proxy url, which call the log service on the task node. - */ - private static final String PROXY_URL = "http://%s:%s/logview?%s=%s&%s=%s"; - - private String host; - - private String summary; - - private List files = new ArrayList(); - - private List dirs = new ArrayList(); - /** - * Http server port - */ - private int port; - - private String portStr; - - private String parent; - - private void getTargetDir(FacesContext ctx) throws Exception { - String dir = null; - if (ctx.getExternalContext().getRequestParameterMap().get("dir") != null) { - - dir = ctx.getExternalContext().getRequestParameterMap().get("dir"); - } - - String paramParent = null; - - if (ctx.getExternalContext().getRequestParameterMap().get("parent") != null) { - - paramParent = ctx.getExternalContext().getRequestParameterMap() - .get("parent"); - } - - if (paramParent == null && dir == null) { - parent = "."; - } else if (paramParent == null && dir != null) { - parent = dir; - } else if (paramParent != null && dir == null) { - parent = paramParent; - } else { - parent = paramParent + File.separator + dir; - } - - } - - public ListLogPage() throws Exception { - FacesContext ctx = FacesContext.getCurrentInstance(); - if (ctx.getExternalContext().getRequestParameterMap().get("host") != null) { - host = (String) ctx.getExternalContext().getRequestParameterMap() - .get("host"); - } - - if (ctx.getExternalContext().getRequestParameterMap().get("port") != null) { - - port = JStormUtils.parseInt(ctx.getExternalContext() - .getRequestParameterMap().get("port"), 0); - } - - getTargetDir(ctx); - - init(); - } - - private void init() throws Exception { - - try { - - if (port == 0) { - Map conf = UIUtils.readUiConfig(); - - port = ConfigExtension.getSupervisorDeamonHttpserverPort(conf); - } - - portStr = String.valueOf(port); - - // proxy call - listLogs(); - - } catch (Exception e) { - LOG.error(e.getCause(), e); - throw e; - } - - } - - private void parseString(String input) { - Map map = (Map) JStormUtils - .from_json(input); - - for (Map jobj : map.values()) { - FileAttribute attribute = FileAttribute.fromJSONObject(jobj); - if (attribute != null) { - - if (JStormUtils.parseBoolean(attribute.getIsDir(), false) == true) { - dirs.add(attribute); - } else { - files.add(attribute); - } - - } - - } - - summary = "There are " + files.size() + " files"; - } - - /** - * proxy query log for the specified task. - * - * @param task - * the specified task - */ - private void listLogs() { - - // PROXY_URL = "http://%s:%s/logview?%s=%s&dir=%s"; - 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); - 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()); - - parseString(data); - - } else { - String data = EntityUtils.toString(response.getEntity()); - summary = ("Failed to get files\n" + data); - } - } catch (Exception e) { - summary = ("Failed to get files\n" + e.getMessage()); - LOG.error(e.getCause(), e); - } - } - - public String getSummary() { - return summary; - } - - public void setSummary(String summary) { - this.summary = summary; - } - - public List getFiles() { - return files; - } - - public void setFiles(List files) { - this.files = files; - } - - public String getHost() { - return host; - } - - public void setHost(String host) { - this.host = host; - } - - public List getDirs() { - return dirs; - } - - public void setDirs(List dirs) { - this.dirs = dirs; - } - - public String getPortStr() { - return portStr; - } - - public void setPortStr(String portStr) { - this.portStr = portStr; - } - - public String getParent() { - return parent; - } - - public void setParent(String parent) { - this.parent = 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 deleted file mode 100644 index cae1a69e4..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/LogPage.java +++ /dev/null @@ -1,409 +0,0 @@ -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 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 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 = "logpage") -@ViewScoped -public class LogPage implements Serializable { - - private static final long serialVersionUID = 4326599394273506083L; - - private static final Logger LOG = Logger.getLogger(LogPage.class); - - /** - * proxy url, which call the log service on the task node. - */ - private static final String PROXY_URL = "http://%s:%s/logview?%s=%s&log=%s"; - - /** - * store the log content. - */ - private String log = ""; - - private List pages = new ArrayList(); - - /** - * Log file name - */ - private String logFileName = "Log"; - - /** - * Http server port - */ - private int port; - - private String position; - - private Map conf; - - private String host; - - private String clusterName; - - public LogPage() throws Exception { - FacesContext ctx = FacesContext.getCurrentInstance(); - if (ctx.getExternalContext().getRequestParameterMap().get("clusterName") != null) { - clusterName = (String) ctx.getExternalContext() - .getRequestParameterMap().get("clusterName"); - } - - if (ctx.getExternalContext().getRequestParameterMap() - .get(HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_POS) != null) { - position = ctx.getExternalContext().getRequestParameterMap() - .get(HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_POS); - } - - 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); - } - - generateLogFileName(); - - // proxy call - queryLog(conf); - - } catch (Exception e) { - LOG.error(e.getCause(), e); - throw e; - } - } - - private void generateLogFileName() throws Exception { - FacesContext ctx = FacesContext.getCurrentInstance(); - if (ctx.getExternalContext().getRequestParameterMap().get("host") != null) { - host = ctx.getExternalContext().getRequestParameterMap() - .get("host"); - } - - String log = null; - if (ctx.getExternalContext().getRequestParameterMap().get("log") != null) { - log = ctx.getExternalContext().getRequestParameterMap() - .get("log"); - } - - String workerPort = null; - if (ctx.getExternalContext().getRequestParameterMap().get("workerPort") != null) { - workerPort = ctx.getExternalContext().getRequestParameterMap() - .get("workerPort"); - } - - if (StringUtils.isBlank(host) == false) { - if (StringUtils.isBlank(log) == false) { - String parent = null; - if (ctx.getExternalContext().getRequestParameterMap().get("parent") != null) { - parent = ctx.getExternalContext().getRequestParameterMap() - .get("parent"); - } - - if (parent == null) { - logFileName = log; - }else { - logFileName = parent + File.separator + log; - } - } else if (StringUtils.isBlank(workerPort) == false) { - String topologyId = null; - if (ctx.getExternalContext().getRequestParameterMap().get("topologyId") != null) { - topologyId = ctx.getExternalContext().getRequestParameterMap() - .get("topologyId"); - } - - NimbusClient client = null; - - try { - client = UIUtils.getNimbusClient(conf, clusterName); - TopologyInfo summ = client.getClient().getTopologyInfo(topologyId); - logFileName = JStormUtils.genLogName(summ.get_name(), Integer.valueOf(workerPort)); - }finally { - if (client != null) { - client.close(); - } - } - } - return; - } - - String topologyid = null; - String taskid = null; - String clusterName = null; - - // resolve the arguments - if (ctx.getExternalContext().getRequestParameterMap().get("clusterName") != null) { - clusterName = (String) ctx.getExternalContext() - .getRequestParameterMap().get("clusterName"); - } - if (ctx.getExternalContext().getRequestParameterMap().get("topologyid") != null) { - topologyid = ctx.getExternalContext().getRequestParameterMap() - .get("topologyid"); - } - if (ctx.getExternalContext().getRequestParameterMap().get("taskid") != null) { - taskid = ctx.getExternalContext().getRequestParameterMap() - .get("taskid"); - } - - if (topologyid == null) { - throw new NotAliveException("Input topologyId is null "); - } - if (taskid == null) { - throw new NotAliveException("Input taskId is null "); - } - - NimbusClient client = null; - - try { - client = UIUtils.getNimbusClient(conf, clusterName); - - TopologyInfo summ = client.getClient().getTopologyInfo(topologyid); - - // find the specified task entity - TaskSummary taskSummary = null; - for (TaskSummary _taskSummary : summ.get_tasks()) { - if (taskid.equals(String.valueOf(_taskSummary.get_task_id()))) { - taskSummary = _taskSummary; - break; - } - } - - if (taskSummary == null) { - throw new NotAliveException("topologyid=" + topologyid - + ", taskid=" + taskid); - } - - ComponentTask componentTask = UIUtils.getComponentTask(taskSummary, - topologyid); - - host = componentTask.getHost(); - -// logFileName = componentTask.getTopologyid() + "-worker-" -// + componentTask.getPort() + ".log"; - logFileName = JStormUtils.genLogName(summ.get_name(), - Integer.valueOf(componentTask.getPort())); - - } catch (TException e) { - LOG.error(e.getCause(), e); - throw e; - } catch (NotAliveException e) { - LOG.error(e.getCause(), e); - throw e; - } finally { - if (client != null) { - client.close(); - } - } - } - - private void insertPage(long index) { - long pos = index * HttpserverUtils.HTTPSERVER_LOGVIEW_PAGESIZE; - - LogPageIndex page = new LogPageIndex(); - page.setIndex(String.valueOf(index)); - page.setPos(String.valueOf(pos)); - - pages.add(page); - } - - private void genPageUrl(String sizeStr) { - long size = Long.valueOf(sizeStr); - - long item = (size + HttpserverUtils.HTTPSERVER_LOGVIEW_PAGESIZE - 1) - / HttpserverUtils.HTTPSERVER_LOGVIEW_PAGESIZE; - - if (item <= 10) { - for (long i = item - 1; i >= 0; i--) { - insertPage(i); - } - return; - } - - long current = item - 1; - - if (position != null) { - current = (Long.valueOf(position) - + HttpserverUtils.HTTPSERVER_LOGVIEW_PAGESIZE - 1) - / HttpserverUtils.HTTPSERVER_LOGVIEW_PAGESIZE; - } - - if (item - current <= 5) { - for (long i = item - 1; i > current; i--) { - insertPage(i); - } - } else { - insertPage(item - 1); - for (long i = current + 4; i > current; i--) { - insertPage(i); - } - } - - if (current >= 5) { - for (long i = 1; i < 5; i++) { - insertPage(current - i); - } - insertPage(Long.valueOf(0)); - } else { - for (long i = current - 1; i >= 0; i--) { - insertPage(i); - } - } - } - - /** - * proxy query log for the specified task. - * - * @param task - * the specified task - */ - private void queryLog(Map conf) { - // 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, logFileName); - String url = baseUrl; - if (position != null) { - url += ("&" + HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_POS + "=" + position); - } - 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)); - - String sizeStr = data.substring(0, 16); - genPageUrl(sizeStr); - - setLog(data); - } else { - setLog(EntityUtils.toString(response.getEntity())); - } - } catch (Exception e) { - setLog(e.getMessage()); - LOG.error(e.getCause(), e); - } - } - - /** - * get the log content - * - * @return log content - */ - public String getLog() { - return log; - } - - /** - * set the log content - * - * @param log - * log content - */ - public void setLog(String log) { - this.log = log; - } - - public List getPages() { - return pages; - } - - public void setPages(List pages) { - this.pages = pages; - } - - public String getPort() { - return String.valueOf(port); - } - - public void setPort(int port) { - this.port = port; - } - - public String getPosition() { - return position; - } - - public void setPosition(String position) { - this.position = position; - } - - public String getLogFileName() { - return logFileName; - } - - public void setLogFileName(String logFileName) { - this.logFileName = logFileName; - } - - public String getHost() { - return host; - } - - public void setHost(String host) { - this.host = host; - } - - public String getClusterName() { - return clusterName; - } - - public void setClusterName(String clusterName) { - this.clusterName = clusterName; - } - - - -} 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 deleted file mode 100644 index 59d4a441a..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/MainPage.java +++ /dev/null @@ -1,208 +0,0 @@ -package com.alibaba.jstorm.ui.model.data; - -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.log4j.Logger; - -import backtype.storm.Config; -import backtype.storm.generated.ClusterSummary; -import backtype.storm.utils.NimbusClient; - -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.cluster.ClusterState; -import com.alibaba.jstorm.common.stats.StatBuckets; -import com.alibaba.jstorm.ui.UIUtils; -import com.alibaba.jstorm.ui.model.ClusterInfo; -import com.alibaba.jstorm.ui.model.ClusterSumm; -import com.alibaba.jstorm.ui.model.NimbusSlave; -import com.alibaba.jstorm.ui.model.SupervisorSumm; -import com.alibaba.jstorm.ui.model.TopologySumm; -import com.alibaba.jstorm.utils.NetWorkUtils; -import com.alibaba.jstorm.zk.ZkTool; -import com.google.common.collect.Lists; - -/** - * - * @author xin.zhou/Longda - */ -@ManagedBean(name = "mainpage") -@ViewScoped -public class MainPage implements Serializable { - - private static final long serialVersionUID = -6103468103521877721L; - - private static final Logger LOG = Logger.getLogger(MainPage.class); - - private String host = "localhost"; - - private String clusterName = null; - private ClusterSummary summ = null; - private List csumm = null; - private List tsumm = null; - private List ssumm = null; - - private List slaves = null; - private List zkServers = null; - private String zkPort = null; - - public MainPage() throws Exception { - FacesContext ctx = FacesContext.getCurrentInstance(); - if (ctx.getExternalContext().getRequestParameterMap().get("clusterName") != null) { - clusterName = (String) ctx.getExternalContext() - .getRequestParameterMap().get("clusterName"); - } - - init(); - } - - @SuppressWarnings("rawtypes") - private void init() throws Exception { - - NimbusClient client = null; - ClusterState cluster_state = null; - try { - LOG.info("MainPage init..."); - Map conf = UIUtils.readUiConfig(); - - client = UIUtils.getNimbusClient(conf, clusterName); - summ = client.getClient().getClusterInfo(); - - tsumm = UIUtils.topologySummary(summ.get_topologies()); - csumm = UIUtils.clusterSummary(summ, client, conf); - ssumm = UIUtils.supervisorSummary(summ.get_supervisors()); - - cluster_state = ZkTool.mk_distributed_cluster_state(client - .getConf()); - slaves = getNimbusSlave(cluster_state, conf); - - zkServers = getZkServer(conf); - zkPort = String.valueOf(conf.get(Config.STORM_ZOOKEEPER_PORT)); - - } catch (Exception e) { - String errorInfo = e.getMessage(); - 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(); - } - if (cluster_state != null) { - cluster_state.close(); - } - } - } - - private List getNimbusSlave(ClusterState cluster_state, - Map conf) throws Exception { - int port = ConfigExtension.getNimbusDeamonHttpserverPort(conf); - List slaves = Lists.newArrayList(); - Map followerMap = ZkTool.get_followers(cluster_state); - if (!followerMap.isEmpty()) { - for (Entry entry : followerMap.entrySet()) { - String uptime = StatBuckets.prettyUptimeStr(Integer - .valueOf(entry.getValue())); - slaves.add(new NimbusSlave(entry.getKey(), uptime, port)); - } - } - return slaves; - } - - private List getZkServer(Map conf) { - List servers = Lists.newArrayList(); - for (String ip : (List) conf - .get(Config.STORM_ZOOKEEPER_SERVERS)) { - servers.add(NetWorkUtils.ip2Host(ip)); - } - return servers; - } - - public List getCsumm() { - return csumm; - } - - public ClusterSummary getSumm() { - return summ; - } - - public void setSumm(ClusterSummary summ) { - this.summ = summ; - } - - public void setCsumm(List csumm) { - this.csumm = csumm; - } - - public void setTsumm(List tsumm) { - this.tsumm = tsumm; - } - - public void setSsumm(List ssumm) { - this.ssumm = ssumm; - } - - public List getTsumm() { - - return tsumm; - } - - public List getSsumm() { - - return ssumm; - } - - public String getHost() { - return host; - } - - public void setHost(String host) { - this.host = host; - } - - public List getSlaves() { - return slaves; - } - - public void setSlaves(List slaves) { - this.slaves = slaves; - } - - public List getZkServers() { - return zkServers; - } - - public void setZkServers(List zkServers) { - this.zkServers = zkServers; - } - - public String getZkPort() { - return zkPort; - } - - public void setZkPort(String zkPort) { - this.zkPort = zkPort; - } - - public static void main(String[] args) { - try { - MainPage m = new MainPage(); - System.out.println(m.getCsumm()); - } catch (Exception e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - - } - -} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/Param.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/Param.java deleted file mode 100644 index 3e7ceecc9..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/Param.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * To change this template, choose Tools | Templates - * and open the template in the editor. - */ -package com.alibaba.jstorm.ui.model.data; - -import java.io.Serializable; - -import javax.faces.bean.ManagedBean; -import javax.faces.bean.ViewScoped; -import javax.faces.context.FacesContext; - -import com.alibaba.jstorm.common.stats.StatBuckets; - -/** - * - * @author xin.zhou - */ -@ManagedBean(name = "pm") -@ViewScoped -public class Param implements Serializable { - - private static final long serialVersionUID = -1087749257427646824L; - private String clusterName = null; - private String topologyid = ""; - private String window = null; - private String componentid = ""; - private String taskid = ""; - - public Param() { - init(); - } - - private void init() { - FacesContext ctx = FacesContext.getCurrentInstance(); - if (ctx.getExternalContext().getRequestParameterMap().get("clusterName") != null) { - clusterName = (String) ctx.getExternalContext() - .getRequestParameterMap().get("clusterName"); - } - if (ctx.getExternalContext().getRequestParameterMap().get("topologyid") != null) { - topologyid = (String) ctx.getExternalContext() - .getRequestParameterMap().get("topologyid"); - } - if (ctx.getExternalContext().getRequestParameterMap().get("window") != null) { - window = (String) ctx.getExternalContext().getRequestParameterMap() - .get("window"); - } - if (ctx.getExternalContext().getRequestParameterMap() - .get("componentid") != null) { - componentid = (String) ctx.getExternalContext() - .getRequestParameterMap().get("componentid"); - } - if (ctx.getExternalContext().getRequestParameterMap().get("taskid") != null) { - taskid = (String) ctx.getExternalContext().getRequestParameterMap() - .get("taskid"); - } - - window = StatBuckets.getShowTimeStr(window); - } - - public String getClusterName() { - return clusterName; - } - - public void setClusterName(String clusterName) { - this.clusterName = clusterName; - } - - public String getTopologyid() { - return topologyid; - } - - public void setTopologyid(String topologyid) { - this.topologyid = topologyid; - } - - public String getWindow() { - return window; - } - - public void setWindow(String window) { - this.window = window; - } - - public String getComponentid() { - return componentid; - } - - public void setComponentid(String componentid) { - this.componentid = componentid; - } - - public String getTaskid() { - return taskid; - } - - public void setTaskid(String taskid) { - this.taskid = taskid; - } -} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/SpoutPage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/SpoutPage.java deleted file mode 100644 index 234635122..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/SpoutPage.java +++ /dev/null @@ -1,398 +0,0 @@ -/* - * To change this template, choose Tools | Templates - * and open the template in the editor. - */ -package com.alibaba.jstorm.ui.model.data; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.HashMap; -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.log4j.Logger; -import org.apache.thrift7.TException; - -import backtype.storm.Config; -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.generated.NotAliveException; -import backtype.storm.generated.StormTopology; -import backtype.storm.generated.TaskStats; -import backtype.storm.generated.TaskSummary; -import backtype.storm.generated.TopologyInfo; -import backtype.storm.generated.TopologyMetricInfo; -import backtype.storm.generated.TaskMetricData; -import backtype.storm.utils.NimbusClient; - -import com.alibaba.jstorm.client.ConfigExtension; -import com.alibaba.jstorm.common.stats.StatBuckets; -import com.alibaba.jstorm.common.stats.StaticsType; -import com.alibaba.jstorm.ui.UIUtils; -import com.alibaba.jstorm.ui.model.ComponentSummary; -import com.alibaba.jstorm.ui.model.ComponentTask; -import com.alibaba.jstorm.ui.model.SpoutOutput; -import com.alibaba.jstorm.ui.model.WinComponentStats; -import com.alibaba.jstorm.ui.model.TaskMetrics; -import com.alibaba.jstorm.utils.JStormUtils; - -/** - * - * @author xin.zhou - */ -@ManagedBean(name = "spoutpage") -@ViewScoped -public class SpoutPage implements Serializable { - - private static final long serialVersionUID = 2629472722725558979L; - - private static final Logger LOG = Logger.getLogger(SpoutPage.class); - - private String clusterName = null; - private String topologyid = null; - private String window = null; - private String componentid = null; - private List coms = null; - private List comstats = null; - private List coos = null; - private List cts = null; - private List taskmetrics = null; - - public SpoutPage() throws TException, NotAliveException { - FacesContext ctx = FacesContext.getCurrentInstance(); - if (ctx.getExternalContext().getRequestParameterMap().get("clusterName") != null) { - clusterName = (String) ctx.getExternalContext() - .getRequestParameterMap().get("clusterName"); - } - - if (ctx.getExternalContext().getRequestParameterMap().get("topologyid") != null) { - topologyid = (String) ctx.getExternalContext() - .getRequestParameterMap().get("topologyid"); - } - - if (ctx.getExternalContext().getRequestParameterMap() - .get("componentid") != null) { - componentid = (String) ctx.getExternalContext() - .getRequestParameterMap().get("componentid"); - } - - window = UIUtils.getWindow(ctx); - - if (topologyid == null) { - throw new NotAliveException("Input topologyId is null "); - } - - init(); - } - - public SpoutPage(String clusterName, String topologyId, String componentId, String window) - throws TException, NotAliveException { - this.clusterName = clusterName; - this.topologyid = topologyId; - this.componentid = componentId; - this.window = window; - - init(); - } - - private List getComponentSummaries(TopologyInfo summ, - List ts) { - List ret = new ArrayList(); - - ComponentSummary cs = new ComponentSummary(); - - cs.setComponentId(componentid); - cs.setTopologyname(summ.get_name()); - cs.setParallelism(String.valueOf(ts.size())); - - ret.add(cs); - - return ret; - } - - private List getComponentTasks(List taskList, - String window) { - List ret = new ArrayList(); - - for (TaskSummary task : taskList) { - ComponentTask componentTask = UIUtils.getComponentTask(task, topologyid); - - if (componentTask.getStatus().equals(ConfigExtension.TASK_STATUS_ACTIVE)) { - TaskStats taskStats = task.get_stats(); - - Map emitted = UIUtils.mergeStream( - taskStats.get_emitted(), Long.valueOf(0)); - Map sendTps = UIUtils.mergeStream( - taskStats.get_send_tps(), Double.valueOf(0)); - Map recvTps = UIUtils.mergeStream( - taskStats.get_recv_tps(), Double.valueOf(0)); - Map acked = UIUtils.mergeStream( - taskStats.get_acked(), Long.valueOf(0)); - Map failed = UIUtils.mergeStream( - taskStats.get_failed(), Long.valueOf(0)); - Map process = UIUtils.mergeStream( - taskStats.get_process_ms_avg(), Double.valueOf(0)); - - componentTask.setEmitted(JStormUtils.formatValue(emitted - .get(window))); - componentTask.setSendTps(JStormUtils.formatValue(sendTps - .get(window))); - componentTask.setRecvTps(JStormUtils.formatValue(recvTps - .get(window))); - componentTask.setAcked(JStormUtils.formatValue(acked.get(window))); - componentTask - .setFailed(JStormUtils.formatValue(failed.get(window))); - componentTask.setProcess(JStormUtils.formatValue(process - .get(window))); - } - ret.add(componentTask); - } - - return ret; - } - - private List getWinComponentStats( - List taskList, String window) { - List ret = new ArrayList(); - - Map staticsMap = UIUtils.mergeTasks(taskList, - window); - - WinComponentStats winComponentStats = new WinComponentStats(); - - winComponentStats.setWindow(window); - winComponentStats.setValues(staticsMap); - - ret.add(winComponentStats); - - return ret; - } - - private void getOutputSummary(List taskSummaries, String window) { - coos = new ArrayList(); - - List> emittedList = new ArrayList>(); - List> sendTpsList = new ArrayList>(); - List> recvTpsList = new ArrayList>(); - List> ackedList = new ArrayList>(); - List> failedList = new ArrayList>(); - List> processList = new ArrayList>(); - - for (TaskSummary taskSummary : taskSummaries) { - if (taskSummary.get_status() == null) { - // this is for old JStorm version - taskSummary.set_status(ConfigExtension.TASK_STATUS_ACTIVE); - } - - if (taskSummary.get_status().equals(ConfigExtension.TASK_STATUS_ACTIVE) == false) - continue; - - TaskStats taskStats = taskSummary.get_stats(); - - emittedList.add(taskStats.get_emitted().get(window)); - sendTpsList.add(taskStats.get_send_tps().get(window)); - recvTpsList.add(taskStats.get_recv_tps().get(window)); - ackedList.add(taskStats.get_acked().get(window)); - failedList.add(taskStats.get_failed().get(window)); - processList.add(taskStats.get_process_ms_avg().get(window)); - - } - - Map emitted = JStormUtils.mergeMapList(emittedList); - Map sendTps = JStormUtils.mergeMapList(sendTpsList); - Map recvTps = JStormUtils - .mergeMapList(recvTpsList); - Map acked = JStormUtils.mergeMapList(ackedList); - Map failed = JStormUtils.mergeMapList(failedList); - Map processSum = JStormUtils - .mergeMapList(processList); - - Map process = new HashMap(); - for (Entry entry : processSum.entrySet()) { - GlobalStreamId streamId = entry.getKey(); - Double value = entry.getValue(); - - if (value != null && processList.size() > 0) { - process.put(streamId, value/processList.size()); - } - } - - for (Entry emittedEntry : emitted.entrySet()) { - String outputStreamId = emittedEntry.getKey(); - Long emittedValue = emittedEntry.getValue(); - Double sendTpsValue = sendTps.get(outputStreamId); - - GlobalStreamId streamId = null; - for (Entry entry : acked.entrySet()) { - String stream = entry.getKey().get_streamId(); - if (outputStreamId.equals(stream)) { - streamId = entry.getKey(); - break; - } - } - - if (streamId == null) { - for (Entry entry : failed.entrySet()) { - String stream = entry.getKey().get_streamId(); - if (outputStreamId.equals(stream)) { - streamId = entry.getKey(); - break; - } - } - } - - Double processValue = process.get(streamId); - Long ackedValue = acked.get(streamId); - Long failedValue = failed.get(streamId); - - SpoutOutput co = new SpoutOutput(); - co.setValues(outputStreamId, emittedValue, sendTpsValue, - processValue, ackedValue, failedValue); - - coos.add(co); - - } - - return; - - } - public List getTaskMetricsList(List totalTskMetrList) { - if (totalTskMetrList == null) return null; - List ret = new ArrayList(); - LOG.debug("get task metrics list: component ID: " + this.componentid); - for (TaskMetricData taskMetricData : totalTskMetrList) { - if ((taskMetricData.get_component_id()).equals(this.componentid)) { - TaskMetrics taskMetircs = new TaskMetrics(); - taskMetircs.updateTaskMetricData(taskMetricData); - ret.add(taskMetircs); - } - } - return ret; - } - - @SuppressWarnings("rawtypes") - private void init() throws TException, NotAliveException { - - NimbusClient client = null; - - try { - Map conf = UIUtils.readUiConfig(); - client = UIUtils.getNimbusClient(conf, clusterName); - - TopologyInfo summ = client.getClient().getTopologyInfo(topologyid); - StormTopology topology = client.getClient().getTopology(topologyid); - TopologyMetricInfo topologyMetricInfo = client.getClient().getTopologyMetric(topologyid); - - String type = UIUtils.componentType(topology, componentid); - - List ts = UIUtils.getTaskList(summ.get_tasks(), - componentid); - - coms = getComponentSummaries(summ, ts); - - cts = getComponentTasks(ts, window); - - comstats = getWinComponentStats(ts, window); - - getOutputSummary(ts, window); - List totoaltaskmetrics = topologyMetricInfo.get_task_metric_list(); - taskmetrics = getTaskMetricsList(totoaltaskmetrics); - - } catch (TException e) { - LOG.error(e.getCause(), e); - throw e; - } catch (NotAliveException e) { - LOG.error(e.getCause(), e); - throw e; - } catch (Exception e) { - LOG.error(e.getCause(), e); - throw new TException(e); - }finally { - if (client != null) { - client.close(); - } - } - - } - - public List getComstats() { - return comstats; - } - - public void setComstats(List comstats) { - this.comstats = comstats; - } - - public List getCts() { - return cts; - } - - public void setCts(List cts) { - this.cts = cts; - } - - public List getComs() { - return coms; - } - - public void setComs(List coms) { - this.coms = coms; - } - - public String getTopologyid() { - return topologyid; - } - - public void setTopologyid(String topologyid) { - this.topologyid = topologyid; - } - - public String getWindow() { - return window; - } - - public void setWindow(String window) { - this.window = window; - } - - public String getComponentid() { - return componentid; - } - - public void setComponentid(String componentid) { - this.componentid = componentid; - } - - public List getCoos() { - return coos; - } - - public void setCoos(List coos) { - this.coos = coos; - } - - public List gettaskmetrics() { - return this.taskmetrics; - } - - public void settaskmetrics(List taskmetrs) { - this.taskmetrics = taskmetrs; - } - - public static void main(String[] args) { - try { - SpoutPage instance = new SpoutPage("/jstorm", "sequence_test-3-1363789458", - "SequenceSpoutge", StatBuckets.ALL_WINDOW_STR); - } catch (TException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } catch (NotAliveException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - } -} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/SupervisorPage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/SupervisorPage.java deleted file mode 100644 index ac702bcd3..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/SupervisorPage.java +++ /dev/null @@ -1,176 +0,0 @@ -package com.alibaba.jstorm.ui.model.data; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -import javax.faces.bean.ManagedBean; -import javax.faces.bean.ViewScoped; -import javax.faces.context.FacesContext; - -import org.apache.log4j.Logger; - -import backtype.storm.Config; -import backtype.storm.generated.SupervisorWorkers; -import backtype.storm.generated.TopologyMetricInfo; -import backtype.storm.generated.WorkerSummary; -import backtype.storm.generated.WorkerMetricData; -import backtype.storm.utils.NimbusClient; - -import com.alibaba.jstorm.ui.UIUtils; -import com.alibaba.jstorm.ui.model.SupervisorSumm; -import com.alibaba.jstorm.ui.model.WorkerSumm; -import com.alibaba.jstorm.ui.model.WorkerMetrics; - -/** - * - * @author xin.zhou/Longda - */ -@ManagedBean(name = "supervisorpage") -@ViewScoped -public class SupervisorPage implements Serializable { - - private static final long serialVersionUID = -6103468103521877721L; - - private static final Logger LOG = Logger.getLogger(SupervisorPage.class); - - private String clusterName = null; - private String host = "localhost"; - private String ip = null; - - private List ssumm = null; - private List wsumm = null; - private List topologyList = null; - private List topologyMetricsList = null; - private List workermetrics = null; - - public SupervisorPage() throws Exception { - FacesContext ctx = FacesContext.getCurrentInstance(); - if (ctx.getExternalContext().getRequestParameterMap().get("clusterName") != null) { - clusterName = (String) ctx.getExternalContext().getRequestParameterMap() - .get("clusterName"); - } - if (ctx.getExternalContext().getRequestParameterMap().get("host") != null) { - host = (String) ctx.getExternalContext().getRequestParameterMap() - .get("host"); - } - - init(host); - } - - @SuppressWarnings("rawtypes") - public void init(String host) throws Exception { - - NimbusClient client = null; - try { - - Map conf = UIUtils.readUiConfig(); - client = UIUtils.getNimbusClient(conf, clusterName); - SupervisorWorkers supervisorWorkers = client.getClient() - .getSupervisorWorkers(host); - ssumm = new ArrayList(); - SupervisorSumm supervSumm = new SupervisorSumm(supervisorWorkers.get_supervisor()); - ssumm.add(supervSumm); - ip = supervSumm.getIp(); - generateWorkerSum(supervisorWorkers.get_workers()); - getTopoList(); - getTopoMetrList(client); - getWorkerMetrData(); - - } catch (Exception e) { - LOG.error("Failed to get cluster information:", e); - throw e; - } finally { - if (client != null) { - client.close(); - } - } - } - - private void generateWorkerSum(List workerSummaries) { - wsumm = new ArrayList(); - - for (WorkerSummary workerSummary : workerSummaries) { - wsumm.add(new WorkerSumm(workerSummary)); - } - } - - public List getSsumm() { - return ssumm; - } - - public void setSsumm(List ssumm) { - this.ssumm = ssumm; - } - - public List getWsumm() { - return wsumm; - } - - public void setWsumm(List wsumm) { - this.wsumm = wsumm; - } - public void setworkermetrics(List wrkMetrList) { - this.workermetrics = wrkMetrList; - } - public List getworkermetrics(){ - return this.workermetrics; - } - public void getTopoList() { - if (topologyList == null) { - topologyList = new ArrayList(); - } - if (wsumm == null) return; - for(WorkerSumm workerSumm : wsumm) { - String topologyId = workerSumm.getTopology(); - if (!(topologyList.contains(topologyId))) { - topologyList.add(topologyId); - } - } - } - public void getTopoMetrList(NimbusClient client) throws Exception { - if (topologyList == null) return; - if (topologyMetricsList == null) { - topologyMetricsList = new ArrayList(); - } - for (String topologyId : topologyList) { - try { - TopologyMetricInfo topoMetrInfo = client.getClient().getTopologyMetric(topologyId); - topologyMetricsList.add(topoMetrInfo); - } catch (Exception e) { - LOG.error("Failed to get topology metrics information:", e); - throw e; - } - } - } - public void getWorkerMetrData() { - if (topologyMetricsList == null) return; - if (workermetrics == null) { - workermetrics = new ArrayList(); - } - for (TopologyMetricInfo topoMetr : topologyMetricsList) { - List wrkMetrLstFromTopo = topoMetr.get_worker_metric_list(); - if (wrkMetrLstFromTopo == null) return; - for (WorkerMetricData wrkMetrData : wrkMetrLstFromTopo) { - if (wrkMetrData.get_hostname().equals(host) || - wrkMetrData.get_hostname().equals(ip)) { - WorkerMetrics workerMetrics = new WorkerMetrics(); - workerMetrics.updateWorkerMetricData(wrkMetrData); - workermetrics.add(workerMetrics); - } - } - } - } - - public static void main(String[] args) { - try { - SupervisorPage m = new SupervisorPage(); - // m.init("free-56-151.shucang.alipay.net"); - } catch (Exception e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - - } -} \ No newline at end of file diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/Taskpage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/Taskpage.java deleted file mode 100644 index b403454ea..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/Taskpage.java +++ /dev/null @@ -1,159 +0,0 @@ -package com.alibaba.jstorm.ui.model.data; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -import javax.faces.bean.ManagedBean; -import javax.faces.bean.ViewScoped; -import javax.faces.context.FacesContext; - -import org.apache.log4j.Logger; -import org.apache.thrift7.TException; - -import backtype.storm.generated.ErrorInfo; -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.common.stats.StatBuckets; -import com.alibaba.jstorm.ui.UIUtils; -import com.alibaba.jstorm.ui.model.ErrorSummary; -import com.alibaba.jstorm.ui.model.TaskSumm; - -@ManagedBean(name = "taskpage") -@ViewScoped -public class Taskpage { - - private static final Logger LOG = Logger.getLogger(Taskpage.class); - - private String topologyid = null; - private String componentId = null; - private String taskid = null; - private String window = null; - private List tsumms = null; - private List esumms = null; - private String clusterName; - - public Taskpage() throws TException, NotAliveException { - FacesContext ctx = FacesContext.getCurrentInstance(); - if (ctx.getExternalContext().getRequestParameterMap().get("clusterName") != null) { - clusterName = (String) ctx.getExternalContext() - .getRequestParameterMap().get("clusterName"); - } - if (ctx.getExternalContext().getRequestParameterMap().get("topologyid") != null) { - topologyid = (String) ctx.getExternalContext() - .getRequestParameterMap().get("topologyid"); - } - if (ctx.getExternalContext().getRequestParameterMap().get("taskid") != null) { - taskid = (String) ctx.getExternalContext().getRequestParameterMap() - .get("taskid"); - } - - if (topologyid == null) { - throw new NotAliveException("Input topologyId is null "); - } - - window = UIUtils.getWindow(ctx); - init(); - } - - private void init() throws TException, NotAliveException { - - NimbusClient client = null; - - try { - Map conf = Utils.readStormConfig(); - client = UIUtils.getNimbusClient(conf, clusterName); - - TopologyInfo summ = client.getClient().getTopologyInfo(topologyid); - - List ts = summ.get_tasks(); - if (ts == null) { - throw new NotAliveException("No TaskSummary"); - } - TaskSummary t = null; - if (ts != null) { - int tssize = ts.size(); - for (int i = 0; i < tssize; i++) { - if (ts.get(i).get_task_id() == Integer.parseInt(taskid)) { - t = ts.get(i); - } - } - } - - tsumms = taskSummaryTable(t, summ); - esumms = taskErrorTable(t); - } catch (TException e) { - LOG.error(e.getCause(), e); - throw e; - } catch (NotAliveException e) { - LOG.error(e.getCause(), e); - throw e; - } catch (Exception e) { - LOG.error(e.getCause(), e); - throw new TException(e); - }finally { - if (client != null) { - client.close(); - } - } - - } - - public List taskSummaryTable(TaskSummary task, TopologyInfo summ) { - List tsums = new ArrayList(); - TaskSumm tsumm = new TaskSumm(String.valueOf(task.get_task_id()), - task.get_host(), String.valueOf(task.get_port()), - summ.get_name(), task.get_component_id(), - StatBuckets.prettyUptimeStr(task.get_uptime_secs())); - - tsums.add(tsumm); - return tsums; - } - - public List taskErrorTable(TaskSummary task) { - List esums = new ArrayList(); - List errors = task.get_errors(); - if (errors != null) { - int errorsize = errors.size(); - - for (int i = 0; i < errorsize; i++) { - ErrorInfo einfo = errors.get(i); - ErrorSummary esumm = new ErrorSummary(String.valueOf(einfo - .get_error_time_secs()), einfo.get_error()); - - esums.add(esumm); - } - } - return esums; - } - - public List getTsumms() { - return tsumms; - } - - public void setTsumms(List tsumms) { - this.tsumms = tsumms; - } - - public List getEsumms() { - return esumms; - } - - public void setEsumms(List esumms) { - this.esumms = esumms; - } - - public String getClusterName() { - return clusterName; - } - - public void setClusterName(String clusterName) { - this.clusterName = clusterName; - } - - -} 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 deleted file mode 100644 index 73a197841..000000000 --- a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/data/TopologyPage.java +++ /dev/null @@ -1,295 +0,0 @@ -/* - * To change this template, choose Tools | Templates - * and open the template in the editor. - */ -package com.alibaba.jstorm.ui.model.data; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.HashMap; -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.log4j.Logger; - -import backtype.storm.Config; -import backtype.storm.generated.NotAliveException; -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; -import com.alibaba.jstorm.ui.model.WinComponentStats; -import com.alibaba.jstorm.utils.JStormUtils; - -/** - * - * @author xin.zhou/Longda - */ -@ManagedBean(name = "topologypage") -@ViewScoped -public class TopologyPage implements Serializable { - - private static final long serialVersionUID = -214838419818487219L; - - private static final Logger LOG = Logger.getLogger(TopologyPage.class); - - private String clusterName = null; - private String topologyid = null; - private String window = null; - private List tsumm = null; - private List tstats = null; - private List scom = null; - private List bcom = null; - private List udm = null; - - public TopologyPage() throws Exception { - - FacesContext ctx = FacesContext.getCurrentInstance(); - if (ctx.getExternalContext().getRequestParameterMap().get("clusterName") != null) { - clusterName = (String) ctx.getExternalContext() - .getRequestParameterMap().get("clusterName"); - } - if (ctx.getExternalContext().getRequestParameterMap().get("topologyid") != null) { - topologyid = (String) ctx.getExternalContext() - .getRequestParameterMap().get("topologyid"); - } - LOG.debug("Query clusterName=" + clusterName + ", topology=" + topologyid); - - window = UIUtils.getWindow(ctx); - LOG.info("Window:" + window); - - if (topologyid == null) { - - LOG.error("Not set topologyid"); - throw new NotAliveException("Input topologyId is null "); - - } - - init(); - } - - public TopologyPage(String clusterName, String topologyId, String window) throws Exception { - this.clusterName = clusterName; - this.topologyid = topologyId; - this.window = window; - - LOG.info("Window:" + window); - init(); - } - - private void init() throws Exception { - - NimbusClient client = null; - - try { - Map conf = UIUtils.readUiConfig(); - - client = UIUtils.getNimbusClient(conf, clusterName); - - TopologyInfo summ = client.getClient().getTopologyInfo(topologyid); - StormTopology topology = client.getClient().getTopology(topologyid); - - List ts = summ.get_tasks(); - - udm = summ.get_userDefMetric(); - - tsumm = UIUtils.topologySummary(summ); - - getComponents(ts, topology); - - tstats = topologyStatsTable(scom, bcom); - - } catch (Exception e) { - LOG.error("Failed to get topology information,", e); - throw e; - } finally { - - if (client != null) { - client.close(); - } - } - } - - private List getComponents( - Map> componentMap, String type) { - List components = new ArrayList(); - - for (Entry> entry : componentMap.entrySet()) { - String componentId = entry.getKey(); - List taskList = entry.getValue(); - - Components component = UIUtils.getComponent(taskList, componentId, - type, window); - - String lastError = UIUtils.mostRecentError(taskList); - component.setLastError(lastError); - - components.add(component); - - } - - return components; - } - - /** - * get spout or bolt's List - * - * - * @param ts - * @param topology - * @throws NotAliveException - */ - private void getComponents(List ts, StormTopology topology) - throws NotAliveException { - if (ts == null) { - LOG.error("Task list is empty"); - throw new NotAliveException("Task list is empty"); - } - - Map> spoutTasks = new HashMap>(); - Map> boltTasks = new HashMap>(); - - for (TaskSummary t : ts) { - if (t == null) { - continue; - } - - String componentid = t.get_component_id(); - String componentType = UIUtils.componentType(topology, componentid); - if (componentType.equals(UIUtils.SPOUT_STR)) { - - UIUtils.addTask(spoutTasks, t, componentid); - } else if (componentType.equals(UIUtils.BOLT_STR)) { - UIUtils.addTask(boltTasks, t, componentid); - } - - } - - scom = getComponents(spoutTasks, UIUtils.SPOUT_STR); - bcom = getComponents(boltTasks, UIUtils.BOLT_STR); - } - - private List topologyStatsTable(List scom, - List bcom) { - List all = new ArrayList(); - all.addAll(scom); - all.addAll(bcom); - - Long emitted = 0l; - Double sendTps = 0.0; - Double recvTps = 0.0; - Long acked = 0l; - Long failed = 0l; - for (Components component : all) { - emitted += Long.valueOf(component.getEmitted()); - sendTps += Double.valueOf(component.getSendTps()); - recvTps += Double.valueOf(component.getRecvTps()); - acked += Long.valueOf(component.getAcked()); - failed += Long.valueOf(component.getFailed()); - - } - - Double process = 0.0; - Long spoutNum = Long.valueOf(0); - for (Components component : scom) { - if (component.getProcess() != null) { - process += (Double.valueOf(component.getProcess()) * Long - .valueOf(component.getParallelism())); - } - spoutNum = Long.valueOf(component.getParallelism()); - } - Double avergProcess = process / spoutNum; - - WinComponentStats topologyStats = new WinComponentStats(); - topologyStats.setWindow(window); - topologyStats.setEmitted(JStormUtils.formatValue(emitted)); - topologyStats.setSendTps(JStormUtils.formatValue(sendTps)); - topologyStats.setRecvTps(JStormUtils.formatValue(recvTps)); - topologyStats.setAcked(JStormUtils.formatValue(acked)); - topologyStats.setFailed(JStormUtils.formatValue(failed)); - topologyStats.setProcess(JStormUtils.formatValue(avergProcess)); - - List tss = new ArrayList(); - tss.add(topologyStats); - return tss; - } - - public List getUdm() { - return udm; - } - - public void setUdm(List udm) { - this.udm = udm; - } - - public String getTopologyid() { - return topologyid; - } - - public void setTopologyid(String topologyid) { - this.topologyid = topologyid; - } - - public String getWindow() { - return window; - } - - public void setWindow(String window) { - this.window = window; - } - - public List getTsumm() { - return tsumm; - } - - public void setTsumm(List tsumm) { - this.tsumm = tsumm; - } - - public List getTstats() { - return tstats; - } - - public void setTstats(List tstats) { - this.tstats = tstats; - } - - public List getScom() { - return scom; - } - - public void setScom(List scom) { - this.scom = scom; - } - - public List getBcom() { - return bcom; - } - - public void setBcom(List bcom) { - this.bcom = bcom; - } - - public static void main(String[] args) { - - try { - TopologyPage instance = new TopologyPage("jstorm", - "sequence_test-1-1386516240", StatBuckets.ALL_WINDOW_STR); - } catch (Exception e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - } -} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/ClusterPage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/ClusterPage.java new file mode 100755 index 000000000..de78bfe7c --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/ClusterPage.java @@ -0,0 +1,324 @@ +/** + * 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.ui.model.pages; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.generated.ClusterSummary; +import backtype.storm.generated.NimbusStat; +import backtype.storm.generated.NimbusSummary; +import backtype.storm.generated.SupervisorSummary; +import backtype.storm.utils.NimbusClient; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.common.metric.window.StatBuckets; +import com.alibaba.jstorm.ui.NimbusClientManager; +import com.alibaba.jstorm.ui.UIDef; +import com.alibaba.jstorm.ui.UIUtils; +import com.alibaba.jstorm.ui.model.ColumnData; +import com.alibaba.jstorm.ui.model.LinkData; +import com.alibaba.jstorm.ui.model.PageGenerator; +import com.alibaba.jstorm.ui.model.PageIndex; +import com.alibaba.jstorm.ui.model.TableData; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.NetWorkUtils; + +public class ClusterPage implements PageGenerator { + private static final Logger LOG = LoggerFactory + .getLogger(ClusterPage.class); + + public static final String TABLE_NAME_MASTER = "Nimbus Master"; + public static final String TABLE_NAME_SLAVES = "Nimbus Slaves"; + + public TableData getNimbusMaster(NimbusSummary nimbusSummary, + Map nimbusConf, Map paramMap) { + TableData nimbusMaster = new TableData(); + + nimbusMaster.setName(TABLE_NAME_MASTER); + List headers = nimbusMaster.getHeaders(); + List> lines = nimbusMaster.getLines(); + + headers.add(UIDef.HOST.toUpperCase()); + headers.add(UIDef.HEADER_UPTIME); + headers.add(UIDef.HEADER_SUPERVISOR); + headers.add(UIDef.HEADER_TOTAL_PORT); + headers.add(UIDef.HEADER_USED_PORT); + headers.add(UIDef.HEADER_FREE_PORT); + headers.add(UIDef.HEADER_VERSION); + headers.add(UIDef.HEADER_CONF); + headers.add(UIDef.HEADER_LOGS); + + Map line = new HashMap(); + lines.add(line); + + ColumnData nimbusMasterColumn = new ColumnData(); + String ipPort = nimbusSummary.get_nimbus_master().get_host(); + String hostPort = UIUtils.getHostIp(ipPort); + nimbusMasterColumn.addText(hostPort); + line.put(UIDef.HOST.toUpperCase(), nimbusMasterColumn); + + ColumnData uptimeColumn = new ColumnData(); + String uptime = nimbusSummary.get_nimbus_master().get_uptime_secs(); + if (uptime == null) { + uptimeColumn.addText(StatBuckets.prettyUptimeStr(0)); + } else { + uptimeColumn.addText(StatBuckets.prettyUptimeStr(Integer + .valueOf(uptime))); + } + line.put(UIDef.HEADER_UPTIME, uptimeColumn); + + ColumnData supervisorColumn = new ColumnData(); + supervisorColumn.addText(String.valueOf(nimbusSummary + .get_supervisor_num())); + line.put(UIDef.HEADER_SUPERVISOR, supervisorColumn); + + ColumnData totalPortColumn = new ColumnData(); + totalPortColumn.addText(String.valueOf(nimbusSummary + .get_total_port_num())); + line.put(UIDef.HEADER_TOTAL_PORT, totalPortColumn); + + ColumnData usedPortColumn = new ColumnData(); + usedPortColumn + .addText(String.valueOf(nimbusSummary.get_used_port_num())); + line.put(UIDef.HEADER_USED_PORT, usedPortColumn); + + ColumnData freePortColumn = new ColumnData(); + freePortColumn + .addText(String.valueOf(nimbusSummary.get_free_port_num())); + line.put(UIDef.HEADER_FREE_PORT, freePortColumn); + + ColumnData versionColumn = new ColumnData(); + versionColumn.addText(nimbusSummary.get_version()); + line.put(UIDef.HEADER_VERSION, versionColumn); + + ColumnData confColumn = new ColumnData(); + LinkData confLink = new LinkData(); + confColumn.addLinkData(confLink); + line.put(UIDef.HEADER_CONF, confColumn); + + confLink.setUrl(UIDef.LINK_TABLE_PAGE); + confLink.setText(UIDef.HEADER_CONF.toLowerCase()); + confLink.addParam(UIDef.CLUSTER, paramMap.get(UIDef.CLUSTER)); + confLink.addParam(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_CONF); + confLink.addParam(UIDef.CONF_TYPE, UIDef.CONF_TYPE_NIMBUS); + + ColumnData logsColumn = new ColumnData(); + LinkData logsLink = new LinkData(); + logsColumn.addLinkData(logsLink); + line.put(UIDef.HEADER_LOGS, logsColumn); + + logsLink.setUrl(UIDef.LINK_TABLE_PAGE); + logsLink.setText(UIDef.HEADER_LOGS.toLowerCase()); + logsLink.addParam(UIDef.CLUSTER, paramMap.get(UIDef.CLUSTER)); + logsLink.addParam(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_LISTLOG); + logsLink.addParam(UIDef.HOST, UIUtils.getHost(ipPort)); + logsLink.addParam(UIDef.PORT, String.valueOf(ConfigExtension + .getNimbusDeamonHttpserverPort(nimbusConf))); + logsLink.addParam(UIDef.DIR, "."); + + return nimbusMaster; + } + + public TableData getNimbusSlaves(NimbusSummary nimbusSummary, + Map nimbusConf, Map paramMap) { + List slaves = nimbusSummary.get_nimbus_slaves(); + if (slaves == null || slaves.size() == 0) { + return null; + } + + TableData slavesTable = new TableData(); + List headers = slavesTable.getHeaders(); + List> lines = slavesTable.getLines(); + slavesTable.setName(TABLE_NAME_SLAVES); + + headers.add(UIDef.HOST.toUpperCase()); + headers.add(UIDef.HEADER_UPTIME); + headers.add(UIDef.HEADER_LOGS); + + for (NimbusStat slave : slaves) { + + Map line = new HashMap(); + lines.add(line); + + ColumnData hostColumn = new ColumnData(); + String ipPort = slave.get_host(); + String hostPort = UIUtils.getHostIp(ipPort); + hostColumn.addText(hostPort); + line.put(UIDef.HOST.toUpperCase(), hostColumn); + + ColumnData uptimeColumn = new ColumnData(); + String uptime = slave.get_uptime_secs(); + if (uptime == null) { + uptimeColumn.addText(StatBuckets.prettyUptimeStr(0)); + } else { + uptimeColumn.addText(StatBuckets.prettyUptimeStr(Integer + .valueOf(uptime))); + } + line.put(UIDef.HEADER_UPTIME, uptimeColumn); + + ColumnData logsColumn = new ColumnData(); + LinkData logsLink = new LinkData(); + logsColumn.addLinkData(logsLink); + line.put(UIDef.HEADER_LOGS, logsColumn); + + logsLink.setUrl(UIDef.LINK_TABLE_PAGE); + logsLink.setText(UIDef.HEADER_LOGS.toLowerCase()); + logsLink.addParam(UIDef.CLUSTER, paramMap.get(UIDef.CLUSTER)); + logsLink.addParam(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_LISTLOG); + logsLink.addParam(UIDef.HOST, UIUtils.getHost(ipPort)); + logsLink.addParam(UIDef.PORT, String.valueOf(ConfigExtension + .getNimbusDeamonHttpserverPort(nimbusConf))); + logsLink.addParam(UIDef.DIR, "."); + } + + return slavesTable; + + } + + public List getNimbus(NimbusSummary nimbusSummary, + Map nimbusConf, Map paramMap) { + List ret = new ArrayList(); + TableData nimbusMaster = + getNimbusMaster(nimbusSummary, nimbusConf, paramMap); + ret.add(nimbusMaster); + + TableData slavesTable = + getNimbusSlaves(nimbusSummary, nimbusConf, paramMap); + if (slavesTable != null) { + ret.add(slavesTable); + } + + return ret; + } + + public TableData getSupervisorTable( + List supervisorSummaries, + Map paramMap) { + TableData table = new TableData(); + List headers = table.getHeaders(); + List> lines = table.getLines(); + table.setName(UIDef.SUPERVISOR.toUpperCase()); + + headers.add(UIDef.HOST.toUpperCase()); + headers.add(UIDef.HEADER_UPTIME); + headers.add(UIDef.HEADER_TOTAL_PORT); + headers.add(UIDef.HEADER_USED_PORT); + + if (supervisorSummaries == null) { + return table; + } + + long pos = JStormUtils.parseLong(paramMap.get(UIDef.POS), 0); + + for ( long index = pos; + index < supervisorSummaries.size() && index < pos + UIUtils.ONE_TABLE_PAGE_SIZE; + index++ ) { + SupervisorSummary supervisorSummary = supervisorSummaries.get((int)index); + Map line = new HashMap(); + lines.add(line); + + ColumnData hostColumn = new ColumnData(); + LinkData linkData = new LinkData(); + hostColumn.addLinkData(linkData); + line.put(UIDef.HOST.toUpperCase(), hostColumn); + + linkData.setUrl(UIDef.LINK_TABLE_PAGE); + String ip = supervisorSummary.get_host(); + linkData.setText(NetWorkUtils.ip2Host(ip)); + linkData.addParam(UIDef.CLUSTER, paramMap.get(UIDef.CLUSTER)); + linkData.addParam(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_SUPERVISOR); + linkData.addParam(UIDef.HOST, ip); + + ColumnData uptimeColumn = new ColumnData(); + int uptime = supervisorSummary.get_uptime_secs(); + uptimeColumn.addText(StatBuckets.prettyUptimeStr(uptime)); + line.put(UIDef.HEADER_UPTIME, uptimeColumn); + + ColumnData totalPortColumn = new ColumnData(); + totalPortColumn.addText(String.valueOf(supervisorSummary + .get_num_workers())); + line.put(UIDef.HEADER_TOTAL_PORT, totalPortColumn); + + ColumnData usedPortColumn = new ColumnData(); + usedPortColumn.addText(String.valueOf(supervisorSummary + .get_num_used_workers())); + line.put(UIDef.HEADER_USED_PORT, usedPortColumn); + + } + + return table; + } + + @Override + public Output generate(Map paramMap) { + // TODO Auto-generated method stub + List tables = null; + + NimbusClient client = null; + try { + client = NimbusClientManager.getNimbusClient(paramMap); + + ClusterSummary clusterSummary = client.getClient().getClusterInfo(); + + String jsonConf = client.getClient().getNimbusConf(); + Map nimbusConf = + (Map) Utils.from_json(jsonConf); + + NimbusSummary nimbusSummary = clusterSummary.get_nimbus(); + + tables = getNimbus(nimbusSummary, nimbusConf, paramMap); + + TableData topologyTable = + UIUtils.getTopologyTable(clusterSummary.get_topologies(), + paramMap); + tables.add(topologyTable); + + tables.add(getSupervisorTable(clusterSummary.get_supervisors(), + paramMap)); + + Output ret = new Output(); + ret.tables = tables; + ret.rawData = ""; + if (clusterSummary.get_supervisors().size() > UIUtils.ONE_TABLE_PAGE_SIZE) { + ret.pages = PageIndex.generatePageIndex( + clusterSummary.get_supervisors().size(), + UIUtils.ONE_TABLE_PAGE_SIZE, + UIDef.LINK_TABLE_PAGE, paramMap); + } + + + return ret; + } catch (Exception e) { + // TODO Auto-generated catch block + NimbusClientManager.removeClient(paramMap); + LOG.error(e.getMessage(), e); + + + return UIUtils.getErrorInfo(e); + } + + } + +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/ComponentPage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/ComponentPage.java new file mode 100755 index 000000000..3d998e9fd --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/ComponentPage.java @@ -0,0 +1,473 @@ +/** + * 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.ui.model.pages; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.TreeMap; + +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.common.metric.window.StatBuckets; +import com.alibaba.jstorm.ui.NimbusClientManager; +import com.alibaba.jstorm.ui.UIDef; +import com.alibaba.jstorm.ui.UIUtils; +import com.alibaba.jstorm.ui.model.ColumnData; +import com.alibaba.jstorm.ui.model.LinkData; +import com.alibaba.jstorm.ui.model.PageGenerator; +import com.alibaba.jstorm.ui.model.PageIndex; +import com.alibaba.jstorm.ui.model.TableData; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.NetWorkUtils; + +import backtype.storm.generated.ComponentSummary; +import backtype.storm.generated.ErrorInfo; +import backtype.storm.generated.MetricInfo; +import backtype.storm.generated.MetricWindow; +import backtype.storm.generated.TaskSummary; +import backtype.storm.generated.TopologyInfo; +import backtype.storm.utils.NimbusClient; +import backtype.storm.utils.Utils; + +public class ComponentPage implements PageGenerator { + private static final Logger LOG = LoggerFactory + .getLogger(ComponentPage.class); + + public String getWindowStr(Integer window) { + return "(" + StatBuckets.prettyUptimeStr(window) + ")"; + } + + public TableData getComponentSummary(TopologyInfo topologyInfo, + ComponentSummary componentSummary, Map paramMap, + Integer window) { + TableData table = new TableData(); + List headers = table.getHeaders(); + List> lines = table.getLines(); + table.setName(componentSummary.get_name() + getWindowStr(window)); + + headers.add(UIDef.HEADER_COMPONENT_NAME); + headers.add(UIDef.HEADER_TOPOLOGY_NAME); + headers.add(UIDef.HEADER_TASK_NUM); + headers.add(UIDef.HEADER_COMPONENT_TYPE); + + Map line = new HashMap(); + lines.add(line); + + ColumnData nameColumn = new ColumnData(); + nameColumn.addText(componentSummary.get_name()); + line.put(UIDef.HEADER_COMPONENT_NAME, nameColumn); + + ColumnData topologyColumn = new ColumnData(); + LinkData linkData = new LinkData(); + topologyColumn.addLinkData(linkData); + line.put(UIDef.HEADER_TOPOLOGY_NAME, topologyColumn); + + linkData.setUrl(UIDef.LINK_WINDOW_TABLE); + linkData.setText(topologyInfo.get_topology().get_name()); + linkData.addParam(UIDef.CLUSTER, paramMap.get(UIDef.CLUSTER)); + linkData.addParam(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_TOPOLOGY); + linkData.addParam(UIDef.TOPOLOGY, topologyInfo.get_topology().get_id()); + linkData.addParam(UIDef.WINDOW, StatBuckets.prettyUptimeStr(window)); + + ColumnData taskNumColumn = new ColumnData(); + taskNumColumn.addText(String.valueOf(componentSummary.get_parallel())); + line.put(UIDef.HEADER_TASK_NUM, taskNumColumn); + + ColumnData typeColumn = new ColumnData(); + typeColumn.addText(componentSummary.get_type()); + line.put(UIDef.HEADER_COMPONENT_TYPE, typeColumn); + + return table; + } + + public TableData getComponentMetrics(TopologyInfo topologyInfo, + ComponentSummary componentSummary, Map paramMap, + Integer window) { + + MetricInfo metricInfo = + topologyInfo.get_metrics().get_componentMetric() + .get(componentSummary.get_name()); + if (metricInfo == null) { + LOG.info("No component metric of " + componentSummary.get_name()); + return null; + } + + TableData table = UIUtils.getMetricTable(metricInfo, window); + table.setName(componentSummary.get_name() + "-Metrics-" + + getWindowStr(window)); + return table; + } + + public TableData getStreamMetrics( + Map> streamMetric, Integer window) { + TableData table = new TableData(); + List headers = table.getHeaders(); + List> lines = table.getLines(); + + headers.add(UIDef.HEADER_STREAM_ID); + headers.addAll(UIUtils.getSortedKeys(streamMetric.keySet())); + + Map> lineMap = + new HashMap>(); + for (Entry> entry : streamMetric + .entrySet()) { + String metricName = entry.getKey(); + Map streamMetricWindow = entry.getValue(); + for (Entry streamEntry : streamMetricWindow + .entrySet()) { + String streamId = streamEntry.getKey(); + MetricWindow metric = streamEntry.getValue(); + + Map line = lineMap.get(streamId); + if (line == null) { + line = new HashMap(); + lineMap.put(streamId, line); + } + + String value = + String.valueOf(metric.get_metricWindow().get(window)); + + ColumnData columnData = new ColumnData(); + columnData.addText(value); + line.put(metricName, columnData); + } + } + + for (Entry> entry : lineMap.entrySet()) { + String streamId = entry.getKey(); + Map line = entry.getValue(); + + lines.add(line); + + ColumnData columnData = new ColumnData(); + columnData.addText(streamId); + line.put(UIDef.HEADER_STREAM_ID, columnData); + + } + + UIUtils.complementingTable(table); + + return table; + } + + public TableData getInputComponentMetrics(TopologyInfo topologyInfo, + ComponentSummary componentSummary, Map paramMap, + Integer window) { + String name = componentSummary.get_name(); + MetricInfo metric = + topologyInfo.get_metrics().get_componentMetric().get(name); + if (metric == null) { + LOG.info("No component metric of " + name); + return null; + } + + Map> input = metric.get_inputMetric(); + if (input == null || input.size() == 0) { + LOG.info("No input metric of " + name); + return null; + } + + TableData table = getStreamMetrics(input, window); + table.setName("Input stats" + getWindowStr(window)); + + return table; + } + + public TableData getOutputComponentMetrics(TopologyInfo topologyInfo, + ComponentSummary componentSummary, Map paramMap, + Integer window) { + String name = componentSummary.get_name(); + MetricInfo metric = + topologyInfo.get_metrics().get_componentMetric().get(name); + if (metric == null) { + LOG.info("No component metric of " + componentSummary.get_name()); + return null; + } + + Map> output = + metric.get_outputMetric(); + if (output == null || output.size() == 0) { + LOG.info("No output metric of " + name); + return null; + } + + TableData table = getStreamMetrics(output, window); + table.setName("Output stats" + getWindowStr(window)); + + return table; + } + + public TableData getTaskSummary(ComponentSummary componentSummary, + List tasks, Map paramMap, + String topologyId, Map nimbusConf, + List showTasks) { + TableData table = new TableData(); + List headers = table.getHeaders(); + List> lines = table.getLines(); + table.setName("Tasks"); + + headers.add(UIDef.HEADER_TASK_ID); + headers.add(UIDef.HEADER_STATUS); + headers.add(UIDef.HEADER_UPTIME); + headers.add(StringUtils.capitalize(UIDef.HOST)); + headers.add(StringUtils.capitalize(UIDef.PORT)); + headers.add(UIDef.HEADER_LOG); + headers.add(UIDef.HEADER_ERROR); + + List taskIds = componentSummary.get_task_ids(); + Set taskIdSet = new HashSet(); + taskIdSet.addAll(taskIds); + + + long pos = JStormUtils.parseLong(paramMap.get(UIDef.POS), 0); + long index = 0; + + Collections.sort(tasks); + for (TaskSummary task : tasks) { + + Integer taskId = task.get_task_id(); + if (taskIdSet.contains(taskId) == false) { + continue; + } + + if (index < pos) { + index ++; + continue; + }else if (pos <= index && index < pos + UIUtils.ONE_TABLE_PAGE_SIZE) { + showTasks.add(task); + taskIdSet.remove(taskId); + index++; + }else { + break; + } + + + } + + int logPort = + ConfigExtension.getSupervisorDeamonHttpserverPort(nimbusConf); + for (TaskSummary task : showTasks) { + Map line = new HashMap(); + lines.add(line); + + ColumnData taskIdColumn = new ColumnData(); + taskIdColumn.addText(String.valueOf(task.get_task_id())); + line.put(UIDef.HEADER_TASK_ID, taskIdColumn); + + ColumnData statusColumn = new ColumnData(); + statusColumn.addText(task.get_status()); + line.put(UIDef.HEADER_STATUS, statusColumn); + + ColumnData uptimeColumn = new ColumnData(); + int uptime = task.get_uptime(); + uptimeColumn.addText(StatBuckets.prettyUptimeStr(uptime)); + line.put(UIDef.HEADER_UPTIME, uptimeColumn); + + ColumnData hostColumn = new ColumnData(); + LinkData linkData = new LinkData(); + hostColumn.addLinkData(linkData); + line.put(StringUtils.capitalize(UIDef.HOST), hostColumn); + + linkData.setUrl(UIDef.LINK_TABLE_PAGE); + linkData.setText(NetWorkUtils.ip2Host(task.get_host())); + linkData.addParam(UIDef.CLUSTER, paramMap.get(UIDef.CLUSTER)); + linkData.addParam(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_SUPERVISOR); + linkData.addParam(UIDef.HOST, task.get_host()); + + ColumnData portColumn = new ColumnData(); + portColumn.addText(String.valueOf(task.get_port())); + line.put(StringUtils.capitalize(UIDef.PORT), portColumn); + + ColumnData logColumn = new ColumnData(); + LinkData logLink = new LinkData(); + logColumn.addLinkData(logLink); + line.put(UIDef.HEADER_LOG, logColumn); + + logLink.setUrl(UIDef.LINK_LOG); + logLink.setText(UIDef.HEADER_LOG.toLowerCase()); + logLink.addParam(UIDef.CLUSTER, paramMap.get(UIDef.CLUSTER)); + logLink.addParam(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_LOG); + logLink.addParam(UIDef.HOST, task.get_host()); + logLink.addParam(UIDef.TOPOLOGY, topologyId); + logLink.addParam(UIDef.PORT, String.valueOf(task.get_port())); + logLink.addParam(UIDef.LOG_SERVER_PORT, String.valueOf(logPort)); + + ColumnData errColumn = new ColumnData(); + List errList = task.get_errors(); + if (errList == null || errList.size() == 0) { + errColumn.addText(""); + } else { + for (ErrorInfo err : errList) { + errColumn.addText(err.get_error() + "\r\n"); + } + } + line.put(UIDef.HEADER_ERROR, errColumn); + } + + return table; + + } + + public TableData getTaskMetrics(TopologyInfo topologyInfo, List showTasks, + ComponentSummary componentSummary, Map paramMap, + Integer window) { + TableData table = new TableData(); + List headers = table.getHeaders(); + List> lines = table.getLines(); + + Map allTaskMetrics = + topologyInfo.get_metrics().get_taskMetric(); + Map taskMetrics = + new TreeMap(); + + for ( TaskSummary taskSummary : showTasks) { + Integer taskId = taskSummary.get_task_id(); + MetricInfo metric = allTaskMetrics.get(taskId); + if (metric == null) { + LOG.error("No task metric of " + taskId); + continue; + } + + taskMetrics.put(taskId, metric); + } + + headers.add(UIDef.HEADER_TASK_ID); + List keys = + UIUtils.getSortedKeys(UIUtils.getKeys(taskMetrics.values())); + headers.addAll(keys); + + for (Entry entry : taskMetrics.entrySet()) { + Integer taskId = entry.getKey(); + MetricInfo metric = entry.getValue(); + + Map line = + UIUtils.getMetricLine(metric, headers, window); + + ColumnData taskIdColumn = new ColumnData(); + taskIdColumn.addText(String.valueOf(taskId)); + line.put(UIDef.HEADER_TASK_ID, taskIdColumn); + + lines.add(line); + } + + return table; + } + + public Output generate(Map paramMap) { + List tables = new ArrayList(); + + NimbusClient client = null; + try { + client = NimbusClientManager.getNimbusClient(paramMap); + + String jsonConf = client.getClient().getNimbusConf(); + Map nimbusConf = + (Map) Utils.from_json(jsonConf); + + String topologyId = paramMap.get(UIDef.TOPOLOGY); + if (topologyId == null) { + throw new IllegalArgumentException("Not set topologyId"); + } + + String componentNam = paramMap.get(UIDef.COMPONENT); + if (componentNam == null) { + throw new IllegalArgumentException("Not set ComponentNam"); + } + + String windowStr = paramMap.get(UIDef.WINDOW); + Integer window = StatBuckets.getTimeKey(windowStr); + + TopologyInfo topologyInfo = + client.getClient().getTopologyInfo(topologyId); + ComponentSummary componentSummary = null; + List componentSummaries = + topologyInfo.get_components(); + for (ComponentSummary item : componentSummaries) { + if (item.get_name().equals(componentNam)) { + componentSummary = item; + break; + } + } + + if (componentSummary == null) { + throw new IllegalArgumentException("No Component of " + + componentNam); + } + + List myComponentSummaryList = + new ArrayList(); + myComponentSummaryList.add(componentSummary); + TableData componentTable = + UIUtils.getComponentTable(topologyInfo, + myComponentSummaryList, topologyInfo.get_metrics() + .get_componentMetric(), paramMap, window); + tables.add(componentTable); + + TableData inputTable = + getInputComponentMetrics(topologyInfo, componentSummary, + paramMap, window); + if (inputTable != null) { + tables.add(inputTable); + } + + TableData outputTable = + getOutputComponentMetrics(topologyInfo, componentSummary, + paramMap, window); + if (outputTable != null) { + tables.add(outputTable); + } + + List showTasks = new ArrayList(); + TableData taskSummaryTable = + getTaskSummary(componentSummary, topologyInfo.get_tasks(), + paramMap, topologyId, nimbusConf, showTasks); + tables.add(taskSummaryTable); + + TableData taskMetric = + getTaskMetrics(topologyInfo, showTasks, componentSummary, paramMap, + window); + tables.add(taskMetric); + + Output ret = new Output(); + ret.tables = tables; + ret.rawData = ""; + if (componentSummary.get_task_ids().size() > UIUtils.ONE_TABLE_PAGE_SIZE) { + ret.pages = PageIndex.generatePageIndex( + componentSummary.get_task_ids().size(), + UIUtils.ONE_TABLE_PAGE_SIZE, + UIDef.LINK_WINDOW_TABLE, paramMap); + } + return ret; + } catch (Exception e) { + NimbusClientManager.removeClient(paramMap); + + LOG.error(e.getMessage(), e); + + return UIUtils.getErrorInfo(e); + } + } +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/ConfPage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/ConfPage.java new file mode 100755 index 000000000..264a76319 --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/ConfPage.java @@ -0,0 +1,167 @@ +/** + * 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.ui.model.pages; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.http.HttpResponse; +import org.apache.http.client.ClientProtocolException; +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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.utils.NimbusClient; + +import com.alibaba.jstorm.ui.NimbusClientManager; +import com.alibaba.jstorm.ui.UIDef; +import com.alibaba.jstorm.ui.UIUtils; +import com.alibaba.jstorm.ui.model.ColumnData; +import com.alibaba.jstorm.ui.model.PageGenerator; +import com.alibaba.jstorm.ui.model.TableData; +import com.alibaba.jstorm.utils.HttpserverUtils; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.NetWorkUtils; + +public class ConfPage implements PageGenerator { + private static final Logger LOG = LoggerFactory.getLogger(ConfPage.class); + + public String getSupervisorConf(Map paramMap) + throws ClientProtocolException, IOException { + String host = paramMap.get(UIDef.HOST); + String port = paramMap.get(UIDef.PORT); + + if (host == null || port == null) { + throw new IllegalArgumentException( + "Invalid parameter, please set supervisor host and port"); + } + + final String PROXY_URL = "http://%s:%s/logview?%s=%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; + + // 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(), "utf-8"); + + return data; + } else { + return (EntityUtils.toString(response.getEntity())); + } + } + + public String getConf(Map paramMap) { + NimbusClient client = null; + try { + client = NimbusClientManager.getNimbusClient(paramMap); + + String confType = paramMap.get(UIDef.CONF_TYPE); + if (UIDef.CONF_TYPE_NIMBUS.equals(confType)) { + String confStr = client.getClient().getNimbusConf(); + return confStr; + } else if (UIDef.CONF_TYPE_TOPOLOGY.equals(confType)) { + String topologyId = paramMap.get(UIDef.TOPOLOGY); + if (topologyId == null) { + throw new IllegalArgumentException( + "Invalid parameter, please set topologyId"); + } + + return client.getClient().getTopologyConf(topologyId); + } else if (UIDef.CONF_TYPE_SUPERVISOR.equals(confType)) { + return getSupervisorConf(paramMap); + } else { + throw new IllegalArgumentException( + "Invalid parameter, please Conf type"); + } + + } catch (Exception e) { + NimbusClientManager.removeClient(paramMap); + LOG.error(e.getMessage(), e); + throw new RuntimeException(e); + } + } + + public TableData getTable(Map conf) { + TableData table = new TableData(); + List headers = table.getHeaders(); + List> lines = table.getLines(); + + table.setName(UIDef.HEADER_CONF); + + final String headerKey = "Key"; + final String headerValue = "value"; + headers.add(headerKey); + headers.add(headerValue); + + for (Entry entry : conf.entrySet()) { + Map line = new HashMap(); + lines.add(line); + + ColumnData keyColumn = new ColumnData(); + keyColumn.addText(entry.getKey()); + line.put(headerKey, keyColumn); + + ColumnData valueColumn = new ColumnData(); + valueColumn.addText(String.valueOf(entry.getValue())); + line.put(headerValue, valueColumn); + } + + return table; + } + + @Override + public Output generate(Map paramMap) { + // TODO Auto-generated method stub + List tables = new ArrayList(); + + try { + String confStr = getConf(paramMap); + Map conf = + (Map) JStormUtils.from_json(confStr); + + TableData table = getTable(conf); + tables.add(table); + + Output ret = new Output(); + ret.tables = tables; + ret.rawData = ""; + return ret; + } catch (Exception e) { + LOG.error(e.getMessage(), e); + + return UIUtils.getErrorInfo(e); + } + + } + +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/HomePage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/HomePage.java new file mode 100755 index 000000000..a5b65a54e --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/HomePage.java @@ -0,0 +1,182 @@ +/** + * 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.ui.model.pages; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import javax.faces.bean.ManagedBean; +import javax.faces.bean.ViewScoped; + +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.ui.UIDef; +import com.alibaba.jstorm.ui.UIMetrics; +import com.alibaba.jstorm.ui.UIUtils; +import com.alibaba.jstorm.ui.model.ColumnData; +import com.alibaba.jstorm.ui.model.LinkData; +import com.alibaba.jstorm.ui.model.TableData; + +/** + * + * @author xin.zhou/Longda + */ +@ManagedBean(name = "homepage") +@ViewScoped +public class HomePage implements Serializable { + + private static final long serialVersionUID = -6103468603521876731L; + + private static final Logger LOG = LoggerFactory.getLogger(HomePage.class); + + public static final String SINGLE_CLUSTER = "single"; + public static final String MULTI_CLUSTER = "multi"; + + public static final String HEADER_ZK_ROOT = "ZK Root"; + public static final String HEADER_ZK_SERVERS = "ZK Servers"; + public static final String HEADER_ZK_PORT = "ZK Port"; + + protected String clusterType; + protected List tables = new ArrayList(); + protected Map parameterMap; + + static { + // add code to load UIMetrics + UIMetrics.mkInstance(); + } + + public HomePage() throws Exception { + // FacesContext ctx = FacesContext.getCurrentInstance(); + // parameterMap = ctx.getExternalContext().getRequestParameterMap(); + + init(); + } + + @SuppressWarnings("rawtypes") + public void init() { + generateTables(); + + if (tables == null || tables.size() == 0 + || tables.get(0).getLines().size() == 1) { + clusterType = SINGLE_CLUSTER; + } else { + clusterType = MULTI_CLUSTER; + } + } + + public void generateTables() { + long start = System.nanoTime(); + try { + LOG.info("ClusterPage init..."); + Map conf = UIUtils.readUiConfig(); + List uiClusters = ConfigExtension.getUiClusters(conf); + if (uiClusters == null) { + return; + } + + TableData table = new TableData(); + tables.add(table); + + List headers = table.getHeaders(); + List> lines = table.getLines(); + table.setName("Cluster List"); + + headers.add(StringUtils.capitalize(UIDef.CLUSTER)); + headers.add(HEADER_ZK_ROOT); + headers.add(HEADER_ZK_SERVERS); + headers.add(HEADER_ZK_PORT); + + for (Map cluster : uiClusters) { + Map line = + new HashMap(); + lines.add(line); + + String clusterName = ConfigExtension.getUiClusterName(cluster); + ColumnData clusterColumn = new ColumnData(); + LinkData linkData = new LinkData(); + linkData.setUrl(UIDef.LINK_TABLE_PAGE); + linkData.setText(clusterName); + linkData.addParam(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_CLUSTER); + linkData.addParam(UIDef.CLUSTER, clusterName); + + clusterColumn.addLinkData(linkData); + + line.put(StringUtils.capitalize(UIDef.CLUSTER), clusterColumn); + + String zkRoot = ConfigExtension.getUiClusterZkRoot(cluster); + ColumnData zkRootColumn = new ColumnData(); + zkRootColumn.addText(zkRoot); + line.put(HEADER_ZK_ROOT, zkRootColumn); + + List servers = + ConfigExtension.getUiClusterZkServers(cluster); + ColumnData zkServerColumn = new ColumnData(); + for (String server : servers) { + zkServerColumn.addText(server); + } + line.put(HEADER_ZK_SERVERS, zkServerColumn); + + String port = + String.valueOf(ConfigExtension + .getUiClusterZkPort(cluster)); + ColumnData zkPortColumn = new ColumnData(); + zkPortColumn.addText(port); + line.put(HEADER_ZK_PORT, zkPortColumn); + } + + } catch (Exception e) { + LOG.error("Failed to get cluster information:", e); + throw new RuntimeException(e); + } finally { + long end = System.nanoTime(); + UIMetrics.updateHistorgram(this.getClass().getSimpleName(), (end - start)/1000000.0d); + + LOG.info("Finish ClusterPage"); + } + } + + public String getClusterType() { + return clusterType; + } + + public List getTables() { + return tables; + } + + public Map getParameterMap() { + return parameterMap; + } + + public static void main(String[] args) { + try { + HomePage c = new HomePage(); + + } catch (Exception e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + + } + +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/JStackPage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/JStackPage.java new file mode 100755 index 000000000..ad05ef1d6 --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/JStackPage.java @@ -0,0 +1,135 @@ +/** + * 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.ui.model.pages; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Map; + +import org.apache.commons.lang.StringUtils; +import org.apache.http.HttpResponse; +import org.apache.http.client.ClientProtocolException; +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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.ui.UIDef; +import com.alibaba.jstorm.ui.UIUtils; +import com.alibaba.jstorm.ui.model.PageGenerator; +import com.alibaba.jstorm.ui.model.TableData; +import com.alibaba.jstorm.utils.HttpserverUtils; +import com.alibaba.jstorm.utils.NetWorkUtils; + +public class JStackPage implements PageGenerator { + private static final long serialVersionUID = 4326599394273506085L; + + private static final Logger LOG = LoggerFactory.getLogger(JStackPage.class); + + protected static class Event { + public String clusterName; + public String host; + public String topologyId; + public String workerport; + public String logServerPort; + + public Event(Map paramMap) { + clusterName = paramMap.get(UIDef.CLUSTER); + host = paramMap.get(UIDef.HOST); + workerport = paramMap.get(UIDef.PORT); + topologyId = paramMap.get(UIDef.TOPOLOGY); + logServerPort = paramMap.get(UIDef.LOG_SERVER_PORT); + + if (StringUtils.isBlank(host)) { + throw new IllegalArgumentException("Please set host"); + } else if (StringUtils.isBlank(workerport)) { + throw new IllegalArgumentException("Please set port"); + } else if (StringUtils.isBlank(logServerPort)) { + throw new IllegalArgumentException("Please set dir"); + } + } + } + + public String getErrMsg(Event event) { + StringBuilder sb = new StringBuilder(); + sb.append(event.topologyId); + sb.append("'s worker on "); + sb.append(event.host); + sb.append(":"); + sb.append(event.workerport); + sb.append(" disappear."); + + return sb.toString(); + } + + public String getJStack(Event event) throws ClientProtocolException, + IOException { + final String PROXY_URL = "http://%s:%s/logview?%s=%s&%s=%s"; + String baseUrl = + String.format(PROXY_URL, NetWorkUtils.host2Ip(event.host), + event.logServerPort, + HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD, + HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD_JSTACK, + HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_WORKER_PORT, + event.workerport); + String url = baseUrl; + // 1. proxy call the task host log view service + HttpClient client = HttpClientBuilder.create().build(); + HttpPost post = new HttpPost(url); + HttpResponse response = client.execute(post); + + String errMsg = getErrMsg(event); + + if (response.getStatusLine().getStatusCode() == 200) { + String data = EntityUtils.toString(response.getEntity()); + + if (StringUtils.isBlank(data)) { + throw new IllegalArgumentException(errMsg); + } + + return data; + } else { + throw new IllegalArgumentException(errMsg); + } + + } + + @Override + public Output generate(Map paramMap) { + // TODO Auto-generated method stub + + try { + Event event = new Event(paramMap); + + String rawData = getJStack(event); + + Output output = new Output(); + output.tables = new ArrayList(); + output.rawData = rawData; + return output; + } catch (Exception e) { + LOG.error(e.getMessage(), e); + + return UIUtils.getErrorInfo(e); + } + + } + +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/ListLogsPage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/ListLogsPage.java new file mode 100755 index 000000000..aea4bb48b --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/ListLogsPage.java @@ -0,0 +1,232 @@ +/** + * 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.ui.model.pages; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.apache.commons.lang.StringUtils; +import org.apache.http.HttpResponse; +import org.apache.http.client.ClientProtocolException; +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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.ui.UIDef; +import com.alibaba.jstorm.ui.UIUtils; +import com.alibaba.jstorm.ui.model.ColumnData; +import com.alibaba.jstorm.ui.model.LinkData; +import com.alibaba.jstorm.ui.model.PageGenerator; +import com.alibaba.jstorm.ui.model.TableData; +import com.alibaba.jstorm.utils.FileAttribute; +import com.alibaba.jstorm.utils.HttpserverUtils; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.NetWorkUtils; + +public class ListLogsPage implements PageGenerator { + private static final long serialVersionUID = 4326599394273506085L; + + private static final Logger LOG = LoggerFactory + .getLogger(ListLogsPage.class); + + protected static class Event { + public String clusterName; + public String host; + public String port; + public String dir; + + public Event(Map paramMap) { + String host = paramMap.get(UIDef.HOST); + String port = paramMap.get(UIDef.PORT); + String dir = paramMap.get(UIDef.DIR); + String clusterName = paramMap.get(UIDef.CLUSTER); + + if (StringUtils.isBlank(host)) { + throw new IllegalArgumentException("Please set host"); + } else if (StringUtils.isBlank(port)) { + throw new IllegalArgumentException("Please set port"); + } else if (StringUtils.isBlank(dir)) { + throw new IllegalArgumentException("Please set dir"); + } + + this.host = host; + this.port = port; + this.dir = dir; + this.clusterName = clusterName; + } + + } + + private List parseString(String input, ListLogsPage.Event event) { + Map> map = + (Map>) JStormUtils.from_json(input); + + List ret = new ArrayList(); + TableData dirTable = new TableData(); + TableData fileTable = new TableData(); + ret.add(dirTable); + ret.add(fileTable); + + dirTable.setName("Dirs"); + fileTable.setName("Files"); + + Set keys = new HashSet(); + + for (Map jobj : map.values()) { + keys.addAll(jobj.keySet()); + + } + + List dirHeaders = dirTable.getHeaders(); + dirHeaders.addAll(keys); + List fileHeaders = fileTable.getHeaders(); + fileHeaders.addAll(keys); + + for (Map jobj : map.values()) { + boolean isDir = false; + Map line = new HashMap(); + + String fileName = null; + ColumnData linkColumn = null; + for (Entry entry : jobj.entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + + if (FileAttribute.FILE_NAME_FIELD.equals(key)) { + fileName = value; + } else if (FileAttribute.IS_DIR_FIELD.equals(key)) { + isDir = Boolean.valueOf(value); + + ColumnData columnData = new ColumnData(); + columnData.addText(value); + line.put(key, columnData); + } else { + + ColumnData columnData = new ColumnData(); + columnData.addText(value); + line.put(key, columnData); + } + } + + if (isDir == true) { + ColumnData column = new ColumnData(); + line.put(FileAttribute.FILE_NAME_FIELD, column); + LinkData linkData = new LinkData(); + column.addLinkData(linkData); + + linkData.setUrl(UIDef.LINK_TABLE_PAGE); + linkData.setText(fileName); + linkData.addParam(UIDef.CLUSTER, event.clusterName); + linkData.addParam(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_LISTLOG); + linkData.addParam(UIDef.HOST, event.host); + linkData.addParam(UIDef.PORT, event.port); + linkData.addParam(UIDef.DIR, event.dir + File.separator + + fileName); + + dirTable.getLines().add(line); + } else { + ColumnData column = new ColumnData(); + line.put(FileAttribute.FILE_NAME_FIELD, column); + LinkData linkData = new LinkData(); + column.addLinkData(linkData); + + linkData.setUrl(UIDef.LINK_LOG); + linkData.setText(fileName); + linkData.addParam(UIDef.CLUSTER, event.clusterName); + linkData.addParam(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_LOG); + linkData.addParam(UIDef.HOST, event.host); + linkData.addParam(UIDef.LOG_SERVER_PORT, event.port); + linkData.addParam(UIDef.LOG_NAME, event.dir + File.separator + + fileName); + fileTable.getLines().add(line); + } + + } + return ret; + } + + /** + * proxy query log for the specified task. + * + * @param task the specified task + * @throws IOException + * @throws ClientProtocolException + */ + private List listLogs(ListLogsPage.Event event) + throws ClientProtocolException, IOException { + + final String PROXY_URL = "http://%s:%s/logview?%s=%s&%s=%s"; + // PROXY_URL = "http://%s:%s/logview?%s=%s&dir=%s"; + String url = + String.format(PROXY_URL, NetWorkUtils.host2Ip(event.host), + event.port, + HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD, + HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD_LIST, + HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_DIR, event.dir); + + // 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()); + + return parseString(data, event); + + } else { + String data = EntityUtils.toString(response.getEntity()); + List ret = new ArrayList(); + ret.add(UIUtils.errorTable("Failed to list dir")); + return ret; + } + + } + + @Override + public Output generate(Map paramMap) { + // TODO Auto-generated method stub + ListLogsPage.Event event = new ListLogsPage.Event(paramMap); + + try { + List tables = listLogs(event); + + Output output = new Output(); + output.tables = tables; + output.rawData = ""; + return output; + } catch (Exception e) { + LOG.error(e.getMessage(), e); + + return UIUtils.getErrorInfo(e); + } + + } + +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/LogPage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/LogPage.java new file mode 100755 index 000000000..6fd7ec8f3 --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/LogPage.java @@ -0,0 +1,300 @@ +/** + * 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.ui.model.pages; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +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.ClientProtocolException; +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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.utils.NimbusClient; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.cluster.Common; +import com.alibaba.jstorm.ui.NimbusClientManager; +import com.alibaba.jstorm.ui.UIDef; +import com.alibaba.jstorm.ui.UIMetrics; +import com.alibaba.jstorm.ui.UIUtils; +import com.alibaba.jstorm.ui.model.LinkData; +import com.alibaba.jstorm.ui.model.PageIndex; +import com.alibaba.jstorm.utils.HttpserverUtils; +import com.alibaba.jstorm.utils.JStormUtils; +import com.alibaba.jstorm.utils.NetWorkUtils; + +@ManagedBean(name = "logpage") +@ViewScoped +public class LogPage implements Serializable { + + private static final long serialVersionUID = 4326599394273506083L; + + private static final Logger LOG = LoggerFactory.getLogger(LogPage.class); + + /** + * proxy url, which call the log service on the task node. + */ + private static final String PROXY_URL = "http://%s:%s/logview?%s=%s&log=%s"; + + /** + * store the log content. + */ + private String log = ""; + + private List pages = new ArrayList(); + + protected Map paramMap; + + protected LogPage.Event event; + + protected Map conf; + + public LogPage() throws Exception { + + FacesContext ctx = FacesContext.getCurrentInstance(); + paramMap = ctx.getExternalContext().getRequestParameterMap(); + + init(); + } + + public LogPage(Map paramMap) throws Exception { + this.paramMap = paramMap; + + init(); + } + + private void init() throws Exception { + + long start = System.nanoTime(); + try { + event = new LogPage.Event(paramMap); + + conf = getNimbusConf(); + + // proxy call + queryLog(event); + + } catch (Exception e) { + LOG.error(e.getMessage(), e); + throw e; + }finally { + long end = System.nanoTime(); + UIMetrics.updateHistorgram(this.getClass().getSimpleName(), (end - start)/1000000.0d); + + } + } + + private Map getNimbusConf() { + NimbusClient client = null; + try { + client = NimbusClientManager.getNimbusClient(paramMap); + + String jsonConf = client.getClient().getNimbusConf(); + Map nimbusConf = + (Map) Utils.from_json(jsonConf); + return nimbusConf; + } catch (Exception e) { + NimbusClientManager.removeClient(paramMap); + LOG.error(e.getMessage(), e); + return UIUtils.readUiConfig(); + } + } + + + + /** + * proxy query log for the specified task. + * + * @param task the specified task + * @throws IOException + * @throws ClientProtocolException + */ + private void queryLog(LogPage.Event event) throws ClientProtocolException, + IOException { + // PROXY_URL = "http://%s:%s/logview?%s=%s&log=%s"; + String baseUrl = + String.format(PROXY_URL, NetWorkUtils.host2Ip(event.host), + event.logServerPort, + HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD, + HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_CMD_SHOW, + event.logName); + String url = baseUrl; + if (event.pos != null) { + url += + ("&" + HttpserverUtils.HTTPSERVER_LOGVIEW_PARAM_POS + "=" + event.pos); + } + + // 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)); + + String sizeStr = data.substring(0, 16); + + + PageIndex.Event pageIndexEvent = new PageIndex.Event(); + pageIndexEvent.totalSize = Long.valueOf(sizeStr); + pageIndexEvent.pos = JStormUtils.parseLong(event.pos, pageIndexEvent.totalSize > 0 ? pageIndexEvent.totalSize - 1: 0); + pageIndexEvent.pageSize = ConfigExtension.getLogPageSize(conf); + pageIndexEvent.url = UIDef.LINK_LOG; + pageIndexEvent.paramMap = event.toMap(); + + pages = PageIndex.generatePageIndex(pageIndexEvent); + + setLog(data); + } else { + setLog(EntityUtils.toString(response.getEntity())); + } + + } + + + public static class Event implements Serializable { + + public final String clusterName; + public final String host; + public final String logServerPort; + public String logName; + public final String topologyId; + public final String workerPort; + public final String pos; + + public Event(Map paramMap) { + clusterName = paramMap.get(UIDef.CLUSTER); + host = paramMap.get(UIDef.HOST); + logServerPort = paramMap.get(UIDef.LOG_SERVER_PORT); + logName = paramMap.get(UIDef.LOG_NAME); + topologyId = paramMap.get(UIDef.TOPOLOGY); + workerPort = paramMap.get(UIDef.PORT); + pos = paramMap.get(UIDef.POS); + + if (host == null) { + throw new IllegalArgumentException("Please set host"); + } else if (logServerPort == null) { + throw new IllegalArgumentException( + "Please set log server's port"); + } + + if (StringUtils.isBlank(logName) == false) { + return; + } + + if (topologyId == null || workerPort == null) { + throw new IllegalArgumentException( + "Please set log fileName or topologyId-workerPort"); + } + + String topologyName; + try { + topologyName = Common.topologyIdToName(topologyId); + } catch (Exception e) { + // TODO Auto-generated catch block + throw new IllegalArgumentException( + "Please set log fileName or topologyId-workerPort"); + } + logName = topologyName + "-worker-" + workerPort + ".log"; + + } + + public Map toMap() { + Map paramMap = new HashMap(); + + paramMap.put(UIDef.CLUSTER, clusterName); + paramMap.put(UIDef.HOST, host); + paramMap.put(UIDef.LOG_SERVER_PORT, logServerPort); + paramMap.put(UIDef.LOG_NAME, logName); + paramMap.put(UIDef.POS, pos); + + return paramMap; + } + + public String getClusterName() { + return clusterName; + } + + public String getHost() { + return host; + } + + public String getLogServerPort() { + return logServerPort; + } + + public String getLogName() { + return logName; + } + + public String getTopologyId() { + return topologyId; + } + + public String getWorkerPort() { + return workerPort; + } + + public String getPos() { + return pos; + } + + } + + public String getLog() { + return log; + } + + public void setLog(String log) { + this.log = log; + } + + public List getPages() { + return pages; + } + + public Event getEvent() { + return event; + } + + public Map getConf() { + return conf; + } + + public static void main() { + + } +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/NettyPage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/NettyPage.java new file mode 100755 index 000000000..be2930bbf --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/NettyPage.java @@ -0,0 +1,131 @@ +/** + * 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.ui.model.pages; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.generated.MetricInfo; +import backtype.storm.generated.NettyMetric; +import backtype.storm.utils.NimbusClient; + +import com.alibaba.jstorm.cluster.Common; +import com.alibaba.jstorm.common.metric.window.StatBuckets; +import com.alibaba.jstorm.metric.MetricDef; +import com.alibaba.jstorm.ui.NimbusClientManager; +import com.alibaba.jstorm.ui.UIDef; +import com.alibaba.jstorm.ui.UIUtils; +import com.alibaba.jstorm.ui.model.PageGenerator; +import com.alibaba.jstorm.ui.model.PageIndex; +import com.alibaba.jstorm.ui.model.TableData; +import com.alibaba.jstorm.ui.model.PageIndex.Event; +import com.alibaba.jstorm.utils.JStormUtils; + +public class NettyPage implements PageGenerator { + private static final Logger LOG = LoggerFactory + .getLogger(NettyPage.class); + + public TableData getNettyMetricTable(NettyMetric nettyMetric, + Map paramMap) { + String windowStr = paramMap.get(UIDef.WINDOW); + Integer window = StatBuckets.getTimeKey(windowStr); + + if (nettyMetric == null || nettyMetric.get_connections() == null) { + throw new RuntimeException("No netty metrics"); + } + + + TreeMap metrics = new TreeMap(); + metrics.putAll(nettyMetric.get_connections()); + return UIUtils.getNettyMetricsTable(metrics, window, paramMap); + } + + public List getPageIndex(NettyMetric nettyMetric, Map paramMap) { + if (nettyMetric.get_connectionNum() == nettyMetric.get_connections().size()) { + return new ArrayList(); + } + + PageIndex.Event event = new PageIndex.Event(); + event.totalSize = nettyMetric.get_connectionNum(); + event.pos = JStormUtils.parseLong(paramMap.get(UIDef.POS), 0); + event.pageSize = MetricDef.NETTY_METRICS_PACKAGE_SIZE; + event.url = UIDef.LINK_WINDOW_TABLE; + event.paramMap = paramMap; + + return PageIndex.generatePageIndex(event); + } + + public Output generate(Map paramMap) { + // TODO Auto-generated method stub + List tables = new ArrayList(); + + NimbusClient client = null; + try { + client = NimbusClientManager.getNimbusClient(paramMap); + + String topologyId = paramMap.get(UIDef.TOPOLOGY); + if (topologyId == null) { + throw new IllegalArgumentException("Not set topologyId"); + } + + + + NettyMetric nettyMetric = null; + try { + + String topologyName = Common.getTopologyNameById(topologyId); + + String server = paramMap.get(UIDef.HOST); + if (server == null) { + int pos = JStormUtils.parseInt(paramMap.get(UIDef.POS), 0); + nettyMetric = client.getClient().getNettyMetric(topologyName, pos); + }else { + nettyMetric = client.getClient().getServerNettyMetric(topologyName, server); + } + + + } catch (org.apache.thrift.TException e) { + throw new IllegalArgumentException( + "Failed to get topologyInfo of " + topologyId + + ", maybe it is dead"); + } + + + TableData nettyMetricTable = + getNettyMetricTable(nettyMetric, paramMap); + tables.add(nettyMetricTable); + + Output ret = new Output(); + ret.tables = tables; + ret.rawData = ""; + ret.pages = getPageIndex(nettyMetric, paramMap); + return ret; + } catch (Exception e) { + NimbusClientManager.removeClient(paramMap); + LOG.error(e.getMessage(), e); + + return UIUtils.getErrorInfo(e); + } + } + +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/SupervisorPage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/SupervisorPage.java new file mode 100755 index 000000000..faf043f08 --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/SupervisorPage.java @@ -0,0 +1,281 @@ +/** + * 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.ui.model.pages; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.generated.MetricInfo; +import backtype.storm.generated.SupervisorSummary; +import backtype.storm.generated.SupervisorWorkers; +import backtype.storm.generated.TaskComponent; +import backtype.storm.generated.WorkerSummary; +import backtype.storm.utils.NimbusClient; +import backtype.storm.utils.Utils; + +import com.alibaba.jstorm.client.ConfigExtension; +import com.alibaba.jstorm.common.metric.window.StatBuckets; +import com.alibaba.jstorm.ui.NimbusClientManager; +import com.alibaba.jstorm.ui.UIDef; +import com.alibaba.jstorm.ui.UIUtils; +import com.alibaba.jstorm.ui.model.ColumnData; +import com.alibaba.jstorm.ui.model.LinkData; +import com.alibaba.jstorm.ui.model.PageGenerator; +import com.alibaba.jstorm.ui.model.TableData; +import com.alibaba.jstorm.utils.NetWorkUtils; + +public class SupervisorPage implements PageGenerator { + private static final Logger LOG = LoggerFactory + .getLogger(SupervisorPage.class); + + public TableData getSupervisorTable(SupervisorSummary supervisorSummary, + Map paramMap, Map nimbusConf) { + TableData table = new TableData(); + List headers = table.getHeaders(); + List> lines = table.getLines(); + table.setName(UIDef.SUPERVISOR.toUpperCase()); + + headers.add(UIDef.HOST.toUpperCase()); + headers.add(UIDef.IP); + headers.add(UIDef.HEADER_UPTIME); + headers.add(UIDef.HEADER_TOTAL_PORT); + headers.add(UIDef.HEADER_USED_PORT); + headers.add(UIDef.HEADER_CONF); + headers.add(UIDef.HEADER_LOGS); + + Map line = new HashMap(); + lines.add(line); + + ColumnData hostColumn = new ColumnData(); + hostColumn.addText(NetWorkUtils.ip2Host(supervisorSummary.get_host())); + line.put(UIDef.HOST.toUpperCase(), hostColumn); + + ColumnData ipColumn = new ColumnData(); + ipColumn.addText(NetWorkUtils.host2Ip(supervisorSummary.get_host())); + line.put(UIDef.IP, ipColumn); + + ColumnData uptimeColumn = new ColumnData(); + int uptime = supervisorSummary.get_uptime_secs(); + uptimeColumn.addText(StatBuckets.prettyUptimeStr(uptime)); + line.put(UIDef.HEADER_UPTIME, uptimeColumn); + + ColumnData totalPortColumn = new ColumnData(); + totalPortColumn.addText(String.valueOf(supervisorSummary + .get_num_workers())); + line.put(UIDef.HEADER_TOTAL_PORT, totalPortColumn); + + ColumnData usedPortColumn = new ColumnData(); + usedPortColumn.addText(String.valueOf(supervisorSummary + .get_num_used_workers())); + line.put(UIDef.HEADER_USED_PORT, usedPortColumn); + + ColumnData confColumn = new ColumnData(); + LinkData confLink = new LinkData(); + confColumn.addLinkData(confLink); + line.put(UIDef.HEADER_CONF, confColumn); + + confLink.setUrl(UIDef.LINK_TABLE_PAGE); + confLink.setText(UIDef.HEADER_CONF.toLowerCase()); + confLink.addParam(UIDef.CLUSTER, paramMap.get(UIDef.CLUSTER)); + confLink.addParam(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_CONF); + confLink.addParam(UIDef.CONF_TYPE, UIDef.CONF_TYPE_SUPERVISOR); + confLink.addParam(UIDef.HOST, + NetWorkUtils.host2Ip(supervisorSummary.get_host())); + confLink.addParam(UIDef.PORT, String.valueOf(ConfigExtension + .getSupervisorDeamonHttpserverPort(nimbusConf))); + + ColumnData logColumn = new ColumnData(); + LinkData logLink = new LinkData(); + logColumn.addLinkData(logLink); + line.put(UIDef.HEADER_LOGS, logColumn); + + logLink.setUrl(UIDef.LINK_TABLE_PAGE); + logLink.setText(UIDef.HEADER_LOGS.toLowerCase()); + logLink.addParam(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_LISTLOG); + logLink.addParam(UIDef.HOST, + NetWorkUtils.host2Ip(supervisorSummary.get_host())); + logLink.addParam(UIDef.PORT, String.valueOf(ConfigExtension + .getSupervisorDeamonHttpserverPort(nimbusConf))); + logLink.addParam(UIDef.DIR, "."); + + return table; + } + + public TableData getWokerTable(SupervisorWorkers supervisorWorkers, + Map paramMap, Map nimbusConf) { + TableData table = new TableData(); + List headers = table.getHeaders(); + List> lines = table.getLines(); + table.setName(StringUtils.capitalize(UIDef.WOKER)); + + headers.add(StringUtils.capitalize(UIDef.PORT)); + headers.add(UIDef.HEADER_UPTIME); + headers.add(StringUtils.capitalize(UIDef.TOPOLOGY)); + headers.add(UIDef.HEADER_TASK_LIST); + headers.add(UIDef.HEADER_LOG); + headers.add(StringUtils.capitalize(UIDef.JSTACK)); + + List workerSummaries = supervisorWorkers.get_workers(); + if (workerSummaries == null) { + LOG.error("Failed to get workers of " + paramMap.get(UIDef.HOST)); + return table; + } + + int logServerPort = + ConfigExtension.getSupervisorDeamonHttpserverPort(nimbusConf); + + for (WorkerSummary workerSummary : workerSummaries) { + Map line = new HashMap(); + lines.add(line); + + ColumnData portColumn = new ColumnData(); + portColumn.addText(String.valueOf(workerSummary.get_port())); + line.put(StringUtils.capitalize(UIDef.PORT), portColumn); + + ColumnData uptimeColumn = new ColumnData(); + int uptime = workerSummary.get_uptime(); + uptimeColumn.addText(StatBuckets.prettyUptimeStr(uptime)); + line.put(UIDef.HEADER_UPTIME, uptimeColumn); + + ColumnData topologyColumn = new ColumnData(); + topologyColumn.addText(workerSummary.get_topology()); + line.put(StringUtils.capitalize(UIDef.TOPOLOGY), topologyColumn); + + ColumnData taskIdColumn = new ColumnData(); + line.put(UIDef.HEADER_TASK_LIST, taskIdColumn); + for (TaskComponent taskComponent : workerSummary.get_tasks()) { + LinkData linkData = new LinkData(); + taskIdColumn.addLinkData(linkData); + linkData.setUrl(UIDef.LINK_WINDOW_TABLE); + linkData.setText(taskComponent.get_component() + "-" + + taskComponent.get_taskId()); + linkData.addParam(UIDef.CLUSTER, paramMap.get(UIDef.CLUSTER)); + linkData.addParam(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_COMPONENT); + linkData.addParam(UIDef.TOPOLOGY, workerSummary.get_topology()); + linkData.addParam(UIDef.COMPONENT, + taskComponent.get_component()); + } + + ColumnData logColumn = new ColumnData(); + LinkData logLink = new LinkData(); + logColumn.addLinkData(logLink); + line.put(UIDef.HEADER_LOG, logColumn); + + logLink.setUrl(UIDef.LINK_LOG); + logLink.setText(UIDef.HEADER_LOG.toLowerCase()); + logLink.addParam(UIDef.CLUSTER, paramMap.get(UIDef.CLUSTER)); + logLink.addParam(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_LOG); + logLink.addParam(UIDef.HOST, NetWorkUtils.host2Ip(supervisorWorkers + .get_supervisor().get_host())); + logLink.addParam(UIDef.TOPOLOGY, workerSummary.get_topology()); + logLink.addParam(UIDef.PORT, + String.valueOf(workerSummary.get_port())); + logLink.addParam(UIDef.LOG_SERVER_PORT, + String.valueOf(logServerPort)); + + ColumnData jstackColumn = new ColumnData(); + LinkData jstackLink = new LinkData(); + jstackColumn.addLinkData(jstackLink); + line.put(StringUtils.capitalize(UIDef.JSTACK), jstackColumn); + + jstackLink.setUrl(UIDef.LINK_TABLE_PAGE); + jstackLink.setText(UIDef.JSTACK); + jstackLink.addParam(UIDef.CLUSTER, paramMap.get(UIDef.CLUSTER)); + jstackLink.addParam(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_JSTACK); + jstackLink.addParam(UIDef.HOST, supervisorWorkers.get_supervisor() + .get_host()); + jstackLink.addParam(UIDef.TOPOLOGY, workerSummary.get_topology()); + jstackLink.addParam(UIDef.PORT, + String.valueOf(workerSummary.get_port())); + jstackLink.addParam(UIDef.LOG_SERVER_PORT, + String.valueOf(logServerPort)); + } + return table; + } + + public List getMetricsTable(SupervisorWorkers supervisorWorkers, + Map paramMap, Map nimbusConf) { + + Map metrics = supervisorWorkers.get_workerMetric(); + if (metrics == null) { + LOG.error("No metrics of " + + supervisorWorkers.get_supervisor().get_host()); + return null; + } + + return UIUtils.getWorkerMetricsTable(metrics, + StatBuckets.ALL_TIME_WINDOW, paramMap); + } + + @Override + public Output generate(Map paramMap) { + // TODO Auto-generated method stub + List tables = new ArrayList(); + + NimbusClient client = null; + try { + client = NimbusClientManager.getNimbusClient(paramMap); + + String jsonConf = client.getClient().getNimbusConf(); + Map nimbusConf = + (Map) Utils.from_json(jsonConf); + + String host = paramMap.get(UIDef.HOST); + if (StringUtils.isBlank(host)) { + throw new IllegalArgumentException("Invalid parameter of host "); + } + + SupervisorWorkers supervisorWorkers = + client.getClient().getSupervisorWorkers(host); + + TableData supervisorTable = + getSupervisorTable(supervisorWorkers.get_supervisor(), + paramMap, nimbusConf); + tables.add(supervisorTable); + + TableData workerTable = + getWokerTable(supervisorWorkers, paramMap, nimbusConf); + tables.add(workerTable); + + List metricsTables = + getMetricsTable(supervisorWorkers, paramMap, nimbusConf); + if (metricsTables != null) { + tables.addAll(metricsTables); + } + + Output ret = new Output(); + ret.tables = tables; + ret.rawData = ""; + return ret; + } catch (Exception e) { + // TODO Auto-generated catch block + NimbusClientManager.removeClient(paramMap); + LOG.error(e.getMessage(), e); + + return UIUtils.getErrorInfo(e); + } + + } + +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/TablePage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/TablePage.java new file mode 100755 index 000000000..eb0825204 --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/TablePage.java @@ -0,0 +1,154 @@ +/** + * 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.ui.model.pages; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import javax.faces.bean.ManagedBean; +import javax.faces.bean.ViewScoped; +import javax.faces.context.FacesContext; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.ui.UIDef; +import com.alibaba.jstorm.ui.UIMetrics; +import com.alibaba.jstorm.ui.model.PageGenerator; +import com.alibaba.jstorm.ui.model.PageIndex; +import com.alibaba.jstorm.ui.model.TableData; + +/** + * + * @author xin.zhou/Longda + */ +@ManagedBean(name = "tablepage") +@ViewScoped +public class TablePage implements Serializable { + private static final Logger LOG = LoggerFactory.getLogger(TablePage.class); + + protected List tables = new ArrayList(); + private List pages; + protected Map parameterMap; + protected String rawData = ""; + + public TablePage() throws Exception { + FacesContext ctx = FacesContext.getCurrentInstance(); + parameterMap = ctx.getExternalContext().getRequestParameterMap(); + + init(); + } + + public TablePage(Map parameterMap) throws Exception { + this.parameterMap = parameterMap; + init(); + } + + public void init() { + LOG.info(parameterMap.toString()); + String pageType = parameterMap.get(UIDef.PAGE_TYPE); + if (pageType == null) { + throw new IllegalArgumentException("Please set " + UIDef.PAGE_TYPE); + } + + long start = System.nanoTime(); + PageGenerator pageGenerator = UIDef.pageGeneratos.get(pageType); + if (pageGenerator == null) { + throw new IllegalArgumentException("Invalid " + UIDef.PAGE_TYPE + + ":" + pageType); + } + + try { + PageGenerator.Output output = pageGenerator.generate(parameterMap); + tables = output.tables; + pages = output.pages; + rawData = output.rawData; + }finally { + long end = System.nanoTime(); + UIMetrics.updateHistorgram(pageGenerator.getClass().getSimpleName(), (end - start)/1000000.0d); + } + } + + public List getTables() { + return tables; + } + + public void addTable(TableData tableData) { + tables.add(tableData); + } + + public Map getParameterMap() { + return parameterMap; + } + + public String getRawData() { + return rawData; + } + + public List getPages() { + return pages; + } + + public void setPages(List pages) { + this.pages = pages; + } + + public void generateTables() { + + } + + public static Map testCluster() { + Map parameterMap = new HashMap(); + parameterMap.put(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_CLUSTER); + return parameterMap; + } + + public static Map testNimbusConfPage() { + Map parameterMap = new HashMap(); + parameterMap.put(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_CONF); + parameterMap.put(UIDef.CONF_TYPE, UIDef.CONF_TYPE_NIMBUS); + return parameterMap; + } + + public static Map testListLogPage() { + Map parameterMap = new HashMap(); + parameterMap.put(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_LISTLOG); + parameterMap.put(UIDef.HOST, "free-56-156.shucang.alipay.net"); + parameterMap.put(UIDef.PORT, "7621"); + parameterMap.put(UIDef.DIR, "."); + + return parameterMap; + } + + public static void main(String[] args) { + // testCluster(); + Map parameterMap = testCluster(); + try { + TablePage table = new TablePage(parameterMap); + + System.out.println(table.getTables()); + } catch (Exception e) { + // TODO Auto-generated catch block + + } + } + +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/TopologyPage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/TopologyPage.java new file mode 100755 index 000000000..b7938ef8b --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/TopologyPage.java @@ -0,0 +1,211 @@ +/** + * 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.ui.model.pages; + +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.generated.ComponentSummary; +import backtype.storm.generated.MetricInfo; +import backtype.storm.generated.TopologyInfo; +import backtype.storm.generated.TopologyMetric; +import backtype.storm.generated.TopologySummary; +import backtype.storm.utils.NimbusClient; + +import com.alibaba.jstorm.common.metric.window.StatBuckets; +import com.alibaba.jstorm.ui.NimbusClientManager; +import com.alibaba.jstorm.ui.UIDef; +import com.alibaba.jstorm.ui.UIUtils; +import com.alibaba.jstorm.ui.model.PageGenerator; +import com.alibaba.jstorm.ui.model.PageIndex; +import com.alibaba.jstorm.ui.model.TableData; + +public class TopologyPage implements PageGenerator { + private static final Logger LOG = LoggerFactory + .getLogger(TopologyPage.class); + + public TableData getTopologyState(TopologyInfo topologyInfo, + Map paramMap) { + + TopologyMetric topologyMetrics = topologyInfo.get_metrics(); + + String windowStr = paramMap.get(UIDef.WINDOW); + Integer window = StatBuckets.getTimeKey(windowStr); + + TableData table = + UIUtils.getMetricTable(topologyMetrics.get_topologyMetric(), + window); + table.setName(UIDef.HEADER_TOPOLOGY_METRICS + "(" + + StatBuckets.getShowTimeStr(window) + ")"); + + return table; + } + + public List getComponentTables(TopologyInfo topologyInfo, + Map paramMap) { + + List componentSummaries = + topologyInfo.get_components(); + Map componentMetric = + topologyInfo.get_metrics().get_componentMetric(); + + List spoutComponentSummary = + new ArrayList(); + Map spoutComponentMetric = + new HashMap(); + + List boltComponentSummary = + new ArrayList(); + Map boltComponentMetric = + new HashMap(); + + for (ComponentSummary componentSummary : componentSummaries) { + String type = componentSummary.get_type(); + String name = componentSummary.get_name(); + if (type.equals(UIDef.BOLT)) { + boltComponentSummary.add(componentSummary); + MetricInfo metric = componentMetric.get(name); + if (metric != null) { + boltComponentMetric.put(name, metric); + } else { + LOG.warn("No component metric of " + name); + } + } else if (type.equals(UIDef.SPOUT)) { + spoutComponentSummary.add(componentSummary); + MetricInfo metric = componentMetric.get(name); + if (metric != null) { + spoutComponentMetric.put(name, metric); + } else { + LOG.warn("No component metric of " + name); + } + } else { + LOG.warn("No component type of " + name + ":" + type); + } + } + + String windowStr = paramMap.get(UIDef.WINDOW); + Integer window = StatBuckets.getTimeKey(windowStr); + + List ret = new ArrayList(); + TableData spoutTable = + UIUtils.getComponentTable(topologyInfo, spoutComponentSummary, + spoutComponentMetric, paramMap, window); + spoutTable.setName(UIDef.SPOUT + "-" + + StatBuckets.getShowTimeStr(window)); + ret.add(spoutTable); + + TableData boltTable = + UIUtils.getComponentTable(topologyInfo, boltComponentSummary, + boltComponentMetric, paramMap, window); + boltTable + .setName(UIDef.BOLT + "-" + StatBuckets.getShowTimeStr(window)); + ret.add(boltTable); + + return ret; + } + + public List getWorkerMetricTable(TopologyInfo topologyInfo, + Map paramMap) { + String windowStr = paramMap.get(UIDef.WINDOW); + Integer window = StatBuckets.getTimeKey(windowStr); + + if (topologyInfo.get_metrics() == null || topologyInfo.get_metrics() + .get_workerMetric() == null) { + return null; + } + + return UIUtils.getWorkerMetricsTable(topologyInfo.get_metrics() + .get_workerMetric(), window, paramMap); + } + + public Output generate(Map paramMap) { + // TODO Auto-generated method stub + LOG.info("Begin TopologyPage " + new Date()); + + + List tables = new ArrayList(); + + NimbusClient client = null; + try { + client = NimbusClientManager.getNimbusClient(paramMap); + + String topologyId = paramMap.get(UIDef.TOPOLOGY); + if (topologyId == null) { + throw new IllegalArgumentException("Not set topologyId"); + } + + TopologyInfo topologyInfo = null; + try { + + topologyInfo = client.getClient().getTopologyInfo(topologyId); + } catch (org.apache.thrift.TException e) { + throw new IllegalArgumentException( + "Failed to get topologyInfo of " + topologyId + + ", maybe it is dead"); + } + + List topologyList = + new ArrayList(); + topologyList.add(topologyInfo.get_topology()); + TableData topologyTable = + UIUtils.getTopologyTable(topologyList, paramMap); + tables.add(topologyTable); + + TableData topologyMetricTable = + getTopologyState(topologyInfo, paramMap); + tables.add(topologyMetricTable); + + List componentTables = + getComponentTables(topologyInfo, paramMap); + tables.addAll(componentTables); + + List workerMetricTables = + getWorkerMetricTable(topologyInfo, paramMap); + if (workerMetricTables != null) { + tables.addAll(workerMetricTables); + } + + Output ret = new Output(); + ret.tables = tables; + ret.rawData = ""; + if (topologyInfo.get_metrics().get_workerMetric().size() > + UIUtils.ONE_TABLE_PAGE_SIZE) { + ret.pages = PageIndex.generatePageIndex( + topologyInfo.get_metrics().get_workerMetric().size(), + UIUtils.ONE_TABLE_PAGE_SIZE, + UIDef.LINK_WINDOW_TABLE, paramMap); + } + + LOG.info("Finish TopologyPage " + new Date()); + return ret; + } catch (Exception e) { + NimbusClientManager.removeClient(paramMap); + + LOG.error(e.getMessage(), e); + + return UIUtils.getErrorInfo(e); + } + } + +} diff --git a/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/WindowTablePage.java b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/WindowTablePage.java new file mode 100755 index 000000000..cbd4a5c68 --- /dev/null +++ b/jstorm-ui/src/main/java/com/alibaba/jstorm/ui/model/pages/WindowTablePage.java @@ -0,0 +1,105 @@ +/** + * 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.ui.model.pages; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import javax.faces.bean.ManagedBean; +import javax.faces.bean.ViewScoped; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.alibaba.jstorm.common.metric.window.StatBuckets; +import com.alibaba.jstorm.ui.UIDef; +import com.alibaba.jstorm.ui.model.LinkData; + +/** + * + * @author xin.zhou/Longda + */ +@ManagedBean(name = "windowtablepage") +@ViewScoped +public class WindowTablePage extends TablePage { + private static final Logger LOG = LoggerFactory + .getLogger(WindowTablePage.class); + protected String windowsTitle; + protected List windowLinks = new ArrayList(); + + public WindowTablePage() throws Exception { + super(); + initWindowLinks(); + } + + public WindowTablePage(Map parameterMap) throws Exception { + super(parameterMap); + + initWindowLinks(); + } + + public void initWindowLinks() { + windowsTitle = "JStorm Time Windows"; + for (Integer timeWindow : StatBuckets.TIME_WINDOWS) { + LinkData link = new LinkData(); + windowLinks.add(link); + + link.setUrl(UIDef.LINK_WINDOW_TABLE); + link.setText(StatBuckets.getShowTimeStr(timeWindow)); + + Map tempParameterMap = + new HashMap(); + tempParameterMap.putAll(parameterMap); + tempParameterMap.put(UIDef.WINDOW, + StatBuckets.getShowTimeStr(timeWindow)); + link.setParamMap(tempParameterMap); + + } + } + + public static Map testTopologyPage() { + Map parameterMap = new HashMap(); + parameterMap.put(UIDef.PAGE_TYPE, UIDef.PAGE_TYPE_TOPOLOGY); + parameterMap.put(UIDef.TOPOLOGY, "SequenceTest-1-1431086898"); + + return parameterMap; + } + + public String getWindowsTitle() { + return windowsTitle; + } + + public List getWindowLinks() { + return windowLinks; + } + + public static void main(String[] args) { + // testCluster(); + Map parameterMap = testTopologyPage(); + try { + WindowTablePage table = new WindowTablePage(parameterMap); + + System.out.println(table.getTables()); + } catch (Exception e) { + // TODO Auto-generated catch block + + } + } +} diff --git a/jstorm-ui/src/main/resources/defaults.yaml b/jstorm-ui/src/main/resources/defaults.yaml deleted file mode 100644 index 806ebc864..000000000 --- a/jstorm-ui/src/main/resources/defaults.yaml +++ /dev/null @@ -1,235 +0,0 @@ -########### These all have default values as shown -########### Additional configuration goes into storm.yaml - -java.library.path: "/usr/local/lib:/opt/local/lib:/usr/lib" - -### storm.* configs are general configurations -# the local dir is where jars are kept -storm.local.dir: "jstorm-local" -storm.zookeeper.servers: - - "localhost" -storm.zookeeper.port: 2181 -storm.zookeeper.root: "/jstorm" -storm.zookeeper.session.timeout: 20000 -storm.zookeeper.connection.timeout: 15000 -storm.zookeeper.retry.times: 5 -storm.zookeeper.retry.interval: 1000 -storm.zookeeper.retry.intervalceiling.millis: 30000 -storm.cluster.mode: "distributed" # can be distributed or local -storm.local.mode.zmq: false -storm.thrift.transport: "backtype.storm.security.auth.SimpleTransportPlugin" -#storm.messaging.transport: "com.alibaba.jstorm.message.zeroMq.MQContext" -storm.messaging.transport: "com.alibaba.jstorm.message.netty.NettyContext" - -### nimbus.* configs are for the master -nimbus.host: "localhost" -nimbus.thrift.port: 7627 -nimbus.thrift.max_buffer_size: 1048576 -nimbus.childopts: " -Xms1g -Xmx1g -Xmn256m -XX:PermSize=96m -XX:+UseConcMarkSweepGC -XX:+UseCMSInitiatingOccupancyOnly -XX:CMSInitiatingOccupancyFraction=80 -XX:CMSFullGCsBeforeCompaction=5 -XX:+HeapDumpOnOutOfMemoryError -XX:+UseCMSCompactAtFullCollection -XX:CMSMaxAbortablePrecleanTime=5000 " -nimbus.task.timeout.secs: 60 -nimbus.supervisor.timeout.secs: 60 -nimbus.monitor.freq.secs: 10 -nimbus.cleanup.inbox.freq.secs: 600 -nimbus.inbox.jar.expiration.secs: 3600 -nimbus.task.launch.secs: 240 -nimbus.reassign: true -nimbus.file.copy.expiration.secs: 600 -nimbus.topology.validator: "backtype.storm.nimbus.DefaultTopologyValidator" -nimbus.classpath: "" - -### ui.* configs are for the master -ui.port: 8080 -ui.childopts: " -Xms1g -Xmx1g -Xmn256m -XX:PermSize=96m -XX:+UseConcMarkSweepGC -XX:+UseCMSInitiatingOccupancyOnly -XX:CMSInitiatingOccupancyFraction=80 -XX:CMSFullGCsBeforeCompaction=5 -XX:+HeapDumpOnOutOfMemoryError -XX:+UseCMSCompactAtFullCollection -XX:CMSMaxAbortablePrecleanTime=5000 " - -drpc.port: 4772 -drpc.worker.threads: 64 -drpc.queue.size: 128 -drpc.invocations.port: 4773 -drpc.request.timeout.secs: 600 -drpc.childopts: " -Xms1g -Xmx1g -Xmn256m -XX:PermSize=96m -Xmn128m -XX:PermSize=64m -XX:+UseConcMarkSweepGC -XX:+UseCMSInitiatingOccupancyOnly -XX:CMSInitiatingOccupancyFraction=80 -XX:CMSFullGCsBeforeCompaction=5 -XX:+HeapDumpOnOutOfMemoryError -XX:+UseCMSCompactAtFullCollection -XX:CMSMaxAbortablePrecleanTime=5000 " - -transactional.zookeeper.root: "/transactional" -transactional.zookeeper.servers: null -transactional.zookeeper.port: null - -### 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 -supervisor.childopts: " -Xms512m -Xmx512m -Xmn128m -XX:PermSize=64m -XX:+UseConcMarkSweepGC -XX:+UseCMSInitiatingOccupancyOnly -XX:CMSInitiatingOccupancyFraction=80 -XX:CMSFullGCsBeforeCompaction=5 -XX:+HeapDumpOnOutOfMemoryError -XX:+UseCMSCompactAtFullCollection -XX:CMSMaxAbortablePrecleanTime=5000 " -#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 -supervisor.worker.timeout.secs: 60 -#how frequently the supervisor checks on the status of the processes it's monitoring and restarts if necessary -supervisor.monitor.frequency.secs: 10 -#how frequently the supervisor heartbeats to the cluster state (for nimbus) -supervisor.heartbeat.frequency.secs: 60 -supervisor.enable: true -#if set null, it will be get by system -supervisor.hostname: null - - -#How many cpu slot one supervisor can support , if it is null, it will be set by JStorm -supervisor.cpu.slot.num: null - -#How many memory slot one supervisor can support , if it is null, it will be set by JStorm -supervisor.mem.slot.num: null - -# How much disk slot one supervisor can support -# if it is null, it will use $(storm.local.dir)/worker_shared_data -supervisor.disk.slot: null -# if use multiple disks, it can be set as the following -#supervisor.disk.slot: -# - /disk0/jstorm/data -# - /disk1/jstorm/data -# - /disk2/jstorm/data -# - /disk3/jstorm/data - - - -### worker.* configs are for task workers -# worker gc configuration -# worker.gc.path will put all gc logs and memory dump file -worker.gc.path: "%JSTORM_HOME%/logs" -worker.gc.childopts: " -XX:SurvivorRatio=4 -XX:MaxTenuringThreshold=20 -XX:+UseConcMarkSweepGC -XX:+UseCMSInitiatingOccupancyOnly -XX:CMSInitiatingOccupancyFraction=80 -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 - -task.heartbeat.frequency.secs: 10 -task.refresh.poll.secs: 10 - -zmq.threads: 1 -zmq.linger.millis: 5000 -zmq.hwm: 0 - -storm.messaging.netty.server_worker_threads: 1 -storm.messaging.netty.client_worker_threads: 1 -storm.messaging.netty.buffer_size: 5242880 #5MB buffer -storm.messaging.netty.max_retries: 30 -storm.messaging.netty.max_wait_ms: 1000 -storm.messaging.netty.min_wait_ms: 100 -storm.messaging.netty.disruptor: true - -### topology.* configs are for specific executing storms -topology.enable.message.timeouts: true -topology.debug: false -topology.optimize: true -topology.workers: 1 -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.skip.missing.kryo.registrations: false -topology.max.task.parallelism: null -topology.max.spout.pending: null -topology.state.synchronization.timeout.secs: 60 -topology.stats.sample.rate: 0.05 -topology.builtin.metrics.bucket.size.secs: 60 -topology.fall.back.on.java.serialization: true -topology.worker.childopts: null -topology.executor.receive.buffer.size: 1024 #batched -topology.executor.send.buffer.size: 1024 #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.tick.tuple.freq.secs: null -topology.worker.shared.thread.pool.size: 4 -topology.disruptor.wait.strategy: "com.lmax.disruptor.BlockingWaitStrategy" -topology.spout.wait.strategy: "backtype.storm.spout.SleepSpoutWaitStrategy" -topology.sleep.spout.wait.strategy.time.ms: 1 -topology.error.throttle.interval.secs: 10 -topology.max.error.report.per.interval: 5 -topology.kryo.factory: "backtype.storm.serialization.DefaultKryoFactory" -topology.tuple.serializer: "backtype.storm.serialization.types.ListDelegateSerializer" -topology.trident.batch.emit.interval.millis: 500 - -# set topology resource assign priority weight -# the new topology resource assign resource will according to CPU/memory/disk/port 4 part -# so different weight, different priority -topology.disk.weight: 5 -topology.cpu.weight: 3 -topology.memory.weight: 1 -topology.port.weight: 2 - -# if this is false, select supervisor whose weight is highest(left_disk_slot_num * disk.weight -# + left_cpu_slot_num * cpu.weight + left_memory_slot_num * memory.weight + -# left_port_slot_num * port.weight) -# it this is true, select which supervisor is best one according to slot level by level, -# select the supervisor whose disk slot is the most -# if the last select result isn't 1, then select supervisor according to cpu -# then according to port, last according to memory -topology.assign.supervisor.bylevel: true - -#the following weight will decide which task will be assign firstly -topology.task.on.different.node.weight: 2 -topology.use.old.assign.ratio.weight: 2 -topology.user.define.assign.ratio.weight: 2 - - -# enable topology use user-define classloader to avoid class conflict -topology.enable.classloader: false - -# enable supervisor use cgroup to make resource isolation -# Before enable it, you should make sure: -# 1. Linux version (>= 2.6.18) -# 2. Have installed cgroup (check the file's existence:/proc/cgroups) -# 3. You should start your supervisor on root -# You can get more about cgroup: -# http://t.cn/8s7nexU -supervisor.enable.cgroup: false - -dev.zookeeper.path: "/tmp/dev-storm-zookeeper" - -#if this configuration has been set, -# the spout or bolt will log all received tuples -# topology.debug just for logging all sent tuples -topology.debug.recv.tuple: false - -#Usually, spout finish preparation before bolt, -#so spout need wait several seconds so that bolt finish preparation -# the default setting is 30 seconds -spout.delay.run: 30 - -# One memory slot size, the unit size of memory, the default setting is 1G -# For example , if it is 1G, and one task set "memory.slots.per.task" as 2 -# then the task will use 2G memory -memory.slot.per.size: 1073741824 - -# This weight means the number of logic cpu slots corresponding to per hardware cpu core -# For example , if a supervisor have 4 cpu cores and set this weight as 2 , this supervisor -# will have 8 logic cpu slots. This weight also effects the cgroup , if this weight is 1 , -# one cpu slot corresponds to 1024 of cgroup's cpu weight ; when you set this weight as 2 , -# one cpu slot corresponds to 512 of cgroup's cpu weight ; -# You can get more about cgroup cpu weight(cpu.share): -# http://t.cn/8std8UV -cpu.slot.per.weight: 2 - -# if you want to start nimbus with group model, you should -# set your group file path like follow: -# use absolute path -# nimbus.groupfile.path: /home/admin/jstorm/conf/group_file.ini -nimbus.groupfile.path: null - - -# if you want to submit topology on the nimbus with group -# module, you must set your gourp name (user.group)here or -# in your topology;The user.name and the user.password is -# useless in this version -user.group: null -user.name: null -user.password: null - - -# container setting means jstorm is running under other system, such as hadoop-yarn/Ali-Apsara -# For example, in Ali-Apsara, Fuxi start c++ container process, -# the C++ container fork process to start nimbus or supervisor -container.heartbeat.timeout.seconds: 240 -container.heartbeat.frequence: 10 - - -# enable java sandbox or not -java.sandbox.enable: false. \ No newline at end of file diff --git a/jstorm-ui/src/main/resources/log4j.properties b/jstorm-ui/src/main/resources/log4j.properties old mode 100644 new mode 100755 diff --git a/jstorm-ui/src/main/resources/logback-test.xml b/jstorm-ui/src/main/resources/logback-test.xml new file mode 100755 index 000000000..d3335e809 --- /dev/null +++ b/jstorm-ui/src/main/resources/logback-test.xml @@ -0,0 +1,49 @@ + + + + + ../logs/jstorm-ui.log + + ../logs/jstorm-ui.log.%i + 1 + 5 + + + + 1GB + + + + [%p %d{yyyy-MM-dd HH:mm:ss} %c{1}:%L %t] %m%n + + + + + + %-4r [%t] %-5p %c - %m%n + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/jstorm-ui/src/main/webapp/META-INF/context.xml b/jstorm-ui/src/main/webapp/META-INF/context.xml old mode 100644 new mode 100755 diff --git a/jstorm-ui/src/main/webapp/WEB-INF/faces-config.xml b/jstorm-ui/src/main/webapp/WEB-INF/faces-config.xml old mode 100644 new mode 100755 diff --git a/jstorm-ui/src/main/webapp/WEB-INF/web.xml b/jstorm-ui/src/main/webapp/WEB-INF/web.xml old mode 100644 new mode 100755 diff --git a/jstorm-ui/src/main/webapp/assets/css/aloha.css b/jstorm-ui/src/main/webapp/assets/css/aloha.css new file mode 100755 index 000000000..740776dc4 --- /dev/null +++ b/jstorm-ui/src/main/webapp/assets/css/aloha.css @@ -0,0 +1,25 @@ +.disp_table { + width: 100%; + line-height: 30px; + border-collapse: collapse; + border-width: 1px; + text-align: center; +} + +.disp_table TH { + color: #428bca; + font-weight: bold; + background-color: #e5ecf9; + border: 1px solid #999; + padding: 2px 3px 2px 3px; +} + +.disp_table TD { + border: 1px solid #999; + padding: 2px 3px 2px 5px; +} + +.disp_table TR { + line-height: 30px; +} + diff --git a/jstorm-ui/src/main/webapp/assets/css/bootstrap-theme.css b/jstorm-ui/src/main/webapp/assets/css/bootstrap-theme.css new file mode 100755 index 000000000..b0fdfcbf9 --- /dev/null +++ b/jstorm-ui/src/main/webapp/assets/css/bootstrap-theme.css @@ -0,0 +1,476 @@ +/*! + * Bootstrap v3.3.4 (http://getbootstrap.com) + * Copyright 2011-2015 Twitter, Inc. + * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE) + */ + +.btn-default, +.btn-primary, +.btn-success, +.btn-info, +.btn-warning, +.btn-danger { + text-shadow: 0 -1px 0 rgba(0, 0, 0, .2); + -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, .15), 0 1px 1px rgba(0, 0, 0, .075); + box-shadow: inset 0 1px 0 rgba(255, 255, 255, .15), 0 1px 1px rgba(0, 0, 0, .075); +} +.btn-default:active, +.btn-primary:active, +.btn-success:active, +.btn-info:active, +.btn-warning:active, +.btn-danger:active, +.btn-default.active, +.btn-primary.active, +.btn-success.active, +.btn-info.active, +.btn-warning.active, +.btn-danger.active { + -webkit-box-shadow: inset 0 3px 5px rgba(0, 0, 0, .125); + box-shadow: inset 0 3px 5px rgba(0, 0, 0, .125); +} +.btn-default .badge, +.btn-primary .badge, +.btn-success .badge, +.btn-info .badge, +.btn-warning .badge, +.btn-danger .badge { + text-shadow: none; +} +.btn:active, +.btn.active { + background-image: none; +} +.btn-default { + text-shadow: 0 1px 0 #fff; + background-image: -webkit-linear-gradient(top, #fff 0%, #e0e0e0 100%); + background-image: -o-linear-gradient(top, #fff 0%, #e0e0e0 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#fff), to(#e0e0e0)); + background-image: linear-gradient(to bottom, #fff 0%, #e0e0e0 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff', endColorstr='#ffe0e0e0', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled = false); + background-repeat: repeat-x; + border-color: #dbdbdb; + border-color: #ccc; +} +.btn-default:hover, +.btn-default:focus { + background-color: #e0e0e0; + background-position: 0 -15px; +} +.btn-default:active, +.btn-default.active { + background-color: #e0e0e0; + border-color: #dbdbdb; +} +.btn-default.disabled, +.btn-default:disabled, +.btn-default[disabled] { + background-color: #e0e0e0; + background-image: none; +} +.btn-primary { + background-image: -webkit-linear-gradient(top, #337ab7 0%, #265a88 100%); + background-image: -o-linear-gradient(top, #337ab7 0%, #265a88 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#337ab7), to(#265a88)); + background-image: linear-gradient(to bottom, #337ab7 0%, #265a88 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff337ab7', endColorstr='#ff265a88', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled = false); + background-repeat: repeat-x; + border-color: #245580; +} +.btn-primary:hover, +.btn-primary:focus { + background-color: #265a88; + background-position: 0 -15px; +} +.btn-primary:active, +.btn-primary.active { + background-color: #265a88; + border-color: #245580; +} +.btn-primary.disabled, +.btn-primary:disabled, +.btn-primary[disabled] { + background-color: #265a88; + background-image: none; +} +.btn-success { + background-image: -webkit-linear-gradient(top, #5cb85c 0%, #419641 100%); + background-image: -o-linear-gradient(top, #5cb85c 0%, #419641 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#5cb85c), to(#419641)); + background-image: linear-gradient(to bottom, #5cb85c 0%, #419641 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5cb85c', endColorstr='#ff419641', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled = false); + background-repeat: repeat-x; + border-color: #3e8f3e; +} +.btn-success:hover, +.btn-success:focus { + background-color: #419641; + background-position: 0 -15px; +} +.btn-success:active, +.btn-success.active { + background-color: #419641; + border-color: #3e8f3e; +} +.btn-success.disabled, +.btn-success:disabled, +.btn-success[disabled] { + background-color: #419641; + background-image: none; +} +.btn-info { + background-image: -webkit-linear-gradient(top, #5bc0de 0%, #2aabd2 100%); + background-image: -o-linear-gradient(top, #5bc0de 0%, #2aabd2 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#5bc0de), to(#2aabd2)); + background-image: linear-gradient(to bottom, #5bc0de 0%, #2aabd2 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de', endColorstr='#ff2aabd2', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled = false); + background-repeat: repeat-x; + border-color: #28a4c9; +} +.btn-info:hover, +.btn-info:focus { + background-color: #2aabd2; + background-position: 0 -15px; +} +.btn-info:active, +.btn-info.active { + background-color: #2aabd2; + border-color: #28a4c9; +} +.btn-info.disabled, +.btn-info:disabled, +.btn-info[disabled] { + background-color: #2aabd2; + background-image: none; +} +.btn-warning { + background-image: -webkit-linear-gradient(top, #f0ad4e 0%, #eb9316 100%); + background-image: -o-linear-gradient(top, #f0ad4e 0%, #eb9316 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#f0ad4e), to(#eb9316)); + background-image: linear-gradient(to bottom, #f0ad4e 0%, #eb9316 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff0ad4e', endColorstr='#ffeb9316', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled = false); + background-repeat: repeat-x; + border-color: #e38d13; +} +.btn-warning:hover, +.btn-warning:focus { + background-color: #eb9316; + background-position: 0 -15px; +} +.btn-warning:active, +.btn-warning.active { + background-color: #eb9316; + border-color: #e38d13; +} +.btn-warning.disabled, +.btn-warning:disabled, +.btn-warning[disabled] { + background-color: #eb9316; + background-image: none; +} +.btn-danger { + background-image: -webkit-linear-gradient(top, #d9534f 0%, #c12e2a 100%); + background-image: -o-linear-gradient(top, #d9534f 0%, #c12e2a 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#d9534f), to(#c12e2a)); + background-image: linear-gradient(to bottom, #d9534f 0%, #c12e2a 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffd9534f', endColorstr='#ffc12e2a', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled = false); + background-repeat: repeat-x; + border-color: #b92c28; +} +.btn-danger:hover, +.btn-danger:focus { + background-color: #c12e2a; + background-position: 0 -15px; +} +.btn-danger:active, +.btn-danger.active { + background-color: #c12e2a; + border-color: #b92c28; +} +.btn-danger.disabled, +.btn-danger:disabled, +.btn-danger[disabled] { + background-color: #c12e2a; + background-image: none; +} +.thumbnail, +.img-thumbnail { + -webkit-box-shadow: 0 1px 2px rgba(0, 0, 0, .075); + box-shadow: 0 1px 2px rgba(0, 0, 0, .075); +} +.dropdown-menu > li > a:hover, +.dropdown-menu > li > a:focus { + background-color: #e8e8e8; + background-image: -webkit-linear-gradient(top, #f5f5f5 0%, #e8e8e8 100%); + background-image: -o-linear-gradient(top, #f5f5f5 0%, #e8e8e8 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#f5f5f5), to(#e8e8e8)); + background-image: linear-gradient(to bottom, #f5f5f5 0%, #e8e8e8 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff5f5f5', endColorstr='#ffe8e8e8', GradientType=0); + background-repeat: repeat-x; +} +.dropdown-menu > .active > a, +.dropdown-menu > .active > a:hover, +.dropdown-menu > .active > a:focus { + background-color: #2e6da4; + background-image: -webkit-linear-gradient(top, #337ab7 0%, #2e6da4 100%); + background-image: -o-linear-gradient(top, #337ab7 0%, #2e6da4 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#337ab7), to(#2e6da4)); + background-image: linear-gradient(to bottom, #337ab7 0%, #2e6da4 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff337ab7', endColorstr='#ff2e6da4', GradientType=0); + background-repeat: repeat-x; +} +.navbar-default { + background-image: -webkit-linear-gradient(top, #fff 0%, #f8f8f8 100%); + background-image: -o-linear-gradient(top, #fff 0%, #f8f8f8 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#fff), to(#f8f8f8)); + background-image: linear-gradient(to bottom, #fff 0%, #f8f8f8 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff', endColorstr='#fff8f8f8', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled = false); + background-repeat: repeat-x; + border-radius: 4px; + -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, .15), 0 1px 5px rgba(0, 0, 0, .075); + box-shadow: inset 0 1px 0 rgba(255, 255, 255, .15), 0 1px 5px rgba(0, 0, 0, .075); +} +.navbar-default .navbar-nav > .open > a, +.navbar-default .navbar-nav > .active > a { + background-image: -webkit-linear-gradient(top, #dbdbdb 0%, #e2e2e2 100%); + background-image: -o-linear-gradient(top, #dbdbdb 0%, #e2e2e2 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#dbdbdb), to(#e2e2e2)); + background-image: linear-gradient(to bottom, #dbdbdb 0%, #e2e2e2 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffdbdbdb', endColorstr='#ffe2e2e2', GradientType=0); + background-repeat: repeat-x; + -webkit-box-shadow: inset 0 3px 9px rgba(0, 0, 0, .075); + box-shadow: inset 0 3px 9px rgba(0, 0, 0, .075); +} +.navbar-brand, +.navbar-nav > li > a { + text-shadow: 0 1px 0 rgba(255, 255, 255, .25); +} +.navbar-inverse { + background-image: -webkit-linear-gradient(top, #3c3c3c 0%, #222 100%); + background-image: -o-linear-gradient(top, #3c3c3c 0%, #222 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#3c3c3c), to(#222)); + background-image: linear-gradient(to bottom, #3c3c3c 0%, #222 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff3c3c3c', endColorstr='#ff222222', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled = false); + background-repeat: repeat-x; +} +.navbar-inverse .navbar-nav > .open > a, +.navbar-inverse .navbar-nav > .active > a { + background-image: -webkit-linear-gradient(top, #080808 0%, #0f0f0f 100%); + background-image: -o-linear-gradient(top, #080808 0%, #0f0f0f 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#080808), to(#0f0f0f)); + background-image: linear-gradient(to bottom, #080808 0%, #0f0f0f 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff080808', endColorstr='#ff0f0f0f', GradientType=0); + background-repeat: repeat-x; + -webkit-box-shadow: inset 0 3px 9px rgba(0, 0, 0, .25); + box-shadow: inset 0 3px 9px rgba(0, 0, 0, .25); +} +.navbar-inverse .navbar-brand, +.navbar-inverse .navbar-nav > li > a { + text-shadow: 0 -1px 0 rgba(0, 0, 0, .25); +} +.navbar-static-top, +.navbar-fixed-top, +.navbar-fixed-bottom { + border-radius: 0; +} +@media (max-width: 767px) { + .navbar .navbar-nav .open .dropdown-menu > .active > a, + .navbar .navbar-nav .open .dropdown-menu > .active > a:hover, + .navbar .navbar-nav .open .dropdown-menu > .active > a:focus { + color: #fff; + background-image: -webkit-linear-gradient(top, #337ab7 0%, #2e6da4 100%); + background-image: -o-linear-gradient(top, #337ab7 0%, #2e6da4 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#337ab7), to(#2e6da4)); + background-image: linear-gradient(to bottom, #337ab7 0%, #2e6da4 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff337ab7', endColorstr='#ff2e6da4', GradientType=0); + background-repeat: repeat-x; + } +} +.alert { + text-shadow: 0 1px 0 rgba(255, 255, 255, .2); + -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, .25), 0 1px 2px rgba(0, 0, 0, .05); + box-shadow: inset 0 1px 0 rgba(255, 255, 255, .25), 0 1px 2px rgba(0, 0, 0, .05); +} +.alert-success { + background-image: -webkit-linear-gradient(top, #dff0d8 0%, #c8e5bc 100%); + background-image: -o-linear-gradient(top, #dff0d8 0%, #c8e5bc 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#dff0d8), to(#c8e5bc)); + background-image: linear-gradient(to bottom, #dff0d8 0%, #c8e5bc 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffdff0d8', endColorstr='#ffc8e5bc', GradientType=0); + background-repeat: repeat-x; + border-color: #b2dba1; +} +.alert-info { + background-image: -webkit-linear-gradient(top, #d9edf7 0%, #b9def0 100%); + background-image: -o-linear-gradient(top, #d9edf7 0%, #b9def0 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#d9edf7), to(#b9def0)); + background-image: linear-gradient(to bottom, #d9edf7 0%, #b9def0 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffd9edf7', endColorstr='#ffb9def0', GradientType=0); + background-repeat: repeat-x; + border-color: #9acfea; +} +.alert-warning { + background-image: -webkit-linear-gradient(top, #fcf8e3 0%, #f8efc0 100%); + background-image: -o-linear-gradient(top, #fcf8e3 0%, #f8efc0 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#fcf8e3), to(#f8efc0)); + background-image: linear-gradient(to bottom, #fcf8e3 0%, #f8efc0 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffcf8e3', endColorstr='#fff8efc0', GradientType=0); + background-repeat: repeat-x; + border-color: #f5e79e; +} +.alert-danger { + background-image: -webkit-linear-gradient(top, #f2dede 0%, #e7c3c3 100%); + background-image: -o-linear-gradient(top, #f2dede 0%, #e7c3c3 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#f2dede), to(#e7c3c3)); + background-image: linear-gradient(to bottom, #f2dede 0%, #e7c3c3 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff2dede', endColorstr='#ffe7c3c3', GradientType=0); + background-repeat: repeat-x; + border-color: #dca7a7; +} +.progress { + background-image: -webkit-linear-gradient(top, #ebebeb 0%, #f5f5f5 100%); + background-image: -o-linear-gradient(top, #ebebeb 0%, #f5f5f5 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#ebebeb), to(#f5f5f5)); + background-image: linear-gradient(to bottom, #ebebeb 0%, #f5f5f5 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffebebeb', endColorstr='#fff5f5f5', GradientType=0); + background-repeat: repeat-x; +} +.progress-bar { + background-image: -webkit-linear-gradient(top, #337ab7 0%, #286090 100%); + background-image: -o-linear-gradient(top, #337ab7 0%, #286090 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#337ab7), to(#286090)); + background-image: linear-gradient(to bottom, #337ab7 0%, #286090 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff337ab7', endColorstr='#ff286090', GradientType=0); + background-repeat: repeat-x; +} +.progress-bar-success { + background-image: -webkit-linear-gradient(top, #5cb85c 0%, #449d44 100%); + background-image: -o-linear-gradient(top, #5cb85c 0%, #449d44 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#5cb85c), to(#449d44)); + background-image: linear-gradient(to bottom, #5cb85c 0%, #449d44 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5cb85c', endColorstr='#ff449d44', GradientType=0); + background-repeat: repeat-x; +} +.progress-bar-info { + background-image: -webkit-linear-gradient(top, #5bc0de 0%, #31b0d5 100%); + background-image: -o-linear-gradient(top, #5bc0de 0%, #31b0d5 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#5bc0de), to(#31b0d5)); + background-image: linear-gradient(to bottom, #5bc0de 0%, #31b0d5 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de', endColorstr='#ff31b0d5', GradientType=0); + background-repeat: repeat-x; +} +.progress-bar-warning { + background-image: -webkit-linear-gradient(top, #f0ad4e 0%, #ec971f 100%); + background-image: -o-linear-gradient(top, #f0ad4e 0%, #ec971f 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#f0ad4e), to(#ec971f)); + background-image: linear-gradient(to bottom, #f0ad4e 0%, #ec971f 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff0ad4e', endColorstr='#ffec971f', GradientType=0); + background-repeat: repeat-x; +} +.progress-bar-danger { + background-image: -webkit-linear-gradient(top, #d9534f 0%, #c9302c 100%); + background-image: -o-linear-gradient(top, #d9534f 0%, #c9302c 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#d9534f), to(#c9302c)); + background-image: linear-gradient(to bottom, #d9534f 0%, #c9302c 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffd9534f', endColorstr='#ffc9302c', GradientType=0); + background-repeat: repeat-x; +} +.progress-bar-striped { + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); +} +.list-group { + border-radius: 4px; + -webkit-box-shadow: 0 1px 2px rgba(0, 0, 0, .075); + box-shadow: 0 1px 2px rgba(0, 0, 0, .075); +} +.list-group-item.active, +.list-group-item.active:hover, +.list-group-item.active:focus { + text-shadow: 0 -1px 0 #286090; + background-image: -webkit-linear-gradient(top, #337ab7 0%, #2b669a 100%); + background-image: -o-linear-gradient(top, #337ab7 0%, #2b669a 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#337ab7), to(#2b669a)); + background-image: linear-gradient(to bottom, #337ab7 0%, #2b669a 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff337ab7', endColorstr='#ff2b669a', GradientType=0); + background-repeat: repeat-x; + border-color: #2b669a; +} +.list-group-item.active .badge, +.list-group-item.active:hover .badge, +.list-group-item.active:focus .badge { + text-shadow: none; +} +.panel { + -webkit-box-shadow: 0 1px 2px rgba(0, 0, 0, .05); + box-shadow: 0 1px 2px rgba(0, 0, 0, .05); +} +.panel-default > .panel-heading { + background-image: -webkit-linear-gradient(top, #f5f5f5 0%, #e8e8e8 100%); + background-image: -o-linear-gradient(top, #f5f5f5 0%, #e8e8e8 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#f5f5f5), to(#e8e8e8)); + background-image: linear-gradient(to bottom, #f5f5f5 0%, #e8e8e8 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff5f5f5', endColorstr='#ffe8e8e8', GradientType=0); + background-repeat: repeat-x; +} +.panel-primary > .panel-heading { + background-image: -webkit-linear-gradient(top, #337ab7 0%, #2e6da4 100%); + background-image: -o-linear-gradient(top, #337ab7 0%, #2e6da4 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#337ab7), to(#2e6da4)); + background-image: linear-gradient(to bottom, #337ab7 0%, #2e6da4 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff337ab7', endColorstr='#ff2e6da4', GradientType=0); + background-repeat: repeat-x; +} +.panel-success > .panel-heading { + background-image: -webkit-linear-gradient(top, #dff0d8 0%, #d0e9c6 100%); + background-image: -o-linear-gradient(top, #dff0d8 0%, #d0e9c6 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#dff0d8), to(#d0e9c6)); + background-image: linear-gradient(to bottom, #dff0d8 0%, #d0e9c6 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffdff0d8', endColorstr='#ffd0e9c6', GradientType=0); + background-repeat: repeat-x; +} +.panel-info > .panel-heading { + background-image: -webkit-linear-gradient(top, #d9edf7 0%, #c4e3f3 100%); + background-image: -o-linear-gradient(top, #d9edf7 0%, #c4e3f3 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#d9edf7), to(#c4e3f3)); + background-image: linear-gradient(to bottom, #d9edf7 0%, #c4e3f3 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffd9edf7', endColorstr='#ffc4e3f3', GradientType=0); + background-repeat: repeat-x; +} +.panel-warning > .panel-heading { + background-image: -webkit-linear-gradient(top, #fcf8e3 0%, #faf2cc 100%); + background-image: -o-linear-gradient(top, #fcf8e3 0%, #faf2cc 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#fcf8e3), to(#faf2cc)); + background-image: linear-gradient(to bottom, #fcf8e3 0%, #faf2cc 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffcf8e3', endColorstr='#fffaf2cc', GradientType=0); + background-repeat: repeat-x; +} +.panel-danger > .panel-heading { + background-image: -webkit-linear-gradient(top, #f2dede 0%, #ebcccc 100%); + background-image: -o-linear-gradient(top, #f2dede 0%, #ebcccc 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#f2dede), to(#ebcccc)); + background-image: linear-gradient(to bottom, #f2dede 0%, #ebcccc 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff2dede', endColorstr='#ffebcccc', GradientType=0); + background-repeat: repeat-x; +} +.well { + background-image: -webkit-linear-gradient(top, #e8e8e8 0%, #f5f5f5 100%); + background-image: -o-linear-gradient(top, #e8e8e8 0%, #f5f5f5 100%); + background-image: -webkit-gradient(linear, left top, left bottom, from(#e8e8e8), to(#f5f5f5)); + background-image: linear-gradient(to bottom, #e8e8e8 0%, #f5f5f5 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffe8e8e8', endColorstr='#fff5f5f5', GradientType=0); + background-repeat: repeat-x; + border-color: #dcdcdc; + -webkit-box-shadow: inset 0 1px 3px rgba(0, 0, 0, .05), 0 1px 0 rgba(255, 255, 255, .1); + box-shadow: inset 0 1px 3px rgba(0, 0, 0, .05), 0 1px 0 rgba(255, 255, 255, .1); +} +/*# sourceMappingURL=bootstrap-theme.css.map */ diff --git a/jstorm-ui/src/main/webapp/assets/css/bootstrap-theme.css.map b/jstorm-ui/src/main/webapp/assets/css/bootstrap-theme.css.map new file mode 100755 index 000000000..5a12d6317 --- /dev/null +++ b/jstorm-ui/src/main/webapp/assets/css/bootstrap-theme.css.map @@ -0,0 +1 @@ +{"version":3,"sources":["less/theme.less","less/mixins/vendor-prefixes.less","bootstrap-theme.css","less/mixins/gradients.less","less/mixins/reset-filter.less"],"names":[],"mappings":"AAcA;;;;;;EAME,0CAAA;ECgDA,6FAAA;EACQ,qFAAA;EC5DT;AFgBC;;;;;;;;;;;;EC2CA,0DAAA;EACQ,kDAAA;EC7CT;AFVD;;;;;;EAiBI,mBAAA;EECH;AFiCC;;EAEE,wBAAA;EE/BH;AFoCD;EGnDI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EAEA,wHAAA;ECnBF,qEAAA;EJiCA,6BAAA;EACA,uBAAA;EAgC2C,2BAAA;EAA2B,oBAAA;EEzBvE;AFLC;;EAEE,2BAAA;EACA,8BAAA;EEOH;AFJC;;EAEE,2BAAA;EACA,uBAAA;EEMH;AFHC;;;EAGE,2BAAA;EACA,wBAAA;EEKH;AFUD;EGpDI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EAEA,wHAAA;ECnBF,qEAAA;EJiCA,6BAAA;EACA,uBAAA;EEgCD;AF9BC;;EAEE,2BAAA;EACA,8BAAA;EEgCH;AF7BC;;EAEE,2BAAA;EACA,uBAAA;EE+BH;AF5BC;;;EAGE,2BAAA;EACA,wBAAA;EE8BH;AFdD;EGrDI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EAEA,wHAAA;ECnBF,qEAAA;EJiCA,6BAAA;EACA,uBAAA;EEyDD;AFvDC;;EAEE,2BAAA;EACA,8BAAA;EEyDH;AFtDC;;EAEE,2BAAA;EACA,uBAAA;EEwDH;AFrDC;;;EAGE,2BAAA;EACA,wBAAA;EEuDH;AFtCD;EGtDI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EAEA,wHAAA;ECnBF,qEAAA;EJiCA,6BAAA;EACA,uBAAA;EEkFD;AFhFC;;EAEE,2BAAA;EACA,8BAAA;EEkFH;AF/EC;;EAEE,2BAAA;EACA,uBAAA;EEiFH;AF9EC;;;EAGE,2BAAA;EACA,wBAAA;EEgFH;AF9DD;EGvDI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EAEA,wHAAA;ECnBF,qEAAA;EJiCA,6BAAA;EACA,uBAAA;EE2GD;AFzGC;;EAEE,2BAAA;EACA,8BAAA;EE2GH;AFxGC;;EAEE,2BAAA;EACA,uBAAA;EE0GH;AFvGC;;;EAGE,2BAAA;EACA,wBAAA;EEyGH;AFtFD;EGxDI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EAEA,wHAAA;ECnBF,qEAAA;EJiCA,6BAAA;EACA,uBAAA;EEoID;AFlIC;;EAEE,2BAAA;EACA,8BAAA;EEoIH;AFjIC;;EAEE,2BAAA;EACA,uBAAA;EEmIH;AFhIC;;;EAGE,2BAAA;EACA,wBAAA;EEkIH;AFxGD;;EChBE,oDAAA;EACQ,4CAAA;EC4HT;AFnGD;;EGzEI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;EHwEF,2BAAA;EEyGD;AFvGD;;;EG9EI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;EH8EF,2BAAA;EE6GD;AFpGD;EG3FI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;ECnBF,qEAAA;EJ6GA,oBAAA;EC/CA,6FAAA;EACQ,qFAAA;EC0JT;AF/GD;;EG3FI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;EF2CF,0DAAA;EACQ,kDAAA;ECoKT;AF5GD;;EAEE,gDAAA;EE8GD;AF1GD;EG9GI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;ECnBF,qEAAA;EF+OD;AFlHD;;EG9GI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;EF2CF,yDAAA;EACQ,iDAAA;EC0LT;AF5HD;;EAYI,2CAAA;EEoHH;AF/GD;;;EAGE,kBAAA;EEiHD;AF5FD;EAfI;;;IAGE,aAAA;IG3IF,0EAAA;IACA,qEAAA;IACA,+FAAA;IAAA,wEAAA;IACA,6BAAA;IACA,wHAAA;ID0PD;EACF;AFxGD;EACE,+CAAA;ECzGA,4FAAA;EACQ,oFAAA;ECoNT;AFhGD;EGpKI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;EH4JF,uBAAA;EE4GD;AFvGD;EGrKI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;EH4JF,uBAAA;EEoHD;AF9GD;EGtKI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;EH4JF,uBAAA;EE4HD;AFrHD;EGvKI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;EH4JF,uBAAA;EEoID;AFrHD;EG/KI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;EDuSH;AFlHD;EGzLI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;ED8SH;AFxHD;EG1LI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;EDqTH;AF9HD;EG3LI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;ED4TH;AFpID;EG5LI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;EDmUH;AF1ID;EG7LI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;ED0UH;AF7ID;EGhKI,+MAAA;EACA,0MAAA;EACA,uMAAA;EDgTH;AFzID;EACE,oBAAA;EC5JA,oDAAA;EACQ,4CAAA;ECwST;AF1ID;;;EAGE,+BAAA;EGjNE,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;EH+MF,uBAAA;EEgJD;AFrJD;;;EAQI,mBAAA;EEkJH;AFxID;ECjLE,mDAAA;EACQ,2CAAA;EC4TT;AFlID;EG1OI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;ED+WH;AFxID;EG3OI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;EDsXH;AF9ID;EG5OI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;ED6XH;AFpJD;EG7OI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;EDoYH;AF1JD;EG9OI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;ED2YH;AFhKD;EG/OI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;EDkZH;AFhKD;EGtPI,0EAAA;EACA,qEAAA;EACA,+FAAA;EAAA,wEAAA;EACA,6BAAA;EACA,wHAAA;EHoPF,uBAAA;ECzMA,2FAAA;EACQ,mFAAA;ECgXT","file":"bootstrap-theme.css","sourcesContent":["\n//\n// Load core variables and mixins\n// --------------------------------------------------\n\n@import \"variables.less\";\n@import \"mixins.less\";\n\n\n//\n// Buttons\n// --------------------------------------------------\n\n// Common styles\n.btn-default,\n.btn-primary,\n.btn-success,\n.btn-info,\n.btn-warning,\n.btn-danger {\n text-shadow: 0 -1px 0 rgba(0,0,0,.2);\n @shadow: inset 0 1px 0 rgba(255,255,255,.15), 0 1px 1px rgba(0,0,0,.075);\n .box-shadow(@shadow);\n\n // Reset the shadow\n &:active,\n &.active {\n .box-shadow(inset 0 3px 5px rgba(0,0,0,.125));\n }\n\n .badge {\n text-shadow: none;\n }\n}\n\n// Mixin for generating new styles\n.btn-styles(@btn-color: #555) {\n #gradient > .vertical(@start-color: @btn-color; @end-color: darken(@btn-color, 12%));\n .reset-filter(); // Disable gradients for IE9 because filter bleeds through rounded corners; see https://github.com/twbs/bootstrap/issues/10620\n background-repeat: repeat-x;\n border-color: darken(@btn-color, 14%);\n\n &:hover,\n &:focus {\n background-color: darken(@btn-color, 12%);\n background-position: 0 -15px;\n }\n\n &:active,\n &.active {\n background-color: darken(@btn-color, 12%);\n border-color: darken(@btn-color, 14%);\n }\n\n &.disabled,\n &:disabled,\n &[disabled] {\n background-color: darken(@btn-color, 12%);\n background-image: none;\n }\n}\n\n// Common styles\n.btn {\n // Remove the gradient for the pressed/active state\n &:active,\n &.active {\n background-image: none;\n }\n}\n\n// Apply the mixin to the buttons\n.btn-default { .btn-styles(@btn-default-bg); text-shadow: 0 1px 0 #fff; border-color: #ccc; }\n.btn-primary { .btn-styles(@btn-primary-bg); }\n.btn-success { .btn-styles(@btn-success-bg); }\n.btn-info { .btn-styles(@btn-info-bg); }\n.btn-warning { .btn-styles(@btn-warning-bg); }\n.btn-danger { .btn-styles(@btn-danger-bg); }\n\n\n//\n// Images\n// --------------------------------------------------\n\n.thumbnail,\n.img-thumbnail {\n .box-shadow(0 1px 2px rgba(0,0,0,.075));\n}\n\n\n//\n// Dropdowns\n// --------------------------------------------------\n\n.dropdown-menu > li > a:hover,\n.dropdown-menu > li > a:focus {\n #gradient > .vertical(@start-color: @dropdown-link-hover-bg; @end-color: darken(@dropdown-link-hover-bg, 5%));\n background-color: darken(@dropdown-link-hover-bg, 5%);\n}\n.dropdown-menu > .active > a,\n.dropdown-menu > .active > a:hover,\n.dropdown-menu > .active > a:focus {\n #gradient > .vertical(@start-color: @dropdown-link-active-bg; @end-color: darken(@dropdown-link-active-bg, 5%));\n background-color: darken(@dropdown-link-active-bg, 5%);\n}\n\n\n//\n// Navbar\n// --------------------------------------------------\n\n// Default navbar\n.navbar-default {\n #gradient > .vertical(@start-color: lighten(@navbar-default-bg, 10%); @end-color: @navbar-default-bg);\n .reset-filter(); // Remove gradient in IE<10 to fix bug where dropdowns don't get triggered\n border-radius: @navbar-border-radius;\n @shadow: inset 0 1px 0 rgba(255,255,255,.15), 0 1px 5px rgba(0,0,0,.075);\n .box-shadow(@shadow);\n\n .navbar-nav > .open > a,\n .navbar-nav > .active > a {\n #gradient > .vertical(@start-color: darken(@navbar-default-link-active-bg, 5%); @end-color: darken(@navbar-default-link-active-bg, 2%));\n .box-shadow(inset 0 3px 9px rgba(0,0,0,.075));\n }\n}\n.navbar-brand,\n.navbar-nav > li > a {\n text-shadow: 0 1px 0 rgba(255,255,255,.25);\n}\n\n// Inverted navbar\n.navbar-inverse {\n #gradient > .vertical(@start-color: lighten(@navbar-inverse-bg, 10%); @end-color: @navbar-inverse-bg);\n .reset-filter(); // Remove gradient in IE<10 to fix bug where dropdowns don't get triggered; see https://github.com/twbs/bootstrap/issues/10257\n\n .navbar-nav > .open > a,\n .navbar-nav > .active > a {\n #gradient > .vertical(@start-color: @navbar-inverse-link-active-bg; @end-color: lighten(@navbar-inverse-link-active-bg, 2.5%));\n .box-shadow(inset 0 3px 9px rgba(0,0,0,.25));\n }\n\n .navbar-brand,\n .navbar-nav > li > a {\n text-shadow: 0 -1px 0 rgba(0,0,0,.25);\n }\n}\n\n// Undo rounded corners in static and fixed navbars\n.navbar-static-top,\n.navbar-fixed-top,\n.navbar-fixed-bottom {\n border-radius: 0;\n}\n\n// Fix active state of dropdown items in collapsed mode\n@media (max-width: @grid-float-breakpoint-max) {\n .navbar .navbar-nav .open .dropdown-menu > .active > a {\n &,\n &:hover,\n &:focus {\n color: #fff;\n #gradient > .vertical(@start-color: @dropdown-link-active-bg; @end-color: darken(@dropdown-link-active-bg, 5%));\n }\n }\n}\n\n\n//\n// Alerts\n// --------------------------------------------------\n\n// Common styles\n.alert {\n text-shadow: 0 1px 0 rgba(255,255,255,.2);\n @shadow: inset 0 1px 0 rgba(255,255,255,.25), 0 1px 2px rgba(0,0,0,.05);\n .box-shadow(@shadow);\n}\n\n// Mixin for generating new styles\n.alert-styles(@color) {\n #gradient > .vertical(@start-color: @color; @end-color: darken(@color, 7.5%));\n border-color: darken(@color, 15%);\n}\n\n// Apply the mixin to the alerts\n.alert-success { .alert-styles(@alert-success-bg); }\n.alert-info { .alert-styles(@alert-info-bg); }\n.alert-warning { .alert-styles(@alert-warning-bg); }\n.alert-danger { .alert-styles(@alert-danger-bg); }\n\n\n//\n// Progress bars\n// --------------------------------------------------\n\n// Give the progress background some depth\n.progress {\n #gradient > .vertical(@start-color: darken(@progress-bg, 4%); @end-color: @progress-bg)\n}\n\n// Mixin for generating new styles\n.progress-bar-styles(@color) {\n #gradient > .vertical(@start-color: @color; @end-color: darken(@color, 10%));\n}\n\n// Apply the mixin to the progress bars\n.progress-bar { .progress-bar-styles(@progress-bar-bg); }\n.progress-bar-success { .progress-bar-styles(@progress-bar-success-bg); }\n.progress-bar-info { .progress-bar-styles(@progress-bar-info-bg); }\n.progress-bar-warning { .progress-bar-styles(@progress-bar-warning-bg); }\n.progress-bar-danger { .progress-bar-styles(@progress-bar-danger-bg); }\n\n// Reset the striped class because our mixins don't do multiple gradients and\n// the above custom styles override the new `.progress-bar-striped` in v3.2.0.\n.progress-bar-striped {\n #gradient > .striped();\n}\n\n\n//\n// List groups\n// --------------------------------------------------\n\n.list-group {\n border-radius: @border-radius-base;\n .box-shadow(0 1px 2px rgba(0,0,0,.075));\n}\n.list-group-item.active,\n.list-group-item.active:hover,\n.list-group-item.active:focus {\n text-shadow: 0 -1px 0 darken(@list-group-active-bg, 10%);\n #gradient > .vertical(@start-color: @list-group-active-bg; @end-color: darken(@list-group-active-bg, 7.5%));\n border-color: darken(@list-group-active-border, 7.5%);\n\n .badge {\n text-shadow: none;\n }\n}\n\n\n//\n// Panels\n// --------------------------------------------------\n\n// Common styles\n.panel {\n .box-shadow(0 1px 2px rgba(0,0,0,.05));\n}\n\n// Mixin for generating new styles\n.panel-heading-styles(@color) {\n #gradient > .vertical(@start-color: @color; @end-color: darken(@color, 5%));\n}\n\n// Apply the mixin to the panel headings only\n.panel-default > .panel-heading { .panel-heading-styles(@panel-default-heading-bg); }\n.panel-primary > .panel-heading { .panel-heading-styles(@panel-primary-heading-bg); }\n.panel-success > .panel-heading { .panel-heading-styles(@panel-success-heading-bg); }\n.panel-info > .panel-heading { .panel-heading-styles(@panel-info-heading-bg); }\n.panel-warning > .panel-heading { .panel-heading-styles(@panel-warning-heading-bg); }\n.panel-danger > .panel-heading { .panel-heading-styles(@panel-danger-heading-bg); }\n\n\n//\n// Wells\n// --------------------------------------------------\n\n.well {\n #gradient > .vertical(@start-color: darken(@well-bg, 5%); @end-color: @well-bg);\n border-color: darken(@well-bg, 10%);\n @shadow: inset 0 1px 3px rgba(0,0,0,.05), 0 1px 0 rgba(255,255,255,.1);\n .box-shadow(@shadow);\n}\n","// Vendor Prefixes\n//\n// All vendor mixins are deprecated as of v3.2.0 due to the introduction of\n// Autoprefixer in our Gruntfile. They will be removed in v4.\n\n// - Animations\n// - Backface visibility\n// - Box shadow\n// - Box sizing\n// - Content columns\n// - Hyphens\n// - Placeholder text\n// - Transformations\n// - Transitions\n// - User Select\n\n\n// Animations\n.animation(@animation) {\n -webkit-animation: @animation;\n -o-animation: @animation;\n animation: @animation;\n}\n.animation-name(@name) {\n -webkit-animation-name: @name;\n animation-name: @name;\n}\n.animation-duration(@duration) {\n -webkit-animation-duration: @duration;\n animation-duration: @duration;\n}\n.animation-timing-function(@timing-function) {\n -webkit-animation-timing-function: @timing-function;\n animation-timing-function: @timing-function;\n}\n.animation-delay(@delay) {\n -webkit-animation-delay: @delay;\n animation-delay: @delay;\n}\n.animation-iteration-count(@iteration-count) {\n -webkit-animation-iteration-count: @iteration-count;\n animation-iteration-count: @iteration-count;\n}\n.animation-direction(@direction) {\n -webkit-animation-direction: @direction;\n animation-direction: @direction;\n}\n.animation-fill-mode(@fill-mode) {\n -webkit-animation-fill-mode: @fill-mode;\n animation-fill-mode: @fill-mode;\n}\n\n// Backface visibility\n// Prevent browsers from flickering when using CSS 3D transforms.\n// Default value is `visible`, but can be changed to `hidden`\n\n.backface-visibility(@visibility){\n -webkit-backface-visibility: @visibility;\n -moz-backface-visibility: @visibility;\n backface-visibility: @visibility;\n}\n\n// Drop shadows\n//\n// Note: Deprecated `.box-shadow()` as of v3.1.0 since all of Bootstrap's\n// supported browsers that have box shadow capabilities now support it.\n\n.box-shadow(@shadow) {\n -webkit-box-shadow: @shadow; // iOS <4.3 & Android <4.1\n box-shadow: @shadow;\n}\n\n// Box sizing\n.box-sizing(@boxmodel) {\n -webkit-box-sizing: @boxmodel;\n -moz-box-sizing: @boxmodel;\n box-sizing: @boxmodel;\n}\n\n// CSS3 Content Columns\n.content-columns(@column-count; @column-gap: @grid-gutter-width) {\n -webkit-column-count: @column-count;\n -moz-column-count: @column-count;\n column-count: @column-count;\n -webkit-column-gap: @column-gap;\n -moz-column-gap: @column-gap;\n column-gap: @column-gap;\n}\n\n// Optional hyphenation\n.hyphens(@mode: auto) {\n word-wrap: break-word;\n -webkit-hyphens: @mode;\n -moz-hyphens: @mode;\n -ms-hyphens: @mode; // IE10+\n -o-hyphens: @mode;\n hyphens: @mode;\n}\n\n// Placeholder text\n.placeholder(@color: @input-color-placeholder) {\n // Firefox\n &::-moz-placeholder {\n color: @color;\n opacity: 1; // Override Firefox's unusual default opacity; see https://github.com/twbs/bootstrap/pull/11526\n }\n &:-ms-input-placeholder { color: @color; } // Internet Explorer 10+\n &::-webkit-input-placeholder { color: @color; } // Safari and Chrome\n}\n\n// Transformations\n.scale(@ratio) {\n -webkit-transform: scale(@ratio);\n -ms-transform: scale(@ratio); // IE9 only\n -o-transform: scale(@ratio);\n transform: scale(@ratio);\n}\n.scale(@ratioX; @ratioY) {\n -webkit-transform: scale(@ratioX, @ratioY);\n -ms-transform: scale(@ratioX, @ratioY); // IE9 only\n -o-transform: scale(@ratioX, @ratioY);\n transform: scale(@ratioX, @ratioY);\n}\n.scaleX(@ratio) {\n -webkit-transform: scaleX(@ratio);\n -ms-transform: scaleX(@ratio); // IE9 only\n -o-transform: scaleX(@ratio);\n transform: scaleX(@ratio);\n}\n.scaleY(@ratio) {\n -webkit-transform: scaleY(@ratio);\n -ms-transform: scaleY(@ratio); // IE9 only\n -o-transform: scaleY(@ratio);\n transform: scaleY(@ratio);\n}\n.skew(@x; @y) {\n -webkit-transform: skewX(@x) skewY(@y);\n -ms-transform: skewX(@x) skewY(@y); // See https://github.com/twbs/bootstrap/issues/4885; IE9+\n -o-transform: skewX(@x) skewY(@y);\n transform: skewX(@x) skewY(@y);\n}\n.translate(@x; @y) {\n -webkit-transform: translate(@x, @y);\n -ms-transform: translate(@x, @y); // IE9 only\n -o-transform: translate(@x, @y);\n transform: translate(@x, @y);\n}\n.translate3d(@x; @y; @z) {\n -webkit-transform: translate3d(@x, @y, @z);\n transform: translate3d(@x, @y, @z);\n}\n.rotate(@degrees) {\n -webkit-transform: rotate(@degrees);\n -ms-transform: rotate(@degrees); // IE9 only\n -o-transform: rotate(@degrees);\n transform: rotate(@degrees);\n}\n.rotateX(@degrees) {\n -webkit-transform: rotateX(@degrees);\n -ms-transform: rotateX(@degrees); // IE9 only\n -o-transform: rotateX(@degrees);\n transform: rotateX(@degrees);\n}\n.rotateY(@degrees) {\n -webkit-transform: rotateY(@degrees);\n -ms-transform: rotateY(@degrees); // IE9 only\n -o-transform: rotateY(@degrees);\n transform: rotateY(@degrees);\n}\n.perspective(@perspective) {\n -webkit-perspective: @perspective;\n -moz-perspective: @perspective;\n perspective: @perspective;\n}\n.perspective-origin(@perspective) {\n -webkit-perspective-origin: @perspective;\n -moz-perspective-origin: @perspective;\n perspective-origin: @perspective;\n}\n.transform-origin(@origin) {\n -webkit-transform-origin: @origin;\n -moz-transform-origin: @origin;\n -ms-transform-origin: @origin; // IE9 only\n transform-origin: @origin;\n}\n\n\n// Transitions\n\n.transition(@transition) {\n -webkit-transition: @transition;\n -o-transition: @transition;\n transition: @transition;\n}\n.transition-property(@transition-property) {\n -webkit-transition-property: @transition-property;\n transition-property: @transition-property;\n}\n.transition-delay(@transition-delay) {\n -webkit-transition-delay: @transition-delay;\n transition-delay: @transition-delay;\n}\n.transition-duration(@transition-duration) {\n -webkit-transition-duration: @transition-duration;\n transition-duration: @transition-duration;\n}\n.transition-timing-function(@timing-function) {\n -webkit-transition-timing-function: @timing-function;\n transition-timing-function: @timing-function;\n}\n.transition-transform(@transition) {\n -webkit-transition: -webkit-transform @transition;\n -moz-transition: -moz-transform @transition;\n -o-transition: -o-transform @transition;\n transition: transform @transition;\n}\n\n\n// User select\n// For selecting text on the page\n\n.user-select(@select) {\n -webkit-user-select: @select;\n -moz-user-select: @select;\n -ms-user-select: @select; // IE10+\n user-select: @select;\n}\n",".btn-default,\n.btn-primary,\n.btn-success,\n.btn-info,\n.btn-warning,\n.btn-danger {\n text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.2);\n -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.15), 0 1px 1px rgba(0, 0, 0, 0.075);\n box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.15), 0 1px 1px rgba(0, 0, 0, 0.075);\n}\n.btn-default:active,\n.btn-primary:active,\n.btn-success:active,\n.btn-info:active,\n.btn-warning:active,\n.btn-danger:active,\n.btn-default.active,\n.btn-primary.active,\n.btn-success.active,\n.btn-info.active,\n.btn-warning.active,\n.btn-danger.active {\n -webkit-box-shadow: inset 0 3px 5px rgba(0, 0, 0, 0.125);\n box-shadow: inset 0 3px 5px rgba(0, 0, 0, 0.125);\n}\n.btn-default .badge,\n.btn-primary .badge,\n.btn-success .badge,\n.btn-info .badge,\n.btn-warning .badge,\n.btn-danger .badge {\n text-shadow: none;\n}\n.btn:active,\n.btn.active {\n background-image: none;\n}\n.btn-default {\n background-image: -webkit-linear-gradient(top, #ffffff 0%, #e0e0e0 100%);\n background-image: -o-linear-gradient(top, #ffffff 0%, #e0e0e0 100%);\n background-image: linear-gradient(to bottom, #ffffff 0%, #e0e0e0 100%);\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff', endColorstr='#ffe0e0e0', GradientType=0);\n filter: progid:DXImageTransform.Microsoft.gradient(enabled = false);\n background-repeat: repeat-x;\n border-color: #dbdbdb;\n text-shadow: 0 1px 0 #fff;\n border-color: #ccc;\n}\n.btn-default:hover,\n.btn-default:focus {\n background-color: #e0e0e0;\n background-position: 0 -15px;\n}\n.btn-default:active,\n.btn-default.active {\n background-color: #e0e0e0;\n border-color: #dbdbdb;\n}\n.btn-default.disabled,\n.btn-default:disabled,\n.btn-default[disabled] {\n background-color: #e0e0e0;\n background-image: none;\n}\n.btn-primary {\n background-image: -webkit-linear-gradient(top, #337ab7 0%, #265a88 100%);\n background-image: -o-linear-gradient(top, #337ab7 0%, #265a88 100%);\n background-image: linear-gradient(to bottom, #337ab7 0%, #265a88 100%);\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff337ab7', endColorstr='#ff265a88', GradientType=0);\n filter: progid:DXImageTransform.Microsoft.gradient(enabled = false);\n background-repeat: repeat-x;\n border-color: #245580;\n}\n.btn-primary:hover,\n.btn-primary:focus {\n background-color: #265a88;\n background-position: 0 -15px;\n}\n.btn-primary:active,\n.btn-primary.active {\n background-color: #265a88;\n border-color: #245580;\n}\n.btn-primary.disabled,\n.btn-primary:disabled,\n.btn-primary[disabled] {\n background-color: #265a88;\n background-image: none;\n}\n.btn-success {\n background-image: -webkit-linear-gradient(top, #5cb85c 0%, #419641 100%);\n background-image: -o-linear-gradient(top, #5cb85c 0%, #419641 100%);\n background-image: linear-gradient(to bottom, #5cb85c 0%, #419641 100%);\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5cb85c', endColorstr='#ff419641', GradientType=0);\n filter: progid:DXImageTransform.Microsoft.gradient(enabled = false);\n background-repeat: repeat-x;\n border-color: #3e8f3e;\n}\n.btn-success:hover,\n.btn-success:focus {\n background-color: #419641;\n background-position: 0 -15px;\n}\n.btn-success:active,\n.btn-success.active {\n background-color: #419641;\n border-color: #3e8f3e;\n}\n.btn-success.disabled,\n.btn-success:disabled,\n.btn-success[disabled] {\n background-color: #419641;\n background-image: none;\n}\n.btn-info {\n background-image: -webkit-linear-gradient(top, #5bc0de 0%, #2aabd2 100%);\n background-image: -o-linear-gradient(top, #5bc0de 0%, #2aabd2 100%);\n background-image: linear-gradient(to bottom, #5bc0de 0%, #2aabd2 100%);\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de', endColorstr='#ff2aabd2', GradientType=0);\n filter: progid:DXImageTransform.Microsoft.gradient(enabled = false);\n background-repeat: repeat-x;\n border-color: #28a4c9;\n}\n.btn-info:hover,\n.btn-info:focus {\n background-color: #2aabd2;\n background-position: 0 -15px;\n}\n.btn-info:active,\n.btn-info.active {\n background-color: #2aabd2;\n border-color: #28a4c9;\n}\n.btn-info.disabled,\n.btn-info:disabled,\n.btn-info[disabled] {\n background-color: #2aabd2;\n background-image: none;\n}\n.btn-warning {\n background-image: -webkit-linear-gradient(top, #f0ad4e 0%, #eb9316 100%);\n background-image: -o-linear-gradient(top, #f0ad4e 0%, #eb9316 100%);\n background-image: linear-gradient(to bottom, #f0ad4e 0%, #eb9316 100%);\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff0ad4e', endColorstr='#ffeb9316', GradientType=0);\n filter: progid:DXImageTransform.Microsoft.gradient(enabled = false);\n background-repeat: repeat-x;\n border-color: #e38d13;\n}\n.btn-warning:hover,\n.btn-warning:focus {\n background-color: #eb9316;\n background-position: 0 -15px;\n}\n.btn-warning:active,\n.btn-warning.active {\n background-color: #eb9316;\n border-color: #e38d13;\n}\n.btn-warning.disabled,\n.btn-warning:disabled,\n.btn-warning[disabled] {\n background-color: #eb9316;\n background-image: none;\n}\n.btn-danger {\n background-image: -webkit-linear-gradient(top, #d9534f 0%, #c12e2a 100%);\n background-image: -o-linear-gradient(top, #d9534f 0%, #c12e2a 100%);\n background-image: linear-gradient(to bottom, #d9534f 0%, #c12e2a 100%);\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffd9534f', endColorstr='#ffc12e2a', GradientType=0);\n filter: progid:DXImageTransform.Microsoft.gradient(enabled = false);\n background-repeat: repeat-x;\n border-color: #b92c28;\n}\n.btn-danger:hover,\n.btn-danger:focus {\n background-color: #c12e2a;\n background-position: 0 -15px;\n}\n.btn-danger:active,\n.btn-danger.active {\n background-color: #c12e2a;\n border-color: #b92c28;\n}\n.btn-danger.disabled,\n.btn-danger:disabled,\n.btn-danger[disabled] {\n background-color: #c12e2a;\n background-image: none;\n}\n.thumbnail,\n.img-thumbnail {\n -webkit-box-shadow: 0 1px 2px rgba(0, 0, 0, 0.075);\n box-shadow: 0 1px 2px rgba(0, 0, 0, 0.075);\n}\n.dropdown-menu > li > a:hover,\n.dropdown-menu > li > a:focus {\n background-image: -webkit-linear-gradient(top, #f5f5f5 0%, #e8e8e8 100%);\n background-image: -o-linear-gradient(top, #f5f5f5 0%, #e8e8e8 100%);\n background-image: linear-gradient(to bottom, #f5f5f5 0%, #e8e8e8 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff5f5f5', endColorstr='#ffe8e8e8', GradientType=0);\n background-color: #e8e8e8;\n}\n.dropdown-menu > .active > a,\n.dropdown-menu > .active > a:hover,\n.dropdown-menu > .active > a:focus {\n background-image: -webkit-linear-gradient(top, #337ab7 0%, #2e6da4 100%);\n background-image: -o-linear-gradient(top, #337ab7 0%, #2e6da4 100%);\n background-image: linear-gradient(to bottom, #337ab7 0%, #2e6da4 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff337ab7', endColorstr='#ff2e6da4', GradientType=0);\n background-color: #2e6da4;\n}\n.navbar-default {\n background-image: -webkit-linear-gradient(top, #ffffff 0%, #f8f8f8 100%);\n background-image: -o-linear-gradient(top, #ffffff 0%, #f8f8f8 100%);\n background-image: linear-gradient(to bottom, #ffffff 0%, #f8f8f8 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff', endColorstr='#fff8f8f8', GradientType=0);\n filter: progid:DXImageTransform.Microsoft.gradient(enabled = false);\n border-radius: 4px;\n -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.15), 0 1px 5px rgba(0, 0, 0, 0.075);\n box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.15), 0 1px 5px rgba(0, 0, 0, 0.075);\n}\n.navbar-default .navbar-nav > .open > a,\n.navbar-default .navbar-nav > .active > a {\n background-image: -webkit-linear-gradient(top, #dbdbdb 0%, #e2e2e2 100%);\n background-image: -o-linear-gradient(top, #dbdbdb 0%, #e2e2e2 100%);\n background-image: linear-gradient(to bottom, #dbdbdb 0%, #e2e2e2 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffdbdbdb', endColorstr='#ffe2e2e2', GradientType=0);\n -webkit-box-shadow: inset 0 3px 9px rgba(0, 0, 0, 0.075);\n box-shadow: inset 0 3px 9px rgba(0, 0, 0, 0.075);\n}\n.navbar-brand,\n.navbar-nav > li > a {\n text-shadow: 0 1px 0 rgba(255, 255, 255, 0.25);\n}\n.navbar-inverse {\n background-image: -webkit-linear-gradient(top, #3c3c3c 0%, #222222 100%);\n background-image: -o-linear-gradient(top, #3c3c3c 0%, #222222 100%);\n background-image: linear-gradient(to bottom, #3c3c3c 0%, #222222 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff3c3c3c', endColorstr='#ff222222', GradientType=0);\n filter: progid:DXImageTransform.Microsoft.gradient(enabled = false);\n}\n.navbar-inverse .navbar-nav > .open > a,\n.navbar-inverse .navbar-nav > .active > a {\n background-image: -webkit-linear-gradient(top, #080808 0%, #0f0f0f 100%);\n background-image: -o-linear-gradient(top, #080808 0%, #0f0f0f 100%);\n background-image: linear-gradient(to bottom, #080808 0%, #0f0f0f 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff080808', endColorstr='#ff0f0f0f', GradientType=0);\n -webkit-box-shadow: inset 0 3px 9px rgba(0, 0, 0, 0.25);\n box-shadow: inset 0 3px 9px rgba(0, 0, 0, 0.25);\n}\n.navbar-inverse .navbar-brand,\n.navbar-inverse .navbar-nav > li > a {\n text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25);\n}\n.navbar-static-top,\n.navbar-fixed-top,\n.navbar-fixed-bottom {\n border-radius: 0;\n}\n@media (max-width: 767px) {\n .navbar .navbar-nav .open .dropdown-menu > .active > a,\n .navbar .navbar-nav .open .dropdown-menu > .active > a:hover,\n .navbar .navbar-nav .open .dropdown-menu > .active > a:focus {\n color: #fff;\n background-image: -webkit-linear-gradient(top, #337ab7 0%, #2e6da4 100%);\n background-image: -o-linear-gradient(top, #337ab7 0%, #2e6da4 100%);\n background-image: linear-gradient(to bottom, #337ab7 0%, #2e6da4 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff337ab7', endColorstr='#ff2e6da4', GradientType=0);\n }\n}\n.alert {\n text-shadow: 0 1px 0 rgba(255, 255, 255, 0.2);\n -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.25), 0 1px 2px rgba(0, 0, 0, 0.05);\n box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.25), 0 1px 2px rgba(0, 0, 0, 0.05);\n}\n.alert-success {\n background-image: -webkit-linear-gradient(top, #dff0d8 0%, #c8e5bc 100%);\n background-image: -o-linear-gradient(top, #dff0d8 0%, #c8e5bc 100%);\n background-image: linear-gradient(to bottom, #dff0d8 0%, #c8e5bc 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffdff0d8', endColorstr='#ffc8e5bc', GradientType=0);\n border-color: #b2dba1;\n}\n.alert-info {\n background-image: -webkit-linear-gradient(top, #d9edf7 0%, #b9def0 100%);\n background-image: -o-linear-gradient(top, #d9edf7 0%, #b9def0 100%);\n background-image: linear-gradient(to bottom, #d9edf7 0%, #b9def0 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffd9edf7', endColorstr='#ffb9def0', GradientType=0);\n border-color: #9acfea;\n}\n.alert-warning {\n background-image: -webkit-linear-gradient(top, #fcf8e3 0%, #f8efc0 100%);\n background-image: -o-linear-gradient(top, #fcf8e3 0%, #f8efc0 100%);\n background-image: linear-gradient(to bottom, #fcf8e3 0%, #f8efc0 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffcf8e3', endColorstr='#fff8efc0', GradientType=0);\n border-color: #f5e79e;\n}\n.alert-danger {\n background-image: -webkit-linear-gradient(top, #f2dede 0%, #e7c3c3 100%);\n background-image: -o-linear-gradient(top, #f2dede 0%, #e7c3c3 100%);\n background-image: linear-gradient(to bottom, #f2dede 0%, #e7c3c3 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff2dede', endColorstr='#ffe7c3c3', GradientType=0);\n border-color: #dca7a7;\n}\n.progress {\n background-image: -webkit-linear-gradient(top, #ebebeb 0%, #f5f5f5 100%);\n background-image: -o-linear-gradient(top, #ebebeb 0%, #f5f5f5 100%);\n background-image: linear-gradient(to bottom, #ebebeb 0%, #f5f5f5 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffebebeb', endColorstr='#fff5f5f5', GradientType=0);\n}\n.progress-bar {\n background-image: -webkit-linear-gradient(top, #337ab7 0%, #286090 100%);\n background-image: -o-linear-gradient(top, #337ab7 0%, #286090 100%);\n background-image: linear-gradient(to bottom, #337ab7 0%, #286090 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff337ab7', endColorstr='#ff286090', GradientType=0);\n}\n.progress-bar-success {\n background-image: -webkit-linear-gradient(top, #5cb85c 0%, #449d44 100%);\n background-image: -o-linear-gradient(top, #5cb85c 0%, #449d44 100%);\n background-image: linear-gradient(to bottom, #5cb85c 0%, #449d44 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5cb85c', endColorstr='#ff449d44', GradientType=0);\n}\n.progress-bar-info {\n background-image: -webkit-linear-gradient(top, #5bc0de 0%, #31b0d5 100%);\n background-image: -o-linear-gradient(top, #5bc0de 0%, #31b0d5 100%);\n background-image: linear-gradient(to bottom, #5bc0de 0%, #31b0d5 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de', endColorstr='#ff31b0d5', GradientType=0);\n}\n.progress-bar-warning {\n background-image: -webkit-linear-gradient(top, #f0ad4e 0%, #ec971f 100%);\n background-image: -o-linear-gradient(top, #f0ad4e 0%, #ec971f 100%);\n background-image: linear-gradient(to bottom, #f0ad4e 0%, #ec971f 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff0ad4e', endColorstr='#ffec971f', GradientType=0);\n}\n.progress-bar-danger {\n background-image: -webkit-linear-gradient(top, #d9534f 0%, #c9302c 100%);\n background-image: -o-linear-gradient(top, #d9534f 0%, #c9302c 100%);\n background-image: linear-gradient(to bottom, #d9534f 0%, #c9302c 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffd9534f', endColorstr='#ffc9302c', GradientType=0);\n}\n.progress-bar-striped {\n background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n}\n.list-group {\n border-radius: 4px;\n -webkit-box-shadow: 0 1px 2px rgba(0, 0, 0, 0.075);\n box-shadow: 0 1px 2px rgba(0, 0, 0, 0.075);\n}\n.list-group-item.active,\n.list-group-item.active:hover,\n.list-group-item.active:focus {\n text-shadow: 0 -1px 0 #286090;\n background-image: -webkit-linear-gradient(top, #337ab7 0%, #2b669a 100%);\n background-image: -o-linear-gradient(top, #337ab7 0%, #2b669a 100%);\n background-image: linear-gradient(to bottom, #337ab7 0%, #2b669a 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff337ab7', endColorstr='#ff2b669a', GradientType=0);\n border-color: #2b669a;\n}\n.list-group-item.active .badge,\n.list-group-item.active:hover .badge,\n.list-group-item.active:focus .badge {\n text-shadow: none;\n}\n.panel {\n -webkit-box-shadow: 0 1px 2px rgba(0, 0, 0, 0.05);\n box-shadow: 0 1px 2px rgba(0, 0, 0, 0.05);\n}\n.panel-default > .panel-heading {\n background-image: -webkit-linear-gradient(top, #f5f5f5 0%, #e8e8e8 100%);\n background-image: -o-linear-gradient(top, #f5f5f5 0%, #e8e8e8 100%);\n background-image: linear-gradient(to bottom, #f5f5f5 0%, #e8e8e8 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff5f5f5', endColorstr='#ffe8e8e8', GradientType=0);\n}\n.panel-primary > .panel-heading {\n background-image: -webkit-linear-gradient(top, #337ab7 0%, #2e6da4 100%);\n background-image: -o-linear-gradient(top, #337ab7 0%, #2e6da4 100%);\n background-image: linear-gradient(to bottom, #337ab7 0%, #2e6da4 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff337ab7', endColorstr='#ff2e6da4', GradientType=0);\n}\n.panel-success > .panel-heading {\n background-image: -webkit-linear-gradient(top, #dff0d8 0%, #d0e9c6 100%);\n background-image: -o-linear-gradient(top, #dff0d8 0%, #d0e9c6 100%);\n background-image: linear-gradient(to bottom, #dff0d8 0%, #d0e9c6 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffdff0d8', endColorstr='#ffd0e9c6', GradientType=0);\n}\n.panel-info > .panel-heading {\n background-image: -webkit-linear-gradient(top, #d9edf7 0%, #c4e3f3 100%);\n background-image: -o-linear-gradient(top, #d9edf7 0%, #c4e3f3 100%);\n background-image: linear-gradient(to bottom, #d9edf7 0%, #c4e3f3 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffd9edf7', endColorstr='#ffc4e3f3', GradientType=0);\n}\n.panel-warning > .panel-heading {\n background-image: -webkit-linear-gradient(top, #fcf8e3 0%, #faf2cc 100%);\n background-image: -o-linear-gradient(top, #fcf8e3 0%, #faf2cc 100%);\n background-image: linear-gradient(to bottom, #fcf8e3 0%, #faf2cc 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffcf8e3', endColorstr='#fffaf2cc', GradientType=0);\n}\n.panel-danger > .panel-heading {\n background-image: -webkit-linear-gradient(top, #f2dede 0%, #ebcccc 100%);\n background-image: -o-linear-gradient(top, #f2dede 0%, #ebcccc 100%);\n background-image: linear-gradient(to bottom, #f2dede 0%, #ebcccc 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff2dede', endColorstr='#ffebcccc', GradientType=0);\n}\n.well {\n background-image: -webkit-linear-gradient(top, #e8e8e8 0%, #f5f5f5 100%);\n background-image: -o-linear-gradient(top, #e8e8e8 0%, #f5f5f5 100%);\n background-image: linear-gradient(to bottom, #e8e8e8 0%, #f5f5f5 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffe8e8e8', endColorstr='#fff5f5f5', GradientType=0);\n border-color: #dcdcdc;\n -webkit-box-shadow: inset 0 1px 3px rgba(0, 0, 0, 0.05), 0 1px 0 rgba(255, 255, 255, 0.1);\n box-shadow: inset 0 1px 3px rgba(0, 0, 0, 0.05), 0 1px 0 rgba(255, 255, 255, 0.1);\n}\n/*# sourceMappingURL=bootstrap-theme.css.map */","// Gradients\n\n#gradient {\n\n // Horizontal gradient, from left to right\n //\n // Creates two color stops, start and end, by specifying a color and position for each color stop.\n // Color stops are not available in IE9 and below.\n .horizontal(@start-color: #555; @end-color: #333; @start-percent: 0%; @end-percent: 100%) {\n background-image: -webkit-linear-gradient(left, @start-color @start-percent, @end-color @end-percent); // Safari 5.1-6, Chrome 10+\n background-image: -o-linear-gradient(left, @start-color @start-percent, @end-color @end-percent); // Opera 12\n background-image: linear-gradient(to right, @start-color @start-percent, @end-color @end-percent); // Standard, IE10, Firefox 16+, Opera 12.10+, Safari 7+, Chrome 26+\n background-repeat: repeat-x;\n filter: e(%(\"progid:DXImageTransform.Microsoft.gradient(startColorstr='%d', endColorstr='%d', GradientType=1)\",argb(@start-color),argb(@end-color))); // IE9 and down\n }\n\n // Vertical gradient, from top to bottom\n //\n // Creates two color stops, start and end, by specifying a color and position for each color stop.\n // Color stops are not available in IE9 and below.\n .vertical(@start-color: #555; @end-color: #333; @start-percent: 0%; @end-percent: 100%) {\n background-image: -webkit-linear-gradient(top, @start-color @start-percent, @end-color @end-percent); // Safari 5.1-6, Chrome 10+\n background-image: -o-linear-gradient(top, @start-color @start-percent, @end-color @end-percent); // Opera 12\n background-image: linear-gradient(to bottom, @start-color @start-percent, @end-color @end-percent); // Standard, IE10, Firefox 16+, Opera 12.10+, Safari 7+, Chrome 26+\n background-repeat: repeat-x;\n filter: e(%(\"progid:DXImageTransform.Microsoft.gradient(startColorstr='%d', endColorstr='%d', GradientType=0)\",argb(@start-color),argb(@end-color))); // IE9 and down\n }\n\n .directional(@start-color: #555; @end-color: #333; @deg: 45deg) {\n background-repeat: repeat-x;\n background-image: -webkit-linear-gradient(@deg, @start-color, @end-color); // Safari 5.1-6, Chrome 10+\n background-image: -o-linear-gradient(@deg, @start-color, @end-color); // Opera 12\n background-image: linear-gradient(@deg, @start-color, @end-color); // Standard, IE10, Firefox 16+, Opera 12.10+, Safari 7+, Chrome 26+\n }\n .horizontal-three-colors(@start-color: #00b3ee; @mid-color: #7a43b6; @color-stop: 50%; @end-color: #c3325f) {\n background-image: -webkit-linear-gradient(left, @start-color, @mid-color @color-stop, @end-color);\n background-image: -o-linear-gradient(left, @start-color, @mid-color @color-stop, @end-color);\n background-image: linear-gradient(to right, @start-color, @mid-color @color-stop, @end-color);\n background-repeat: no-repeat;\n filter: e(%(\"progid:DXImageTransform.Microsoft.gradient(startColorstr='%d', endColorstr='%d', GradientType=1)\",argb(@start-color),argb(@end-color))); // IE9 and down, gets no color-stop at all for proper fallback\n }\n .vertical-three-colors(@start-color: #00b3ee; @mid-color: #7a43b6; @color-stop: 50%; @end-color: #c3325f) {\n background-image: -webkit-linear-gradient(@start-color, @mid-color @color-stop, @end-color);\n background-image: -o-linear-gradient(@start-color, @mid-color @color-stop, @end-color);\n background-image: linear-gradient(@start-color, @mid-color @color-stop, @end-color);\n background-repeat: no-repeat;\n filter: e(%(\"progid:DXImageTransform.Microsoft.gradient(startColorstr='%d', endColorstr='%d', GradientType=0)\",argb(@start-color),argb(@end-color))); // IE9 and down, gets no color-stop at all for proper fallback\n }\n .radial(@inner-color: #555; @outer-color: #333) {\n background-image: -webkit-radial-gradient(circle, @inner-color, @outer-color);\n background-image: radial-gradient(circle, @inner-color, @outer-color);\n background-repeat: no-repeat;\n }\n .striped(@color: rgba(255,255,255,.15); @angle: 45deg) {\n background-image: -webkit-linear-gradient(@angle, @color 25%, transparent 25%, transparent 50%, @color 50%, @color 75%, transparent 75%, transparent);\n background-image: -o-linear-gradient(@angle, @color 25%, transparent 25%, transparent 50%, @color 50%, @color 75%, transparent 75%, transparent);\n background-image: linear-gradient(@angle, @color 25%, transparent 25%, transparent 50%, @color 50%, @color 75%, transparent 75%, transparent);\n }\n}\n","// Reset filters for IE\n//\n// When you need to remove a gradient background, do not forget to use this to reset\n// the IE filter for IE9 and below.\n\n.reset-filter() {\n filter: e(%(\"progid:DXImageTransform.Microsoft.gradient(enabled = false)\"));\n}\n"]} \ No newline at end of file diff --git a/jstorm-ui/src/main/webapp/assets/css/bootstrap-theme.min.css b/jstorm-ui/src/main/webapp/assets/css/bootstrap-theme.min.css new file mode 100755 index 000000000..cefa3d1ae --- /dev/null +++ b/jstorm-ui/src/main/webapp/assets/css/bootstrap-theme.min.css @@ -0,0 +1,5 @@ +/*! + * Bootstrap v3.3.4 (http://getbootstrap.com) + * Copyright 2011-2015 Twitter, Inc. + * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE) + */.btn-danger,.btn-default,.btn-info,.btn-primary,.btn-success,.btn-warning{text-shadow:0 -1px 0 rgba(0,0,0,.2);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,.15),0 1px 1px rgba(0,0,0,.075);box-shadow:inset 0 1px 0 rgba(255,255,255,.15),0 1px 1px rgba(0,0,0,.075)}.btn-danger.active,.btn-danger:active,.btn-default.active,.btn-default:active,.btn-info.active,.btn-info:active,.btn-primary.active,.btn-primary:active,.btn-success.active,.btn-success:active,.btn-warning.active,.btn-warning:active{-webkit-box-shadow:inset 0 3px 5px rgba(0,0,0,.125);box-shadow:inset 0 3px 5px rgba(0,0,0,.125)}.btn-danger .badge,.btn-default .badge,.btn-info .badge,.btn-primary .badge,.btn-success .badge,.btn-warning .badge{text-shadow:none}.btn.active,.btn:active{background-image:none}.btn-default{text-shadow:0 1px 0 #fff;background-image:-webkit-linear-gradient(top,#fff 0,#e0e0e0 100%);background-image:-o-linear-gradient(top,#fff 0,#e0e0e0 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#fff),to(#e0e0e0));background-image:linear-gradient(to bottom,#fff 0,#e0e0e0 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff', endColorstr='#ffe0e0e0', GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false);background-repeat:repeat-x;border-color:#dbdbdb;border-color:#ccc}.btn-default:focus,.btn-default:hover{background-color:#e0e0e0;background-position:0 -15px}.btn-default.active,.btn-default:active{background-color:#e0e0e0;border-color:#dbdbdb}.btn-default.disabled,.btn-default:disabled,.btn-default[disabled]{background-color:#e0e0e0;background-image:none}.btn-primary{background-image:-webkit-linear-gradient(top,#337ab7 0,#265a88 100%);background-image:-o-linear-gradient(top,#337ab7 0,#265a88 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#337ab7),to(#265a88));background-image:linear-gradient(to bottom,#337ab7 0,#265a88 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff337ab7', endColorstr='#ff265a88', GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false);background-repeat:repeat-x;border-color:#245580}.btn-primary:focus,.btn-primary:hover{background-color:#265a88;background-position:0 -15px}.btn-primary.active,.btn-primary:active{background-color:#265a88;border-color:#245580}.btn-primary.disabled,.btn-primary:disabled,.btn-primary[disabled]{background-color:#265a88;background-image:none}.btn-success{background-image:-webkit-linear-gradient(top,#5cb85c 0,#419641 100%);background-image:-o-linear-gradient(top,#5cb85c 0,#419641 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#5cb85c),to(#419641));background-image:linear-gradient(to bottom,#5cb85c 0,#419641 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5cb85c', endColorstr='#ff419641', GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false);background-repeat:repeat-x;border-color:#3e8f3e}.btn-success:focus,.btn-success:hover{background-color:#419641;background-position:0 -15px}.btn-success.active,.btn-success:active{background-color:#419641;border-color:#3e8f3e}.btn-success.disabled,.btn-success:disabled,.btn-success[disabled]{background-color:#419641;background-image:none}.btn-info{background-image:-webkit-linear-gradient(top,#5bc0de 0,#2aabd2 100%);background-image:-o-linear-gradient(top,#5bc0de 0,#2aabd2 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#5bc0de),to(#2aabd2));background-image:linear-gradient(to bottom,#5bc0de 0,#2aabd2 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de', endColorstr='#ff2aabd2', GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false);background-repeat:repeat-x;border-color:#28a4c9}.btn-info:focus,.btn-info:hover{background-color:#2aabd2;background-position:0 -15px}.btn-info.active,.btn-info:active{background-color:#2aabd2;border-color:#28a4c9}.btn-info.disabled,.btn-info:disabled,.btn-info[disabled]{background-color:#2aabd2;background-image:none}.btn-warning{background-image:-webkit-linear-gradient(top,#f0ad4e 0,#eb9316 100%);background-image:-o-linear-gradient(top,#f0ad4e 0,#eb9316 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#f0ad4e),to(#eb9316));background-image:linear-gradient(to bottom,#f0ad4e 0,#eb9316 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff0ad4e', endColorstr='#ffeb9316', GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false);background-repeat:repeat-x;border-color:#e38d13}.btn-warning:focus,.btn-warning:hover{background-color:#eb9316;background-position:0 -15px}.btn-warning.active,.btn-warning:active{background-color:#eb9316;border-color:#e38d13}.btn-warning.disabled,.btn-warning:disabled,.btn-warning[disabled]{background-color:#eb9316;background-image:none}.btn-danger{background-image:-webkit-linear-gradient(top,#d9534f 0,#c12e2a 100%);background-image:-o-linear-gradient(top,#d9534f 0,#c12e2a 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#d9534f),to(#c12e2a));background-image:linear-gradient(to bottom,#d9534f 0,#c12e2a 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffd9534f', endColorstr='#ffc12e2a', GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false);background-repeat:repeat-x;border-color:#b92c28}.btn-danger:focus,.btn-danger:hover{background-color:#c12e2a;background-position:0 -15px}.btn-danger.active,.btn-danger:active{background-color:#c12e2a;border-color:#b92c28}.btn-danger.disabled,.btn-danger:disabled,.btn-danger[disabled]{background-color:#c12e2a;background-image:none}.img-thumbnail,.thumbnail{-webkit-box-shadow:0 1px 2px rgba(0,0,0,.075);box-shadow:0 1px 2px rgba(0,0,0,.075)}.dropdown-menu>li>a:focus,.dropdown-menu>li>a:hover{background-color:#e8e8e8;background-image:-webkit-linear-gradient(top,#f5f5f5 0,#e8e8e8 100%);background-image:-o-linear-gradient(top,#f5f5f5 0,#e8e8e8 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#f5f5f5),to(#e8e8e8));background-image:linear-gradient(to bottom,#f5f5f5 0,#e8e8e8 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff5f5f5', endColorstr='#ffe8e8e8', GradientType=0);background-repeat:repeat-x}.dropdown-menu>.active>a,.dropdown-menu>.active>a:focus,.dropdown-menu>.active>a:hover{background-color:#2e6da4;background-image:-webkit-linear-gradient(top,#337ab7 0,#2e6da4 100%);background-image:-o-linear-gradient(top,#337ab7 0,#2e6da4 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#337ab7),to(#2e6da4));background-image:linear-gradient(to bottom,#337ab7 0,#2e6da4 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff337ab7', endColorstr='#ff2e6da4', GradientType=0);background-repeat:repeat-x}.navbar-default{background-image:-webkit-linear-gradient(top,#fff 0,#f8f8f8 100%);background-image:-o-linear-gradient(top,#fff 0,#f8f8f8 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#fff),to(#f8f8f8));background-image:linear-gradient(to bottom,#fff 0,#f8f8f8 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff', endColorstr='#fff8f8f8', GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false);background-repeat:repeat-x;border-radius:4px;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,.15),0 1px 5px rgba(0,0,0,.075);box-shadow:inset 0 1px 0 rgba(255,255,255,.15),0 1px 5px rgba(0,0,0,.075)}.navbar-default .navbar-nav>.active>a,.navbar-default .navbar-nav>.open>a{background-image:-webkit-linear-gradient(top,#dbdbdb 0,#e2e2e2 100%);background-image:-o-linear-gradient(top,#dbdbdb 0,#e2e2e2 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#dbdbdb),to(#e2e2e2));background-image:linear-gradient(to bottom,#dbdbdb 0,#e2e2e2 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffdbdbdb', endColorstr='#ffe2e2e2', GradientType=0);background-repeat:repeat-x;-webkit-box-shadow:inset 0 3px 9px rgba(0,0,0,.075);box-shadow:inset 0 3px 9px rgba(0,0,0,.075)}.navbar-brand,.navbar-nav>li>a{text-shadow:0 1px 0 rgba(255,255,255,.25)}.navbar-inverse{background-image:-webkit-linear-gradient(top,#3c3c3c 0,#222 100%);background-image:-o-linear-gradient(top,#3c3c3c 0,#222 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#3c3c3c),to(#222));background-image:linear-gradient(to bottom,#3c3c3c 0,#222 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff3c3c3c', endColorstr='#ff222222', GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false);background-repeat:repeat-x}.navbar-inverse .navbar-nav>.active>a,.navbar-inverse .navbar-nav>.open>a{background-image:-webkit-linear-gradient(top,#080808 0,#0f0f0f 100%);background-image:-o-linear-gradient(top,#080808 0,#0f0f0f 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#080808),to(#0f0f0f));background-image:linear-gradient(to bottom,#080808 0,#0f0f0f 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff080808', endColorstr='#ff0f0f0f', GradientType=0);background-repeat:repeat-x;-webkit-box-shadow:inset 0 3px 9px rgba(0,0,0,.25);box-shadow:inset 0 3px 9px rgba(0,0,0,.25)}.navbar-inverse .navbar-brand,.navbar-inverse .navbar-nav>li>a{text-shadow:0 -1px 0 rgba(0,0,0,.25)}.navbar-fixed-bottom,.navbar-fixed-top,.navbar-static-top{border-radius:0}@media (max-width:767px){.navbar .navbar-nav .open .dropdown-menu>.active>a,.navbar .navbar-nav .open .dropdown-menu>.active>a:focus,.navbar .navbar-nav .open .dropdown-menu>.active>a:hover{color:#fff;background-image:-webkit-linear-gradient(top,#337ab7 0,#2e6da4 100%);background-image:-o-linear-gradient(top,#337ab7 0,#2e6da4 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#337ab7),to(#2e6da4));background-image:linear-gradient(to bottom,#337ab7 0,#2e6da4 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff337ab7', endColorstr='#ff2e6da4', GradientType=0);background-repeat:repeat-x}}.alert{text-shadow:0 1px 0 rgba(255,255,255,.2);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,.25),0 1px 2px rgba(0,0,0,.05);box-shadow:inset 0 1px 0 rgba(255,255,255,.25),0 1px 2px rgba(0,0,0,.05)}.alert-success{background-image:-webkit-linear-gradient(top,#dff0d8 0,#c8e5bc 100%);background-image:-o-linear-gradient(top,#dff0d8 0,#c8e5bc 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#dff0d8),to(#c8e5bc));background-image:linear-gradient(to bottom,#dff0d8 0,#c8e5bc 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffdff0d8', endColorstr='#ffc8e5bc', GradientType=0);background-repeat:repeat-x;border-color:#b2dba1}.alert-info{background-image:-webkit-linear-gradient(top,#d9edf7 0,#b9def0 100%);background-image:-o-linear-gradient(top,#d9edf7 0,#b9def0 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#d9edf7),to(#b9def0));background-image:linear-gradient(to bottom,#d9edf7 0,#b9def0 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffd9edf7', endColorstr='#ffb9def0', GradientType=0);background-repeat:repeat-x;border-color:#9acfea}.alert-warning{background-image:-webkit-linear-gradient(top,#fcf8e3 0,#f8efc0 100%);background-image:-o-linear-gradient(top,#fcf8e3 0,#f8efc0 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#fcf8e3),to(#f8efc0));background-image:linear-gradient(to bottom,#fcf8e3 0,#f8efc0 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffcf8e3', endColorstr='#fff8efc0', GradientType=0);background-repeat:repeat-x;border-color:#f5e79e}.alert-danger{background-image:-webkit-linear-gradient(top,#f2dede 0,#e7c3c3 100%);background-image:-o-linear-gradient(top,#f2dede 0,#e7c3c3 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#f2dede),to(#e7c3c3));background-image:linear-gradient(to bottom,#f2dede 0,#e7c3c3 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff2dede', endColorstr='#ffe7c3c3', GradientType=0);background-repeat:repeat-x;border-color:#dca7a7}.progress{background-image:-webkit-linear-gradient(top,#ebebeb 0,#f5f5f5 100%);background-image:-o-linear-gradient(top,#ebebeb 0,#f5f5f5 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#ebebeb),to(#f5f5f5));background-image:linear-gradient(to bottom,#ebebeb 0,#f5f5f5 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffebebeb', endColorstr='#fff5f5f5', GradientType=0);background-repeat:repeat-x}.progress-bar{background-image:-webkit-linear-gradient(top,#337ab7 0,#286090 100%);background-image:-o-linear-gradient(top,#337ab7 0,#286090 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#337ab7),to(#286090));background-image:linear-gradient(to bottom,#337ab7 0,#286090 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff337ab7', endColorstr='#ff286090', GradientType=0);background-repeat:repeat-x}.progress-bar-success{background-image:-webkit-linear-gradient(top,#5cb85c 0,#449d44 100%);background-image:-o-linear-gradient(top,#5cb85c 0,#449d44 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#5cb85c),to(#449d44));background-image:linear-gradient(to bottom,#5cb85c 0,#449d44 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5cb85c', endColorstr='#ff449d44', GradientType=0);background-repeat:repeat-x}.progress-bar-info{background-image:-webkit-linear-gradient(top,#5bc0de 0,#31b0d5 100%);background-image:-o-linear-gradient(top,#5bc0de 0,#31b0d5 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#5bc0de),to(#31b0d5));background-image:linear-gradient(to bottom,#5bc0de 0,#31b0d5 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de', endColorstr='#ff31b0d5', GradientType=0);background-repeat:repeat-x}.progress-bar-warning{background-image:-webkit-linear-gradient(top,#f0ad4e 0,#ec971f 100%);background-image:-o-linear-gradient(top,#f0ad4e 0,#ec971f 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#f0ad4e),to(#ec971f));background-image:linear-gradient(to bottom,#f0ad4e 0,#ec971f 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff0ad4e', endColorstr='#ffec971f', GradientType=0);background-repeat:repeat-x}.progress-bar-danger{background-image:-webkit-linear-gradient(top,#d9534f 0,#c9302c 100%);background-image:-o-linear-gradient(top,#d9534f 0,#c9302c 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#d9534f),to(#c9302c));background-image:linear-gradient(to bottom,#d9534f 0,#c9302c 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffd9534f', endColorstr='#ffc9302c', GradientType=0);background-repeat:repeat-x}.progress-bar-striped{background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent)}.list-group{border-radius:4px;-webkit-box-shadow:0 1px 2px rgba(0,0,0,.075);box-shadow:0 1px 2px rgba(0,0,0,.075)}.list-group-item.active,.list-group-item.active:focus,.list-group-item.active:hover{text-shadow:0 -1px 0 #286090;background-image:-webkit-linear-gradient(top,#337ab7 0,#2b669a 100%);background-image:-o-linear-gradient(top,#337ab7 0,#2b669a 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#337ab7),to(#2b669a));background-image:linear-gradient(to bottom,#337ab7 0,#2b669a 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff337ab7', endColorstr='#ff2b669a', GradientType=0);background-repeat:repeat-x;border-color:#2b669a}.list-group-item.active .badge,.list-group-item.active:focus .badge,.list-group-item.active:hover .badge{text-shadow:none}.panel{-webkit-box-shadow:0 1px 2px rgba(0,0,0,.05);box-shadow:0 1px 2px rgba(0,0,0,.05)}.panel-default>.panel-heading{background-image:-webkit-linear-gradient(top,#f5f5f5 0,#e8e8e8 100%);background-image:-o-linear-gradient(top,#f5f5f5 0,#e8e8e8 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#f5f5f5),to(#e8e8e8));background-image:linear-gradient(to bottom,#f5f5f5 0,#e8e8e8 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff5f5f5', endColorstr='#ffe8e8e8', GradientType=0);background-repeat:repeat-x}.panel-primary>.panel-heading{background-image:-webkit-linear-gradient(top,#337ab7 0,#2e6da4 100%);background-image:-o-linear-gradient(top,#337ab7 0,#2e6da4 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#337ab7),to(#2e6da4));background-image:linear-gradient(to bottom,#337ab7 0,#2e6da4 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff337ab7', endColorstr='#ff2e6da4', GradientType=0);background-repeat:repeat-x}.panel-success>.panel-heading{background-image:-webkit-linear-gradient(top,#dff0d8 0,#d0e9c6 100%);background-image:-o-linear-gradient(top,#dff0d8 0,#d0e9c6 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#dff0d8),to(#d0e9c6));background-image:linear-gradient(to bottom,#dff0d8 0,#d0e9c6 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffdff0d8', endColorstr='#ffd0e9c6', GradientType=0);background-repeat:repeat-x}.panel-info>.panel-heading{background-image:-webkit-linear-gradient(top,#d9edf7 0,#c4e3f3 100%);background-image:-o-linear-gradient(top,#d9edf7 0,#c4e3f3 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#d9edf7),to(#c4e3f3));background-image:linear-gradient(to bottom,#d9edf7 0,#c4e3f3 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffd9edf7', endColorstr='#ffc4e3f3', GradientType=0);background-repeat:repeat-x}.panel-warning>.panel-heading{background-image:-webkit-linear-gradient(top,#fcf8e3 0,#faf2cc 100%);background-image:-o-linear-gradient(top,#fcf8e3 0,#faf2cc 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#fcf8e3),to(#faf2cc));background-image:linear-gradient(to bottom,#fcf8e3 0,#faf2cc 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffcf8e3', endColorstr='#fffaf2cc', GradientType=0);background-repeat:repeat-x}.panel-danger>.panel-heading{background-image:-webkit-linear-gradient(top,#f2dede 0,#ebcccc 100%);background-image:-o-linear-gradient(top,#f2dede 0,#ebcccc 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#f2dede),to(#ebcccc));background-image:linear-gradient(to bottom,#f2dede 0,#ebcccc 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff2dede', endColorstr='#ffebcccc', GradientType=0);background-repeat:repeat-x}.well{background-image:-webkit-linear-gradient(top,#e8e8e8 0,#f5f5f5 100%);background-image:-o-linear-gradient(top,#e8e8e8 0,#f5f5f5 100%);background-image:-webkit-gradient(linear,left top,left bottom,from(#e8e8e8),to(#f5f5f5));background-image:linear-gradient(to bottom,#e8e8e8 0,#f5f5f5 100%);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffe8e8e8', endColorstr='#fff5f5f5', GradientType=0);background-repeat:repeat-x;border-color:#dcdcdc;-webkit-box-shadow:inset 0 1px 3px rgba(0,0,0,.05),0 1px 0 rgba(255,255,255,.1);box-shadow:inset 0 1px 3px rgba(0,0,0,.05),0 1px 0 rgba(255,255,255,.1)} \ No newline at end of file diff --git a/jstorm-ui/src/main/webapp/assets/css/bootstrap.css b/jstorm-ui/src/main/webapp/assets/css/bootstrap.css new file mode 100755 index 000000000..fb15e3d69 --- /dev/null +++ b/jstorm-ui/src/main/webapp/assets/css/bootstrap.css @@ -0,0 +1,6584 @@ +/*! + * Bootstrap v3.3.4 (http://getbootstrap.com) + * Copyright 2011-2015 Twitter, Inc. + * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE) + */ + +/*! normalize.css v3.0.2 | MIT License | git.io/normalize */ +html { + font-family: sans-serif; + -webkit-text-size-adjust: 100%; + -ms-text-size-adjust: 100%; +} +body { + margin: 0; +} +article, +aside, +details, +figcaption, +figure, +footer, +header, +hgroup, +main, +menu, +nav, +section, +summary { + display: block; +} +audio, +canvas, +progress, +video { + display: inline-block; + vertical-align: baseline; +} +audio:not([controls]) { + display: none; + height: 0; +} +[hidden], +template { + display: none; +} +a { + background-color: transparent; +} +a:active, +a:hover { + outline: 0; +} +abbr[title] { + border-bottom: 1px dotted; +} +b, +strong { + font-weight: bold; +} +dfn { + font-style: italic; +} +h1 { + margin: .67em 0; + font-size: 2em; +} +mark { + color: #000; + background: #ff0; +} +small { + font-size: 80%; +} +sub, +sup { + position: relative; + font-size: 75%; + line-height: 0; + vertical-align: baseline; +} +sup { + top: -.5em; +} +sub { + bottom: -.25em; +} +img { + border: 0; +} +svg:not(:root) { + overflow: hidden; +} +figure { + margin: 1em 40px; +} +hr { + height: 0; + -webkit-box-sizing: content-box; + -moz-box-sizing: content-box; + box-sizing: content-box; +} +pre { + overflow: auto; +} +code, +kbd, +pre, +samp { + font-family: monospace, monospace; + font-size: 1em; +} +button, +input, +optgroup, +select, +textarea { + margin: 0; + font: inherit; + color: inherit; +} +button { + overflow: visible; +} +button, +select { + text-transform: none; +} +button, +html input[type="button"], +input[type="reset"], +input[type="submit"] { + -webkit-appearance: button; + cursor: pointer; +} +button[disabled], +html input[disabled] { + cursor: default; +} +button::-moz-focus-inner, +input::-moz-focus-inner { + padding: 0; + border: 0; +} +input { + line-height: normal; +} +input[type="checkbox"], +input[type="radio"] { + -webkit-box-sizing: border-box; + -moz-box-sizing: border-box; + box-sizing: border-box; + padding: 0; +} +input[type="number"]::-webkit-inner-spin-button, +input[type="number"]::-webkit-outer-spin-button { + height: auto; +} +input[type="search"] { + -webkit-box-sizing: content-box; + -moz-box-sizing: content-box; + box-sizing: content-box; + -webkit-appearance: textfield; +} +input[type="search"]::-webkit-search-cancel-button, +input[type="search"]::-webkit-search-decoration { + -webkit-appearance: none; +} +fieldset { + padding: .35em .625em .75em; + margin: 0 2px; + border: 1px solid #c0c0c0; +} +legend { + padding: 0; + border: 0; +} +textarea { + overflow: auto; +} +optgroup { + font-weight: bold; +} +table { + border-spacing: 0; + border-collapse: collapse; +} +td, +th { + padding: 0; +} +/*! Source: https://github.com/h5bp/html5-boilerplate/blob/master/src/css/main.css */ +@media print { + *, + *:before, + *:after { + color: #000 !important; + text-shadow: none !important; + background: transparent !important; + -webkit-box-shadow: none !important; + box-shadow: none !important; + } + a, + a:visited { + text-decoration: underline; + } + a[href]:after { + content: " (" attr(href) ")"; + } + abbr[title]:after { + content: " (" attr(title) ")"; + } + a[href^="#"]:after, + a[href^="javascript:"]:after { + content: ""; + } + pre, + blockquote { + border: 1px solid #999; + + page-break-inside: avoid; + } + thead { + display: table-header-group; + } + tr, + img { + page-break-inside: avoid; + } + img { + max-width: 100% !important; + } + p, + h2, + h3 { + orphans: 3; + widows: 3; + } + h2, + h3 { + page-break-after: avoid; + } + select { + background: #fff !important; + } + .navbar { + display: none; + } + .btn > .caret, + .dropup > .btn > .caret { + border-top-color: #000 !important; + } + .label { + border: 1px solid #000; + } + .table { + border-collapse: collapse !important; + } + .table td, + .table th { + background-color: #fff !important; + } + .table-bordered th, + .table-bordered td { + border: 1px solid #ddd !important; + } +} +@font-face { + font-family: 'Glyphicons Halflings'; + + src: url('../fonts/glyphicons-halflings-regular.eot'); + src: url('../fonts/glyphicons-halflings-regular.eot?#iefix') format('embedded-opentype'), url('../fonts/glyphicons-halflings-regular.woff2') format('woff2'), url('../fonts/glyphicons-halflings-regular.woff') format('woff'), url('../fonts/glyphicons-halflings-regular.ttf') format('truetype'), url('../fonts/glyphicons-halflings-regular.svg#glyphicons_halflingsregular') format('svg'); +} +.glyphicon { + position: relative; + top: 1px; + display: inline-block; + font-family: 'Glyphicons Halflings'; + font-style: normal; + font-weight: normal; + line-height: 1; + + -webkit-font-smoothing: antialiased; + -moz-osx-font-smoothing: grayscale; +} +.glyphicon-asterisk:before { + content: "\2a"; +} +.glyphicon-plus:before { + content: "\2b"; +} +.glyphicon-euro:before, +.glyphicon-eur:before { + content: "\20ac"; +} +.glyphicon-minus:before { + content: "\2212"; +} +.glyphicon-cloud:before { + content: "\2601"; +} +.glyphicon-envelope:before { + content: "\2709"; +} +.glyphicon-pencil:before { + content: "\270f"; +} +.glyphicon-glass:before { + content: "\e001"; +} +.glyphicon-music:before { + content: "\e002"; +} +.glyphicon-search:before { + content: "\e003"; +} +.glyphicon-heart:before { + content: "\e005"; +} +.glyphicon-star:before { + content: "\e006"; +} +.glyphicon-star-empty:before { + content: "\e007"; +} +.glyphicon-user:before { + content: "\e008"; +} +.glyphicon-film:before { + content: "\e009"; +} +.glyphicon-th-large:before { + content: "\e010"; +} +.glyphicon-th:before { + content: "\e011"; +} +.glyphicon-th-list:before { + content: "\e012"; +} +.glyphicon-ok:before { + content: "\e013"; +} +.glyphicon-remove:before { + content: "\e014"; +} +.glyphicon-zoom-in:before { + content: "\e015"; +} +.glyphicon-zoom-out:before { + content: "\e016"; +} +.glyphicon-off:before { + content: "\e017"; +} +.glyphicon-signal:before { + content: "\e018"; +} +.glyphicon-cog:before { + content: "\e019"; +} +.glyphicon-trash:before { + content: "\e020"; +} +.glyphicon-home:before { + content: "\e021"; +} +.glyphicon-file:before { + content: "\e022"; +} +.glyphicon-time:before { + content: "\e023"; +} +.glyphicon-road:before { + content: "\e024"; +} +.glyphicon-download-alt:before { + content: "\e025"; +} +.glyphicon-download:before { + content: "\e026"; +} +.glyphicon-upload:before { + content: "\e027"; +} +.glyphicon-inbox:before { + content: "\e028"; +} +.glyphicon-play-circle:before { + content: "\e029"; +} +.glyphicon-repeat:before { + content: "\e030"; +} +.glyphicon-refresh:before { + content: "\e031"; +} +.glyphicon-list-alt:before { + content: "\e032"; +} +.glyphicon-lock:before { + content: "\e033"; +} +.glyphicon-flag:before { + content: "\e034"; +} +.glyphicon-headphones:before { + content: "\e035"; +} +.glyphicon-volume-off:before { + content: "\e036"; +} +.glyphicon-volume-down:before { + content: "\e037"; +} +.glyphicon-volume-up:before { + content: "\e038"; +} +.glyphicon-qrcode:before { + content: "\e039"; +} +.glyphicon-barcode:before { + content: "\e040"; +} +.glyphicon-tag:before { + content: "\e041"; +} +.glyphicon-tags:before { + content: "\e042"; +} +.glyphicon-book:before { + content: "\e043"; +} +.glyphicon-bookmark:before { + content: "\e044"; +} +.glyphicon-print:before { + content: "\e045"; +} +.glyphicon-camera:before { + content: "\e046"; +} +.glyphicon-font:before { + content: "\e047"; +} +.glyphicon-bold:before { + content: "\e048"; +} +.glyphicon-italic:before { + content: "\e049"; +} +.glyphicon-text-height:before { + content: "\e050"; +} +.glyphicon-text-width:before { + content: "\e051"; +} +.glyphicon-align-left:before { + content: "\e052"; +} +.glyphicon-align-center:before { + content: "\e053"; +} +.glyphicon-align-right:before { + content: "\e054"; +} +.glyphicon-align-justify:before { + content: "\e055"; +} +.glyphicon-list:before { + content: "\e056"; +} +.glyphicon-indent-left:before { + content: "\e057"; +} +.glyphicon-indent-right:before { + content: "\e058"; +} +.glyphicon-facetime-video:before { + content: "\e059"; +} +.glyphicon-picture:before { + content: "\e060"; +} +.glyphicon-map-marker:before { + content: "\e062"; +} +.glyphicon-adjust:before { + content: "\e063"; +} +.glyphicon-tint:before { + content: "\e064"; +} +.glyphicon-edit:before { + content: "\e065"; +} +.glyphicon-share:before { + content: "\e066"; +} +.glyphicon-check:before { + content: "\e067"; +} +.glyphicon-move:before { + content: "\e068"; +} +.glyphicon-step-backward:before { + content: "\e069"; +} +.glyphicon-fast-backward:before { + content: "\e070"; +} +.glyphicon-backward:before { + content: "\e071"; +} +.glyphicon-play:before { + content: "\e072"; +} +.glyphicon-pause:before { + content: "\e073"; +} +.glyphicon-stop:before { + content: "\e074"; +} +.glyphicon-forward:before { + content: "\e075"; +} +.glyphicon-fast-forward:before { + content: "\e076"; +} +.glyphicon-step-forward:before { + content: "\e077"; +} +.glyphicon-eject:before { + content: "\e078"; +} +.glyphicon-chevron-left:before { + content: "\e079"; +} +.glyphicon-chevron-right:before { + content: "\e080"; +} +.glyphicon-plus-sign:before { + content: "\e081"; +} +.glyphicon-minus-sign:before { + content: "\e082"; +} +.glyphicon-remove-sign:before { + content: "\e083"; +} +.glyphicon-ok-sign:before { + content: "\e084"; +} +.glyphicon-question-sign:before { + content: "\e085"; +} +.glyphicon-info-sign:before { + content: "\e086"; +} +.glyphicon-screenshot:before { + content: "\e087"; +} +.glyphicon-remove-circle:before { + content: "\e088"; +} +.glyphicon-ok-circle:before { + content: "\e089"; +} +.glyphicon-ban-circle:before { + content: "\e090"; +} +.glyphicon-arrow-left:before { + content: "\e091"; +} +.glyphicon-arrow-right:before { + content: "\e092"; +} +.glyphicon-arrow-up:before { + content: "\e093"; +} +.glyphicon-arrow-down:before { + content: "\e094"; +} +.glyphicon-share-alt:before { + content: "\e095"; +} +.glyphicon-resize-full:before { + content: "\e096"; +} +.glyphicon-resize-small:before { + content: "\e097"; +} +.glyphicon-exclamation-sign:before { + content: "\e101"; +} +.glyphicon-gift:before { + content: "\e102"; +} +.glyphicon-leaf:before { + content: "\e103"; +} +.glyphicon-fire:before { + content: "\e104"; +} +.glyphicon-eye-open:before { + content: "\e105"; +} +.glyphicon-eye-close:before { + content: "\e106"; +} +.glyphicon-warning-sign:before { + content: "\e107"; +} +.glyphicon-plane:before { + content: "\e108"; +} +.glyphicon-calendar:before { + content: "\e109"; +} +.glyphicon-random:before { + content: "\e110"; +} +.glyphicon-comment:before { + content: "\e111"; +} +.glyphicon-magnet:before { + content: "\e112"; +} +.glyphicon-chevron-up:before { + content: "\e113"; +} +.glyphicon-chevron-down:before { + content: "\e114"; +} +.glyphicon-retweet:before { + content: "\e115"; +} +.glyphicon-shopping-cart:before { + content: "\e116"; +} +.glyphicon-folder-close:before { + content: "\e117"; +} +.glyphicon-folder-open:before { + content: "\e118"; +} +.glyphicon-resize-vertical:before { + content: "\e119"; +} +.glyphicon-resize-horizontal:before { + content: "\e120"; +} +.glyphicon-hdd:before { + content: "\e121"; +} +.glyphicon-bullhorn:before { + content: "\e122"; +} +.glyphicon-bell:before { + content: "\e123"; +} +.glyphicon-certificate:before { + content: "\e124"; +} +.glyphicon-thumbs-up:before { + content: "\e125"; +} +.glyphicon-thumbs-down:before { + content: "\e126"; +} +.glyphicon-hand-right:before { + content: "\e127"; +} +.glyphicon-hand-left:before { + content: "\e128"; +} +.glyphicon-hand-up:before { + content: "\e129"; +} +.glyphicon-hand-down:before { + content: "\e130"; +} +.glyphicon-circle-arrow-right:before { + content: "\e131"; +} +.glyphicon-circle-arrow-left:before { + content: "\e132"; +} +.glyphicon-circle-arrow-up:before { + content: "\e133"; +} +.glyphicon-circle-arrow-down:before { + content: "\e134"; +} +.glyphicon-globe:before { + content: "\e135"; +} +.glyphicon-wrench:before { + content: "\e136"; +} +.glyphicon-tasks:before { + content: "\e137"; +} +.glyphicon-filter:before { + content: "\e138"; +} +.glyphicon-briefcase:before { + content: "\e139"; +} +.glyphicon-fullscreen:before { + content: "\e140"; +} +.glyphicon-dashboard:before { + content: "\e141"; +} +.glyphicon-paperclip:before { + content: "\e142"; +} +.glyphicon-heart-empty:before { + content: "\e143"; +} +.glyphicon-link:before { + content: "\e144"; +} +.glyphicon-phone:before { + content: "\e145"; +} +.glyphicon-pushpin:before { + content: "\e146"; +} +.glyphicon-usd:before { + content: "\e148"; +} +.glyphicon-gbp:before { + content: "\e149"; +} +.glyphicon-sort:before { + content: "\e150"; +} +.glyphicon-sort-by-alphabet:before { + content: "\e151"; +} +.glyphicon-sort-by-alphabet-alt:before { + content: "\e152"; +} +.glyphicon-sort-by-order:before { + content: "\e153"; +} +.glyphicon-sort-by-order-alt:before { + content: "\e154"; +} +.glyphicon-sort-by-attributes:before { + content: "\e155"; +} +.glyphicon-sort-by-attributes-alt:before { + content: "\e156"; +} +.glyphicon-unchecked:before { + content: "\e157"; +} +.glyphicon-expand:before { + content: "\e158"; +} +.glyphicon-collapse-down:before { + content: "\e159"; +} +.glyphicon-collapse-up:before { + content: "\e160"; +} +.glyphicon-log-in:before { + content: "\e161"; +} +.glyphicon-flash:before { + content: "\e162"; +} +.glyphicon-log-out:before { + content: "\e163"; +} +.glyphicon-new-window:before { + content: "\e164"; +} +.glyphicon-record:before { + content: "\e165"; +} +.glyphicon-save:before { + content: "\e166"; +} +.glyphicon-open:before { + content: "\e167"; +} +.glyphicon-saved:before { + content: "\e168"; +} +.glyphicon-import:before { + content: "\e169"; +} +.glyphicon-export:before { + content: "\e170"; +} +.glyphicon-send:before { + content: "\e171"; +} +.glyphicon-floppy-disk:before { + content: "\e172"; +} +.glyphicon-floppy-saved:before { + content: "\e173"; +} +.glyphicon-floppy-remove:before { + content: "\e174"; +} +.glyphicon-floppy-save:before { + content: "\e175"; +} +.glyphicon-floppy-open:before { + content: "\e176"; +} +.glyphicon-credit-card:before { + content: "\e177"; +} +.glyphicon-transfer:before { + content: "\e178"; +} +.glyphicon-cutlery:before { + content: "\e179"; +} +.glyphicon-header:before { + content: "\e180"; +} +.glyphicon-compressed:before { + content: "\e181"; +} +.glyphicon-earphone:before { + content: "\e182"; +} +.glyphicon-phone-alt:before { + content: "\e183"; +} +.glyphicon-tower:before { + content: "\e184"; +} +.glyphicon-stats:before { + content: "\e185"; +} +.glyphicon-sd-video:before { + content: "\e186"; +} +.glyphicon-hd-video:before { + content: "\e187"; +} +.glyphicon-subtitles:before { + content: "\e188"; +} +.glyphicon-sound-stereo:before { + content: "\e189"; +} +.glyphicon-sound-dolby:before { + content: "\e190"; +} +.glyphicon-sound-5-1:before { + content: "\e191"; +} +.glyphicon-sound-6-1:before { + content: "\e192"; +} +.glyphicon-sound-7-1:before { + content: "\e193"; +} +.glyphicon-copyright-mark:before { + content: "\e194"; +} +.glyphicon-registration-mark:before { + content: "\e195"; +} +.glyphicon-cloud-download:before { + content: "\e197"; +} +.glyphicon-cloud-upload:before { + content: "\e198"; +} +.glyphicon-tree-conifer:before { + content: "\e199"; +} +.glyphicon-tree-deciduous:before { + content: "\e200"; +} +.glyphicon-cd:before { + content: "\e201"; +} +.glyphicon-save-file:before { + content: "\e202"; +} +.glyphicon-open-file:before { + content: "\e203"; +} +.glyphicon-level-up:before { + content: "\e204"; +} +.glyphicon-copy:before { + content: "\e205"; +} +.glyphicon-paste:before { + content: "\e206"; +} +.glyphicon-alert:before { + content: "\e209"; +} +.glyphicon-equalizer:before { + content: "\e210"; +} +.glyphicon-king:before { + content: "\e211"; +} +.glyphicon-queen:before { + content: "\e212"; +} +.glyphicon-pawn:before { + content: "\e213"; +} +.glyphicon-bishop:before { + content: "\e214"; +} +.glyphicon-knight:before { + content: "\e215"; +} +.glyphicon-baby-formula:before { + content: "\e216"; +} +.glyphicon-tent:before { + content: "\26fa"; +} +.glyphicon-blackboard:before { + content: "\e218"; +} +.glyphicon-bed:before { + content: "\e219"; +} +.glyphicon-apple:before { + content: "\f8ff"; +} +.glyphicon-erase:before { + content: "\e221"; +} +.glyphicon-hourglass:before { + content: "\231b"; +} +.glyphicon-lamp:before { + content: "\e223"; +} +.glyphicon-duplicate:before { + content: "\e224"; +} +.glyphicon-piggy-bank:before { + content: "\e225"; +} +.glyphicon-scissors:before { + content: "\e226"; +} +.glyphicon-bitcoin:before { + content: "\e227"; +} +.glyphicon-btc:before { + content: "\e227"; +} +.glyphicon-xbt:before { + content: "\e227"; +} +.glyphicon-yen:before { + content: "\00a5"; +} +.glyphicon-jpy:before { + content: "\00a5"; +} +.glyphicon-ruble:before { + content: "\20bd"; +} +.glyphicon-rub:before { + content: "\20bd"; +} +.glyphicon-scale:before { + content: "\e230"; +} +.glyphicon-ice-lolly:before { + content: "\e231"; +} +.glyphicon-ice-lolly-tasted:before { + content: "\e232"; +} +.glyphicon-education:before { + content: "\e233"; +} +.glyphicon-option-horizontal:before { + content: "\e234"; +} +.glyphicon-option-vertical:before { + content: "\e235"; +} +.glyphicon-menu-hamburger:before { + content: "\e236"; +} +.glyphicon-modal-window:before { + content: "\e237"; +} +.glyphicon-oil:before { + content: "\e238"; +} +.glyphicon-grain:before { + content: "\e239"; +} +.glyphicon-sunglasses:before { + content: "\e240"; +} +.glyphicon-text-size:before { + content: "\e241"; +} +.glyphicon-text-color:before { + content: "\e242"; +} +.glyphicon-text-background:before { + content: "\e243"; +} +.glyphicon-object-align-top:before { + content: "\e244"; +} +.glyphicon-object-align-bottom:before { + content: "\e245"; +} +.glyphicon-object-align-horizontal:before { + content: "\e246"; +} +.glyphicon-object-align-left:before { + content: "\e247"; +} +.glyphicon-object-align-vertical:before { + content: "\e248"; +} +.glyphicon-object-align-right:before { + content: "\e249"; +} +.glyphicon-triangle-right:before { + content: "\e250"; +} +.glyphicon-triangle-left:before { + content: "\e251"; +} +.glyphicon-triangle-bottom:before { + content: "\e252"; +} +.glyphicon-triangle-top:before { + content: "\e253"; +} +.glyphicon-console:before { + content: "\e254"; +} +.glyphicon-superscript:before { + content: "\e255"; +} +.glyphicon-subscript:before { + content: "\e256"; +} +.glyphicon-menu-left:before { + content: "\e257"; +} +.glyphicon-menu-right:before { + content: "\e258"; +} +.glyphicon-menu-down:before { + content: "\e259"; +} +.glyphicon-menu-up:before { + content: "\e260"; +} +* { + -webkit-box-sizing: border-box; + -moz-box-sizing: border-box; + box-sizing: border-box; +} +*:before, +*:after { + -webkit-box-sizing: border-box; + -moz-box-sizing: border-box; + box-sizing: border-box; +} +html { + font-size: 10px; + + -webkit-tap-highlight-color: rgba(0, 0, 0, 0); +} +body { + font-family: "Helvetica Neue", Helvetica, Arial, sans-serif; + font-size: 14px; + line-height: 1.42857143; + color: #333; + background-color: #fff; +} +input, +button, +select, +textarea { + font-family: inherit; + font-size: inherit; + line-height: inherit; +} +a { + color: #337ab7; + text-decoration: none; +} +a:hover, +a:focus { + color: #23527c; + text-decoration: underline; +} +a:focus { + outline: thin dotted; + outline: 5px auto -webkit-focus-ring-color; + outline-offset: -2px; +} +figure { + margin: 0; +} +img { + vertical-align: middle; +} +.img-responsive, +.thumbnail > img, +.thumbnail a > img, +.carousel-inner > .item > img, +.carousel-inner > .item > a > img { + display: block; + max-width: 100%; + height: auto; +} +.img-rounded { + border-radius: 6px; +} +.img-thumbnail { + display: inline-block; + max-width: 100%; + height: auto; + padding: 4px; + line-height: 1.42857143; + background-color: #fff; + border: 1px solid #ddd; + border-radius: 4px; + -webkit-transition: all .2s ease-in-out; + -o-transition: all .2s ease-in-out; + transition: all .2s ease-in-out; +} +.img-circle { + border-radius: 50%; +} +hr { + margin-top: 20px; + margin-bottom: 20px; + border: 0; + border-top: 1px solid #eee; +} +.sr-only { + position: absolute; + width: 1px; + height: 1px; + padding: 0; + margin: -1px; + overflow: hidden; + clip: rect(0, 0, 0, 0); + border: 0; +} +.sr-only-focusable:active, +.sr-only-focusable:focus { + position: static; + width: auto; + height: auto; + margin: 0; + overflow: visible; + clip: auto; +} +[role="button"] { + cursor: pointer; +} +h1, +h2, +h3, +h4, +h5, +h6, +.h1, +.h2, +.h3, +.h4, +.h5, +.h6 { + font-family: inherit; + font-weight: 500; + line-height: 1.1; + color: inherit; +} +h1 small, +h2 small, +h3 small, +h4 small, +h5 small, +h6 small, +.h1 small, +.h2 small, +.h3 small, +.h4 small, +.h5 small, +.h6 small, +h1 .small, +h2 .small, +h3 .small, +h4 .small, +h5 .small, +h6 .small, +.h1 .small, +.h2 .small, +.h3 .small, +.h4 .small, +.h5 .small, +.h6 .small { + font-weight: normal; + line-height: 1; + color: #777; +} +h1, +.h1, +h2, +.h2, +h3, +.h3 { + margin-top: 20px; + margin-bottom: 10px; +} +h1 small, +.h1 small, +h2 small, +.h2 small, +h3 small, +.h3 small, +h1 .small, +.h1 .small, +h2 .small, +.h2 .small, +h3 .small, +.h3 .small { + font-size: 65%; +} +h4, +.h4, +h5, +.h5, +h6, +.h6 { + margin-top: 10px; + margin-bottom: 10px; +} +h4 small, +.h4 small, +h5 small, +.h5 small, +h6 small, +.h6 small, +h4 .small, +.h4 .small, +h5 .small, +.h5 .small, +h6 .small, +.h6 .small { + font-size: 75%; +} +h1, +.h1 { + font-size: 36px; +} +h2, +.h2 { + font-size: 30px; +} +h3, +.h3 { + font-size: 24px; +} +h4, +.h4 { + font-size: 18px; +} +h5, +.h5 { + font-size: 14px; +} +h6, +.h6 { + font-size: 12px; +} +p { + margin: 0 0 10px; +} +.lead { + margin-bottom: 20px; + font-size: 16px; + font-weight: 300; + line-height: 1.4; +} +@media (min-width: 768px) { + .lead { + font-size: 21px; + } +} +small, +.small { + font-size: 85%; +} +mark, +.mark { + padding: .2em; + background-color: #fcf8e3; +} +.text-left { + text-align: left; +} +.text-right { + text-align: right; +} +.text-center { + text-align: center; +} +.text-justify { + text-align: justify; +} +.text-nowrap { + white-space: nowrap; +} +.text-lowercase { + text-transform: lowercase; +} +.text-uppercase { + text-transform: uppercase; +} +.text-capitalize { + text-transform: capitalize; +} +.text-muted { + color: #777; +} +.text-primary { + color: #337ab7; +} +a.text-primary:hover { + color: #286090; +} +.text-success { + color: #3c763d; +} +a.text-success:hover { + color: #2b542c; +} +.text-info { + color: #31708f; +} +a.text-info:hover { + color: #245269; +} +.text-warning { + color: #8a6d3b; +} +a.text-warning:hover { + color: #66512c; +} +.text-danger { + color: #a94442; +} +a.text-danger:hover { + color: #843534; +} +.bg-primary { + color: #fff; + background-color: #337ab7; +} +a.bg-primary:hover { + background-color: #286090; +} +.bg-success { + background-color: #dff0d8; +} +a.bg-success:hover { + background-color: #c1e2b3; +} +.bg-info { + background-color: #d9edf7; +} +a.bg-info:hover { + background-color: #afd9ee; +} +.bg-warning { + background-color: #fcf8e3; +} +a.bg-warning:hover { + background-color: #f7ecb5; +} +.bg-danger { + background-color: #f2dede; +} +a.bg-danger:hover { + background-color: #e4b9b9; +} +.page-header { + padding-bottom: 9px; + margin: 40px 0 20px; + border-bottom: 1px solid #eee; +} +ul, +ol { + margin-top: 0; + margin-bottom: 10px; +} +ul ul, +ol ul, +ul ol, +ol ol { + margin-bottom: 0; +} +.list-unstyled { + padding-left: 0; + list-style: none; +} +.list-inline { + padding-left: 0; + margin-left: -5px; + list-style: none; +} +.list-inline > li { + display: inline-block; + padding-right: 5px; + padding-left: 5px; +} +dl { + margin-top: 0; + margin-bottom: 20px; +} +dt, +dd { + line-height: 1.42857143; +} +dt { + font-weight: bold; +} +dd { + margin-left: 0; +} +@media (min-width: 768px) { + .dl-horizontal dt { + float: left; + width: 160px; + overflow: hidden; + clear: left; + text-align: right; + text-overflow: ellipsis; + white-space: nowrap; + } + .dl-horizontal dd { + margin-left: 180px; + } +} +abbr[title], +abbr[data-original-title] { + cursor: help; + border-bottom: 1px dotted #777; +} +.initialism { + font-size: 90%; + text-transform: uppercase; +} +blockquote { + padding: 10px 20px; + margin: 0 0 20px; + font-size: 17.5px; + border-left: 5px solid #eee; +} +blockquote p:last-child, +blockquote ul:last-child, +blockquote ol:last-child { + margin-bottom: 0; +} +blockquote footer, +blockquote small, +blockquote .small { + display: block; + font-size: 80%; + line-height: 1.42857143; + color: #777; +} +blockquote footer:before, +blockquote small:before, +blockquote .small:before { + content: '\2014 \00A0'; +} +.blockquote-reverse, +blockquote.pull-right { + padding-right: 15px; + padding-left: 0; + text-align: right; + border-right: 5px solid #eee; + border-left: 0; +} +.blockquote-reverse footer:before, +blockquote.pull-right footer:before, +.blockquote-reverse small:before, +blockquote.pull-right small:before, +.blockquote-reverse .small:before, +blockquote.pull-right .small:before { + content: ''; +} +.blockquote-reverse footer:after, +blockquote.pull-right footer:after, +.blockquote-reverse small:after, +blockquote.pull-right small:after, +.blockquote-reverse .small:after, +blockquote.pull-right .small:after { + content: '\00A0 \2014'; +} +address { + margin-bottom: 20px; + font-style: normal; + line-height: 1.42857143; +} +code, +kbd, +pre, +samp { + font-family: Menlo, Monaco, Consolas, "Courier New", monospace; +} +code { + padding: 2px 4px; + font-size: 90%; + color: #c7254e; + background-color: #f9f2f4; + border-radius: 4px; +} +kbd { + padding: 2px 4px; + font-size: 90%; + color: #fff; + background-color: #333; + border-radius: 3px; + -webkit-box-shadow: inset 0 -1px 0 rgba(0, 0, 0, .25); + box-shadow: inset 0 -1px 0 rgba(0, 0, 0, .25); +} +kbd kbd { + padding: 0; + font-size: 100%; + font-weight: bold; + -webkit-box-shadow: none; + box-shadow: none; +} +pre { + display: block; + padding: 9.5px; + margin: 0 0 10px; + font-size: 13px; + line-height: 1.42857143; + color: #333; + word-break: break-all; + word-wrap: break-word; + background-color: #f5f5f5; + border: 1px solid #ccc; + border-radius: 4px; +} +pre code { + padding: 0; + font-size: inherit; + color: inherit; + white-space: pre-wrap; + background-color: transparent; + border-radius: 0; +} +.pre-scrollable { + max-height: 340px; + overflow-y: scroll; +} +.container { + padding-right: 15px; + padding-left: 15px; + margin-right: auto; + margin-left: auto; +} +@media (min-width: 768px) { + .container { + width: 750px; + } +} +@media (min-width: 992px) { + .container { + width: 970px; + } +} +@media (min-width: 1200px) { + .container { + width: 1170px; + } +} +.container-fluid { + padding-right: 15px; + padding-left: 15px; + margin-right: auto; + margin-left: auto; +} +.row { + margin-right: -15px; + margin-left: -15px; +} +.col-xs-1, .col-sm-1, .col-md-1, .col-lg-1, .col-xs-2, .col-sm-2, .col-md-2, .col-lg-2, .col-xs-3, .col-sm-3, .col-md-3, .col-lg-3, .col-xs-4, .col-sm-4, .col-md-4, .col-lg-4, .col-xs-5, .col-sm-5, .col-md-5, .col-lg-5, .col-xs-6, .col-sm-6, .col-md-6, .col-lg-6, .col-xs-7, .col-sm-7, .col-md-7, .col-lg-7, .col-xs-8, .col-sm-8, .col-md-8, .col-lg-8, .col-xs-9, .col-sm-9, .col-md-9, .col-lg-9, .col-xs-10, .col-sm-10, .col-md-10, .col-lg-10, .col-xs-11, .col-sm-11, .col-md-11, .col-lg-11, .col-xs-12, .col-sm-12, .col-md-12, .col-lg-12 { + position: relative; + min-height: 1px; + padding-right: 15px; + padding-left: 15px; +} +.col-xs-1, .col-xs-2, .col-xs-3, .col-xs-4, .col-xs-5, .col-xs-6, .col-xs-7, .col-xs-8, .col-xs-9, .col-xs-10, .col-xs-11, .col-xs-12 { + float: left; +} +.col-xs-12 { + width: 100%; +} +.col-xs-11 { + width: 91.66666667%; +} +.col-xs-10 { + width: 83.33333333%; +} +.col-xs-9 { + width: 75%; +} +.col-xs-8 { + width: 66.66666667%; +} +.col-xs-7 { + width: 58.33333333%; +} +.col-xs-6 { + width: 50%; +} +.col-xs-5 { + width: 41.66666667%; +} +.col-xs-4 { + width: 33.33333333%; +} +.col-xs-3 { + width: 25%; +} +.col-xs-2 { + width: 16.66666667%; +} +.col-xs-1 { + width: 8.33333333%; +} +.col-xs-pull-12 { + right: 100%; +} +.col-xs-pull-11 { + right: 91.66666667%; +} +.col-xs-pull-10 { + right: 83.33333333%; +} +.col-xs-pull-9 { + right: 75%; +} +.col-xs-pull-8 { + right: 66.66666667%; +} +.col-xs-pull-7 { + right: 58.33333333%; +} +.col-xs-pull-6 { + right: 50%; +} +.col-xs-pull-5 { + right: 41.66666667%; +} +.col-xs-pull-4 { + right: 33.33333333%; +} +.col-xs-pull-3 { + right: 25%; +} +.col-xs-pull-2 { + right: 16.66666667%; +} +.col-xs-pull-1 { + right: 8.33333333%; +} +.col-xs-pull-0 { + right: auto; +} +.col-xs-push-12 { + left: 100%; +} +.col-xs-push-11 { + left: 91.66666667%; +} +.col-xs-push-10 { + left: 83.33333333%; +} +.col-xs-push-9 { + left: 75%; +} +.col-xs-push-8 { + left: 66.66666667%; +} +.col-xs-push-7 { + left: 58.33333333%; +} +.col-xs-push-6 { + left: 50%; +} +.col-xs-push-5 { + left: 41.66666667%; +} +.col-xs-push-4 { + left: 33.33333333%; +} +.col-xs-push-3 { + left: 25%; +} +.col-xs-push-2 { + left: 16.66666667%; +} +.col-xs-push-1 { + left: 8.33333333%; +} +.col-xs-push-0 { + left: auto; +} +.col-xs-offset-12 { + margin-left: 100%; +} +.col-xs-offset-11 { + margin-left: 91.66666667%; +} +.col-xs-offset-10 { + margin-left: 83.33333333%; +} +.col-xs-offset-9 { + margin-left: 75%; +} +.col-xs-offset-8 { + margin-left: 66.66666667%; +} +.col-xs-offset-7 { + margin-left: 58.33333333%; +} +.col-xs-offset-6 { + margin-left: 50%; +} +.col-xs-offset-5 { + margin-left: 41.66666667%; +} +.col-xs-offset-4 { + margin-left: 33.33333333%; +} +.col-xs-offset-3 { + margin-left: 25%; +} +.col-xs-offset-2 { + margin-left: 16.66666667%; +} +.col-xs-offset-1 { + margin-left: 8.33333333%; +} +.col-xs-offset-0 { + margin-left: 0; +} +@media (min-width: 768px) { + .col-sm-1, .col-sm-2, .col-sm-3, .col-sm-4, .col-sm-5, .col-sm-6, .col-sm-7, .col-sm-8, .col-sm-9, .col-sm-10, .col-sm-11, .col-sm-12 { + float: left; + } + .col-sm-12 { + width: 100%; + } + .col-sm-11 { + width: 91.66666667%; + } + .col-sm-10 { + width: 83.33333333%; + } + .col-sm-9 { + width: 75%; + } + .col-sm-8 { + width: 66.66666667%; + } + .col-sm-7 { + width: 58.33333333%; + } + .col-sm-6 { + width: 50%; + } + .col-sm-5 { + width: 41.66666667%; + } + .col-sm-4 { + width: 33.33333333%; + } + .col-sm-3 { + width: 25%; + } + .col-sm-2 { + width: 16.66666667%; + } + .col-sm-1 { + width: 8.33333333%; + } + .col-sm-pull-12 { + right: 100%; + } + .col-sm-pull-11 { + right: 91.66666667%; + } + .col-sm-pull-10 { + right: 83.33333333%; + } + .col-sm-pull-9 { + right: 75%; + } + .col-sm-pull-8 { + right: 66.66666667%; + } + .col-sm-pull-7 { + right: 58.33333333%; + } + .col-sm-pull-6 { + right: 50%; + } + .col-sm-pull-5 { + right: 41.66666667%; + } + .col-sm-pull-4 { + right: 33.33333333%; + } + .col-sm-pull-3 { + right: 25%; + } + .col-sm-pull-2 { + right: 16.66666667%; + } + .col-sm-pull-1 { + right: 8.33333333%; + } + .col-sm-pull-0 { + right: auto; + } + .col-sm-push-12 { + left: 100%; + } + .col-sm-push-11 { + left: 91.66666667%; + } + .col-sm-push-10 { + left: 83.33333333%; + } + .col-sm-push-9 { + left: 75%; + } + .col-sm-push-8 { + left: 66.66666667%; + } + .col-sm-push-7 { + left: 58.33333333%; + } + .col-sm-push-6 { + left: 50%; + } + .col-sm-push-5 { + left: 41.66666667%; + } + .col-sm-push-4 { + left: 33.33333333%; + } + .col-sm-push-3 { + left: 25%; + } + .col-sm-push-2 { + left: 16.66666667%; + } + .col-sm-push-1 { + left: 8.33333333%; + } + .col-sm-push-0 { + left: auto; + } + .col-sm-offset-12 { + margin-left: 100%; + } + .col-sm-offset-11 { + margin-left: 91.66666667%; + } + .col-sm-offset-10 { + margin-left: 83.33333333%; + } + .col-sm-offset-9 { + margin-left: 75%; + } + .col-sm-offset-8 { + margin-left: 66.66666667%; + } + .col-sm-offset-7 { + margin-left: 58.33333333%; + } + .col-sm-offset-6 { + margin-left: 50%; + } + .col-sm-offset-5 { + margin-left: 41.66666667%; + } + .col-sm-offset-4 { + margin-left: 33.33333333%; + } + .col-sm-offset-3 { + margin-left: 25%; + } + .col-sm-offset-2 { + margin-left: 16.66666667%; + } + .col-sm-offset-1 { + margin-left: 8.33333333%; + } + .col-sm-offset-0 { + margin-left: 0; + } +} +@media (min-width: 992px) { + .col-md-1, .col-md-2, .col-md-3, .col-md-4, .col-md-5, .col-md-6, .col-md-7, .col-md-8, .col-md-9, .col-md-10, .col-md-11, .col-md-12 { + float: left; + } + .col-md-12 { + width: 100%; + } + .col-md-11 { + width: 91.66666667%; + } + .col-md-10 { + width: 83.33333333%; + } + .col-md-9 { + width: 75%; + } + .col-md-8 { + width: 66.66666667%; + } + .col-md-7 { + width: 58.33333333%; + } + .col-md-6 { + width: 50%; + } + .col-md-5 { + width: 41.66666667%; + } + .col-md-4 { + width: 33.33333333%; + } + .col-md-3 { + width: 25%; + } + .col-md-2 { + width: 16.66666667%; + } + .col-md-1 { + width: 8.33333333%; + } + .col-md-pull-12 { + right: 100%; + } + .col-md-pull-11 { + right: 91.66666667%; + } + .col-md-pull-10 { + right: 83.33333333%; + } + .col-md-pull-9 { + right: 75%; + } + .col-md-pull-8 { + right: 66.66666667%; + } + .col-md-pull-7 { + right: 58.33333333%; + } + .col-md-pull-6 { + right: 50%; + } + .col-md-pull-5 { + right: 41.66666667%; + } + .col-md-pull-4 { + right: 33.33333333%; + } + .col-md-pull-3 { + right: 25%; + } + .col-md-pull-2 { + right: 16.66666667%; + } + .col-md-pull-1 { + right: 8.33333333%; + } + .col-md-pull-0 { + right: auto; + } + .col-md-push-12 { + left: 100%; + } + .col-md-push-11 { + left: 91.66666667%; + } + .col-md-push-10 { + left: 83.33333333%; + } + .col-md-push-9 { + left: 75%; + } + .col-md-push-8 { + left: 66.66666667%; + } + .col-md-push-7 { + left: 58.33333333%; + } + .col-md-push-6 { + left: 50%; + } + .col-md-push-5 { + left: 41.66666667%; + } + .col-md-push-4 { + left: 33.33333333%; + } + .col-md-push-3 { + left: 25%; + } + .col-md-push-2 { + left: 16.66666667%; + } + .col-md-push-1 { + left: 8.33333333%; + } + .col-md-push-0 { + left: auto; + } + .col-md-offset-12 { + margin-left: 100%; + } + .col-md-offset-11 { + margin-left: 91.66666667%; + } + .col-md-offset-10 { + margin-left: 83.33333333%; + } + .col-md-offset-9 { + margin-left: 75%; + } + .col-md-offset-8 { + margin-left: 66.66666667%; + } + .col-md-offset-7 { + margin-left: 58.33333333%; + } + .col-md-offset-6 { + margin-left: 50%; + } + .col-md-offset-5 { + margin-left: 41.66666667%; + } + .col-md-offset-4 { + margin-left: 33.33333333%; + } + .col-md-offset-3 { + margin-left: 25%; + } + .col-md-offset-2 { + margin-left: 16.66666667%; + } + .col-md-offset-1 { + margin-left: 8.33333333%; + } + .col-md-offset-0 { + margin-left: 0; + } +} +@media (min-width: 1200px) { + .col-lg-1, .col-lg-2, .col-lg-3, .col-lg-4, .col-lg-5, .col-lg-6, .col-lg-7, .col-lg-8, .col-lg-9, .col-lg-10, .col-lg-11, .col-lg-12 { + float: left; + } + .col-lg-12 { + width: 100%; + } + .col-lg-11 { + width: 91.66666667%; + } + .col-lg-10 { + width: 83.33333333%; + } + .col-lg-9 { + width: 75%; + } + .col-lg-8 { + width: 66.66666667%; + } + .col-lg-7 { + width: 58.33333333%; + } + .col-lg-6 { + width: 50%; + } + .col-lg-5 { + width: 41.66666667%; + } + .col-lg-4 { + width: 33.33333333%; + } + .col-lg-3 { + width: 25%; + } + .col-lg-2 { + width: 16.66666667%; + } + .col-lg-1 { + width: 8.33333333%; + } + .col-lg-pull-12 { + right: 100%; + } + .col-lg-pull-11 { + right: 91.66666667%; + } + .col-lg-pull-10 { + right: 83.33333333%; + } + .col-lg-pull-9 { + right: 75%; + } + .col-lg-pull-8 { + right: 66.66666667%; + } + .col-lg-pull-7 { + right: 58.33333333%; + } + .col-lg-pull-6 { + right: 50%; + } + .col-lg-pull-5 { + right: 41.66666667%; + } + .col-lg-pull-4 { + right: 33.33333333%; + } + .col-lg-pull-3 { + right: 25%; + } + .col-lg-pull-2 { + right: 16.66666667%; + } + .col-lg-pull-1 { + right: 8.33333333%; + } + .col-lg-pull-0 { + right: auto; + } + .col-lg-push-12 { + left: 100%; + } + .col-lg-push-11 { + left: 91.66666667%; + } + .col-lg-push-10 { + left: 83.33333333%; + } + .col-lg-push-9 { + left: 75%; + } + .col-lg-push-8 { + left: 66.66666667%; + } + .col-lg-push-7 { + left: 58.33333333%; + } + .col-lg-push-6 { + left: 50%; + } + .col-lg-push-5 { + left: 41.66666667%; + } + .col-lg-push-4 { + left: 33.33333333%; + } + .col-lg-push-3 { + left: 25%; + } + .col-lg-push-2 { + left: 16.66666667%; + } + .col-lg-push-1 { + left: 8.33333333%; + } + .col-lg-push-0 { + left: auto; + } + .col-lg-offset-12 { + margin-left: 100%; + } + .col-lg-offset-11 { + margin-left: 91.66666667%; + } + .col-lg-offset-10 { + margin-left: 83.33333333%; + } + .col-lg-offset-9 { + margin-left: 75%; + } + .col-lg-offset-8 { + margin-left: 66.66666667%; + } + .col-lg-offset-7 { + margin-left: 58.33333333%; + } + .col-lg-offset-6 { + margin-left: 50%; + } + .col-lg-offset-5 { + margin-left: 41.66666667%; + } + .col-lg-offset-4 { + margin-left: 33.33333333%; + } + .col-lg-offset-3 { + margin-left: 25%; + } + .col-lg-offset-2 { + margin-left: 16.66666667%; + } + .col-lg-offset-1 { + margin-left: 8.33333333%; + } + .col-lg-offset-0 { + margin-left: 0; + } +} +table { + background-color: transparent; +} +caption { + padding-top: 8px; + padding-bottom: 8px; + color: #777; + text-align: left; +} +th { + text-align: left; +} +.table { + width: 100%; + max-width: 100%; + margin-bottom: 20px; +} +.table > thead > tr > th, +.table > tbody > tr > th, +.table > tfoot > tr > th, +.table > thead > tr > td, +.table > tbody > tr > td, +.table > tfoot > tr > td { + padding: 8px; + line-height: 1.42857143; + vertical-align: top; + border-top: 1px solid #ddd; +} +.table > thead > tr > th { + vertical-align: bottom; + border-bottom: 2px solid #ddd; +} +.table > caption + thead > tr:first-child > th, +.table > colgroup + thead > tr:first-child > th, +.table > thead:first-child > tr:first-child > th, +.table > caption + thead > tr:first-child > td, +.table > colgroup + thead > tr:first-child > td, +.table > thead:first-child > tr:first-child > td { + border-top: 0; +} +.table > tbody + tbody { + border-top: 2px solid #ddd; +} +.table .table { + background-color: #fff; +} +.table-condensed > thead > tr > th, +.table-condensed > tbody > tr > th, +.table-condensed > tfoot > tr > th, +.table-condensed > thead > tr > td, +.table-condensed > tbody > tr > td, +.table-condensed > tfoot > tr > td { + padding: 5px; +} +.table-bordered { + border: 1px solid #ddd; +} +.table-bordered > thead > tr > th, +.table-bordered > tbody > tr > th, +.table-bordered > tfoot > tr > th, +.table-bordered > thead > tr > td, +.table-bordered > tbody > tr > td, +.table-bordered > tfoot > tr > td { + border: 1px solid #ddd; +} +.table-bordered > thead > tr > th, +.table-bordered > thead > tr > td { + border-bottom-width: 2px; +} +.table-striped > tbody > tr:nth-of-type(odd) { + background-color: #f9f9f9; +} +.table-hover > tbody > tr:hover { + background-color: #f5f5f5; +} +table col[class*="col-"] { + position: static; + display: table-column; + float: none; +} +table td[class*="col-"], +table th[class*="col-"] { + position: static; + display: table-cell; + float: none; +} +.table > thead > tr > td.active, +.table > tbody > tr > td.active, +.table > tfoot > tr > td.active, +.table > thead > tr > th.active, +.table > tbody > tr > th.active, +.table > tfoot > tr > th.active, +.table > thead > tr.active > td, +.table > tbody > tr.active > td, +.table > tfoot > tr.active > td, +.table > thead > tr.active > th, +.table > tbody > tr.active > th, +.table > tfoot > tr.active > th { + background-color: #f5f5f5; +} +.table-hover > tbody > tr > td.active:hover, +.table-hover > tbody > tr > th.active:hover, +.table-hover > tbody > tr.active:hover > td, +.table-hover > tbody > tr:hover > .active, +.table-hover > tbody > tr.active:hover > th { + background-color: #e8e8e8; +} +.table > thead > tr > td.success, +.table > tbody > tr > td.success, +.table > tfoot > tr > td.success, +.table > thead > tr > th.success, +.table > tbody > tr > th.success, +.table > tfoot > tr > th.success, +.table > thead > tr.success > td, +.table > tbody > tr.success > td, +.table > tfoot > tr.success > td, +.table > thead > tr.success > th, +.table > tbody > tr.success > th, +.table > tfoot > tr.success > th { + background-color: #dff0d8; +} +.table-hover > tbody > tr > td.success:hover, +.table-hover > tbody > tr > th.success:hover, +.table-hover > tbody > tr.success:hover > td, +.table-hover > tbody > tr:hover > .success, +.table-hover > tbody > tr.success:hover > th { + background-color: #d0e9c6; +} +.table > thead > tr > td.info, +.table > tbody > tr > td.info, +.table > tfoot > tr > td.info, +.table > thead > tr > th.info, +.table > tbody > tr > th.info, +.table > tfoot > tr > th.info, +.table > thead > tr.info > td, +.table > tbody > tr.info > td, +.table > tfoot > tr.info > td, +.table > thead > tr.info > th, +.table > tbody > tr.info > th, +.table > tfoot > tr.info > th { + background-color: #d9edf7; +} +.table-hover > tbody > tr > td.info:hover, +.table-hover > tbody > tr > th.info:hover, +.table-hover > tbody > tr.info:hover > td, +.table-hover > tbody > tr:hover > .info, +.table-hover > tbody > tr.info:hover > th { + background-color: #c4e3f3; +} +.table > thead > tr > td.warning, +.table > tbody > tr > td.warning, +.table > tfoot > tr > td.warning, +.table > thead > tr > th.warning, +.table > tbody > tr > th.warning, +.table > tfoot > tr > th.warning, +.table > thead > tr.warning > td, +.table > tbody > tr.warning > td, +.table > tfoot > tr.warning > td, +.table > thead > tr.warning > th, +.table > tbody > tr.warning > th, +.table > tfoot > tr.warning > th { + background-color: #fcf8e3; +} +.table-hover > tbody > tr > td.warning:hover, +.table-hover > tbody > tr > th.warning:hover, +.table-hover > tbody > tr.warning:hover > td, +.table-hover > tbody > tr:hover > .warning, +.table-hover > tbody > tr.warning:hover > th { + background-color: #faf2cc; +} +.table > thead > tr > td.danger, +.table > tbody > tr > td.danger, +.table > tfoot > tr > td.danger, +.table > thead > tr > th.danger, +.table > tbody > tr > th.danger, +.table > tfoot > tr > th.danger, +.table > thead > tr.danger > td, +.table > tbody > tr.danger > td, +.table > tfoot > tr.danger > td, +.table > thead > tr.danger > th, +.table > tbody > tr.danger > th, +.table > tfoot > tr.danger > th { + background-color: #f2dede; +} +.table-hover > tbody > tr > td.danger:hover, +.table-hover > tbody > tr > th.danger:hover, +.table-hover > tbody > tr.danger:hover > td, +.table-hover > tbody > tr:hover > .danger, +.table-hover > tbody > tr.danger:hover > th { + background-color: #ebcccc; +} +.table-responsive { + min-height: .01%; + overflow-x: auto; +} +@media screen and (max-width: 767px) { + .table-responsive { + width: 100%; + margin-bottom: 15px; + overflow-y: hidden; + -ms-overflow-style: -ms-autohiding-scrollbar; + border: 1px solid #ddd; + } + .table-responsive > .table { + margin-bottom: 0; + } + .table-responsive > .table > thead > tr > th, + .table-responsive > .table > tbody > tr > th, + .table-responsive > .table > tfoot > tr > th, + .table-responsive > .table > thead > tr > td, + .table-responsive > .table > tbody > tr > td, + .table-responsive > .table > tfoot > tr > td { + white-space: nowrap; + } + .table-responsive > .table-bordered { + border: 0; + } + .table-responsive > .table-bordered > thead > tr > th:first-child, + .table-responsive > .table-bordered > tbody > tr > th:first-child, + .table-responsive > .table-bordered > tfoot > tr > th:first-child, + .table-responsive > .table-bordered > thead > tr > td:first-child, + .table-responsive > .table-bordered > tbody > tr > td:first-child, + .table-responsive > .table-bordered > tfoot > tr > td:first-child { + border-left: 0; + } + .table-responsive > .table-bordered > thead > tr > th:last-child, + .table-responsive > .table-bordered > tbody > tr > th:last-child, + .table-responsive > .table-bordered > tfoot > tr > th:last-child, + .table-responsive > .table-bordered > thead > tr > td:last-child, + .table-responsive > .table-bordered > tbody > tr > td:last-child, + .table-responsive > .table-bordered > tfoot > tr > td:last-child { + border-right: 0; + } + .table-responsive > .table-bordered > tbody > tr:last-child > th, + .table-responsive > .table-bordered > tfoot > tr:last-child > th, + .table-responsive > .table-bordered > tbody > tr:last-child > td, + .table-responsive > .table-bordered > tfoot > tr:last-child > td { + border-bottom: 0; + } +} +fieldset { + min-width: 0; + padding: 0; + margin: 0; + border: 0; +} +legend { + display: block; + width: 100%; + padding: 0; + margin-bottom: 20px; + font-size: 21px; + line-height: inherit; + color: #333; + border: 0; + border-bottom: 1px solid #e5e5e5; +} +label { + display: inline-block; + max-width: 100%; + margin-bottom: 5px; + font-weight: bold; +} +input[type="search"] { + -webkit-box-sizing: border-box; + -moz-box-sizing: border-box; + box-sizing: border-box; +} +input[type="radio"], +input[type="checkbox"] { + margin: 4px 0 0; + margin-top: 1px \9; + line-height: normal; +} +input[type="file"] { + display: block; +} +input[type="range"] { + display: block; + width: 100%; +} +select[multiple], +select[size] { + height: auto; +} +input[type="file"]:focus, +input[type="radio"]:focus, +input[type="checkbox"]:focus { + outline: thin dotted; + outline: 5px auto -webkit-focus-ring-color; + outline-offset: -2px; +} +output { + display: block; + padding-top: 7px; + font-size: 14px; + line-height: 1.42857143; + color: #555; +} +.form-control { + display: block; + width: 100%; + height: 34px; + padding: 6px 12px; + font-size: 14px; + line-height: 1.42857143; + color: #555; + background-color: #fff; + background-image: none; + border: 1px solid #ccc; + border-radius: 4px; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); + -webkit-transition: border-color ease-in-out .15s, -webkit-box-shadow ease-in-out .15s; + -o-transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s; + transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s; +} +.form-control:focus { + border-color: #66afe9; + outline: 0; + -webkit-box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, .6); + box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, .6); +} +.form-control::-moz-placeholder { + color: #999; + opacity: 1; +} +.form-control:-ms-input-placeholder { + color: #999; +} +.form-control::-webkit-input-placeholder { + color: #999; +} +.form-control[disabled], +.form-control[readonly], +fieldset[disabled] .form-control { + background-color: #eee; + opacity: 1; +} +.form-control[disabled], +fieldset[disabled] .form-control { + cursor: not-allowed; +} +textarea.form-control { + height: auto; +} +input[type="search"] { + -webkit-appearance: none; +} +@media screen and (-webkit-min-device-pixel-ratio: 0) { + input[type="date"], + input[type="time"], + input[type="datetime-local"], + input[type="month"] { + line-height: 34px; + } + input[type="date"].input-sm, + input[type="time"].input-sm, + input[type="datetime-local"].input-sm, + input[type="month"].input-sm, + .input-group-sm input[type="date"], + .input-group-sm input[type="time"], + .input-group-sm input[type="datetime-local"], + .input-group-sm input[type="month"] { + line-height: 30px; + } + input[type="date"].input-lg, + input[type="time"].input-lg, + input[type="datetime-local"].input-lg, + input[type="month"].input-lg, + .input-group-lg input[type="date"], + .input-group-lg input[type="time"], + .input-group-lg input[type="datetime-local"], + .input-group-lg input[type="month"] { + line-height: 46px; + } +} +.form-group { + margin-bottom: 15px; +} +.radio, +.checkbox { + position: relative; + display: block; + margin-top: 10px; + margin-bottom: 10px; +} +.radio label, +.checkbox label { + min-height: 20px; + padding-left: 20px; + margin-bottom: 0; + font-weight: normal; + cursor: pointer; +} +.radio input[type="radio"], +.radio-inline input[type="radio"], +.checkbox input[type="checkbox"], +.checkbox-inline input[type="checkbox"] { + position: absolute; + margin-top: 4px \9; + margin-left: -20px; +} +.radio + .radio, +.checkbox + .checkbox { + margin-top: -5px; +} +.radio-inline, +.checkbox-inline { + position: relative; + display: inline-block; + padding-left: 20px; + margin-bottom: 0; + font-weight: normal; + vertical-align: middle; + cursor: pointer; +} +.radio-inline + .radio-inline, +.checkbox-inline + .checkbox-inline { + margin-top: 0; + margin-left: 10px; +} +input[type="radio"][disabled], +input[type="checkbox"][disabled], +input[type="radio"].disabled, +input[type="checkbox"].disabled, +fieldset[disabled] input[type="radio"], +fieldset[disabled] input[type="checkbox"] { + cursor: not-allowed; +} +.radio-inline.disabled, +.checkbox-inline.disabled, +fieldset[disabled] .radio-inline, +fieldset[disabled] .checkbox-inline { + cursor: not-allowed; +} +.radio.disabled label, +.checkbox.disabled label, +fieldset[disabled] .radio label, +fieldset[disabled] .checkbox label { + cursor: not-allowed; +} +.form-control-static { + min-height: 34px; + padding-top: 7px; + padding-bottom: 7px; + margin-bottom: 0; +} +.form-control-static.input-lg, +.form-control-static.input-sm { + padding-right: 0; + padding-left: 0; +} +.input-sm { + height: 30px; + padding: 5px 10px; + font-size: 12px; + line-height: 1.5; + border-radius: 3px; +} +select.input-sm { + height: 30px; + line-height: 30px; +} +textarea.input-sm, +select[multiple].input-sm { + height: auto; +} +.form-group-sm .form-control { + height: 30px; + padding: 5px 10px; + font-size: 12px; + line-height: 1.5; + border-radius: 3px; +} +select.form-group-sm .form-control { + height: 30px; + line-height: 30px; +} +textarea.form-group-sm .form-control, +select[multiple].form-group-sm .form-control { + height: auto; +} +.form-group-sm .form-control-static { + height: 30px; + min-height: 32px; + padding: 5px 10px; + font-size: 12px; + line-height: 1.5; +} +.input-lg { + height: 46px; + padding: 10px 16px; + font-size: 18px; + line-height: 1.3333333; + border-radius: 6px; +} +select.input-lg { + height: 46px; + line-height: 46px; +} +textarea.input-lg, +select[multiple].input-lg { + height: auto; +} +.form-group-lg .form-control { + height: 46px; + padding: 10px 16px; + font-size: 18px; + line-height: 1.3333333; + border-radius: 6px; +} +select.form-group-lg .form-control { + height: 46px; + line-height: 46px; +} +textarea.form-group-lg .form-control, +select[multiple].form-group-lg .form-control { + height: auto; +} +.form-group-lg .form-control-static { + height: 46px; + min-height: 38px; + padding: 10px 16px; + font-size: 18px; + line-height: 1.3333333; +} +.has-feedback { + position: relative; +} +.has-feedback .form-control { + padding-right: 42.5px; +} +.form-control-feedback { + position: absolute; + top: 0; + right: 0; + z-index: 2; + display: block; + width: 34px; + height: 34px; + line-height: 34px; + text-align: center; + pointer-events: none; +} +.input-lg + .form-control-feedback { + width: 46px; + height: 46px; + line-height: 46px; +} +.input-sm + .form-control-feedback { + width: 30px; + height: 30px; + line-height: 30px; +} +.has-success .help-block, +.has-success .control-label, +.has-success .radio, +.has-success .checkbox, +.has-success .radio-inline, +.has-success .checkbox-inline, +.has-success.radio label, +.has-success.checkbox label, +.has-success.radio-inline label, +.has-success.checkbox-inline label { + color: #3c763d; +} +.has-success .form-control { + border-color: #3c763d; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); +} +.has-success .form-control:focus { + border-color: #2b542c; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #67b168; + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #67b168; +} +.has-success .input-group-addon { + color: #3c763d; + background-color: #dff0d8; + border-color: #3c763d; +} +.has-success .form-control-feedback { + color: #3c763d; +} +.has-warning .help-block, +.has-warning .control-label, +.has-warning .radio, +.has-warning .checkbox, +.has-warning .radio-inline, +.has-warning .checkbox-inline, +.has-warning.radio label, +.has-warning.checkbox label, +.has-warning.radio-inline label, +.has-warning.checkbox-inline label { + color: #8a6d3b; +} +.has-warning .form-control { + border-color: #8a6d3b; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); +} +.has-warning .form-control:focus { + border-color: #66512c; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #c0a16b; + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #c0a16b; +} +.has-warning .input-group-addon { + color: #8a6d3b; + background-color: #fcf8e3; + border-color: #8a6d3b; +} +.has-warning .form-control-feedback { + color: #8a6d3b; +} +.has-error .help-block, +.has-error .control-label, +.has-error .radio, +.has-error .checkbox, +.has-error .radio-inline, +.has-error .checkbox-inline, +.has-error.radio label, +.has-error.checkbox label, +.has-error.radio-inline label, +.has-error.checkbox-inline label { + color: #a94442; +} +.has-error .form-control { + border-color: #a94442; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075); +} +.has-error .form-control:focus { + border-color: #843534; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #ce8483; + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .075), 0 0 6px #ce8483; +} +.has-error .input-group-addon { + color: #a94442; + background-color: #f2dede; + border-color: #a94442; +} +.has-error .form-control-feedback { + color: #a94442; +} +.has-feedback label ~ .form-control-feedback { + top: 25px; +} +.has-feedback label.sr-only ~ .form-control-feedback { + top: 0; +} +.help-block { + display: block; + margin-top: 5px; + margin-bottom: 10px; + color: #737373; +} +@media (min-width: 768px) { + .form-inline .form-group { + display: inline-block; + margin-bottom: 0; + vertical-align: middle; + } + .form-inline .form-control { + display: inline-block; + width: auto; + vertical-align: middle; + } + .form-inline .form-control-static { + display: inline-block; + } + .form-inline .input-group { + display: inline-table; + vertical-align: middle; + } + .form-inline .input-group .input-group-addon, + .form-inline .input-group .input-group-btn, + .form-inline .input-group .form-control { + width: auto; + } + .form-inline .input-group > .form-control { + width: 100%; + } + .form-inline .control-label { + margin-bottom: 0; + vertical-align: middle; + } + .form-inline .radio, + .form-inline .checkbox { + display: inline-block; + margin-top: 0; + margin-bottom: 0; + vertical-align: middle; + } + .form-inline .radio label, + .form-inline .checkbox label { + padding-left: 0; + } + .form-inline .radio input[type="radio"], + .form-inline .checkbox input[type="checkbox"] { + position: relative; + margin-left: 0; + } + .form-inline .has-feedback .form-control-feedback { + top: 0; + } +} +.form-horizontal .radio, +.form-horizontal .checkbox, +.form-horizontal .radio-inline, +.form-horizontal .checkbox-inline { + padding-top: 7px; + margin-top: 0; + margin-bottom: 0; +} +.form-horizontal .radio, +.form-horizontal .checkbox { + min-height: 27px; +} +.form-horizontal .form-group { + margin-right: -15px; + margin-left: -15px; +} +@media (min-width: 768px) { + .form-horizontal .control-label { + padding-top: 7px; + margin-bottom: 0; + text-align: right; + } +} +.form-horizontal .has-feedback .form-control-feedback { + right: 15px; +} +@media (min-width: 768px) { + .form-horizontal .form-group-lg .control-label { + padding-top: 14.333333px; + } +} +@media (min-width: 768px) { + .form-horizontal .form-group-sm .control-label { + padding-top: 6px; + } +} +.btn { + display: inline-block; + padding: 6px 12px; + margin-bottom: 0; + font-size: 14px; + font-weight: normal; + line-height: 1.42857143; + text-align: center; + white-space: nowrap; + vertical-align: middle; + -ms-touch-action: manipulation; + touch-action: manipulation; + cursor: pointer; + -webkit-user-select: none; + -moz-user-select: none; + -ms-user-select: none; + user-select: none; + background-image: none; + border: 1px solid transparent; + border-radius: 4px; +} +.btn:focus, +.btn:active:focus, +.btn.active:focus, +.btn.focus, +.btn:active.focus, +.btn.active.focus { + outline: thin dotted; + outline: 5px auto -webkit-focus-ring-color; + outline-offset: -2px; +} +.btn:hover, +.btn:focus, +.btn.focus { + color: #333; + text-decoration: none; +} +.btn:active, +.btn.active { + background-image: none; + outline: 0; + -webkit-box-shadow: inset 0 3px 5px rgba(0, 0, 0, .125); + box-shadow: inset 0 3px 5px rgba(0, 0, 0, .125); +} +.btn.disabled, +.btn[disabled], +fieldset[disabled] .btn { + pointer-events: none; + cursor: not-allowed; + filter: alpha(opacity=65); + -webkit-box-shadow: none; + box-shadow: none; + opacity: .65; +} +.btn-default { + color: #333; + background-color: #fff; + border-color: #ccc; +} +.btn-default:hover, +.btn-default:focus, +.btn-default.focus, +.btn-default:active, +.btn-default.active, +.open > .dropdown-toggle.btn-default { + color: #333; + background-color: #e6e6e6; + border-color: #adadad; +} +.btn-default:active, +.btn-default.active, +.open > .dropdown-toggle.btn-default { + background-image: none; +} +.btn-default.disabled, +.btn-default[disabled], +fieldset[disabled] .btn-default, +.btn-default.disabled:hover, +.btn-default[disabled]:hover, +fieldset[disabled] .btn-default:hover, +.btn-default.disabled:focus, +.btn-default[disabled]:focus, +fieldset[disabled] .btn-default:focus, +.btn-default.disabled.focus, +.btn-default[disabled].focus, +fieldset[disabled] .btn-default.focus, +.btn-default.disabled:active, +.btn-default[disabled]:active, +fieldset[disabled] .btn-default:active, +.btn-default.disabled.active, +.btn-default[disabled].active, +fieldset[disabled] .btn-default.active { + background-color: #fff; + border-color: #ccc; +} +.btn-default .badge { + color: #fff; + background-color: #333; +} +.btn-primary { + color: #fff; + background-color: #337ab7; + border-color: #2e6da4; +} +.btn-primary:hover, +.btn-primary:focus, +.btn-primary.focus, +.btn-primary:active, +.btn-primary.active, +.open > .dropdown-toggle.btn-primary { + color: #fff; + background-color: #286090; + border-color: #204d74; +} +.btn-primary:active, +.btn-primary.active, +.open > .dropdown-toggle.btn-primary { + background-image: none; +} +.btn-primary.disabled, +.btn-primary[disabled], +fieldset[disabled] .btn-primary, +.btn-primary.disabled:hover, +.btn-primary[disabled]:hover, +fieldset[disabled] .btn-primary:hover, +.btn-primary.disabled:focus, +.btn-primary[disabled]:focus, +fieldset[disabled] .btn-primary:focus, +.btn-primary.disabled.focus, +.btn-primary[disabled].focus, +fieldset[disabled] .btn-primary.focus, +.btn-primary.disabled:active, +.btn-primary[disabled]:active, +fieldset[disabled] .btn-primary:active, +.btn-primary.disabled.active, +.btn-primary[disabled].active, +fieldset[disabled] .btn-primary.active { + background-color: #337ab7; + border-color: #2e6da4; +} +.btn-primary .badge { + color: #337ab7; + background-color: #fff; +} +.btn-success { + color: #fff; + background-color: #5cb85c; + border-color: #4cae4c; +} +.btn-success:hover, +.btn-success:focus, +.btn-success.focus, +.btn-success:active, +.btn-success.active, +.open > .dropdown-toggle.btn-success { + color: #fff; + background-color: #449d44; + border-color: #398439; +} +.btn-success:active, +.btn-success.active, +.open > .dropdown-toggle.btn-success { + background-image: none; +} +.btn-success.disabled, +.btn-success[disabled], +fieldset[disabled] .btn-success, +.btn-success.disabled:hover, +.btn-success[disabled]:hover, +fieldset[disabled] .btn-success:hover, +.btn-success.disabled:focus, +.btn-success[disabled]:focus, +fieldset[disabled] .btn-success:focus, +.btn-success.disabled.focus, +.btn-success[disabled].focus, +fieldset[disabled] .btn-success.focus, +.btn-success.disabled:active, +.btn-success[disabled]:active, +fieldset[disabled] .btn-success:active, +.btn-success.disabled.active, +.btn-success[disabled].active, +fieldset[disabled] .btn-success.active { + background-color: #5cb85c; + border-color: #4cae4c; +} +.btn-success .badge { + color: #5cb85c; + background-color: #fff; +} +.btn-info { + color: #fff; + background-color: #5bc0de; + border-color: #46b8da; +} +.btn-info:hover, +.btn-info:focus, +.btn-info.focus, +.btn-info:active, +.btn-info.active, +.open > .dropdown-toggle.btn-info { + color: #fff; + background-color: #31b0d5; + border-color: #269abc; +} +.btn-info:active, +.btn-info.active, +.open > .dropdown-toggle.btn-info { + background-image: none; +} +.btn-info.disabled, +.btn-info[disabled], +fieldset[disabled] .btn-info, +.btn-info.disabled:hover, +.btn-info[disabled]:hover, +fieldset[disabled] .btn-info:hover, +.btn-info.disabled:focus, +.btn-info[disabled]:focus, +fieldset[disabled] .btn-info:focus, +.btn-info.disabled.focus, +.btn-info[disabled].focus, +fieldset[disabled] .btn-info.focus, +.btn-info.disabled:active, +.btn-info[disabled]:active, +fieldset[disabled] .btn-info:active, +.btn-info.disabled.active, +.btn-info[disabled].active, +fieldset[disabled] .btn-info.active { + background-color: #5bc0de; + border-color: #46b8da; +} +.btn-info .badge { + color: #5bc0de; + background-color: #fff; +} +.btn-warning { + color: #fff; + background-color: #f0ad4e; + border-color: #eea236; +} +.btn-warning:hover, +.btn-warning:focus, +.btn-warning.focus, +.btn-warning:active, +.btn-warning.active, +.open > .dropdown-toggle.btn-warning { + color: #fff; + background-color: #ec971f; + border-color: #d58512; +} +.btn-warning:active, +.btn-warning.active, +.open > .dropdown-toggle.btn-warning { + background-image: none; +} +.btn-warning.disabled, +.btn-warning[disabled], +fieldset[disabled] .btn-warning, +.btn-warning.disabled:hover, +.btn-warning[disabled]:hover, +fieldset[disabled] .btn-warning:hover, +.btn-warning.disabled:focus, +.btn-warning[disabled]:focus, +fieldset[disabled] .btn-warning:focus, +.btn-warning.disabled.focus, +.btn-warning[disabled].focus, +fieldset[disabled] .btn-warning.focus, +.btn-warning.disabled:active, +.btn-warning[disabled]:active, +fieldset[disabled] .btn-warning:active, +.btn-warning.disabled.active, +.btn-warning[disabled].active, +fieldset[disabled] .btn-warning.active { + background-color: #f0ad4e; + border-color: #eea236; +} +.btn-warning .badge { + color: #f0ad4e; + background-color: #fff; +} +.btn-danger { + color: #fff; + background-color: #d9534f; + border-color: #d43f3a; +} +.btn-danger:hover, +.btn-danger:focus, +.btn-danger.focus, +.btn-danger:active, +.btn-danger.active, +.open > .dropdown-toggle.btn-danger { + color: #fff; + background-color: #c9302c; + border-color: #ac2925; +} +.btn-danger:active, +.btn-danger.active, +.open > .dropdown-toggle.btn-danger { + background-image: none; +} +.btn-danger.disabled, +.btn-danger[disabled], +fieldset[disabled] .btn-danger, +.btn-danger.disabled:hover, +.btn-danger[disabled]:hover, +fieldset[disabled] .btn-danger:hover, +.btn-danger.disabled:focus, +.btn-danger[disabled]:focus, +fieldset[disabled] .btn-danger:focus, +.btn-danger.disabled.focus, +.btn-danger[disabled].focus, +fieldset[disabled] .btn-danger.focus, +.btn-danger.disabled:active, +.btn-danger[disabled]:active, +fieldset[disabled] .btn-danger:active, +.btn-danger.disabled.active, +.btn-danger[disabled].active, +fieldset[disabled] .btn-danger.active { + background-color: #d9534f; + border-color: #d43f3a; +} +.btn-danger .badge { + color: #d9534f; + background-color: #fff; +} +.btn-link { + font-weight: normal; + color: #337ab7; + border-radius: 0; +} +.btn-link, +.btn-link:active, +.btn-link.active, +.btn-link[disabled], +fieldset[disabled] .btn-link { + background-color: transparent; + -webkit-box-shadow: none; + box-shadow: none; +} +.btn-link, +.btn-link:hover, +.btn-link:focus, +.btn-link:active { + border-color: transparent; +} +.btn-link:hover, +.btn-link:focus { + color: #23527c; + text-decoration: underline; + background-color: transparent; +} +.btn-link[disabled]:hover, +fieldset[disabled] .btn-link:hover, +.btn-link[disabled]:focus, +fieldset[disabled] .btn-link:focus { + color: #777; + text-decoration: none; +} +.btn-lg, +.btn-group-lg > .btn { + padding: 10px 16px; + font-size: 18px; + line-height: 1.3333333; + border-radius: 6px; +} +.btn-sm, +.btn-group-sm > .btn { + padding: 5px 10px; + font-size: 12px; + line-height: 1.5; + border-radius: 3px; +} +.btn-xs, +.btn-group-xs > .btn { + padding: 1px 5px; + font-size: 12px; + line-height: 1.5; + border-radius: 3px; +} +.btn-block { + display: block; + width: 100%; +} +.btn-block + .btn-block { + margin-top: 5px; +} +input[type="submit"].btn-block, +input[type="reset"].btn-block, +input[type="button"].btn-block { + width: 100%; +} +.fade { + opacity: 0; + -webkit-transition: opacity .15s linear; + -o-transition: opacity .15s linear; + transition: opacity .15s linear; +} +.fade.in { + opacity: 1; +} +.collapse { + display: none; +} +.collapse.in { + display: block; +} +tr.collapse.in { + display: table-row; +} +tbody.collapse.in { + display: table-row-group; +} +.collapsing { + position: relative; + height: 0; + overflow: hidden; + -webkit-transition-timing-function: ease; + -o-transition-timing-function: ease; + transition-timing-function: ease; + -webkit-transition-duration: .35s; + -o-transition-duration: .35s; + transition-duration: .35s; + -webkit-transition-property: height, visibility; + -o-transition-property: height, visibility; + transition-property: height, visibility; +} +.caret { + display: inline-block; + width: 0; + height: 0; + margin-left: 2px; + vertical-align: middle; + border-top: 4px dashed; + border-right: 4px solid transparent; + border-left: 4px solid transparent; +} +.dropup, +.dropdown { + position: relative; +} +.dropdown-toggle:focus { + outline: 0; +} +.dropdown-menu { + position: absolute; + top: 100%; + left: 0; + z-index: 1000; + display: none; + float: left; + min-width: 160px; + padding: 5px 0; + margin: 2px 0 0; + font-size: 14px; + text-align: left; + list-style: none; + background-color: #fff; + -webkit-background-clip: padding-box; + background-clip: padding-box; + border: 1px solid #ccc; + border: 1px solid rgba(0, 0, 0, .15); + border-radius: 4px; + -webkit-box-shadow: 0 6px 12px rgba(0, 0, 0, .175); + box-shadow: 0 6px 12px rgba(0, 0, 0, .175); +} +.dropdown-menu.pull-right { + right: 0; + left: auto; +} +.dropdown-menu .divider { + height: 1px; + margin: 9px 0; + overflow: hidden; + background-color: #e5e5e5; +} +.dropdown-menu > li > a { + display: block; + padding: 3px 20px; + clear: both; + font-weight: normal; + line-height: 1.42857143; + color: #333; + white-space: nowrap; +} +.dropdown-menu > li > a:hover, +.dropdown-menu > li > a:focus { + color: #262626; + text-decoration: none; + background-color: #f5f5f5; +} +.dropdown-menu > .active > a, +.dropdown-menu > .active > a:hover, +.dropdown-menu > .active > a:focus { + color: #fff; + text-decoration: none; + background-color: #337ab7; + outline: 0; +} +.dropdown-menu > .disabled > a, +.dropdown-menu > .disabled > a:hover, +.dropdown-menu > .disabled > a:focus { + color: #777; +} +.dropdown-menu > .disabled > a:hover, +.dropdown-menu > .disabled > a:focus { + text-decoration: none; + cursor: not-allowed; + background-color: transparent; + background-image: none; + filter: progid:DXImageTransform.Microsoft.gradient(enabled = false); +} +.open > .dropdown-menu { + display: block; +} +.open > a { + outline: 0; +} +.dropdown-menu-right { + right: 0; + left: auto; +} +.dropdown-menu-left { + right: auto; + left: 0; +} +.dropdown-header { + display: block; + padding: 3px 20px; + font-size: 12px; + line-height: 1.42857143; + color: #777; + white-space: nowrap; +} +.dropdown-backdrop { + position: fixed; + top: 0; + right: 0; + bottom: 0; + left: 0; + z-index: 990; +} +.pull-right > .dropdown-menu { + right: 0; + left: auto; +} +.dropup .caret, +.navbar-fixed-bottom .dropdown .caret { + content: ""; + border-top: 0; + border-bottom: 4px solid; +} +.dropup .dropdown-menu, +.navbar-fixed-bottom .dropdown .dropdown-menu { + top: auto; + bottom: 100%; + margin-bottom: 2px; +} +@media (min-width: 768px) { + .navbar-right .dropdown-menu { + right: 0; + left: auto; + } + .navbar-right .dropdown-menu-left { + right: auto; + left: 0; + } +} +.btn-group, +.btn-group-vertical { + position: relative; + display: inline-block; + vertical-align: middle; +} +.btn-group > .btn, +.btn-group-vertical > .btn { + position: relative; + float: left; +} +.btn-group > .btn:hover, +.btn-group-vertical > .btn:hover, +.btn-group > .btn:focus, +.btn-group-vertical > .btn:focus, +.btn-group > .btn:active, +.btn-group-vertical > .btn:active, +.btn-group > .btn.active, +.btn-group-vertical > .btn.active { + z-index: 2; +} +.btn-group .btn + .btn, +.btn-group .btn + .btn-group, +.btn-group .btn-group + .btn, +.btn-group .btn-group + .btn-group { + margin-left: -1px; +} +.btn-toolbar { + margin-left: -5px; +} +.btn-toolbar .btn-group, +.btn-toolbar .input-group { + float: left; +} +.btn-toolbar > .btn, +.btn-toolbar > .btn-group, +.btn-toolbar > .input-group { + margin-left: 5px; +} +.btn-group > .btn:not(:first-child):not(:last-child):not(.dropdown-toggle) { + border-radius: 0; +} +.btn-group > .btn:first-child { + margin-left: 0; +} +.btn-group > .btn:first-child:not(:last-child):not(.dropdown-toggle) { + border-top-right-radius: 0; + border-bottom-right-radius: 0; +} +.btn-group > .btn:last-child:not(:first-child), +.btn-group > .dropdown-toggle:not(:first-child) { + border-top-left-radius: 0; + border-bottom-left-radius: 0; +} +.btn-group > .btn-group { + float: left; +} +.btn-group > .btn-group:not(:first-child):not(:last-child) > .btn { + border-radius: 0; +} +.btn-group > .btn-group:first-child:not(:last-child) > .btn:last-child, +.btn-group > .btn-group:first-child:not(:last-child) > .dropdown-toggle { + border-top-right-radius: 0; + border-bottom-right-radius: 0; +} +.btn-group > .btn-group:last-child:not(:first-child) > .btn:first-child { + border-top-left-radius: 0; + border-bottom-left-radius: 0; +} +.btn-group .dropdown-toggle:active, +.btn-group.open .dropdown-toggle { + outline: 0; +} +.btn-group > .btn + .dropdown-toggle { + padding-right: 8px; + padding-left: 8px; +} +.btn-group > .btn-lg + .dropdown-toggle { + padding-right: 12px; + padding-left: 12px; +} +.btn-group.open .dropdown-toggle { + -webkit-box-shadow: inset 0 3px 5px rgba(0, 0, 0, .125); + box-shadow: inset 0 3px 5px rgba(0, 0, 0, .125); +} +.btn-group.open .dropdown-toggle.btn-link { + -webkit-box-shadow: none; + box-shadow: none; +} +.btn .caret { + margin-left: 0; +} +.btn-lg .caret { + border-width: 5px 5px 0; + border-bottom-width: 0; +} +.dropup .btn-lg .caret { + border-width: 0 5px 5px; +} +.btn-group-vertical > .btn, +.btn-group-vertical > .btn-group, +.btn-group-vertical > .btn-group > .btn { + display: block; + float: none; + width: 100%; + max-width: 100%; +} +.btn-group-vertical > .btn-group > .btn { + float: none; +} +.btn-group-vertical > .btn + .btn, +.btn-group-vertical > .btn + .btn-group, +.btn-group-vertical > .btn-group + .btn, +.btn-group-vertical > .btn-group + .btn-group { + margin-top: -1px; + margin-left: 0; +} +.btn-group-vertical > .btn:not(:first-child):not(:last-child) { + border-radius: 0; +} +.btn-group-vertical > .btn:first-child:not(:last-child) { + border-top-right-radius: 4px; + border-bottom-right-radius: 0; + border-bottom-left-radius: 0; +} +.btn-group-vertical > .btn:last-child:not(:first-child) { + border-top-left-radius: 0; + border-top-right-radius: 0; + border-bottom-left-radius: 4px; +} +.btn-group-vertical > .btn-group:not(:first-child):not(:last-child) > .btn { + border-radius: 0; +} +.btn-group-vertical > .btn-group:first-child:not(:last-child) > .btn:last-child, +.btn-group-vertical > .btn-group:first-child:not(:last-child) > .dropdown-toggle { + border-bottom-right-radius: 0; + border-bottom-left-radius: 0; +} +.btn-group-vertical > .btn-group:last-child:not(:first-child) > .btn:first-child { + border-top-left-radius: 0; + border-top-right-radius: 0; +} +.btn-group-justified { + display: table; + width: 100%; + table-layout: fixed; + border-collapse: separate; +} +.btn-group-justified > .btn, +.btn-group-justified > .btn-group { + display: table-cell; + float: none; + width: 1%; +} +.btn-group-justified > .btn-group .btn { + width: 100%; +} +.btn-group-justified > .btn-group .dropdown-menu { + left: auto; +} +[data-toggle="buttons"] > .btn input[type="radio"], +[data-toggle="buttons"] > .btn-group > .btn input[type="radio"], +[data-toggle="buttons"] > .btn input[type="checkbox"], +[data-toggle="buttons"] > .btn-group > .btn input[type="checkbox"] { + position: absolute; + clip: rect(0, 0, 0, 0); + pointer-events: none; +} +.input-group { + position: relative; + display: table; + border-collapse: separate; +} +.input-group[class*="col-"] { + float: none; + padding-right: 0; + padding-left: 0; +} +.input-group .form-control { + position: relative; + z-index: 2; + float: left; + width: 100%; + margin-bottom: 0; +} +.input-group-lg > .form-control, +.input-group-lg > .input-group-addon, +.input-group-lg > .input-group-btn > .btn { + height: 46px; + padding: 10px 16px; + font-size: 18px; + line-height: 1.3333333; + border-radius: 6px; +} +select.input-group-lg > .form-control, +select.input-group-lg > .input-group-addon, +select.input-group-lg > .input-group-btn > .btn { + height: 46px; + line-height: 46px; +} +textarea.input-group-lg > .form-control, +textarea.input-group-lg > .input-group-addon, +textarea.input-group-lg > .input-group-btn > .btn, +select[multiple].input-group-lg > .form-control, +select[multiple].input-group-lg > .input-group-addon, +select[multiple].input-group-lg > .input-group-btn > .btn { + height: auto; +} +.input-group-sm > .form-control, +.input-group-sm > .input-group-addon, +.input-group-sm > .input-group-btn > .btn { + height: 30px; + padding: 5px 10px; + font-size: 12px; + line-height: 1.5; + border-radius: 3px; +} +select.input-group-sm > .form-control, +select.input-group-sm > .input-group-addon, +select.input-group-sm > .input-group-btn > .btn { + height: 30px; + line-height: 30px; +} +textarea.input-group-sm > .form-control, +textarea.input-group-sm > .input-group-addon, +textarea.input-group-sm > .input-group-btn > .btn, +select[multiple].input-group-sm > .form-control, +select[multiple].input-group-sm > .input-group-addon, +select[multiple].input-group-sm > .input-group-btn > .btn { + height: auto; +} +.input-group-addon, +.input-group-btn, +.input-group .form-control { + display: table-cell; +} +.input-group-addon:not(:first-child):not(:last-child), +.input-group-btn:not(:first-child):not(:last-child), +.input-group .form-control:not(:first-child):not(:last-child) { + border-radius: 0; +} +.input-group-addon, +.input-group-btn { + width: 1%; + white-space: nowrap; + vertical-align: middle; +} +.input-group-addon { + padding: 6px 12px; + font-size: 14px; + font-weight: normal; + line-height: 1; + color: #555; + text-align: center; + background-color: #eee; + border: 1px solid #ccc; + border-radius: 4px; +} +.input-group-addon.input-sm { + padding: 5px 10px; + font-size: 12px; + border-radius: 3px; +} +.input-group-addon.input-lg { + padding: 10px 16px; + font-size: 18px; + border-radius: 6px; +} +.input-group-addon input[type="radio"], +.input-group-addon input[type="checkbox"] { + margin-top: 0; +} +.input-group .form-control:first-child, +.input-group-addon:first-child, +.input-group-btn:first-child > .btn, +.input-group-btn:first-child > .btn-group > .btn, +.input-group-btn:first-child > .dropdown-toggle, +.input-group-btn:last-child > .btn:not(:last-child):not(.dropdown-toggle), +.input-group-btn:last-child > .btn-group:not(:last-child) > .btn { + border-top-right-radius: 0; + border-bottom-right-radius: 0; +} +.input-group-addon:first-child { + border-right: 0; +} +.input-group .form-control:last-child, +.input-group-addon:last-child, +.input-group-btn:last-child > .btn, +.input-group-btn:last-child > .btn-group > .btn, +.input-group-btn:last-child > .dropdown-toggle, +.input-group-btn:first-child > .btn:not(:first-child), +.input-group-btn:first-child > .btn-group:not(:first-child) > .btn { + border-top-left-radius: 0; + border-bottom-left-radius: 0; +} +.input-group-addon:last-child { + border-left: 0; +} +.input-group-btn { + position: relative; + font-size: 0; + white-space: nowrap; +} +.input-group-btn > .btn { + position: relative; +} +.input-group-btn > .btn + .btn { + margin-left: -1px; +} +.input-group-btn > .btn:hover, +.input-group-btn > .btn:focus, +.input-group-btn > .btn:active { + z-index: 2; +} +.input-group-btn:first-child > .btn, +.input-group-btn:first-child > .btn-group { + margin-right: -1px; +} +.input-group-btn:last-child > .btn, +.input-group-btn:last-child > .btn-group { + margin-left: -1px; +} +.nav { + padding-left: 0; + margin-bottom: 0; + list-style: none; +} +.nav > li { + position: relative; + display: block; +} +.nav > li > a { + position: relative; + display: block; + padding: 10px 15px; +} +.nav > li > a:hover, +.nav > li > a:focus { + text-decoration: none; + background-color: #eee; +} +.nav > li.disabled > a { + color: #777; +} +.nav > li.disabled > a:hover, +.nav > li.disabled > a:focus { + color: #777; + text-decoration: none; + cursor: not-allowed; + background-color: transparent; +} +.nav .open > a, +.nav .open > a:hover, +.nav .open > a:focus { + background-color: #eee; + border-color: #337ab7; +} +.nav .nav-divider { + height: 1px; + margin: 9px 0; + overflow: hidden; + background-color: #e5e5e5; +} +.nav > li > a > img { + max-width: none; +} +.nav-tabs { + border-bottom: 1px solid #ddd; +} +.nav-tabs > li { + float: left; + margin-bottom: -1px; +} +.nav-tabs > li > a { + margin-right: 2px; + line-height: 1.42857143; + border: 1px solid transparent; + border-radius: 4px 4px 0 0; +} +.nav-tabs > li > a:hover { + border-color: #eee #eee #ddd; +} +.nav-tabs > li.active > a, +.nav-tabs > li.active > a:hover, +.nav-tabs > li.active > a:focus { + color: #555; + cursor: default; + background-color: #fff; + border: 1px solid #ddd; + border-bottom-color: transparent; +} +.nav-tabs.nav-justified { + width: 100%; + border-bottom: 0; +} +.nav-tabs.nav-justified > li { + float: none; +} +.nav-tabs.nav-justified > li > a { + margin-bottom: 5px; + text-align: center; +} +.nav-tabs.nav-justified > .dropdown .dropdown-menu { + top: auto; + left: auto; +} +@media (min-width: 768px) { + .nav-tabs.nav-justified > li { + display: table-cell; + width: 1%; + } + .nav-tabs.nav-justified > li > a { + margin-bottom: 0; + } +} +.nav-tabs.nav-justified > li > a { + margin-right: 0; + border-radius: 4px; +} +.nav-tabs.nav-justified > .active > a, +.nav-tabs.nav-justified > .active > a:hover, +.nav-tabs.nav-justified > .active > a:focus { + border: 1px solid #ddd; +} +@media (min-width: 768px) { + .nav-tabs.nav-justified > li > a { + border-bottom: 1px solid #ddd; + border-radius: 4px 4px 0 0; + } + .nav-tabs.nav-justified > .active > a, + .nav-tabs.nav-justified > .active > a:hover, + .nav-tabs.nav-justified > .active > a:focus { + border-bottom-color: #fff; + } +} +.nav-pills > li { + float: left; +} +.nav-pills > li > a { + border-radius: 4px; +} +.nav-pills > li + li { + margin-left: 2px; +} +.nav-pills > li.active > a, +.nav-pills > li.active > a:hover, +.nav-pills > li.active > a:focus { + color: #fff; + background-color: #337ab7; +} +.nav-stacked > li { + float: none; +} +.nav-stacked > li + li { + margin-top: 2px; + margin-left: 0; +} +.nav-justified { + width: 100%; +} +.nav-justified > li { + float: none; +} +.nav-justified > li > a { + margin-bottom: 5px; + text-align: center; +} +.nav-justified > .dropdown .dropdown-menu { + top: auto; + left: auto; +} +@media (min-width: 768px) { + .nav-justified > li { + display: table-cell; + width: 1%; + } + .nav-justified > li > a { + margin-bottom: 0; + } +} +.nav-tabs-justified { + border-bottom: 0; +} +.nav-tabs-justified > li > a { + margin-right: 0; + border-radius: 4px; +} +.nav-tabs-justified > .active > a, +.nav-tabs-justified > .active > a:hover, +.nav-tabs-justified > .active > a:focus { + border: 1px solid #ddd; +} +@media (min-width: 768px) { + .nav-tabs-justified > li > a { + border-bottom: 1px solid #ddd; + border-radius: 4px 4px 0 0; + } + .nav-tabs-justified > .active > a, + .nav-tabs-justified > .active > a:hover, + .nav-tabs-justified > .active > a:focus { + border-bottom-color: #fff; + } +} +.tab-content > .tab-pane { + display: none; +} +.tab-content > .active { + display: block; +} +.nav-tabs .dropdown-menu { + margin-top: -1px; + border-top-left-radius: 0; + border-top-right-radius: 0; +} +.navbar { + position: relative; + min-height: 50px; + margin-bottom: 20px; + border: 1px solid transparent; +} +@media (min-width: 768px) { + .navbar { + border-radius: 4px; + } +} +@media (min-width: 768px) { + .navbar-header { + float: left; + } +} +.navbar-collapse { + padding-right: 15px; + padding-left: 15px; + overflow-x: visible; + -webkit-overflow-scrolling: touch; + border-top: 1px solid transparent; + -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, .1); + box-shadow: inset 0 1px 0 rgba(255, 255, 255, .1); +} +.navbar-collapse.in { + overflow-y: auto; +} +@media (min-width: 768px) { + .navbar-collapse { + width: auto; + border-top: 0; + -webkit-box-shadow: none; + box-shadow: none; + } + .navbar-collapse.collapse { + display: block !important; + height: auto !important; + padding-bottom: 0; + overflow: visible !important; + } + .navbar-collapse.in { + overflow-y: visible; + } + .navbar-fixed-top .navbar-collapse, + .navbar-static-top .navbar-collapse, + .navbar-fixed-bottom .navbar-collapse { + padding-right: 0; + padding-left: 0; + } +} +.navbar-fixed-top .navbar-collapse, +.navbar-fixed-bottom .navbar-collapse { + max-height: 340px; +} +@media (max-device-width: 480px) and (orientation: landscape) { + .navbar-fixed-top .navbar-collapse, + .navbar-fixed-bottom .navbar-collapse { + max-height: 200px; + } +} +.container > .navbar-header, +.container-fluid > .navbar-header, +.container > .navbar-collapse, +.container-fluid > .navbar-collapse { + margin-right: -15px; + margin-left: -15px; +} +@media (min-width: 768px) { + .container > .navbar-header, + .container-fluid > .navbar-header, + .container > .navbar-collapse, + .container-fluid > .navbar-collapse { + margin-right: 0; + margin-left: 0; + } +} +.navbar-static-top { + z-index: 1000; + border-width: 0 0 1px; +} +@media (min-width: 768px) { + .navbar-static-top { + border-radius: 0; + } +} +.navbar-fixed-top, +.navbar-fixed-bottom { + position: fixed; + right: 0; + left: 0; + z-index: 1030; +} +@media (min-width: 768px) { + .navbar-fixed-top, + .navbar-fixed-bottom { + border-radius: 0; + } +} +.navbar-fixed-top { + top: 0; + border-width: 0 0 1px; +} +.navbar-fixed-bottom { + bottom: 0; + margin-bottom: 0; + border-width: 1px 0 0; +} +.navbar-brand { + float: left; + height: 50px; + padding: 15px 15px; + font-size: 18px; + line-height: 20px; +} +.navbar-brand:hover, +.navbar-brand:focus { + text-decoration: none; +} +.navbar-brand > img { + display: block; +} +@media (min-width: 768px) { + .navbar > .container .navbar-brand, + .navbar > .container-fluid .navbar-brand { + margin-left: -15px; + } +} +.navbar-toggle { + position: relative; + float: right; + padding: 9px 10px; + margin-top: 8px; + margin-right: 15px; + margin-bottom: 8px; + background-color: transparent; + background-image: none; + border: 1px solid transparent; + border-radius: 4px; +} +.navbar-toggle:focus { + outline: 0; +} +.navbar-toggle .icon-bar { + display: block; + width: 22px; + height: 2px; + border-radius: 1px; +} +.navbar-toggle .icon-bar + .icon-bar { + margin-top: 4px; +} +@media (min-width: 768px) { + .navbar-toggle { + display: none; + } +} +.navbar-nav { + margin: 7.5px -15px; +} +.navbar-nav > li > a { + padding-top: 10px; + padding-bottom: 10px; + line-height: 20px; +} +@media (max-width: 767px) { + .navbar-nav .open .dropdown-menu { + position: static; + float: none; + width: auto; + margin-top: 0; + background-color: transparent; + border: 0; + -webkit-box-shadow: none; + box-shadow: none; + } + .navbar-nav .open .dropdown-menu > li > a, + .navbar-nav .open .dropdown-menu .dropdown-header { + padding: 5px 15px 5px 25px; + } + .navbar-nav .open .dropdown-menu > li > a { + line-height: 20px; + } + .navbar-nav .open .dropdown-menu > li > a:hover, + .navbar-nav .open .dropdown-menu > li > a:focus { + background-image: none; + } +} +@media (min-width: 768px) { + .navbar-nav { + float: left; + margin: 0; + } + .navbar-nav > li { + float: left; + } + .navbar-nav > li > a { + padding-top: 15px; + padding-bottom: 15px; + } +} +.navbar-form { + padding: 10px 15px; + margin-top: 8px; + margin-right: -15px; + margin-bottom: 8px; + margin-left: -15px; + border-top: 1px solid transparent; + border-bottom: 1px solid transparent; + -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, .1), 0 1px 0 rgba(255, 255, 255, .1); + box-shadow: inset 0 1px 0 rgba(255, 255, 255, .1), 0 1px 0 rgba(255, 255, 255, .1); +} +@media (min-width: 768px) { + .navbar-form .form-group { + display: inline-block; + margin-bottom: 0; + vertical-align: middle; + } + .navbar-form .form-control { + display: inline-block; + width: auto; + vertical-align: middle; + } + .navbar-form .form-control-static { + display: inline-block; + } + .navbar-form .input-group { + display: inline-table; + vertical-align: middle; + } + .navbar-form .input-group .input-group-addon, + .navbar-form .input-group .input-group-btn, + .navbar-form .input-group .form-control { + width: auto; + } + .navbar-form .input-group > .form-control { + width: 100%; + } + .navbar-form .control-label { + margin-bottom: 0; + vertical-align: middle; + } + .navbar-form .radio, + .navbar-form .checkbox { + display: inline-block; + margin-top: 0; + margin-bottom: 0; + vertical-align: middle; + } + .navbar-form .radio label, + .navbar-form .checkbox label { + padding-left: 0; + } + .navbar-form .radio input[type="radio"], + .navbar-form .checkbox input[type="checkbox"] { + position: relative; + margin-left: 0; + } + .navbar-form .has-feedback .form-control-feedback { + top: 0; + } +} +@media (max-width: 767px) { + .navbar-form .form-group { + margin-bottom: 5px; + } + .navbar-form .form-group:last-child { + margin-bottom: 0; + } +} +@media (min-width: 768px) { + .navbar-form { + width: auto; + padding-top: 0; + padding-bottom: 0; + margin-right: 0; + margin-left: 0; + border: 0; + -webkit-box-shadow: none; + box-shadow: none; + } +} +.navbar-nav > li > .dropdown-menu { + margin-top: 0; + border-top-left-radius: 0; + border-top-right-radius: 0; +} +.navbar-fixed-bottom .navbar-nav > li > .dropdown-menu { + margin-bottom: 0; + border-top-left-radius: 4px; + border-top-right-radius: 4px; + border-bottom-right-radius: 0; + border-bottom-left-radius: 0; +} +.navbar-btn { + margin-top: 8px; + margin-bottom: 8px; +} +.navbar-btn.btn-sm { + margin-top: 10px; + margin-bottom: 10px; +} +.navbar-btn.btn-xs { + margin-top: 14px; + margin-bottom: 14px; +} +.navbar-text { + margin-top: 15px; + margin-bottom: 15px; +} +@media (min-width: 768px) { + .navbar-text { + float: left; + margin-right: 15px; + margin-left: 15px; + } +} +@media (min-width: 768px) { + .navbar-left { + float: left !important; + } + .navbar-right { + float: right !important; + margin-right: -15px; + } + .navbar-right ~ .navbar-right { + margin-right: 0; + } +} +.navbar-default { + background-color: #f8f8f8; + border-color: #e7e7e7; +} +.navbar-default .navbar-brand { + color: #777; +} +.navbar-default .navbar-brand:hover, +.navbar-default .navbar-brand:focus { + color: #5e5e5e; + background-color: transparent; +} +.navbar-default .navbar-text { + color: #777; +} +.navbar-default .navbar-nav > li > a { + color: #777; +} +.navbar-default .navbar-nav > li > a:hover, +.navbar-default .navbar-nav > li > a:focus { + color: #333; + background-color: transparent; +} +.navbar-default .navbar-nav > .active > a, +.navbar-default .navbar-nav > .active > a:hover, +.navbar-default .navbar-nav > .active > a:focus { + color: #555; + background-color: #e7e7e7; +} +.navbar-default .navbar-nav > .disabled > a, +.navbar-default .navbar-nav > .disabled > a:hover, +.navbar-default .navbar-nav > .disabled > a:focus { + color: #ccc; + background-color: transparent; +} +.navbar-default .navbar-toggle { + border-color: #ddd; +} +.navbar-default .navbar-toggle:hover, +.navbar-default .navbar-toggle:focus { + background-color: #ddd; +} +.navbar-default .navbar-toggle .icon-bar { + background-color: #888; +} +.navbar-default .navbar-collapse, +.navbar-default .navbar-form { + border-color: #e7e7e7; +} +.navbar-default .navbar-nav > .open > a, +.navbar-default .navbar-nav > .open > a:hover, +.navbar-default .navbar-nav > .open > a:focus { + color: #555; + background-color: #e7e7e7; +} +@media (max-width: 767px) { + .navbar-default .navbar-nav .open .dropdown-menu > li > a { + color: #777; + } + .navbar-default .navbar-nav .open .dropdown-menu > li > a:hover, + .navbar-default .navbar-nav .open .dropdown-menu > li > a:focus { + color: #333; + background-color: transparent; + } + .navbar-default .navbar-nav .open .dropdown-menu > .active > a, + .navbar-default .navbar-nav .open .dropdown-menu > .active > a:hover, + .navbar-default .navbar-nav .open .dropdown-menu > .active > a:focus { + color: #555; + background-color: #e7e7e7; + } + .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a, + .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a:hover, + .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a:focus { + color: #ccc; + background-color: transparent; + } +} +.navbar-default .navbar-link { + color: #777; +} +.navbar-default .navbar-link:hover { + color: #333; +} +.navbar-default .btn-link { + color: #777; +} +.navbar-default .btn-link:hover, +.navbar-default .btn-link:focus { + color: #333; +} +.navbar-default .btn-link[disabled]:hover, +fieldset[disabled] .navbar-default .btn-link:hover, +.navbar-default .btn-link[disabled]:focus, +fieldset[disabled] .navbar-default .btn-link:focus { + color: #ccc; +} +.navbar-inverse { + background-color: #222; + border-color: #080808; +} +.navbar-inverse .navbar-brand { + color: #9d9d9d; +} +.navbar-inverse .navbar-brand:hover, +.navbar-inverse .navbar-brand:focus { + color: #fff; + background-color: transparent; +} +.navbar-inverse .navbar-text { + color: #9d9d9d; +} +.navbar-inverse .navbar-nav > li > a { + color: #9d9d9d; +} +.navbar-inverse .navbar-nav > li > a:hover, +.navbar-inverse .navbar-nav > li > a:focus { + color: #fff; + background-color: transparent; +} +.navbar-inverse .navbar-nav > .active > a, +.navbar-inverse .navbar-nav > .active > a:hover, +.navbar-inverse .navbar-nav > .active > a:focus { + color: #fff; + background-color: #080808; +} +.navbar-inverse .navbar-nav > .disabled > a, +.navbar-inverse .navbar-nav > .disabled > a:hover, +.navbar-inverse .navbar-nav > .disabled > a:focus { + color: #444; + background-color: transparent; +} +.navbar-inverse .navbar-toggle { + border-color: #333; +} +.navbar-inverse .navbar-toggle:hover, +.navbar-inverse .navbar-toggle:focus { + background-color: #333; +} +.navbar-inverse .navbar-toggle .icon-bar { + background-color: #fff; +} +.navbar-inverse .navbar-collapse, +.navbar-inverse .navbar-form { + border-color: #101010; +} +.navbar-inverse .navbar-nav > .open > a, +.navbar-inverse .navbar-nav > .open > a:hover, +.navbar-inverse .navbar-nav > .open > a:focus { + color: #fff; + background-color: #080808; +} +@media (max-width: 767px) { + .navbar-inverse .navbar-nav .open .dropdown-menu > .dropdown-header { + border-color: #080808; + } + .navbar-inverse .navbar-nav .open .dropdown-menu .divider { + background-color: #080808; + } + .navbar-inverse .navbar-nav .open .dropdown-menu > li > a { + color: #9d9d9d; + } + .navbar-inverse .navbar-nav .open .dropdown-menu > li > a:hover, + .navbar-inverse .navbar-nav .open .dropdown-menu > li > a:focus { + color: #fff; + background-color: transparent; + } + .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a, + .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a:hover, + .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a:focus { + color: #fff; + background-color: #080808; + } + .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a, + .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a:hover, + .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a:focus { + color: #444; + background-color: transparent; + } +} +.navbar-inverse .navbar-link { + color: #9d9d9d; +} +.navbar-inverse .navbar-link:hover { + color: #fff; +} +.navbar-inverse .btn-link { + color: #9d9d9d; +} +.navbar-inverse .btn-link:hover, +.navbar-inverse .btn-link:focus { + color: #fff; +} +.navbar-inverse .btn-link[disabled]:hover, +fieldset[disabled] .navbar-inverse .btn-link:hover, +.navbar-inverse .btn-link[disabled]:focus, +fieldset[disabled] .navbar-inverse .btn-link:focus { + color: #444; +} +.breadcrumb { + padding: 8px 15px; + margin-bottom: 20px; + list-style: none; + background-color: #f5f5f5; + border-radius: 4px; +} +.breadcrumb > li { + display: inline-block; +} +.breadcrumb > li + li:before { + padding: 0 5px; + color: #ccc; + content: "/\00a0"; +} +.breadcrumb > .active { + color: #777; +} +.pagination { + display: inline-block; + padding-left: 0; + margin: 20px 0; + border-radius: 4px; +} +.pagination > li { + display: inline; +} +.pagination > li > a, +.pagination > li > span { + position: relative; + float: left; + padding: 6px 12px; + margin-left: -1px; + line-height: 1.42857143; + color: #337ab7; + text-decoration: none; + background-color: #fff; + border: 1px solid #ddd; +} +.pagination > li:first-child > a, +.pagination > li:first-child > span { + margin-left: 0; + border-top-left-radius: 4px; + border-bottom-left-radius: 4px; +} +.pagination > li:last-child > a, +.pagination > li:last-child > span { + border-top-right-radius: 4px; + border-bottom-right-radius: 4px; +} +.pagination > li > a:hover, +.pagination > li > span:hover, +.pagination > li > a:focus, +.pagination > li > span:focus { + color: #23527c; + background-color: #eee; + border-color: #ddd; +} +.pagination > .active > a, +.pagination > .active > span, +.pagination > .active > a:hover, +.pagination > .active > span:hover, +.pagination > .active > a:focus, +.pagination > .active > span:focus { + z-index: 2; + color: #fff; + cursor: default; + background-color: #337ab7; + border-color: #337ab7; +} +.pagination > .disabled > span, +.pagination > .disabled > span:hover, +.pagination > .disabled > span:focus, +.pagination > .disabled > a, +.pagination > .disabled > a:hover, +.pagination > .disabled > a:focus { + color: #777; + cursor: not-allowed; + background-color: #fff; + border-color: #ddd; +} +.pagination-lg > li > a, +.pagination-lg > li > span { + padding: 10px 16px; + font-size: 18px; +} +.pagination-lg > li:first-child > a, +.pagination-lg > li:first-child > span { + border-top-left-radius: 6px; + border-bottom-left-radius: 6px; +} +.pagination-lg > li:last-child > a, +.pagination-lg > li:last-child > span { + border-top-right-radius: 6px; + border-bottom-right-radius: 6px; +} +.pagination-sm > li > a, +.pagination-sm > li > span { + padding: 5px 10px; + font-size: 12px; +} +.pagination-sm > li:first-child > a, +.pagination-sm > li:first-child > span { + border-top-left-radius: 3px; + border-bottom-left-radius: 3px; +} +.pagination-sm > li:last-child > a, +.pagination-sm > li:last-child > span { + border-top-right-radius: 3px; + border-bottom-right-radius: 3px; +} +.pager { + padding-left: 0; + margin: 20px 0; + text-align: center; + list-style: none; +} +.pager li { + display: inline; +} +.pager li > a, +.pager li > span { + display: inline-block; + padding: 5px 14px; + background-color: #fff; + border: 1px solid #ddd; + border-radius: 15px; +} +.pager li > a:hover, +.pager li > a:focus { + text-decoration: none; + background-color: #eee; +} +.pager .next > a, +.pager .next > span { + float: right; +} +.pager .previous > a, +.pager .previous > span { + float: left; +} +.pager .disabled > a, +.pager .disabled > a:hover, +.pager .disabled > a:focus, +.pager .disabled > span { + color: #777; + cursor: not-allowed; + background-color: #fff; +} +.label { + display: inline; + padding: .2em .6em .3em; + font-size: 75%; + font-weight: bold; + line-height: 1; + color: #fff; + text-align: center; + white-space: nowrap; + vertical-align: baseline; + border-radius: .25em; +} +a.label:hover, +a.label:focus { + color: #fff; + text-decoration: none; + cursor: pointer; +} +.label:empty { + display: none; +} +.btn .label { + position: relative; + top: -1px; +} +.label-default { + background-color: #777; +} +.label-default[href]:hover, +.label-default[href]:focus { + background-color: #5e5e5e; +} +.label-primary { + background-color: #337ab7; +} +.label-primary[href]:hover, +.label-primary[href]:focus { + background-color: #286090; +} +.label-success { + background-color: #5cb85c; +} +.label-success[href]:hover, +.label-success[href]:focus { + background-color: #449d44; +} +.label-info { + background-color: #5bc0de; +} +.label-info[href]:hover, +.label-info[href]:focus { + background-color: #31b0d5; +} +.label-warning { + background-color: #f0ad4e; +} +.label-warning[href]:hover, +.label-warning[href]:focus { + background-color: #ec971f; +} +.label-danger { + background-color: #d9534f; +} +.label-danger[href]:hover, +.label-danger[href]:focus { + background-color: #c9302c; +} +.badge { + display: inline-block; + min-width: 10px; + padding: 3px 7px; + font-size: 12px; + font-weight: bold; + line-height: 1; + color: #fff; + text-align: center; + white-space: nowrap; + vertical-align: baseline; + background-color: #777; + border-radius: 10px; +} +.badge:empty { + display: none; +} +.btn .badge { + position: relative; + top: -1px; +} +.btn-xs .badge, +.btn-group-xs > .btn .badge { + top: 0; + padding: 1px 5px; +} +a.badge:hover, +a.badge:focus { + color: #fff; + text-decoration: none; + cursor: pointer; +} +.list-group-item.active > .badge, +.nav-pills > .active > a > .badge { + color: #337ab7; + background-color: #fff; +} +.list-group-item > .badge { + float: right; +} +.list-group-item > .badge + .badge { + margin-right: 5px; +} +.nav-pills > li > a > .badge { + margin-left: 3px; +} +.jumbotron { + padding: 30px 15px; + margin-bottom: 30px; + color: inherit; + background-color: #eee; +} +.jumbotron h1, +.jumbotron .h1 { + color: inherit; +} +.jumbotron p { + margin-bottom: 15px; + font-size: 21px; + font-weight: 200; +} +.jumbotron > hr { + border-top-color: #d5d5d5; +} +.container .jumbotron, +.container-fluid .jumbotron { + border-radius: 6px; +} +.jumbotron .container { + max-width: 100%; +} +@media screen and (min-width: 768px) { + .jumbotron { + padding: 48px 0; + } + .container .jumbotron, + .container-fluid .jumbotron { + padding-right: 60px; + padding-left: 60px; + } + .jumbotron h1, + .jumbotron .h1 { + font-size: 63px; + } +} +.thumbnail { + display: block; + padding: 4px; + margin-bottom: 20px; + line-height: 1.42857143; + background-color: #fff; + border: 1px solid #ddd; + border-radius: 4px; + -webkit-transition: border .2s ease-in-out; + -o-transition: border .2s ease-in-out; + transition: border .2s ease-in-out; +} +.thumbnail > img, +.thumbnail a > img { + margin-right: auto; + margin-left: auto; +} +a.thumbnail:hover, +a.thumbnail:focus, +a.thumbnail.active { + border-color: #337ab7; +} +.thumbnail .caption { + padding: 9px; + color: #333; +} +.alert { + padding: 15px; + margin-bottom: 20px; + border: 1px solid transparent; + border-radius: 4px; +} +.alert h4 { + margin-top: 0; + color: inherit; +} +.alert .alert-link { + font-weight: bold; +} +.alert > p, +.alert > ul { + margin-bottom: 0; +} +.alert > p + p { + margin-top: 5px; +} +.alert-dismissable, +.alert-dismissible { + padding-right: 35px; +} +.alert-dismissable .close, +.alert-dismissible .close { + position: relative; + top: -2px; + right: -21px; + color: inherit; +} +.alert-success { + color: #3c763d; + background-color: #dff0d8; + border-color: #d6e9c6; +} +.alert-success hr { + border-top-color: #c9e2b3; +} +.alert-success .alert-link { + color: #2b542c; +} +.alert-info { + color: #31708f; + background-color: #d9edf7; + border-color: #bce8f1; +} +.alert-info hr { + border-top-color: #a6e1ec; +} +.alert-info .alert-link { + color: #245269; +} +.alert-warning { + color: #8a6d3b; + background-color: #fcf8e3; + border-color: #faebcc; +} +.alert-warning hr { + border-top-color: #f7e1b5; +} +.alert-warning .alert-link { + color: #66512c; +} +.alert-danger { + color: #a94442; + background-color: #f2dede; + border-color: #ebccd1; +} +.alert-danger hr { + border-top-color: #e4b9c0; +} +.alert-danger .alert-link { + color: #843534; +} +@-webkit-keyframes progress-bar-stripes { + from { + background-position: 40px 0; + } + to { + background-position: 0 0; + } +} +@-o-keyframes progress-bar-stripes { + from { + background-position: 40px 0; + } + to { + background-position: 0 0; + } +} +@keyframes progress-bar-stripes { + from { + background-position: 40px 0; + } + to { + background-position: 0 0; + } +} +.progress { + height: 20px; + margin-bottom: 20px; + overflow: hidden; + background-color: #f5f5f5; + border-radius: 4px; + -webkit-box-shadow: inset 0 1px 2px rgba(0, 0, 0, .1); + box-shadow: inset 0 1px 2px rgba(0, 0, 0, .1); +} +.progress-bar { + float: left; + width: 0; + height: 100%; + font-size: 12px; + line-height: 20px; + color: #fff; + text-align: center; + background-color: #337ab7; + -webkit-box-shadow: inset 0 -1px 0 rgba(0, 0, 0, .15); + box-shadow: inset 0 -1px 0 rgba(0, 0, 0, .15); + -webkit-transition: width .6s ease; + -o-transition: width .6s ease; + transition: width .6s ease; +} +.progress-striped .progress-bar, +.progress-bar-striped { + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + -webkit-background-size: 40px 40px; + background-size: 40px 40px; +} +.progress.active .progress-bar, +.progress-bar.active { + -webkit-animation: progress-bar-stripes 2s linear infinite; + -o-animation: progress-bar-stripes 2s linear infinite; + animation: progress-bar-stripes 2s linear infinite; +} +.progress-bar-success { + background-color: #5cb85c; +} +.progress-striped .progress-bar-success { + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); +} +.progress-bar-info { + background-color: #5bc0de; +} +.progress-striped .progress-bar-info { + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); +} +.progress-bar-warning { + background-color: #f0ad4e; +} +.progress-striped .progress-bar-warning { + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); +} +.progress-bar-danger { + background-color: #d9534f; +} +.progress-striped .progress-bar-danger { + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, .15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, .15) 50%, rgba(255, 255, 255, .15) 75%, transparent 75%, transparent); +} +.media { + margin-top: 15px; +} +.media:first-child { + margin-top: 0; +} +.media, +.media-body { + overflow: hidden; + zoom: 1; +} +.media-body { + width: 10000px; +} +.media-object { + display: block; +} +.media-right, +.media > .pull-right { + padding-left: 10px; +} +.media-left, +.media > .pull-left { + padding-right: 10px; +} +.media-left, +.media-right, +.media-body { + display: table-cell; + vertical-align: top; +} +.media-middle { + vertical-align: middle; +} +.media-bottom { + vertical-align: bottom; +} +.media-heading { + margin-top: 0; + margin-bottom: 5px; +} +.media-list { + padding-left: 0; + list-style: none; +} +.list-group { + padding-left: 0; + margin-bottom: 20px; +} +.list-group-item { + position: relative; + display: block; + padding: 10px 15px; + margin-bottom: -1px; + background-color: #fff; + border: 1px solid #ddd; +} +.list-group-item:first-child { + border-top-left-radius: 4px; + border-top-right-radius: 4px; +} +.list-group-item:last-child { + margin-bottom: 0; + border-bottom-right-radius: 4px; + border-bottom-left-radius: 4px; +} +a.list-group-item { + color: #555; +} +a.list-group-item .list-group-item-heading { + color: #333; +} +a.list-group-item:hover, +a.list-group-item:focus { + color: #555; + text-decoration: none; + background-color: #f5f5f5; +} +.list-group-item.disabled, +.list-group-item.disabled:hover, +.list-group-item.disabled:focus { + color: #777; + cursor: not-allowed; + background-color: #eee; +} +.list-group-item.disabled .list-group-item-heading, +.list-group-item.disabled:hover .list-group-item-heading, +.list-group-item.disabled:focus .list-group-item-heading { + color: inherit; +} +.list-group-item.disabled .list-group-item-text, +.list-group-item.disabled:hover .list-group-item-text, +.list-group-item.disabled:focus .list-group-item-text { + color: #777; +} +.list-group-item.active, +.list-group-item.active:hover, +.list-group-item.active:focus { + z-index: 2; + color: #fff; + background-color: #337ab7; + border-color: #337ab7; +} +.list-group-item.active .list-group-item-heading, +.list-group-item.active:hover .list-group-item-heading, +.list-group-item.active:focus .list-group-item-heading, +.list-group-item.active .list-group-item-heading > small, +.list-group-item.active:hover .list-group-item-heading > small, +.list-group-item.active:focus .list-group-item-heading > small, +.list-group-item.active .list-group-item-heading > .small, +.list-group-item.active:hover .list-group-item-heading > .small, +.list-group-item.active:focus .list-group-item-heading > .small { + color: inherit; +} +.list-group-item.active .list-group-item-text, +.list-group-item.active:hover .list-group-item-text, +.list-group-item.active:focus .list-group-item-text { + color: #c7ddef; +} +.list-group-item-success { + color: #3c763d; + background-color: #dff0d8; +} +a.list-group-item-success { + color: #3c763d; +} +a.list-group-item-success .list-group-item-heading { + color: inherit; +} +a.list-group-item-success:hover, +a.list-group-item-success:focus { + color: #3c763d; + background-color: #d0e9c6; +} +a.list-group-item-success.active, +a.list-group-item-success.active:hover, +a.list-group-item-success.active:focus { + color: #fff; + background-color: #3c763d; + border-color: #3c763d; +} +.list-group-item-info { + color: #31708f; + background-color: #d9edf7; +} +a.list-group-item-info { + color: #31708f; +} +a.list-group-item-info .list-group-item-heading { + color: inherit; +} +a.list-group-item-info:hover, +a.list-group-item-info:focus { + color: #31708f; + background-color: #c4e3f3; +} +a.list-group-item-info.active, +a.list-group-item-info.active:hover, +a.list-group-item-info.active:focus { + color: #fff; + background-color: #31708f; + border-color: #31708f; +} +.list-group-item-warning { + color: #8a6d3b; + background-color: #fcf8e3; +} +a.list-group-item-warning { + color: #8a6d3b; +} +a.list-group-item-warning .list-group-item-heading { + color: inherit; +} +a.list-group-item-warning:hover, +a.list-group-item-warning:focus { + color: #8a6d3b; + background-color: #faf2cc; +} +a.list-group-item-warning.active, +a.list-group-item-warning.active:hover, +a.list-group-item-warning.active:focus { + color: #fff; + background-color: #8a6d3b; + border-color: #8a6d3b; +} +.list-group-item-danger { + color: #a94442; + background-color: #f2dede; +} +a.list-group-item-danger { + color: #a94442; +} +a.list-group-item-danger .list-group-item-heading { + color: inherit; +} +a.list-group-item-danger:hover, +a.list-group-item-danger:focus { + color: #a94442; + background-color: #ebcccc; +} +a.list-group-item-danger.active, +a.list-group-item-danger.active:hover, +a.list-group-item-danger.active:focus { + color: #fff; + background-color: #a94442; + border-color: #a94442; +} +.list-group-item-heading { + margin-top: 0; + margin-bottom: 5px; +} +.list-group-item-text { + margin-bottom: 0; + line-height: 1.3; +} +.panel { + margin-bottom: 20px; + background-color: #fff; + border: 1px solid transparent; + border-radius: 4px; + -webkit-box-shadow: 0 1px 1px rgba(0, 0, 0, .05); + box-shadow: 0 1px 1px rgba(0, 0, 0, .05); +} +.panel-body { + padding: 15px; +} +.panel-heading { + padding: 10px 15px; + border-bottom: 1px solid transparent; + border-top-left-radius: 3px; + border-top-right-radius: 3px; +} +.panel-heading > .dropdown .dropdown-toggle { + color: inherit; +} +.panel-title { + margin-top: 0; + margin-bottom: 0; + font-size: 16px; + color: inherit; +} +.panel-title > a, +.panel-title > small, +.panel-title > .small, +.panel-title > small > a, +.panel-title > .small > a { + color: inherit; +} +.panel-footer { + padding: 10px 15px; + background-color: #f5f5f5; + border-top: 1px solid #ddd; + border-bottom-right-radius: 3px; + border-bottom-left-radius: 3px; +} +.panel > .list-group, +.panel > .panel-collapse > .list-group { + margin-bottom: 0; +} +.panel > .list-group .list-group-item, +.panel > .panel-collapse > .list-group .list-group-item { + border-width: 1px 0; + border-radius: 0; +} +.panel > .list-group:first-child .list-group-item:first-child, +.panel > .panel-collapse > .list-group:first-child .list-group-item:first-child { + border-top: 0; + border-top-left-radius: 3px; + border-top-right-radius: 3px; +} +.panel > .list-group:last-child .list-group-item:last-child, +.panel > .panel-collapse > .list-group:last-child .list-group-item:last-child { + border-bottom: 0; + border-bottom-right-radius: 3px; + border-bottom-left-radius: 3px; +} +.panel-heading + .list-group .list-group-item:first-child { + border-top-width: 0; +} +.list-group + .panel-footer { + border-top-width: 0; +} +.panel > .table, +.panel > .table-responsive > .table, +.panel > .panel-collapse > .table { + margin-bottom: 0; +} +.panel > .table caption, +.panel > .table-responsive > .table caption, +.panel > .panel-collapse > .table caption { + padding-right: 15px; + padding-left: 15px; +} +.panel > .table:first-child, +.panel > .table-responsive:first-child > .table:first-child { + border-top-left-radius: 3px; + border-top-right-radius: 3px; +} +.panel > .table:first-child > thead:first-child > tr:first-child, +.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child, +.panel > .table:first-child > tbody:first-child > tr:first-child, +.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child { + border-top-left-radius: 3px; + border-top-right-radius: 3px; +} +.panel > .table:first-child > thead:first-child > tr:first-child td:first-child, +.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child td:first-child, +.panel > .table:first-child > tbody:first-child > tr:first-child td:first-child, +.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child td:first-child, +.panel > .table:first-child > thead:first-child > tr:first-child th:first-child, +.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child th:first-child, +.panel > .table:first-child > tbody:first-child > tr:first-child th:first-child, +.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child th:first-child { + border-top-left-radius: 3px; +} +.panel > .table:first-child > thead:first-child > tr:first-child td:last-child, +.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child td:last-child, +.panel > .table:first-child > tbody:first-child > tr:first-child td:last-child, +.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child td:last-child, +.panel > .table:first-child > thead:first-child > tr:first-child th:last-child, +.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child th:last-child, +.panel > .table:first-child > tbody:first-child > tr:first-child th:last-child, +.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child th:last-child { + border-top-right-radius: 3px; +} +.panel > .table:last-child, +.panel > .table-responsive:last-child > .table:last-child { + border-bottom-right-radius: 3px; + border-bottom-left-radius: 3px; +} +.panel > .table:last-child > tbody:last-child > tr:last-child, +.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child, +.panel > .table:last-child > tfoot:last-child > tr:last-child, +.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child { + border-bottom-right-radius: 3px; + border-bottom-left-radius: 3px; +} +.panel > .table:last-child > tbody:last-child > tr:last-child td:first-child, +.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child td:first-child, +.panel > .table:last-child > tfoot:last-child > tr:last-child td:first-child, +.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child td:first-child, +.panel > .table:last-child > tbody:last-child > tr:last-child th:first-child, +.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child th:first-child, +.panel > .table:last-child > tfoot:last-child > tr:last-child th:first-child, +.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child th:first-child { + border-bottom-left-radius: 3px; +} +.panel > .table:last-child > tbody:last-child > tr:last-child td:last-child, +.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child td:last-child, +.panel > .table:last-child > tfoot:last-child > tr:last-child td:last-child, +.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child td:last-child, +.panel > .table:last-child > tbody:last-child > tr:last-child th:last-child, +.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child th:last-child, +.panel > .table:last-child > tfoot:last-child > tr:last-child th:last-child, +.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child th:last-child { + border-bottom-right-radius: 3px; +} +.panel > .panel-body + .table, +.panel > .panel-body + .table-responsive, +.panel > .table + .panel-body, +.panel > .table-responsive + .panel-body { + border-top: 1px solid #ddd; +} +.panel > .table > tbody:first-child > tr:first-child th, +.panel > .table > tbody:first-child > tr:first-child td { + border-top: 0; +} +.panel > .table-bordered, +.panel > .table-responsive > .table-bordered { + border: 0; +} +.panel > .table-bordered > thead > tr > th:first-child, +.panel > .table-responsive > .table-bordered > thead > tr > th:first-child, +.panel > .table-bordered > tbody > tr > th:first-child, +.panel > .table-responsive > .table-bordered > tbody > tr > th:first-child, +.panel > .table-bordered > tfoot > tr > th:first-child, +.panel > .table-responsive > .table-bordered > tfoot > tr > th:first-child, +.panel > .table-bordered > thead > tr > td:first-child, +.panel > .table-responsive > .table-bordered > thead > tr > td:first-child, +.panel > .table-bordered > tbody > tr > td:first-child, +.panel > .table-responsive > .table-bordered > tbody > tr > td:first-child, +.panel > .table-bordered > tfoot > tr > td:first-child, +.panel > .table-responsive > .table-bordered > tfoot > tr > td:first-child { + border-left: 0; +} +.panel > .table-bordered > thead > tr > th:last-child, +.panel > .table-responsive > .table-bordered > thead > tr > th:last-child, +.panel > .table-bordered > tbody > tr > th:last-child, +.panel > .table-responsive > .table-bordered > tbody > tr > th:last-child, +.panel > .table-bordered > tfoot > tr > th:last-child, +.panel > .table-responsive > .table-bordered > tfoot > tr > th:last-child, +.panel > .table-bordered > thead > tr > td:last-child, +.panel > .table-responsive > .table-bordered > thead > tr > td:last-child, +.panel > .table-bordered > tbody > tr > td:last-child, +.panel > .table-responsive > .table-bordered > tbody > tr > td:last-child, +.panel > .table-bordered > tfoot > tr > td:last-child, +.panel > .table-responsive > .table-bordered > tfoot > tr > td:last-child { + border-right: 0; +} +.panel > .table-bordered > thead > tr:first-child > td, +.panel > .table-responsive > .table-bordered > thead > tr:first-child > td, +.panel > .table-bordered > tbody > tr:first-child > td, +.panel > .table-responsive > .table-bordered > tbody > tr:first-child > td, +.panel > .table-bordered > thead > tr:first-child > th, +.panel > .table-responsive > .table-bordered > thead > tr:first-child > th, +.panel > .table-bordered > tbody > tr:first-child > th, +.panel > .table-responsive > .table-bordered > tbody > tr:first-child > th { + border-bottom: 0; +} +.panel > .table-bordered > tbody > tr:last-child > td, +.panel > .table-responsive > .table-bordered > tbody > tr:last-child > td, +.panel > .table-bordered > tfoot > tr:last-child > td, +.panel > .table-responsive > .table-bordered > tfoot > tr:last-child > td, +.panel > .table-bordered > tbody > tr:last-child > th, +.panel > .table-responsive > .table-bordered > tbody > tr:last-child > th, +.panel > .table-bordered > tfoot > tr:last-child > th, +.panel > .table-responsive > .table-bordered > tfoot > tr:last-child > th { + border-bottom: 0; +} +.panel > .table-responsive { + margin-bottom: 0; + border: 0; +} +.panel-group { + margin-bottom: 20px; +} +.panel-group .panel { + margin-bottom: 0; + border-radius: 4px; +} +.panel-group .panel + .panel { + margin-top: 5px; +} +.panel-group .panel-heading { + border-bottom: 0; +} +.panel-group .panel-heading + .panel-collapse > .panel-body, +.panel-group .panel-heading + .panel-collapse > .list-group { + border-top: 1px solid #ddd; +} +.panel-group .panel-footer { + border-top: 0; +} +.panel-group .panel-footer + .panel-collapse .panel-body { + border-bottom: 1px solid #ddd; +} +.panel-default { + border-color: #ddd; +} +.panel-default > .panel-heading { + color: #333; + background-color: #f5f5f5; + border-color: #ddd; +} +.panel-default > .panel-heading + .panel-collapse > .panel-body { + border-top-color: #ddd; +} +.panel-default > .panel-heading .badge { + color: #f5f5f5; + background-color: #333; +} +.panel-default > .panel-footer + .panel-collapse > .panel-body { + border-bottom-color: #ddd; +} +.panel-primary { + border-color: #337ab7; +} +.panel-primary > .panel-heading { + color: #fff; + background-color: #337ab7; + border-color: #337ab7; +} +.panel-primary > .panel-heading + .panel-collapse > .panel-body { + border-top-color: #337ab7; +} +.panel-primary > .panel-heading .badge { + color: #337ab7; + background-color: #fff; +} +.panel-primary > .panel-footer + .panel-collapse > .panel-body { + border-bottom-color: #337ab7; +} +.panel-success { + border-color: #d6e9c6; +} +.panel-success > .panel-heading { + color: #3c763d; + background-color: #dff0d8; + border-color: #d6e9c6; +} +.panel-success > .panel-heading + .panel-collapse > .panel-body { + border-top-color: #d6e9c6; +} +.panel-success > .panel-heading .badge { + color: #dff0d8; + background-color: #3c763d; +} +.panel-success > .panel-footer + .panel-collapse > .panel-body { + border-bottom-color: #d6e9c6; +} +.panel-info { + border-color: #bce8f1; +} +.panel-info > .panel-heading { + color: #31708f; + background-color: #d9edf7; + border-color: #bce8f1; +} +.panel-info > .panel-heading + .panel-collapse > .panel-body { + border-top-color: #bce8f1; +} +.panel-info > .panel-heading .badge { + color: #d9edf7; + background-color: #31708f; +} +.panel-info > .panel-footer + .panel-collapse > .panel-body { + border-bottom-color: #bce8f1; +} +.panel-warning { + border-color: #faebcc; +} +.panel-warning > .panel-heading { + color: #8a6d3b; + background-color: #fcf8e3; + border-color: #faebcc; +} +.panel-warning > .panel-heading + .panel-collapse > .panel-body { + border-top-color: #faebcc; +} +.panel-warning > .panel-heading .badge { + color: #fcf8e3; + background-color: #8a6d3b; +} +.panel-warning > .panel-footer + .panel-collapse > .panel-body { + border-bottom-color: #faebcc; +} +.panel-danger { + border-color: #ebccd1; +} +.panel-danger > .panel-heading { + color: #a94442; + background-color: #f2dede; + border-color: #ebccd1; +} +.panel-danger > .panel-heading + .panel-collapse > .panel-body { + border-top-color: #ebccd1; +} +.panel-danger > .panel-heading .badge { + color: #f2dede; + background-color: #a94442; +} +.panel-danger > .panel-footer + .panel-collapse > .panel-body { + border-bottom-color: #ebccd1; +} +.embed-responsive { + position: relative; + display: block; + height: 0; + padding: 0; + overflow: hidden; +} +.embed-responsive .embed-responsive-item, +.embed-responsive iframe, +.embed-responsive embed, +.embed-responsive object, +.embed-responsive video { + position: absolute; + top: 0; + bottom: 0; + left: 0; + width: 100%; + height: 100%; + border: 0; +} +.embed-responsive-16by9 { + padding-bottom: 56.25%; +} +.embed-responsive-4by3 { + padding-bottom: 75%; +} +.well { + min-height: 20px; + padding: 19px; + margin-bottom: 20px; + background-color: #f5f5f5; + border: 1px solid #e3e3e3; + border-radius: 4px; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, .05); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, .05); +} +.well blockquote { + border-color: #ddd; + border-color: rgba(0, 0, 0, .15); +} +.well-lg { + padding: 24px; + border-radius: 6px; +} +.well-sm { + padding: 9px; + border-radius: 3px; +} +.close { + float: right; + font-size: 21px; + font-weight: bold; + line-height: 1; + color: #000; + text-shadow: 0 1px 0 #fff; + filter: alpha(opacity=20); + opacity: .2; +} +.close:hover, +.close:focus { + color: #000; + text-decoration: none; + cursor: pointer; + filter: alpha(opacity=50); + opacity: .5; +} +button.close { + -webkit-appearance: none; + padding: 0; + cursor: pointer; + background: transparent; + border: 0; +} +.modal-open { + overflow: hidden; +} +.modal { + position: fixed; + top: 0; + right: 0; + bottom: 0; + left: 0; + z-index: 1050; + display: none; + overflow: hidden; + -webkit-overflow-scrolling: touch; + outline: 0; +} +.modal.fade .modal-dialog { + -webkit-transition: -webkit-transform .3s ease-out; + -o-transition: -o-transform .3s ease-out; + transition: transform .3s ease-out; + -webkit-transform: translate(0, -25%); + -ms-transform: translate(0, -25%); + -o-transform: translate(0, -25%); + transform: translate(0, -25%); +} +.modal.in .modal-dialog { + -webkit-transform: translate(0, 0); + -ms-transform: translate(0, 0); + -o-transform: translate(0, 0); + transform: translate(0, 0); +} +.modal-open .modal { + overflow-x: hidden; + overflow-y: auto; +} +.modal-dialog { + position: relative; + width: auto; + margin: 10px; +} +.modal-content { + position: relative; + background-color: #fff; + -webkit-background-clip: padding-box; + background-clip: padding-box; + border: 1px solid #999; + border: 1px solid rgba(0, 0, 0, .2); + border-radius: 6px; + outline: 0; + -webkit-box-shadow: 0 3px 9px rgba(0, 0, 0, .5); + box-shadow: 0 3px 9px rgba(0, 0, 0, .5); +} +.modal-backdrop { + position: fixed; + top: 0; + right: 0; + bottom: 0; + left: 0; + z-index: 1040; + background-color: #000; +} +.modal-backdrop.fade { + filter: alpha(opacity=0); + opacity: 0; +} +.modal-backdrop.in { + filter: alpha(opacity=50); + opacity: .5; +} +.modal-header { + min-height: 16.42857143px; + padding: 15px; + border-bottom: 1px solid #e5e5e5; +} +.modal-header .close { + margin-top: -2px; +} +.modal-title { + margin: 0; + line-height: 1.42857143; +} +.modal-body { + position: relative; + padding: 15px; +} +.modal-footer { + padding: 15px; + text-align: right; + border-top: 1px solid #e5e5e5; +} +.modal-footer .btn + .btn { + margin-bottom: 0; + margin-left: 5px; +} +.modal-footer .btn-group .btn + .btn { + margin-left: -1px; +} +.modal-footer .btn-block + .btn-block { + margin-left: 0; +} +.modal-scrollbar-measure { + position: absolute; + top: -9999px; + width: 50px; + height: 50px; + overflow: scroll; +} +@media (min-width: 768px) { + .modal-dialog { + width: 600px; + margin: 30px auto; + } + .modal-content { + -webkit-box-shadow: 0 5px 15px rgba(0, 0, 0, .5); + box-shadow: 0 5px 15px rgba(0, 0, 0, .5); + } + .modal-sm { + width: 300px; + } +} +@media (min-width: 992px) { + .modal-lg { + width: 900px; + } +} +.tooltip { + position: absolute; + z-index: 1070; + display: block; + font-family: "Helvetica Neue", Helvetica, Arial, sans-serif; + font-size: 12px; + font-weight: normal; + line-height: 1.4; + filter: alpha(opacity=0); + opacity: 0; +} +.tooltip.in { + filter: alpha(opacity=90); + opacity: .9; +} +.tooltip.top { + padding: 5px 0; + margin-top: -3px; +} +.tooltip.right { + padding: 0 5px; + margin-left: 3px; +} +.tooltip.bottom { + padding: 5px 0; + margin-top: 3px; +} +.tooltip.left { + padding: 0 5px; + margin-left: -3px; +} +.tooltip-inner { + max-width: 200px; + padding: 3px 8px; + color: #fff; + text-align: center; + text-decoration: none; + background-color: #000; + border-radius: 4px; +} +.tooltip-arrow { + position: absolute; + width: 0; + height: 0; + border-color: transparent; + border-style: solid; +} +.tooltip.top .tooltip-arrow { + bottom: 0; + left: 50%; + margin-left: -5px; + border-width: 5px 5px 0; + border-top-color: #000; +} +.tooltip.top-left .tooltip-arrow { + right: 5px; + bottom: 0; + margin-bottom: -5px; + border-width: 5px 5px 0; + border-top-color: #000; +} +.tooltip.top-right .tooltip-arrow { + bottom: 0; + left: 5px; + margin-bottom: -5px; + border-width: 5px 5px 0; + border-top-color: #000; +} +.tooltip.right .tooltip-arrow { + top: 50%; + left: 0; + margin-top: -5px; + border-width: 5px 5px 5px 0; + border-right-color: #000; +} +.tooltip.left .tooltip-arrow { + top: 50%; + right: 0; + margin-top: -5px; + border-width: 5px 0 5px 5px; + border-left-color: #000; +} +.tooltip.bottom .tooltip-arrow { + top: 0; + left: 50%; + margin-left: -5px; + border-width: 0 5px 5px; + border-bottom-color: #000; +} +.tooltip.bottom-left .tooltip-arrow { + top: 0; + right: 5px; + margin-top: -5px; + border-width: 0 5px 5px; + border-bottom-color: #000; +} +.tooltip.bottom-right .tooltip-arrow { + top: 0; + left: 5px; + margin-top: -5px; + border-width: 0 5px 5px; + border-bottom-color: #000; +} +.popover { + position: absolute; + top: 0; + left: 0; + z-index: 1060; + display: none; + max-width: 276px; + padding: 1px; + font-family: "Helvetica Neue", Helvetica, Arial, sans-serif; + font-size: 14px; + font-weight: normal; + line-height: 1.42857143; + text-align: left; + white-space: normal; + background-color: #fff; + -webkit-background-clip: padding-box; + background-clip: padding-box; + border: 1px solid #ccc; + border: 1px solid rgba(0, 0, 0, .2); + border-radius: 6px; + -webkit-box-shadow: 0 5px 10px rgba(0, 0, 0, .2); + box-shadow: 0 5px 10px rgba(0, 0, 0, .2); +} +.popover.top { + margin-top: -10px; +} +.popover.right { + margin-left: 10px; +} +.popover.bottom { + margin-top: 10px; +} +.popover.left { + margin-left: -10px; +} +.popover-title { + padding: 8px 14px; + margin: 0; + font-size: 14px; + background-color: #f7f7f7; + border-bottom: 1px solid #ebebeb; + border-radius: 5px 5px 0 0; +} +.popover-content { + padding: 9px 14px; +} +.popover > .arrow, +.popover > .arrow:after { + position: absolute; + display: block; + width: 0; + height: 0; + border-color: transparent; + border-style: solid; +} +.popover > .arrow { + border-width: 11px; +} +.popover > .arrow:after { + content: ""; + border-width: 10px; +} +.popover.top > .arrow { + bottom: -11px; + left: 50%; + margin-left: -11px; + border-top-color: #999; + border-top-color: rgba(0, 0, 0, .25); + border-bottom-width: 0; +} +.popover.top > .arrow:after { + bottom: 1px; + margin-left: -10px; + content: " "; + border-top-color: #fff; + border-bottom-width: 0; +} +.popover.right > .arrow { + top: 50%; + left: -11px; + margin-top: -11px; + border-right-color: #999; + border-right-color: rgba(0, 0, 0, .25); + border-left-width: 0; +} +.popover.right > .arrow:after { + bottom: -10px; + left: 1px; + content: " "; + border-right-color: #fff; + border-left-width: 0; +} +.popover.bottom > .arrow { + top: -11px; + left: 50%; + margin-left: -11px; + border-top-width: 0; + border-bottom-color: #999; + border-bottom-color: rgba(0, 0, 0, .25); +} +.popover.bottom > .arrow:after { + top: 1px; + margin-left: -10px; + content: " "; + border-top-width: 0; + border-bottom-color: #fff; +} +.popover.left > .arrow { + top: 50%; + right: -11px; + margin-top: -11px; + border-right-width: 0; + border-left-color: #999; + border-left-color: rgba(0, 0, 0, .25); +} +.popover.left > .arrow:after { + right: 1px; + bottom: -10px; + content: " "; + border-right-width: 0; + border-left-color: #fff; +} +.carousel { + position: relative; +} +.carousel-inner { + position: relative; + width: 100%; + overflow: hidden; +} +.carousel-inner > .item { + position: relative; + display: none; + -webkit-transition: .6s ease-in-out left; + -o-transition: .6s ease-in-out left; + transition: .6s ease-in-out left; +} +.carousel-inner > .item > img, +.carousel-inner > .item > a > img { + line-height: 1; +} +@media all and (transform-3d), (-webkit-transform-3d) { + .carousel-inner > .item { + -webkit-transition: -webkit-transform .6s ease-in-out; + -o-transition: -o-transform .6s ease-in-out; + transition: transform .6s ease-in-out; + + -webkit-backface-visibility: hidden; + backface-visibility: hidden; + -webkit-perspective: 1000; + perspective: 1000; + } + .carousel-inner > .item.next, + .carousel-inner > .item.active.right { + left: 0; + -webkit-transform: translate3d(100%, 0, 0); + transform: translate3d(100%, 0, 0); + } + .carousel-inner > .item.prev, + .carousel-inner > .item.active.left { + left: 0; + -webkit-transform: translate3d(-100%, 0, 0); + transform: translate3d(-100%, 0, 0); + } + .carousel-inner > .item.next.left, + .carousel-inner > .item.prev.right, + .carousel-inner > .item.active { + left: 0; + -webkit-transform: translate3d(0, 0, 0); + transform: translate3d(0, 0, 0); + } +} +.carousel-inner > .active, +.carousel-inner > .next, +.carousel-inner > .prev { + display: block; +} +.carousel-inner > .active { + left: 0; +} +.carousel-inner > .next, +.carousel-inner > .prev { + position: absolute; + top: 0; + width: 100%; +} +.carousel-inner > .next { + left: 100%; +} +.carousel-inner > .prev { + left: -100%; +} +.carousel-inner > .next.left, +.carousel-inner > .prev.right { + left: 0; +} +.carousel-inner > .active.left { + left: -100%; +} +.carousel-inner > .active.right { + left: 100%; +} +.carousel-control { + position: absolute; + top: 0; + bottom: 0; + left: 0; + width: 15%; + font-size: 20px; + color: #fff; + text-align: center; + text-shadow: 0 1px 2px rgba(0, 0, 0, .6); + filter: alpha(opacity=50); + opacity: .5; +} +.carousel-control.left { + background-image: -webkit-linear-gradient(left, rgba(0, 0, 0, .5) 0%, rgba(0, 0, 0, .0001) 100%); + background-image: -o-linear-gradient(left, rgba(0, 0, 0, .5) 0%, rgba(0, 0, 0, .0001) 100%); + background-image: -webkit-gradient(linear, left top, right top, from(rgba(0, 0, 0, .5)), to(rgba(0, 0, 0, .0001))); + background-image: linear-gradient(to right, rgba(0, 0, 0, .5) 0%, rgba(0, 0, 0, .0001) 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#80000000', endColorstr='#00000000', GradientType=1); + background-repeat: repeat-x; +} +.carousel-control.right { + right: 0; + left: auto; + background-image: -webkit-linear-gradient(left, rgba(0, 0, 0, .0001) 0%, rgba(0, 0, 0, .5) 100%); + background-image: -o-linear-gradient(left, rgba(0, 0, 0, .0001) 0%, rgba(0, 0, 0, .5) 100%); + background-image: -webkit-gradient(linear, left top, right top, from(rgba(0, 0, 0, .0001)), to(rgba(0, 0, 0, .5))); + background-image: linear-gradient(to right, rgba(0, 0, 0, .0001) 0%, rgba(0, 0, 0, .5) 100%); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#00000000', endColorstr='#80000000', GradientType=1); + background-repeat: repeat-x; +} +.carousel-control:hover, +.carousel-control:focus { + color: #fff; + text-decoration: none; + filter: alpha(opacity=90); + outline: 0; + opacity: .9; +} +.carousel-control .icon-prev, +.carousel-control .icon-next, +.carousel-control .glyphicon-chevron-left, +.carousel-control .glyphicon-chevron-right { + position: absolute; + top: 50%; + z-index: 5; + display: inline-block; +} +.carousel-control .icon-prev, +.carousel-control .glyphicon-chevron-left { + left: 50%; + margin-left: -10px; +} +.carousel-control .icon-next, +.carousel-control .glyphicon-chevron-right { + right: 50%; + margin-right: -10px; +} +.carousel-control .icon-prev, +.carousel-control .icon-next { + width: 20px; + height: 20px; + margin-top: -10px; + font-family: serif; + line-height: 1; +} +.carousel-control .icon-prev:before { + content: '\2039'; +} +.carousel-control .icon-next:before { + content: '\203a'; +} +.carousel-indicators { + position: absolute; + bottom: 10px; + left: 50%; + z-index: 15; + width: 60%; + padding-left: 0; + margin-left: -30%; + text-align: center; + list-style: none; +} +.carousel-indicators li { + display: inline-block; + width: 10px; + height: 10px; + margin: 1px; + text-indent: -999px; + cursor: pointer; + background-color: #000 \9; + background-color: rgba(0, 0, 0, 0); + border: 1px solid #fff; + border-radius: 10px; +} +.carousel-indicators .active { + width: 12px; + height: 12px; + margin: 0; + background-color: #fff; +} +.carousel-caption { + position: absolute; + right: 15%; + bottom: 20px; + left: 15%; + z-index: 10; + padding-top: 20px; + padding-bottom: 20px; + color: #fff; + text-align: center; + text-shadow: 0 1px 2px rgba(0, 0, 0, .6); +} +.carousel-caption .btn { + text-shadow: none; +} +@media screen and (min-width: 768px) { + .carousel-control .glyphicon-chevron-left, + .carousel-control .glyphicon-chevron-right, + .carousel-control .icon-prev, + .carousel-control .icon-next { + width: 30px; + height: 30px; + margin-top: -15px; + font-size: 30px; + } + .carousel-control .glyphicon-chevron-left, + .carousel-control .icon-prev { + margin-left: -15px; + } + .carousel-control .glyphicon-chevron-right, + .carousel-control .icon-next { + margin-right: -15px; + } + .carousel-caption { + right: 20%; + left: 20%; + padding-bottom: 30px; + } + .carousel-indicators { + bottom: 20px; + } +} +.clearfix:before, +.clearfix:after, +.dl-horizontal dd:before, +.dl-horizontal dd:after, +.container:before, +.container:after, +.container-fluid:before, +.container-fluid:after, +.row:before, +.row:after, +.form-horizontal .form-group:before, +.form-horizontal .form-group:after, +.btn-toolbar:before, +.btn-toolbar:after, +.btn-group-vertical > .btn-group:before, +.btn-group-vertical > .btn-group:after, +.nav:before, +.nav:after, +.navbar:before, +.navbar:after, +.navbar-header:before, +.navbar-header:after, +.navbar-collapse:before, +.navbar-collapse:after, +.pager:before, +.pager:after, +.panel-body:before, +.panel-body:after, +.modal-footer:before, +.modal-footer:after { + display: table; + content: " "; +} +.clearfix:after, +.dl-horizontal dd:after, +.container:after, +.container-fluid:after, +.row:after, +.form-horizontal .form-group:after, +.btn-toolbar:after, +.btn-group-vertical > .btn-group:after, +.nav:after, +.navbar:after, +.navbar-header:after, +.navbar-collapse:after, +.pager:after, +.panel-body:after, +.modal-footer:after { + clear: both; +} +.center-block { + display: block; + margin-right: auto; + margin-left: auto; +} +.pull-right { + float: right !important; +} +.pull-left { + float: left !important; +} +.hide { + display: none !important; +} +.show { + display: block !important; +} +.invisible { + visibility: hidden; +} +.text-hide { + font: 0/0 a; + color: transparent; + text-shadow: none; + background-color: transparent; + border: 0; +} +.hidden { + display: none !important; +} +.affix { + position: fixed; +} +@-ms-viewport { + width: device-width; +} +.visible-xs, +.visible-sm, +.visible-md, +.visible-lg { + display: none !important; +} +.visible-xs-block, +.visible-xs-inline, +.visible-xs-inline-block, +.visible-sm-block, +.visible-sm-inline, +.visible-sm-inline-block, +.visible-md-block, +.visible-md-inline, +.visible-md-inline-block, +.visible-lg-block, +.visible-lg-inline, +.visible-lg-inline-block { + display: none !important; +} +@media (max-width: 767px) { + .visible-xs { + display: block !important; + } + table.visible-xs { + display: table; + } + tr.visible-xs { + display: table-row !important; + } + th.visible-xs, + td.visible-xs { + display: table-cell !important; + } +} +@media (max-width: 767px) { + .visible-xs-block { + display: block !important; + } +} +@media (max-width: 767px) { + .visible-xs-inline { + display: inline !important; + } +} +@media (max-width: 767px) { + .visible-xs-inline-block { + display: inline-block !important; + } +} +@media (min-width: 768px) and (max-width: 991px) { + .visible-sm { + display: block !important; + } + table.visible-sm { + display: table; + } + tr.visible-sm { + display: table-row !important; + } + th.visible-sm, + td.visible-sm { + display: table-cell !important; + } +} +@media (min-width: 768px) and (max-width: 991px) { + .visible-sm-block { + display: block !important; + } +} +@media (min-width: 768px) and (max-width: 991px) { + .visible-sm-inline { + display: inline !important; + } +} +@media (min-width: 768px) and (max-width: 991px) { + .visible-sm-inline-block { + display: inline-block !important; + } +} +@media (min-width: 992px) and (max-width: 1199px) { + .visible-md { + display: block !important; + } + table.visible-md { + display: table; + } + tr.visible-md { + display: table-row !important; + } + th.visible-md, + td.visible-md { + display: table-cell !important; + } +} +@media (min-width: 992px) and (max-width: 1199px) { + .visible-md-block { + display: block !important; + } +} +@media (min-width: 992px) and (max-width: 1199px) { + .visible-md-inline { + display: inline !important; + } +} +@media (min-width: 992px) and (max-width: 1199px) { + .visible-md-inline-block { + display: inline-block !important; + } +} +@media (min-width: 1200px) { + .visible-lg { + display: block !important; + } + table.visible-lg { + display: table; + } + tr.visible-lg { + display: table-row !important; + } + th.visible-lg, + td.visible-lg { + display: table-cell !important; + } +} +@media (min-width: 1200px) { + .visible-lg-block { + display: block !important; + } +} +@media (min-width: 1200px) { + .visible-lg-inline { + display: inline !important; + } +} +@media (min-width: 1200px) { + .visible-lg-inline-block { + display: inline-block !important; + } +} +@media (max-width: 767px) { + .hidden-xs { + display: none !important; + } +} +@media (min-width: 768px) and (max-width: 991px) { + .hidden-sm { + display: none !important; + } +} +@media (min-width: 992px) and (max-width: 1199px) { + .hidden-md { + display: none !important; + } +} +@media (min-width: 1200px) { + .hidden-lg { + display: none !important; + } +} +.visible-print { + display: none !important; +} +@media print { + .visible-print { + display: block !important; + } + table.visible-print { + display: table; + } + tr.visible-print { + display: table-row !important; + } + th.visible-print, + td.visible-print { + display: table-cell !important; + } +} +.visible-print-block { + display: none !important; +} +@media print { + .visible-print-block { + display: block !important; + } +} +.visible-print-inline { + display: none !important; +} +@media print { + .visible-print-inline { + display: inline !important; + } +} +.visible-print-inline-block { + display: none !important; +} +@media print { + .visible-print-inline-block { + display: inline-block !important; + } +} +@media print { + .hidden-print { + display: none !important; + } +} +/*# sourceMappingURL=bootstrap.css.map */ diff --git a/jstorm-ui/src/main/webapp/assets/css/bootstrap.css.map b/jstorm-ui/src/main/webapp/assets/css/bootstrap.css.map new file mode 100755 index 000000000..2fd84f36e --- /dev/null +++ b/jstorm-ui/src/main/webapp/assets/css/bootstrap.css.map @@ -0,0 +1 @@ +{"version":3,"sources":["bootstrap.css","less/normalize.less","less/print.less","less/glyphicons.less","less/scaffolding.less","less/mixins/vendor-prefixes.less","less/mixins/tab-focus.less","less/mixins/image.less","less/type.less","less/mixins/text-emphasis.less","less/mixins/background-variant.less","less/mixins/text-overflow.less","less/code.less","less/grid.less","less/mixins/grid.less","less/mixins/grid-framework.less","less/tables.less","less/mixins/table-row.less","less/forms.less","less/mixins/forms.less","less/buttons.less","less/mixins/buttons.less","less/mixins/opacity.less","less/component-animations.less","less/dropdowns.less","less/mixins/nav-divider.less","less/mixins/reset-filter.less","less/button-groups.less","less/mixins/border-radius.less","less/input-groups.less","less/navs.less","less/navbar.less","less/mixins/nav-vertical-align.less","less/utilities.less","less/breadcrumbs.less","less/pagination.less","less/mixins/pagination.less","less/pager.less","less/labels.less","less/mixins/labels.less","less/badges.less","less/jumbotron.less","less/thumbnails.less","less/alerts.less","less/mixins/alerts.less","less/progress-bars.less","less/mixins/gradients.less","less/mixins/progress-bar.less","less/media.less","less/list-group.less","less/mixins/list-group.less","less/panels.less","less/mixins/panels.less","less/responsive-embed.less","less/wells.less","less/close.less","less/modals.less","less/tooltip.less","less/popovers.less","less/carousel.less","less/mixins/clearfix.less","less/mixins/center-block.less","less/mixins/hide-text.less","less/responsive-utilities.less","less/mixins/responsive-visibility.less"],"names":[],"mappings":"AAAA,6DAA4D;ACQ5D;EACE,yBAAA;EACA,4BAAA;EACA,gCAAA;EDND;ACaD;EACE,WAAA;EDXD;ACwBD;;;;;;;;;;;;;EAaE,gBAAA;EDtBD;AC8BD;;;;EAIE,uBAAA;EACA,0BAAA;ED5BD;ACoCD;EACE,eAAA;EACA,WAAA;EDlCD;AC0CD;;EAEE,eAAA;EDxCD;ACkDD;EACE,+BAAA;EDhDD;ACuDD;;EAEE,YAAA;EDrDD;AC+DD;EACE,2BAAA;ED7DD;ACoED;;EAEE,mBAAA;EDlED;ACyED;EACE,oBAAA;EDvED;AC+ED;EACE,gBAAA;EACA,kBAAA;ED7ED;ACoFD;EACE,kBAAA;EACA,aAAA;EDlFD;ACyFD;EACE,gBAAA;EDvFD;AC8FD;;EAEE,gBAAA;EACA,gBAAA;EACA,oBAAA;EACA,0BAAA;ED5FD;AC+FD;EACE,aAAA;ED7FD;ACgGD;EACE,iBAAA;ED9FD;ACwGD;EACE,WAAA;EDtGD;AC6GD;EACE,kBAAA;ED3GD;ACqHD;EACE,kBAAA;EDnHD;AC0HD;EACE,8BAAA;EACA,iCAAA;UAAA,yBAAA;EACA,WAAA;EDxHD;AC+HD;EACE,gBAAA;ED7HD;ACoID;;;;EAIE,mCAAA;EACA,gBAAA;EDlID;ACoJD;;;;;EAKE,gBAAA;EACA,eAAA;EACA,WAAA;EDlJD;ACyJD;EACE,mBAAA;EDvJD;ACiKD;;EAEE,sBAAA;ED/JD;AC0KD;;;;EAIE,4BAAA;EACA,iBAAA;EDxKD;AC+KD;;EAEE,iBAAA;ED7KD;ACoLD;;EAEE,WAAA;EACA,YAAA;EDlLD;AC0LD;EACE,qBAAA;EDxLD;ACmMD;;EAEE,gCAAA;KAAA,6BAAA;UAAA,wBAAA;EACA,YAAA;EDjMD;AC0MD;;EAEE,cAAA;EDxMD;ACiND;EACE,+BAAA;EACA,8BAAA;EACA,iCAAA;EACA,yBAAA;ED/MD;ACwND;;EAEE,0BAAA;EDtND;AC6ND;EACE,2BAAA;EACA,eAAA;EACA,gCAAA;ED3ND;ACmOD;EACE,WAAA;EACA,YAAA;EDjOD;ACwOD;EACE,gBAAA;EDtOD;AC8OD;EACE,mBAAA;ED5OD;ACsPD;EACE,2BAAA;EACA,mBAAA;EDpPD;ACuPD;;EAEE,YAAA;EDrPD;AACD,sFAAqF;AE1ErF;EAnGI;;;IAGI,oCAAA;IACA,wBAAA;IACA,qCAAA;YAAA,6BAAA;IACA,8BAAA;IFgLL;EE7KC;;IAEI,4BAAA;IF+KL;EE5KC;IACI,8BAAA;IF8KL;EE3KC;IACI,+BAAA;IF6KL;EExKC;;IAEI,aAAA;IF0KL;EEvKC;;IAEI,wBAAA;IACA,0BAAA;IFyKL;EEtKC;IACI,6BAAA;IFwKL;EErKC;;IAEI,0BAAA;IFuKL;EEpKC;IACI,4BAAA;IFsKL;EEnKC;;;IAGI,YAAA;IACA,WAAA;IFqKL;EElKC;;IAEI,yBAAA;IFoKL;EE7JC;IACI,6BAAA;IF+JL;EE3JC;IACI,eAAA;IF6JL;EE3JC;;IAGQ,mCAAA;IF4JT;EEzJC;IACI,wBAAA;IF2JL;EExJC;IACI,sCAAA;IF0JL;EE3JC;;IAKQ,mCAAA;IF0JT;EEvJC;;IAGQ,mCAAA;IFwJT;EACF;AGpPD;EACE,qCAAA;EACA,uDAAA;EACA,iYAAA;EHsPD;AG9OD;EACE,oBAAA;EACA,UAAA;EACA,uBAAA;EACA,qCAAA;EACA,oBAAA;EACA,qBAAA;EACA,gBAAA;EACA,qCAAA;EACA,oCAAA;EHgPD;AG5OmC;EAAW,gBAAA;EH+O9C;AG9OmC;EAAW,gBAAA;EHiP9C;AG/OmC;;EAAW,kBAAA;EHmP9C;AGlPmC;EAAW,kBAAA;EHqP9C;AGpPmC;EAAW,kBAAA;EHuP9C;AGtPmC;EAAW,kBAAA;EHyP9C;AGxPmC;EAAW,kBAAA;EH2P9C;AG1PmC;EAAW,kBAAA;EH6P9C;AG5PmC;EAAW,kBAAA;EH+P9C;AG9PmC;EAAW,kBAAA;EHiQ9C;AGhQmC;EAAW,kBAAA;EHmQ9C;AGlQmC;EAAW,kBAAA;EHqQ9C;AGpQmC;EAAW,kBAAA;EHuQ9C;AGtQmC;EAAW,kBAAA;EHyQ9C;AGxQmC;EAAW,kBAAA;EH2Q9C;AG1QmC;EAAW,kBAAA;EH6Q9C;AG5QmC;EAAW,kBAAA;EH+Q9C;AG9QmC;EAAW,kBAAA;EHiR9C;AGhRmC;EAAW,kBAAA;EHmR9C;AGlRmC;EAAW,kBAAA;EHqR9C;AGpRmC;EAAW,kBAAA;EHuR9C;AGtRmC;EAAW,kBAAA;EHyR9C;AGxRmC;EAAW,kBAAA;EH2R9C;AG1RmC;EAAW,kBAAA;EH6R9C;AG5RmC;EAAW,kBAAA;EH+R9C;AG9RmC;EAAW,kBAAA;EHiS9C;AGhSmC;EAAW,kBAAA;EHmS9C;AGlSmC;EAAW,kBAAA;EHqS9C;AGpSmC;EAAW,kBAAA;EHuS9C;AGtSmC;EAAW,kBAAA;EHyS9C;AGxSmC;EAAW,kBAAA;EH2S9C;AG1SmC;EAAW,kBAAA;EH6S9C;AG5SmC;EAAW,kBAAA;EH+S9C;AG9SmC;EAAW,kBAAA;EHiT9C;AGhTmC;EAAW,kBAAA;EHmT9C;AGlTmC;EAAW,kBAAA;EHqT9C;AGpTmC;EAAW,kBAAA;EHuT9C;AGtTmC;EAAW,kBAAA;EHyT9C;AGxTmC;EAAW,kBAAA;EH2T9C;AG1TmC;EAAW,kBAAA;EH6T9C;AG5TmC;EAAW,kBAAA;EH+T9C;AG9TmC;EAAW,kBAAA;EHiU9C;AGhUmC;EAAW,kBAAA;EHmU9C;AGlUmC;EAAW,kBAAA;EHqU9C;AGpUmC;EAAW,kBAAA;EHuU9C;AGtUmC;EAAW,kBAAA;EHyU9C;AGxUmC;EAAW,kBAAA;EH2U9C;AG1UmC;EAAW,kBAAA;EH6U9C;AG5UmC;EAAW,kBAAA;EH+U9C;AG9UmC;EAAW,kBAAA;EHiV9C;AGhVmC;EAAW,kBAAA;EHmV9C;AGlVmC;EAAW,kBAAA;EHqV9C;AGpVmC;EAAW,kBAAA;EHuV9C;AGtVmC;EAAW,kBAAA;EHyV9C;AGxVmC;EAAW,kBAAA;EH2V9C;AG1VmC;EAAW,kBAAA;EH6V9C;AG5VmC;EAAW,kBAAA;EH+V9C;AG9VmC;EAAW,kBAAA;EHiW9C;AGhWmC;EAAW,kBAAA;EHmW9C;AGlWmC;EAAW,kBAAA;EHqW9C;AGpWmC;EAAW,kBAAA;EHuW9C;AGtWmC;EAAW,kBAAA;EHyW9C;AGxWmC;EAAW,kBAAA;EH2W9C;AG1WmC;EAAW,kBAAA;EH6W9C;AG5WmC;EAAW,kBAAA;EH+W9C;AG9WmC;EAAW,kBAAA;EHiX9C;AGhXmC;EAAW,kBAAA;EHmX9C;AGlXmC;EAAW,kBAAA;EHqX9C;AGpXmC;EAAW,kBAAA;EHuX9C;AGtXmC;EAAW,kBAAA;EHyX9C;AGxXmC;EAAW,kBAAA;EH2X9C;AG1XmC;EAAW,kBAAA;EH6X9C;AG5XmC;EAAW,kBAAA;EH+X9C;AG9XmC;EAAW,kBAAA;EHiY9C;AGhYmC;EAAW,kBAAA;EHmY9C;AGlYmC;EAAW,kBAAA;EHqY9C;AGpYmC;EAAW,kBAAA;EHuY9C;AGtYmC;EAAW,kBAAA;EHyY9C;AGxYmC;EAAW,kBAAA;EH2Y9C;AG1YmC;EAAW,kBAAA;EH6Y9C;AG5YmC;EAAW,kBAAA;EH+Y9C;AG9YmC;EAAW,kBAAA;EHiZ9C;AGhZmC;EAAW,kBAAA;EHmZ9C;AGlZmC;EAAW,kBAAA;EHqZ9C;AGpZmC;EAAW,kBAAA;EHuZ9C;AGtZmC;EAAW,kBAAA;EHyZ9C;AGxZmC;EAAW,kBAAA;EH2Z9C;AG1ZmC;EAAW,kBAAA;EH6Z9C;AG5ZmC;EAAW,kBAAA;EH+Z9C;AG9ZmC;EAAW,kBAAA;EHia9C;AGhamC;EAAW,kBAAA;EHma9C;AGlamC;EAAW,kBAAA;EHqa9C;AGpamC;EAAW,kBAAA;EHua9C;AGtamC;EAAW,kBAAA;EHya9C;AGxamC;EAAW,kBAAA;EH2a9C;AG1amC;EAAW,kBAAA;EH6a9C;AG5amC;EAAW,kBAAA;EH+a9C;AG9amC;EAAW,kBAAA;EHib9C;AGhbmC;EAAW,kBAAA;EHmb9C;AGlbmC;EAAW,kBAAA;EHqb9C;AGpbmC;EAAW,kBAAA;EHub9C;AGtbmC;EAAW,kBAAA;EHyb9C;AGxbmC;EAAW,kBAAA;EH2b9C;AG1bmC;EAAW,kBAAA;EH6b9C;AG5bmC;EAAW,kBAAA;EH+b9C;AG9bmC;EAAW,kBAAA;EHic9C;AGhcmC;EAAW,kBAAA;EHmc9C;AGlcmC;EAAW,kBAAA;EHqc9C;AGpcmC;EAAW,kBAAA;EHuc9C;AGtcmC;EAAW,kBAAA;EHyc9C;AGxcmC;EAAW,kBAAA;EH2c9C;AG1cmC;EAAW,kBAAA;EH6c9C;AG5cmC;EAAW,kBAAA;EH+c9C;AG9cmC;EAAW,kBAAA;EHid9C;AGhdmC;EAAW,kBAAA;EHmd9C;AGldmC;EAAW,kBAAA;EHqd9C;AGpdmC;EAAW,kBAAA;EHud9C;AGtdmC;EAAW,kBAAA;EHyd9C;AGxdmC;EAAW,kBAAA;EH2d9C;AG1dmC;EAAW,kBAAA;EH6d9C;AG5dmC;EAAW,kBAAA;EH+d9C;AG9dmC;EAAW,kBAAA;EHie9C;AGhemC;EAAW,kBAAA;EHme9C;AGlemC;EAAW,kBAAA;EHqe9C;AGpemC;EAAW,kBAAA;EHue9C;AGtemC;EAAW,kBAAA;EHye9C;AGxemC;EAAW,kBAAA;EH2e9C;AG1emC;EAAW,kBAAA;EH6e9C;AG5emC;EAAW,kBAAA;EH+e9C;AG9emC;EAAW,kBAAA;EHif9C;AGhfmC;EAAW,kBAAA;EHmf9C;AGlfmC;EAAW,kBAAA;EHqf9C;AGpfmC;EAAW,kBAAA;EHuf9C;AGtfmC;EAAW,kBAAA;EHyf9C;AGxfmC;EAAW,kBAAA;EH2f9C;AG1fmC;EAAW,kBAAA;EH6f9C;AG5fmC;EAAW,kBAAA;EH+f9C;AG9fmC;EAAW,kBAAA;EHigB9C;AGhgBmC;EAAW,kBAAA;EHmgB9C;AGlgBmC;EAAW,kBAAA;EHqgB9C;AGpgBmC;EAAW,kBAAA;EHugB9C;AGtgBmC;EAAW,kBAAA;EHygB9C;AGxgBmC;EAAW,kBAAA;EH2gB9C;AG1gBmC;EAAW,kBAAA;EH6gB9C;AG5gBmC;EAAW,kBAAA;EH+gB9C;AG9gBmC;EAAW,kBAAA;EHihB9C;AGhhBmC;EAAW,kBAAA;EHmhB9C;AGlhBmC;EAAW,kBAAA;EHqhB9C;AGphBmC;EAAW,kBAAA;EHuhB9C;AGthBmC;EAAW,kBAAA;EHyhB9C;AGxhBmC;EAAW,kBAAA;EH2hB9C;AG1hBmC;EAAW,kBAAA;EH6hB9C;AG5hBmC;EAAW,kBAAA;EH+hB9C;AG9hBmC;EAAW,kBAAA;EHiiB9C;AGhiBmC;EAAW,kBAAA;EHmiB9C;AGliBmC;EAAW,kBAAA;EHqiB9C;AGpiBmC;EAAW,kBAAA;EHuiB9C;AGtiBmC;EAAW,kBAAA;EHyiB9C;AGxiBmC;EAAW,kBAAA;EH2iB9C;AG1iBmC;EAAW,kBAAA;EH6iB9C;AG5iBmC;EAAW,kBAAA;EH+iB9C;AG9iBmC;EAAW,kBAAA;EHijB9C;AGhjBmC;EAAW,kBAAA;EHmjB9C;AGljBmC;EAAW,kBAAA;EHqjB9C;AGpjBmC;EAAW,kBAAA;EHujB9C;AGtjBmC;EAAW,kBAAA;EHyjB9C;AGxjBmC;EAAW,kBAAA;EH2jB9C;AG1jBmC;EAAW,kBAAA;EH6jB9C;AG5jBmC;EAAW,kBAAA;EH+jB9C;AG9jBmC;EAAW,kBAAA;EHikB9C;AGhkBmC;EAAW,kBAAA;EHmkB9C;AGlkBmC;EAAW,kBAAA;EHqkB9C;AGpkBmC;EAAW,kBAAA;EHukB9C;AGtkBmC;EAAW,kBAAA;EHykB9C;AGxkBmC;EAAW,kBAAA;EH2kB9C;AG1kBmC;EAAW,kBAAA;EH6kB9C;AG5kBmC;EAAW,kBAAA;EH+kB9C;AG9kBmC;EAAW,kBAAA;EHilB9C;AGhlBmC;EAAW,kBAAA;EHmlB9C;AGllBmC;EAAW,kBAAA;EHqlB9C;AGplBmC;EAAW,kBAAA;EHulB9C;AGtlBmC;EAAW,kBAAA;EHylB9C;AGxlBmC;EAAW,kBAAA;EH2lB9C;AG1lBmC;EAAW,kBAAA;EH6lB9C;AG5lBmC;EAAW,kBAAA;EH+lB9C;AG9lBmC;EAAW,kBAAA;EHimB9C;AGhmBmC;EAAW,kBAAA;EHmmB9C;AGlmBmC;EAAW,kBAAA;EHqmB9C;AGpmBmC;EAAW,kBAAA;EHumB9C;AGtmBmC;EAAW,kBAAA;EHymB9C;AGxmBmC;EAAW,kBAAA;EH2mB9C;AG1mBmC;EAAW,kBAAA;EH6mB9C;AG5mBmC;EAAW,kBAAA;EH+mB9C;AG9mBmC;EAAW,kBAAA;EHinB9C;AGhnBmC;EAAW,kBAAA;EHmnB9C;AGlnBmC;EAAW,kBAAA;EHqnB9C;AGpnBmC;EAAW,kBAAA;EHunB9C;AGtnBmC;EAAW,kBAAA;EHynB9C;AGxnBmC;EAAW,kBAAA;EH2nB9C;AG1nBmC;EAAW,kBAAA;EH6nB9C;AG5nBmC;EAAW,kBAAA;EH+nB9C;AG9nBmC;EAAW,kBAAA;EHioB9C;AGhoBmC;EAAW,kBAAA;EHmoB9C;AGloBmC;EAAW,kBAAA;EHqoB9C;AGpoBmC;EAAW,kBAAA;EHuoB9C;AGtoBmC;EAAW,kBAAA;EHyoB9C;AGhoBmC;EAAW,kBAAA;EHmoB9C;AGloBmC;EAAW,kBAAA;EHqoB9C;AGpoBmC;EAAW,kBAAA;EHuoB9C;AGtoBmC;EAAW,kBAAA;EHyoB9C;AGxoBmC;EAAW,kBAAA;EH2oB9C;AG1oBmC;EAAW,kBAAA;EH6oB9C;AG5oBmC;EAAW,kBAAA;EH+oB9C;AG9oBmC;EAAW,kBAAA;EHipB9C;AGhpBmC;EAAW,kBAAA;EHmpB9C;AGlpBmC;EAAW,kBAAA;EHqpB9C;AGppBmC;EAAW,kBAAA;EHupB9C;AGtpBmC;EAAW,kBAAA;EHypB9C;AGxpBmC;EAAW,kBAAA;EH2pB9C;AG1pBmC;EAAW,kBAAA;EH6pB9C;AG5pBmC;EAAW,kBAAA;EH+pB9C;AG9pBmC;EAAW,kBAAA;EHiqB9C;AGhqBmC;EAAW,kBAAA;EHmqB9C;AGlqBmC;EAAW,kBAAA;EHqqB9C;AGpqBmC;EAAW,kBAAA;EHuqB9C;AGtqBmC;EAAW,kBAAA;EHyqB9C;AGxqBmC;EAAW,kBAAA;EH2qB9C;AG1qBmC;EAAW,kBAAA;EH6qB9C;AG5qBmC;EAAW,kBAAA;EH+qB9C;AG9qBmC;EAAW,kBAAA;EHirB9C;AGhrBmC;EAAW,kBAAA;EHmrB9C;AGlrBmC;EAAW,kBAAA;EHqrB9C;AGprBmC;EAAW,kBAAA;EHurB9C;AGtrBmC;EAAW,kBAAA;EHyrB9C;AGxrBmC;EAAW,kBAAA;EH2rB9C;AG1rBmC;EAAW,kBAAA;EH6rB9C;AG5rBmC;EAAW,kBAAA;EH+rB9C;AG9rBmC;EAAW,kBAAA;EHisB9C;AGhsBmC;EAAW,kBAAA;EHmsB9C;AGlsBmC;EAAW,kBAAA;EHqsB9C;AGpsBmC;EAAW,kBAAA;EHusB9C;AGtsBmC;EAAW,kBAAA;EHysB9C;AGxsBmC;EAAW,kBAAA;EH2sB9C;AG1sBmC;EAAW,kBAAA;EH6sB9C;AG5sBmC;EAAW,kBAAA;EH+sB9C;AG9sBmC;EAAW,kBAAA;EHitB9C;AGhtBmC;EAAW,kBAAA;EHmtB9C;AGltBmC;EAAW,kBAAA;EHqtB9C;AGptBmC;EAAW,kBAAA;EHutB9C;AGttBmC;EAAW,kBAAA;EHytB9C;AGxtBmC;EAAW,kBAAA;EH2tB9C;AG1tBmC;EAAW,kBAAA;EH6tB9C;AG5tBmC;EAAW,kBAAA;EH+tB9C;AG9tBmC;EAAW,kBAAA;EHiuB9C;AGhuBmC;EAAW,kBAAA;EHmuB9C;AGluBmC;EAAW,kBAAA;EHquB9C;AGpuBmC;EAAW,kBAAA;EHuuB9C;AGtuBmC;EAAW,kBAAA;EHyuB9C;AGxuBmC;EAAW,kBAAA;EH2uB9C;AG1uBmC;EAAW,kBAAA;EH6uB9C;AG5uBmC;EAAW,kBAAA;EH+uB9C;AG9uBmC;EAAW,kBAAA;EHivB9C;AIvhCD;ECgEE,gCAAA;EACG,6BAAA;EACK,wBAAA;EL09BT;AIzhCD;;EC6DE,gCAAA;EACG,6BAAA;EACK,wBAAA;ELg+BT;AIvhCD;EACE,iBAAA;EACA,+CAAA;EJyhCD;AIthCD;EACE,6DAAA;EACA,iBAAA;EACA,yBAAA;EACA,gBAAA;EACA,2BAAA;EJwhCD;AIphCD;;;;EAIE,sBAAA;EACA,oBAAA;EACA,sBAAA;EJshCD;AIhhCD;EACE,gBAAA;EACA,uBAAA;EJkhCD;AIhhCC;;EAEE,gBAAA;EACA,4BAAA;EJkhCH;AI/gCC;EErDA,sBAAA;EAEA,4CAAA;EACA,sBAAA;ENskCD;AIzgCD;EACE,WAAA;EJ2gCD;AIrgCD;EACE,wBAAA;EJugCD;AIngCD;;;;;EGvEE,gBAAA;EACA,iBAAA;EACA,cAAA;EPilCD;AIvgCD;EACE,oBAAA;EJygCD;AIngCD;EACE,cAAA;EACA,yBAAA;EACA,2BAAA;EACA,2BAAA;EACA,oBAAA;EC6FA,0CAAA;EACK,qCAAA;EACG,kCAAA;EEvLR,uBAAA;EACA,iBAAA;EACA,cAAA;EPimCD;AIngCD;EACE,oBAAA;EJqgCD;AI//BD;EACE,kBAAA;EACA,qBAAA;EACA,WAAA;EACA,+BAAA;EJigCD;AIz/BD;EACE,oBAAA;EACA,YAAA;EACA,aAAA;EACA,cAAA;EACA,YAAA;EACA,kBAAA;EACA,wBAAA;EACA,WAAA;EJ2/BD;AIn/BC;;EAEE,kBAAA;EACA,aAAA;EACA,cAAA;EACA,WAAA;EACA,mBAAA;EACA,YAAA;EJq/BH;AIz+BD;EACE,iBAAA;EJ2+BD;AQnoCD;;;;;;;;;;;;EAEE,sBAAA;EACA,kBAAA;EACA,kBAAA;EACA,gBAAA;ER+oCD;AQppCD;;;;;;;;;;;;;;;;;;;;;;;;EASI,qBAAA;EACA,gBAAA;EACA,gBAAA;ERqqCH;AQjqCD;;;;;;EAGE,kBAAA;EACA,qBAAA;ERsqCD;AQ1qCD;;;;;;;;;;;;EAQI,gBAAA;ERgrCH;AQ7qCD;;;;;;EAGE,kBAAA;EACA,qBAAA;ERkrCD;AQtrCD;;;;;;;;;;;;EAQI,gBAAA;ER4rCH;AQxrCD;;EAAU,iBAAA;ER4rCT;AQ3rCD;;EAAU,iBAAA;ER+rCT;AQ9rCD;;EAAU,iBAAA;ERksCT;AQjsCD;;EAAU,iBAAA;ERqsCT;AQpsCD;;EAAU,iBAAA;ERwsCT;AQvsCD;;EAAU,iBAAA;ER2sCT;AQrsCD;EACE,kBAAA;ERusCD;AQpsCD;EACE,qBAAA;EACA,iBAAA;EACA,kBAAA;EACA,kBAAA;ERssCD;AQjsCD;EAAA;IAFI,iBAAA;IRusCD;EACF;AQ/rCD;;EAEE,gBAAA;ERisCD;AQ9rCD;;EAEE,2BAAA;EACA,eAAA;ERgsCD;AQ5rCD;EAAuB,kBAAA;ER+rCtB;AQ9rCD;EAAuB,mBAAA;ERisCtB;AQhsCD;EAAuB,oBAAA;ERmsCtB;AQlsCD;EAAuB,qBAAA;ERqsCtB;AQpsCD;EAAuB,qBAAA;ERusCtB;AQpsCD;EAAuB,2BAAA;ERusCtB;AQtsCD;EAAuB,2BAAA;ERysCtB;AQxsCD;EAAuB,4BAAA;ER2sCtB;AQxsCD;EACE,gBAAA;ER0sCD;AQxsCD;ECrGE,gBAAA;ETgzCD;AS/yCC;EACE,gBAAA;ETizCH;AQ3sCD;ECxGE,gBAAA;ETszCD;ASrzCC;EACE,gBAAA;ETuzCH;AQ9sCD;EC3GE,gBAAA;ET4zCD;AS3zCC;EACE,gBAAA;ET6zCH;AQjtCD;EC9GE,gBAAA;ETk0CD;ASj0CC;EACE,gBAAA;ETm0CH;AQptCD;ECjHE,gBAAA;ETw0CD;ASv0CC;EACE,gBAAA;ETy0CH;AQntCD;EAGE,aAAA;EE3HA,2BAAA;EV+0CD;AU90CC;EACE,2BAAA;EVg1CH;AQptCD;EE9HE,2BAAA;EVq1CD;AUp1CC;EACE,2BAAA;EVs1CH;AQvtCD;EEjIE,2BAAA;EV21CD;AU11CC;EACE,2BAAA;EV41CH;AQ1tCD;EEpIE,2BAAA;EVi2CD;AUh2CC;EACE,2BAAA;EVk2CH;AQ7tCD;EEvIE,2BAAA;EVu2CD;AUt2CC;EACE,2BAAA;EVw2CH;AQ3tCD;EACE,qBAAA;EACA,qBAAA;EACA,kCAAA;ER6tCD;AQrtCD;;EAEE,eAAA;EACA,qBAAA;ERutCD;AQ1tCD;;;;EAMI,kBAAA;ER0tCH;AQntCD;EACE,iBAAA;EACA,kBAAA;ERqtCD;AQjtCD;EALE,iBAAA;EACA,kBAAA;EAMA,mBAAA;ERotCD;AQttCD;EAKI,uBAAA;EACA,mBAAA;EACA,oBAAA;ERotCH;AQ/sCD;EACE,eAAA;EACA,qBAAA;ERitCD;AQ/sCD;;EAEE,yBAAA;ERitCD;AQ/sCD;EACE,mBAAA;ERitCD;AQ/sCD;EACE,gBAAA;ERitCD;AQxrCD;EAAA;IAVM,aAAA;IACA,cAAA;IACA,aAAA;IACA,mBAAA;IGtNJ,kBAAA;IACA,yBAAA;IACA,qBAAA;IX65CC;EQlsCH;IAHM,oBAAA;IRwsCH;EACF;AQ/rCD;;EAGE,cAAA;EACA,mCAAA;ERgsCD;AQ9rCD;EACE,gBAAA;EA9IqB,2BAAA;ER+0CtB;AQ5rCD;EACE,oBAAA;EACA,kBAAA;EACA,mBAAA;EACA,gCAAA;ER8rCD;AQzrCG;;;EACE,kBAAA;ER6rCL;AQvsCD;;;EAmBI,gBAAA;EACA,gBAAA;EACA,yBAAA;EACA,gBAAA;ERyrCH;AQvrCG;;;EACE,wBAAA;ER2rCL;AQnrCD;;EAEE,qBAAA;EACA,iBAAA;EACA,iCAAA;EACA,gBAAA;EACA,mBAAA;ERqrCD;AQ/qCG;;;;;;EAAW,aAAA;ERurCd;AQtrCG;;;;;;EACE,wBAAA;ER6rCL;AQvrCD;EACE,qBAAA;EACA,oBAAA;EACA,yBAAA;ERyrCD;AY/9CD;;;;EAIE,gEAAA;EZi+CD;AY79CD;EACE,kBAAA;EACA,gBAAA;EACA,gBAAA;EACA,2BAAA;EACA,oBAAA;EZ+9CD;AY39CD;EACE,kBAAA;EACA,gBAAA;EACA,gBAAA;EACA,2BAAA;EACA,oBAAA;EACA,wDAAA;UAAA,gDAAA;EZ69CD;AYn+CD;EASI,YAAA;EACA,iBAAA;EACA,mBAAA;EACA,0BAAA;UAAA,kBAAA;EZ69CH;AYx9CD;EACE,gBAAA;EACA,gBAAA;EACA,kBAAA;EACA,iBAAA;EACA,yBAAA;EACA,uBAAA;EACA,uBAAA;EACA,gBAAA;EACA,2BAAA;EACA,2BAAA;EACA,oBAAA;EZ09CD;AYr+CD;EAeI,YAAA;EACA,oBAAA;EACA,gBAAA;EACA,uBAAA;EACA,+BAAA;EACA,kBAAA;EZy9CH;AYp9CD;EACE,mBAAA;EACA,oBAAA;EZs9CD;AahhDD;ECHE,oBAAA;EACA,mBAAA;EACA,oBAAA;EACA,qBAAA;EdshDD;AahhDC;EAAA;IAFE,cAAA;IbshDD;EACF;AalhDC;EAAA;IAFE,cAAA;IbwhDD;EACF;AaphDD;EAAA;IAFI,eAAA;Ib0hDD;EACF;AajhDD;ECvBE,oBAAA;EACA,mBAAA;EACA,oBAAA;EACA,qBAAA;Ed2iDD;Aa9gDD;ECvBE,oBAAA;EACA,qBAAA;EdwiDD;AexiDG;EACE,oBAAA;EAEA,iBAAA;EAEA,oBAAA;EACA,qBAAA;EfwiDL;AexhDG;EACE,aAAA;Ef0hDL;AenhDC;EACE,aAAA;EfqhDH;AethDC;EACE,qBAAA;EfwhDH;AezhDC;EACE,qBAAA;Ef2hDH;Ae5hDC;EACE,YAAA;Ef8hDH;Ae/hDC;EACE,qBAAA;EfiiDH;AeliDC;EACE,qBAAA;EfoiDH;AeriDC;EACE,YAAA;EfuiDH;AexiDC;EACE,qBAAA;Ef0iDH;Ae3iDC;EACE,qBAAA;Ef6iDH;Ae9iDC;EACE,YAAA;EfgjDH;AejjDC;EACE,qBAAA;EfmjDH;AepjDC;EACE,oBAAA;EfsjDH;AexiDC;EACE,aAAA;Ef0iDH;Ae3iDC;EACE,qBAAA;Ef6iDH;Ae9iDC;EACE,qBAAA;EfgjDH;AejjDC;EACE,YAAA;EfmjDH;AepjDC;EACE,qBAAA;EfsjDH;AevjDC;EACE,qBAAA;EfyjDH;Ae1jDC;EACE,YAAA;Ef4jDH;Ae7jDC;EACE,qBAAA;Ef+jDH;AehkDC;EACE,qBAAA;EfkkDH;AenkDC;EACE,YAAA;EfqkDH;AetkDC;EACE,qBAAA;EfwkDH;AezkDC;EACE,oBAAA;Ef2kDH;AevkDC;EACE,aAAA;EfykDH;AezlDC;EACE,YAAA;Ef2lDH;Ae5lDC;EACE,oBAAA;Ef8lDH;Ae/lDC;EACE,oBAAA;EfimDH;AelmDC;EACE,WAAA;EfomDH;AermDC;EACE,oBAAA;EfumDH;AexmDC;EACE,oBAAA;Ef0mDH;Ae3mDC;EACE,WAAA;Ef6mDH;Ae9mDC;EACE,oBAAA;EfgnDH;AejnDC;EACE,oBAAA;EfmnDH;AepnDC;EACE,WAAA;EfsnDH;AevnDC;EACE,oBAAA;EfynDH;Ae1nDC;EACE,mBAAA;Ef4nDH;AexnDC;EACE,YAAA;Ef0nDH;Ae5mDC;EACE,mBAAA;Ef8mDH;Ae/mDC;EACE,2BAAA;EfinDH;AelnDC;EACE,2BAAA;EfonDH;AernDC;EACE,kBAAA;EfunDH;AexnDC;EACE,2BAAA;Ef0nDH;Ae3nDC;EACE,2BAAA;Ef6nDH;Ae9nDC;EACE,kBAAA;EfgoDH;AejoDC;EACE,2BAAA;EfmoDH;AepoDC;EACE,2BAAA;EfsoDH;AevoDC;EACE,kBAAA;EfyoDH;Ae1oDC;EACE,2BAAA;Ef4oDH;Ae7oDC;EACE,0BAAA;Ef+oDH;AehpDC;EACE,iBAAA;EfkpDH;AalpDD;EElCI;IACE,aAAA;IfurDH;EehrDD;IACE,aAAA;IfkrDD;EenrDD;IACE,qBAAA;IfqrDD;EetrDD;IACE,qBAAA;IfwrDD;EezrDD;IACE,YAAA;If2rDD;Ee5rDD;IACE,qBAAA;If8rDD;Ee/rDD;IACE,qBAAA;IfisDD;EelsDD;IACE,YAAA;IfosDD;EersDD;IACE,qBAAA;IfusDD;EexsDD;IACE,qBAAA;If0sDD;Ee3sDD;IACE,YAAA;If6sDD;Ee9sDD;IACE,qBAAA;IfgtDD;EejtDD;IACE,oBAAA;IfmtDD;EersDD;IACE,aAAA;IfusDD;EexsDD;IACE,qBAAA;If0sDD;Ee3sDD;IACE,qBAAA;If6sDD;Ee9sDD;IACE,YAAA;IfgtDD;EejtDD;IACE,qBAAA;IfmtDD;EeptDD;IACE,qBAAA;IfstDD;EevtDD;IACE,YAAA;IfytDD;Ee1tDD;IACE,qBAAA;If4tDD;Ee7tDD;IACE,qBAAA;If+tDD;EehuDD;IACE,YAAA;IfkuDD;EenuDD;IACE,qBAAA;IfquDD;EetuDD;IACE,oBAAA;IfwuDD;EepuDD;IACE,aAAA;IfsuDD;EetvDD;IACE,YAAA;IfwvDD;EezvDD;IACE,oBAAA;If2vDD;Ee5vDD;IACE,oBAAA;If8vDD;Ee/vDD;IACE,WAAA;IfiwDD;EelwDD;IACE,oBAAA;IfowDD;EerwDD;IACE,oBAAA;IfuwDD;EexwDD;IACE,WAAA;If0wDD;Ee3wDD;IACE,oBAAA;If6wDD;Ee9wDD;IACE,oBAAA;IfgxDD;EejxDD;IACE,WAAA;IfmxDD;EepxDD;IACE,oBAAA;IfsxDD;EevxDD;IACE,mBAAA;IfyxDD;EerxDD;IACE,YAAA;IfuxDD;EezwDD;IACE,mBAAA;If2wDD;Ee5wDD;IACE,2BAAA;If8wDD;Ee/wDD;IACE,2BAAA;IfixDD;EelxDD;IACE,kBAAA;IfoxDD;EerxDD;IACE,2BAAA;IfuxDD;EexxDD;IACE,2BAAA;If0xDD;Ee3xDD;IACE,kBAAA;If6xDD;Ee9xDD;IACE,2BAAA;IfgyDD;EejyDD;IACE,2BAAA;IfmyDD;EepyDD;IACE,kBAAA;IfsyDD;EevyDD;IACE,2BAAA;IfyyDD;Ee1yDD;IACE,0BAAA;If4yDD;Ee7yDD;IACE,iBAAA;If+yDD;EACF;AavyDD;EE3CI;IACE,aAAA;Ifq1DH;Ee90DD;IACE,aAAA;Ifg1DD;Eej1DD;IACE,qBAAA;Ifm1DD;Eep1DD;IACE,qBAAA;Ifs1DD;Eev1DD;IACE,YAAA;Ify1DD;Ee11DD;IACE,qBAAA;If41DD;Ee71DD;IACE,qBAAA;If+1DD;Eeh2DD;IACE,YAAA;Ifk2DD;Een2DD;IACE,qBAAA;Ifq2DD;Eet2DD;IACE,qBAAA;Ifw2DD;Eez2DD;IACE,YAAA;If22DD;Ee52DD;IACE,qBAAA;If82DD;Ee/2DD;IACE,oBAAA;Ifi3DD;Een2DD;IACE,aAAA;Ifq2DD;Eet2DD;IACE,qBAAA;Ifw2DD;Eez2DD;IACE,qBAAA;If22DD;Ee52DD;IACE,YAAA;If82DD;Ee/2DD;IACE,qBAAA;Ifi3DD;Eel3DD;IACE,qBAAA;Ifo3DD;Eer3DD;IACE,YAAA;Ifu3DD;Eex3DD;IACE,qBAAA;If03DD;Ee33DD;IACE,qBAAA;If63DD;Ee93DD;IACE,YAAA;Ifg4DD;Eej4DD;IACE,qBAAA;Ifm4DD;Eep4DD;IACE,oBAAA;Ifs4DD;Eel4DD;IACE,aAAA;Ifo4DD;Eep5DD;IACE,YAAA;Ifs5DD;Eev5DD;IACE,oBAAA;Ify5DD;Ee15DD;IACE,oBAAA;If45DD;Ee75DD;IACE,WAAA;If+5DD;Eeh6DD;IACE,oBAAA;Ifk6DD;Een6DD;IACE,oBAAA;Ifq6DD;Eet6DD;IACE,WAAA;Ifw6DD;Eez6DD;IACE,oBAAA;If26DD;Ee56DD;IACE,oBAAA;If86DD;Ee/6DD;IACE,WAAA;Ifi7DD;Eel7DD;IACE,oBAAA;Ifo7DD;Eer7DD;IACE,mBAAA;Ifu7DD;Een7DD;IACE,YAAA;Ifq7DD;Eev6DD;IACE,mBAAA;Ify6DD;Ee16DD;IACE,2BAAA;If46DD;Ee76DD;IACE,2BAAA;If+6DD;Eeh7DD;IACE,kBAAA;Ifk7DD;Een7DD;IACE,2BAAA;Ifq7DD;Eet7DD;IACE,2BAAA;Ifw7DD;Eez7DD;IACE,kBAAA;If27DD;Ee57DD;IACE,2BAAA;If87DD;Ee/7DD;IACE,2BAAA;Ifi8DD;Eel8DD;IACE,kBAAA;Ifo8DD;Eer8DD;IACE,2BAAA;Ifu8DD;Eex8DD;IACE,0BAAA;If08DD;Ee38DD;IACE,iBAAA;If68DD;EACF;Aal8DD;EE9CI;IACE,aAAA;Ifm/DH;Ee5+DD;IACE,aAAA;If8+DD;Ee/+DD;IACE,qBAAA;Ifi/DD;Eel/DD;IACE,qBAAA;Ifo/DD;Eer/DD;IACE,YAAA;Ifu/DD;Eex/DD;IACE,qBAAA;If0/DD;Ee3/DD;IACE,qBAAA;If6/DD;Ee9/DD;IACE,YAAA;IfggED;EejgED;IACE,qBAAA;IfmgED;EepgED;IACE,qBAAA;IfsgED;EevgED;IACE,YAAA;IfygED;Ee1gED;IACE,qBAAA;If4gED;Ee7gED;IACE,oBAAA;If+gED;EejgED;IACE,aAAA;IfmgED;EepgED;IACE,qBAAA;IfsgED;EevgED;IACE,qBAAA;IfygED;Ee1gED;IACE,YAAA;If4gED;Ee7gED;IACE,qBAAA;If+gED;EehhED;IACE,qBAAA;IfkhED;EenhED;IACE,YAAA;IfqhED;EethED;IACE,qBAAA;IfwhED;EezhED;IACE,qBAAA;If2hED;Ee5hED;IACE,YAAA;If8hED;Ee/hED;IACE,qBAAA;IfiiED;EeliED;IACE,oBAAA;IfoiED;EehiED;IACE,aAAA;IfkiED;EeljED;IACE,YAAA;IfojED;EerjED;IACE,oBAAA;IfujED;EexjED;IACE,oBAAA;If0jED;Ee3jED;IACE,WAAA;If6jED;Ee9jED;IACE,oBAAA;IfgkED;EejkED;IACE,oBAAA;IfmkED;EepkED;IACE,WAAA;IfskED;EevkED;IACE,oBAAA;IfykED;Ee1kED;IACE,oBAAA;If4kED;Ee7kED;IACE,WAAA;If+kED;EehlED;IACE,oBAAA;IfklED;EenlED;IACE,mBAAA;IfqlED;EejlED;IACE,YAAA;IfmlED;EerkED;IACE,mBAAA;IfukED;EexkED;IACE,2BAAA;If0kED;Ee3kED;IACE,2BAAA;If6kED;Ee9kED;IACE,kBAAA;IfglED;EejlED;IACE,2BAAA;IfmlED;EeplED;IACE,2BAAA;IfslED;EevlED;IACE,kBAAA;IfylED;Ee1lED;IACE,2BAAA;If4lED;Ee7lED;IACE,2BAAA;If+lED;EehmED;IACE,kBAAA;IfkmED;EenmED;IACE,2BAAA;IfqmED;EetmED;IACE,0BAAA;IfwmED;EezmED;IACE,iBAAA;If2mED;EACF;AgB/qED;EACE,+BAAA;EhBirED;AgB/qED;EACE,kBAAA;EACA,qBAAA;EACA,gBAAA;EACA,kBAAA;EhBirED;AgB/qED;EACE,kBAAA;EhBirED;AgB3qED;EACE,aAAA;EACA,iBAAA;EACA,qBAAA;EhB6qED;AgBhrED;;;;;;EAWQ,cAAA;EACA,yBAAA;EACA,qBAAA;EACA,+BAAA;EhB6qEP;AgB3rED;EAoBI,wBAAA;EACA,kCAAA;EhB0qEH;AgB/rED;;;;;;EA8BQ,eAAA;EhByqEP;AgBvsED;EAoCI,+BAAA;EhBsqEH;AgB1sED;EAyCI,2BAAA;EhBoqEH;AgB7pED;;;;;;EAOQ,cAAA;EhB8pEP;AgBnpED;EACE,2BAAA;EhBqpED;AgBtpED;;;;;;EAQQ,2BAAA;EhBspEP;AgB9pED;;EAeM,0BAAA;EhBmpEL;AgBzoED;EAEI,2BAAA;EhB0oEH;AgBjoED;EAEI,2BAAA;EhBkoEH;AgBznED;EACE,kBAAA;EACA,aAAA;EACA,uBAAA;EhB2nED;AgBtnEG;;EACE,kBAAA;EACA,aAAA;EACA,qBAAA;EhBynEL;AiBrwEC;;;;;;;;;;;;EAOI,2BAAA;EjB4wEL;AiBtwEC;;;;;EAMI,2BAAA;EjBuwEL;AiB1xEC;;;;;;;;;;;;EAOI,2BAAA;EjBiyEL;AiB3xEC;;;;;EAMI,2BAAA;EjB4xEL;AiB/yEC;;;;;;;;;;;;EAOI,2BAAA;EjBszEL;AiBhzEC;;;;;EAMI,2BAAA;EjBizEL;AiBp0EC;;;;;;;;;;;;EAOI,2BAAA;EjB20EL;AiBr0EC;;;;;EAMI,2BAAA;EjBs0EL;AiBz1EC;;;;;;;;;;;;EAOI,2BAAA;EjBg2EL;AiB11EC;;;;;EAMI,2BAAA;EjB21EL;AgBzsED;EACE,kBAAA;EACA,mBAAA;EhB2sED;AgB9oED;EAAA;IA1DI,aAAA;IACA,qBAAA;IACA,oBAAA;IACA,8CAAA;IACA,2BAAA;IhB4sED;EgBtpEH;IAlDM,kBAAA;IhB2sEH;EgBzpEH;;;;;;IAzCY,qBAAA;IhB0sET;EgBjqEH;IAjCM,WAAA;IhBqsEH;EgBpqEH;;;;;;IAxBY,gBAAA;IhBosET;EgB5qEH;;;;;;IApBY,iBAAA;IhBwsET;EgBprEH;;;;IAPY,kBAAA;IhBisET;EACF;AkB35ED;EACE,YAAA;EACA,WAAA;EACA,WAAA;EAIA,cAAA;ElB05ED;AkBv5ED;EACE,gBAAA;EACA,aAAA;EACA,YAAA;EACA,qBAAA;EACA,iBAAA;EACA,sBAAA;EACA,gBAAA;EACA,WAAA;EACA,kCAAA;ElBy5ED;AkBt5ED;EACE,uBAAA;EACA,iBAAA;EACA,oBAAA;EACA,mBAAA;ElBw5ED;AkB74ED;Eb4BE,gCAAA;EACG,6BAAA;EACK,wBAAA;ELo3ET;AkB74ED;;EAEE,iBAAA;EACA,oBAAA;EACA,qBAAA;ElB+4ED;AkB34ED;EACE,gBAAA;ElB64ED;AkBz4ED;EACE,gBAAA;EACA,aAAA;ElB24ED;AkBv4ED;;EAEE,cAAA;ElBy4ED;AkBr4ED;;;EZxEE,sBAAA;EAEA,4CAAA;EACA,sBAAA;ENi9ED;AkBr4ED;EACE,gBAAA;EACA,kBAAA;EACA,iBAAA;EACA,yBAAA;EACA,gBAAA;ElBu4ED;AkB72ED;EACE,gBAAA;EACA,aAAA;EACA,cAAA;EACA,mBAAA;EACA,iBAAA;EACA,yBAAA;EACA,gBAAA;EACA,2BAAA;EACA,wBAAA;EACA,2BAAA;EACA,oBAAA;EbzDA,0DAAA;EACQ,kDAAA;EAyHR,wFAAA;EACK,2EAAA;EACG,wEAAA;ELizET;AmBz7EC;EACE,uBAAA;EACA,YAAA;EdUF,wFAAA;EACQ,gFAAA;ELk7ET;AKj5EC;EACE,gBAAA;EACA,YAAA;ELm5EH;AKj5EC;EAA0B,gBAAA;ELo5E3B;AKn5EC;EAAgC,gBAAA;ELs5EjC;AkBr3EC;;;EAGE,2BAAA;EACA,YAAA;ElBu3EH;AkBp3EC;;EAEE,qBAAA;ElBs3EH;AkBl3EC;EACE,cAAA;ElBo3EH;AkBx2ED;EACE,0BAAA;ElB02ED;AkBt0ED;EAxBE;;;;IAIE,mBAAA;IlBi2ED;EkB/1EC;;;;;;;;IAEE,mBAAA;IlBu2EH;EkBp2EC;;;;;;;;IAEE,mBAAA;IlB42EH;EACF;AkBl2ED;EACE,qBAAA;ElBo2ED;AkB51ED;;EAEE,oBAAA;EACA,gBAAA;EACA,kBAAA;EACA,qBAAA;ElB81ED;AkBn2ED;;EAQI,kBAAA;EACA,oBAAA;EACA,kBAAA;EACA,qBAAA;EACA,iBAAA;ElB+1EH;AkB51ED;;;;EAIE,oBAAA;EACA,oBAAA;EACA,oBAAA;ElB81ED;AkB31ED;;EAEE,kBAAA;ElB61ED;AkBz1ED;;EAEE,oBAAA;EACA,uBAAA;EACA,oBAAA;EACA,kBAAA;EACA,wBAAA;EACA,qBAAA;EACA,iBAAA;ElB21ED;AkBz1ED;;EAEE,eAAA;EACA,mBAAA;ElB21ED;AkBl1EC;;;;;;EAGE,qBAAA;ElBu1EH;AkBj1EC;;;;EAEE,qBAAA;ElBq1EH;AkB/0EC;;;;EAGI,qBAAA;ElBk1EL;AkBv0ED;EAEE,kBAAA;EACA,qBAAA;EAEA,kBAAA;EACA,kBAAA;ElBu0ED;AkBr0EC;;EAEE,iBAAA;EACA,kBAAA;ElBu0EH;AkB1zED;EC1PE,cAAA;EACA,mBAAA;EACA,iBAAA;EACA,kBAAA;EACA,oBAAA;EnBujFD;AmBrjFC;EACE,cAAA;EACA,mBAAA;EnBujFH;AmBpjFC;;EAEE,cAAA;EnBsjFH;AkBt0ED;EC7PE,cAAA;EACA,mBAAA;EACA,iBAAA;EACA,kBAAA;EACA,oBAAA;EnBskFD;AmBpkFC;EACE,cAAA;EACA,mBAAA;EnBskFH;AmBnkFC;;EAEE,cAAA;EnBqkFH;AkBr1ED;EAKI,cAAA;EACA,mBAAA;EACA,iBAAA;EACA,kBAAA;EACA,kBAAA;ElBm1EH;AkB/0ED;EC1QE,cAAA;EACA,oBAAA;EACA,iBAAA;EACA,wBAAA;EACA,oBAAA;EnB4lFD;AmB1lFC;EACE,cAAA;EACA,mBAAA;EnB4lFH;AmBzlFC;;EAEE,cAAA;EnB2lFH;AkB31ED;EC7QE,cAAA;EACA,oBAAA;EACA,iBAAA;EACA,wBAAA;EACA,oBAAA;EnB2mFD;AmBzmFC;EACE,cAAA;EACA,mBAAA;EnB2mFH;AmBxmFC;;EAEE,cAAA;EnB0mFH;AkB12ED;EAKI,cAAA;EACA,oBAAA;EACA,iBAAA;EACA,wBAAA;EACA,kBAAA;ElBw2EH;AkB/1ED;EAEE,oBAAA;ElBg2ED;AkBl2ED;EAMI,uBAAA;ElB+1EH;AkB31ED;EACE,oBAAA;EACA,QAAA;EACA,UAAA;EACA,YAAA;EACA,gBAAA;EACA,aAAA;EACA,cAAA;EACA,mBAAA;EACA,oBAAA;EACA,sBAAA;ElB61ED;AkB31ED;EACE,aAAA;EACA,cAAA;EACA,mBAAA;ElB61ED;AkB31ED;EACE,aAAA;EACA,cAAA;EACA,mBAAA;ElB61ED;AkBz1ED;;;;;;;;;;ECrXI,gBAAA;EnB0tFH;AkBr2ED;ECjXI,uBAAA;Ed+CF,0DAAA;EACQ,kDAAA;EL2qFT;AmBztFG;EACE,uBAAA;Ed4CJ,2EAAA;EACQ,mEAAA;ELgrFT;AkB/2ED;ECvWI,gBAAA;EACA,uBAAA;EACA,2BAAA;EnBytFH;AkBp3ED;ECjWI,gBAAA;EnBwtFH;AkBp3ED;;;;;;;;;;ECxXI,gBAAA;EnBwvFH;AkBh4ED;ECpXI,uBAAA;Ed+CF,0DAAA;EACQ,kDAAA;ELysFT;AmBvvFG;EACE,uBAAA;Ed4CJ,2EAAA;EACQ,mEAAA;EL8sFT;AkB14ED;EC1WI,gBAAA;EACA,uBAAA;EACA,2BAAA;EnBuvFH;AkB/4ED;ECpWI,gBAAA;EnBsvFH;AkB/4ED;;;;;;;;;;EC3XI,gBAAA;EnBsxFH;AkB35ED;ECvXI,uBAAA;Ed+CF,0DAAA;EACQ,kDAAA;ELuuFT;AmBrxFG;EACE,uBAAA;Ed4CJ,2EAAA;EACQ,mEAAA;EL4uFT;AkBr6ED;EC7WI,gBAAA;EACA,uBAAA;EACA,2BAAA;EnBqxFH;AkB16ED;ECvWI,gBAAA;EnBoxFH;AkBt6EC;EACG,WAAA;ElBw6EJ;AkBt6EC;EACG,QAAA;ElBw6EJ;AkB95ED;EACE,gBAAA;EACA,iBAAA;EACA,qBAAA;EACA,gBAAA;ElBg6ED;AkB70ED;EAAA;IA9DM,uBAAA;IACA,kBAAA;IACA,wBAAA;IlB+4EH;EkBn1EH;IAvDM,uBAAA;IACA,aAAA;IACA,wBAAA;IlB64EH;EkBx1EH;IAhDM,uBAAA;IlB24EH;EkB31EH;IA5CM,uBAAA;IACA,wBAAA;IlB04EH;EkB/1EH;;;IAtCQ,aAAA;IlB04EL;EkBp2EH;IAhCM,aAAA;IlBu4EH;EkBv2EH;IA5BM,kBAAA;IACA,wBAAA;IlBs4EH;EkB32EH;;IApBM,uBAAA;IACA,eAAA;IACA,kBAAA;IACA,wBAAA;IlBm4EH;EkBl3EH;;IAdQ,iBAAA;IlBo4EL;EkBt3EH;;IATM,oBAAA;IACA,gBAAA;IlBm4EH;EkB33EH;IAHM,QAAA;IlBi4EH;EACF;AkBv3ED;;;;EASI,eAAA;EACA,kBAAA;EACA,kBAAA;ElBo3EH;AkB/3ED;;EAiBI,kBAAA;ElBk3EH;AkBn4ED;EJjfE,oBAAA;EACA,qBAAA;Edu3FD;AkBh2EC;EAAA;IAVI,mBAAA;IACA,kBAAA;IACA,kBAAA;IlB82EH;EACF;AkB94ED;EAwCI,aAAA;ElBy2EH;AkB51EC;EAAA;IAHM,0BAAA;IlBm2EL;EACF;AkB11EC;EAAA;IAHM,kBAAA;IlBi2EL;EACF;AoBn5FD;EACE,uBAAA;EACA,kBAAA;EACA,qBAAA;EACA,oBAAA;EACA,wBAAA;EACA,gCAAA;MAAA,4BAAA;EACA,iBAAA;EACA,wBAAA;EACA,+BAAA;EACA,qBAAA;EC6BA,mBAAA;EACA,iBAAA;EACA,yBAAA;EACA,oBAAA;EhB4KA,2BAAA;EACG,wBAAA;EACC,uBAAA;EACI,mBAAA;EL8sFT;AoBt5FG;;;;;;EdrBF,sBAAA;EAEA,4CAAA;EACA,sBAAA;ENk7FD;AoB15FC;;;EAGE,gBAAA;EACA,uBAAA;EpB45FH;AoBz5FC;;EAEE,YAAA;EACA,wBAAA;Ef2BF,0DAAA;EACQ,kDAAA;ELi4FT;AoBz5FC;;;EAGE,qBAAA;EACA,sBAAA;EE9CF,eAAA;EAGA,2BAAA;EjB8DA,0BAAA;EACQ,kBAAA;EL24FT;AoBr5FD;ECrDE,gBAAA;EACA,2BAAA;EACA,uBAAA;ErB68FD;AqB38FC;;;;;;EAME,gBAAA;EACA,2BAAA;EACI,uBAAA;ErB68FP;AqB38FC;;;EAGE,wBAAA;ErB68FH;AqBx8FG;;;;;;;;;;;;;;;;;;EAME,2BAAA;EACI,uBAAA;ErBs9FT;AoB97FD;ECnBI,gBAAA;EACA,2BAAA;ErBo9FH;AoB/7FD;ECxDE,gBAAA;EACA,2BAAA;EACA,uBAAA;ErB0/FD;AqBx/FC;;;;;;EAME,gBAAA;EACA,2BAAA;EACI,uBAAA;ErB0/FP;AqBx/FC;;;EAGE,wBAAA;ErB0/FH;AqBr/FG;;;;;;;;;;;;;;;;;;EAME,2BAAA;EACI,uBAAA;ErBmgGT;AoBx+FD;ECtBI,gBAAA;EACA,2BAAA;ErBigGH;AoBx+FD;EC5DE,gBAAA;EACA,2BAAA;EACA,uBAAA;ErBuiGD;AqBriGC;;;;;;EAME,gBAAA;EACA,2BAAA;EACI,uBAAA;ErBuiGP;AqBriGC;;;EAGE,wBAAA;ErBuiGH;AqBliGG;;;;;;;;;;;;;;;;;;EAME,2BAAA;EACI,uBAAA;ErBgjGT;AoBjhGD;EC1BI,gBAAA;EACA,2BAAA;ErB8iGH;AoBjhGD;EChEE,gBAAA;EACA,2BAAA;EACA,uBAAA;ErBolGD;AqBllGC;;;;;;EAME,gBAAA;EACA,2BAAA;EACI,uBAAA;ErBolGP;AqBllGC;;;EAGE,wBAAA;ErBolGH;AqB/kGG;;;;;;;;;;;;;;;;;;EAME,2BAAA;EACI,uBAAA;ErB6lGT;AoB1jGD;EC9BI,gBAAA;EACA,2BAAA;ErB2lGH;AoB1jGD;ECpEE,gBAAA;EACA,2BAAA;EACA,uBAAA;ErBioGD;AqB/nGC;;;;;;EAME,gBAAA;EACA,2BAAA;EACI,uBAAA;ErBioGP;AqB/nGC;;;EAGE,wBAAA;ErBioGH;AqB5nGG;;;;;;;;;;;;;;;;;;EAME,2BAAA;EACI,uBAAA;ErB0oGT;AoBnmGD;EClCI,gBAAA;EACA,2BAAA;ErBwoGH;AoBnmGD;ECxEE,gBAAA;EACA,2BAAA;EACA,uBAAA;ErB8qGD;AqB5qGC;;;;;;EAME,gBAAA;EACA,2BAAA;EACI,uBAAA;ErB8qGP;AqB5qGC;;;EAGE,wBAAA;ErB8qGH;AqBzqGG;;;;;;;;;;;;;;;;;;EAME,2BAAA;EACI,uBAAA;ErBurGT;AoB5oGD;ECtCI,gBAAA;EACA,2BAAA;ErBqrGH;AoBvoGD;EACE,gBAAA;EACA,qBAAA;EACA,kBAAA;EpByoGD;AoBvoGC;;;;;EAKE,+BAAA;Ef7BF,0BAAA;EACQ,kBAAA;ELuqGT;AoBxoGC;;;;EAIE,2BAAA;EpB0oGH;AoBxoGC;;EAEE,gBAAA;EACA,4BAAA;EACA,+BAAA;EpB0oGH;AoBtoGG;;;;EAEE,gBAAA;EACA,uBAAA;EpB0oGL;AoBjoGD;;EC/EE,oBAAA;EACA,iBAAA;EACA,wBAAA;EACA,oBAAA;ErBotGD;AoBpoGD;;ECnFE,mBAAA;EACA,iBAAA;EACA,kBAAA;EACA,oBAAA;ErB2tGD;AoBvoGD;;ECvFE,kBAAA;EACA,iBAAA;EACA,kBAAA;EACA,oBAAA;ErBkuGD;AoBtoGD;EACE,gBAAA;EACA,aAAA;EpBwoGD;AoBpoGD;EACE,iBAAA;EpBsoGD;AoB/nGC;;;EACE,aAAA;EpBmoGH;AuBvxGD;EACE,YAAA;ElBoLA,0CAAA;EACK,qCAAA;EACG,kCAAA;ELsmGT;AuB1xGC;EACE,YAAA;EvB4xGH;AuBxxGD;EACE,eAAA;EvB0xGD;AuBxxGC;EAAY,gBAAA;EvB2xGb;AuB1xGC;EAAY,oBAAA;EvB6xGb;AuB5xGC;EAAY,0BAAA;EvB+xGb;AuB5xGD;EACE,oBAAA;EACA,WAAA;EACA,kBAAA;ElBuKA,iDAAA;EACQ,4CAAA;KAAA,yCAAA;EAOR,oCAAA;EACQ,+BAAA;KAAA,4BAAA;EAGR,0CAAA;EACQ,qCAAA;KAAA,kCAAA;ELgnGT;AwB1zGD;EACE,uBAAA;EACA,UAAA;EACA,WAAA;EACA,kBAAA;EACA,wBAAA;EACA,wBAAA;EACA,qCAAA;EACA,oCAAA;ExB4zGD;AwBxzGD;;EAEE,oBAAA;ExB0zGD;AwBtzGD;EACE,YAAA;ExBwzGD;AwBpzGD;EACE,oBAAA;EACA,WAAA;EACA,SAAA;EACA,eAAA;EACA,eAAA;EACA,aAAA;EACA,kBAAA;EACA,gBAAA;EACA,iBAAA;EACA,kBAAA;EACA,iBAAA;EACA,kBAAA;EACA,2BAAA;EACA,2BAAA;EACA,uCAAA;EACA,oBAAA;EnBuBA,qDAAA;EACQ,6CAAA;EmBtBR,sCAAA;UAAA,8BAAA;ExBuzGD;AwBlzGC;EACE,UAAA;EACA,YAAA;ExBozGH;AwB70GD;ECxBE,aAAA;EACA,eAAA;EACA,kBAAA;EACA,2BAAA;EzBw2GD;AwBn1GD;EAmCI,gBAAA;EACA,mBAAA;EACA,aAAA;EACA,qBAAA;EACA,yBAAA;EACA,gBAAA;EACA,qBAAA;ExBmzGH;AwB7yGC;;EAEE,uBAAA;EACA,gBAAA;EACA,2BAAA;ExB+yGH;AwBzyGC;;;EAGE,gBAAA;EACA,uBAAA;EACA,YAAA;EACA,2BAAA;ExB2yGH;AwBlyGC;;;EAGE,gBAAA;ExBoyGH;AwBhyGC;;EAEE,uBAAA;EACA,+BAAA;EACA,wBAAA;EE1GF,qEAAA;EF4GE,qBAAA;ExBkyGH;AwB7xGD;EAGI,gBAAA;ExB6xGH;AwBhyGD;EAQI,YAAA;ExB2xGH;AwBnxGD;EACE,YAAA;EACA,UAAA;ExBqxGD;AwB7wGD;EACE,SAAA;EACA,aAAA;ExB+wGD;AwB3wGD;EACE,gBAAA;EACA,mBAAA;EACA,iBAAA;EACA,yBAAA;EACA,gBAAA;EACA,qBAAA;ExB6wGD;AwBzwGD;EACE,iBAAA;EACA,SAAA;EACA,UAAA;EACA,WAAA;EACA,QAAA;EACA,cAAA;ExB2wGD;AwBvwGD;EACE,UAAA;EACA,YAAA;ExBywGD;AwBjwGD;;EAII,eAAA;EACA,0BAAA;EACA,aAAA;ExBiwGH;AwBvwGD;;EAUI,WAAA;EACA,cAAA;EACA,oBAAA;ExBiwGH;AwB5uGD;EAXE;IAnEA,YAAA;IACA,UAAA;IxB8zGC;EwB5vGD;IAzDA,SAAA;IACA,aAAA;IxBwzGC;EACF;A2Bv8GD;;EAEE,oBAAA;EACA,uBAAA;EACA,wBAAA;E3By8GD;A2B78GD;;EAMI,oBAAA;EACA,aAAA;E3B28GH;A2Bz8GG;;;;;;;;EAIE,YAAA;E3B+8GL;A2Bz8GD;;;;EAKI,mBAAA;E3B08GH;A2Br8GD;EACE,mBAAA;E3Bu8GD;A2Bx8GD;;EAMI,aAAA;E3Bs8GH;A2B58GD;;;EAWI,kBAAA;E3Bs8GH;A2Bl8GD;EACE,kBAAA;E3Bo8GD;A2Bh8GD;EACE,gBAAA;E3Bk8GD;A2Bj8GC;ECjDA,+BAAA;EACG,4BAAA;E5Bq/GJ;A2Bh8GD;;EC9CE,8BAAA;EACG,2BAAA;E5Bk/GJ;A2B/7GD;EACE,aAAA;E3Bi8GD;A2B/7GD;EACE,kBAAA;E3Bi8GD;A2B/7GD;;EClEE,+BAAA;EACG,4BAAA;E5BqgHJ;A2B97GD;EChEE,8BAAA;EACG,2BAAA;E5BigHJ;A2B77GD;;EAEE,YAAA;E3B+7GD;A2B96GD;EACE,mBAAA;EACA,oBAAA;E3Bg7GD;A2B96GD;EACE,oBAAA;EACA,qBAAA;E3Bg7GD;A2B36GD;EtB9CE,0DAAA;EACQ,kDAAA;EL49GT;A2B36GC;EtBlDA,0BAAA;EACQ,kBAAA;ELg+GT;A2Bx6GD;EACE,gBAAA;E3B06GD;A2Bv6GD;EACE,yBAAA;EACA,wBAAA;E3By6GD;A2Bt6GD;EACE,yBAAA;E3Bw6GD;A2Bj6GD;;;EAII,gBAAA;EACA,aAAA;EACA,aAAA;EACA,iBAAA;E3Bk6GH;A2Bz6GD;EAcM,aAAA;E3B85GL;A2B56GD;;;;EAsBI,kBAAA;EACA,gBAAA;E3B45GH;A2Bv5GC;EACE,kBAAA;E3By5GH;A2Bv5GC;EACE,8BAAA;ECnKF,+BAAA;EACC,8BAAA;E5B6jHF;A2Bx5GC;EACE,gCAAA;EC/KF,4BAAA;EACC,2BAAA;E5B0kHF;A2Bx5GD;EACE,kBAAA;E3B05GD;A2Bx5GD;;EC9KE,+BAAA;EACC,8BAAA;E5B0kHF;A2Bv5GD;EC5LE,4BAAA;EACC,2BAAA;E5BslHF;A2Bn5GD;EACE,gBAAA;EACA,aAAA;EACA,qBAAA;EACA,2BAAA;E3Bq5GD;A2Bz5GD;;EAOI,aAAA;EACA,qBAAA;EACA,WAAA;E3Bs5GH;A2B/5GD;EAYI,aAAA;E3Bs5GH;A2Bl6GD;EAgBI,YAAA;E3Bq5GH;A2Bp4GD;;;;EAKM,oBAAA;EACA,wBAAA;EACA,sBAAA;E3Bq4GL;A6B9mHD;EACE,oBAAA;EACA,gBAAA;EACA,2BAAA;E7BgnHD;A6B7mHC;EACE,aAAA;EACA,iBAAA;EACA,kBAAA;E7B+mHH;A6BxnHD;EAeI,oBAAA;EACA,YAAA;EAKA,aAAA;EAEA,aAAA;EACA,kBAAA;E7BumHH;A6B9lHD;;;EV8BE,cAAA;EACA,oBAAA;EACA,iBAAA;EACA,wBAAA;EACA,oBAAA;EnBqkHD;AmBnkHC;;;EACE,cAAA;EACA,mBAAA;EnBukHH;AmBpkHC;;;;;;EAEE,cAAA;EnB0kHH;A6BhnHD;;;EVyBE,cAAA;EACA,mBAAA;EACA,iBAAA;EACA,kBAAA;EACA,oBAAA;EnB4lHD;AmB1lHC;;;EACE,cAAA;EACA,mBAAA;EnB8lHH;AmB3lHC;;;;;;EAEE,cAAA;EnBimHH;A6B9nHD;;;EAGE,qBAAA;E7BgoHD;A6B9nHC;;;EACE,kBAAA;E7BkoHH;A6B9nHD;;EAEE,WAAA;EACA,qBAAA;EACA,wBAAA;E7BgoHD;A6B3nHD;EACE,mBAAA;EACA,iBAAA;EACA,qBAAA;EACA,gBAAA;EACA,gBAAA;EACA,oBAAA;EACA,2BAAA;EACA,2BAAA;EACA,oBAAA;E7B6nHD;A6B1nHC;EACE,mBAAA;EACA,iBAAA;EACA,oBAAA;E7B4nHH;A6B1nHC;EACE,oBAAA;EACA,iBAAA;EACA,oBAAA;E7B4nHH;A6BhpHD;;EA0BI,eAAA;E7B0nHH;A6BrnHD;;;;;;;EDhGE,+BAAA;EACG,4BAAA;E5B8tHJ;A6BtnHD;EACE,iBAAA;E7BwnHD;A6BtnHD;;;;;;;EDpGE,8BAAA;EACG,2BAAA;E5BmuHJ;A6BvnHD;EACE,gBAAA;E7BynHD;A6BpnHD;EACE,oBAAA;EAGA,cAAA;EACA,qBAAA;E7BonHD;A6BznHD;EAUI,oBAAA;E7BknHH;A6B5nHD;EAYM,mBAAA;E7BmnHL;A6BhnHG;;;EAGE,YAAA;E7BknHL;A6B7mHC;;EAGI,oBAAA;E7B8mHL;A6B3mHC;;EAGI,mBAAA;E7B4mHL;A8BtwHD;EACE,kBAAA;EACA,iBAAA;EACA,kBAAA;E9BwwHD;A8B3wHD;EAOI,oBAAA;EACA,gBAAA;E9BuwHH;A8B/wHD;EAWM,oBAAA;EACA,gBAAA;EACA,oBAAA;E9BuwHL;A8BtwHK;;EAEE,uBAAA;EACA,2BAAA;E9BwwHP;A8BnwHG;EACE,gBAAA;E9BqwHL;A8BnwHK;;EAEE,gBAAA;EACA,uBAAA;EACA,+BAAA;EACA,qBAAA;E9BqwHP;A8B9vHG;;;EAGE,2BAAA;EACA,uBAAA;E9BgwHL;A8BzyHD;ELHE,aAAA;EACA,eAAA;EACA,kBAAA;EACA,2BAAA;EzB+yHD;A8B/yHD;EA0DI,iBAAA;E9BwvHH;A8B/uHD;EACE,kCAAA;E9BivHD;A8BlvHD;EAGI,aAAA;EAEA,qBAAA;E9BivHH;A8BtvHD;EASM,mBAAA;EACA,yBAAA;EACA,+BAAA;EACA,4BAAA;E9BgvHL;A8B/uHK;EACE,uCAAA;E9BivHP;A8B3uHK;;;EAGE,gBAAA;EACA,2BAAA;EACA,2BAAA;EACA,kCAAA;EACA,iBAAA;E9B6uHP;A8BxuHC;EAqDA,aAAA;EA8BA,kBAAA;E9BypHD;A8B5uHC;EAwDE,aAAA;E9BurHH;A8B/uHC;EA0DI,oBAAA;EACA,oBAAA;E9BwrHL;A8BnvHC;EAgEE,WAAA;EACA,YAAA;E9BsrHH;A8B1qHD;EAAA;IAPM,qBAAA;IACA,WAAA;I9BqrHH;E8B/qHH;IAJQ,kBAAA;I9BsrHL;EACF;A8BhwHC;EAuFE,iBAAA;EACA,oBAAA;E9B4qHH;A8BpwHC;;;EA8FE,2BAAA;E9B2qHH;A8B7pHD;EAAA;IATM,kCAAA;IACA,4BAAA;I9B0qHH;E8BlqHH;;;IAHM,8BAAA;I9B0qHH;EACF;A8B3wHD;EAEI,aAAA;E9B4wHH;A8B9wHD;EAMM,oBAAA;E9B2wHL;A8BjxHD;EASM,kBAAA;E9B2wHL;A8BtwHK;;;EAGE,gBAAA;EACA,2BAAA;E9BwwHP;A8BhwHD;EAEI,aAAA;E9BiwHH;A8BnwHD;EAIM,iBAAA;EACA,gBAAA;E9BkwHL;A8BtvHD;EACE,aAAA;E9BwvHD;A8BzvHD;EAII,aAAA;E9BwvHH;A8B5vHD;EAMM,oBAAA;EACA,oBAAA;E9ByvHL;A8BhwHD;EAYI,WAAA;EACA,YAAA;E9BuvHH;A8B3uHD;EAAA;IAPM,qBAAA;IACA,WAAA;I9BsvHH;E8BhvHH;IAJQ,kBAAA;I9BuvHL;EACF;A8B/uHD;EACE,kBAAA;E9BivHD;A8BlvHD;EAKI,iBAAA;EACA,oBAAA;E9BgvHH;A8BtvHD;;;EAYI,2BAAA;E9B+uHH;A8BjuHD;EAAA;IATM,kCAAA;IACA,4BAAA;I9B8uHH;E8BtuHH;;;IAHM,8BAAA;I9B8uHH;EACF;A8BruHD;EAEI,eAAA;E9BsuHH;A8BxuHD;EAKI,gBAAA;E9BsuHH;A8B7tHD;EAEE,kBAAA;EF3OA,4BAAA;EACC,2BAAA;E5B08HF;A+Bp8HD;EACE,oBAAA;EACA,kBAAA;EACA,qBAAA;EACA,+BAAA;E/Bs8HD;A+B97HD;EAAA;IAFI,oBAAA;I/Bo8HD;EACF;A+Br7HD;EAAA;IAFI,aAAA;I/B27HD;EACF;A+B76HD;EACE,qBAAA;EACA,qBAAA;EACA,oBAAA;EACA,mCAAA;EACA,4DAAA;UAAA,oDAAA;EAEA,mCAAA;E/B86HD;A+B56HC;EACE,kBAAA;E/B86HH;A+Bl5HD;EAAA;IAxBI,aAAA;IACA,eAAA;IACA,0BAAA;YAAA,kBAAA;I/B86HD;E+B56HC;IACE,2BAAA;IACA,yBAAA;IACA,mBAAA;IACA,8BAAA;I/B86HH;E+B36HC;IACE,qBAAA;I/B66HH;E+Bx6HC;;;IAGE,iBAAA;IACA,kBAAA;I/B06HH;EACF;A+Bt6HD;;EAGI,mBAAA;E/Bu6HH;A+Bl6HC;EAAA;;IAFI,mBAAA;I/By6HH;EACF;A+Bh6HD;;;;EAII,qBAAA;EACA,oBAAA;E/Bk6HH;A+B55HC;EAAA;;;;IAHI,iBAAA;IACA,gBAAA;I/Bs6HH;EACF;A+B15HD;EACE,eAAA;EACA,uBAAA;E/B45HD;A+Bv5HD;EAAA;IAFI,kBAAA;I/B65HD;EACF;A+Bz5HD;;EAEE,iBAAA;EACA,UAAA;EACA,SAAA;EACA,eAAA;E/B25HD;A+Br5HD;EAAA;;IAFI,kBAAA;I/B45HD;EACF;A+B15HD;EACE,QAAA;EACA,uBAAA;E/B45HD;A+B15HD;EACE,WAAA;EACA,kBAAA;EACA,uBAAA;E/B45HD;A+Bt5HD;EACE,aAAA;EACA,oBAAA;EACA,iBAAA;EACA,mBAAA;EACA,cAAA;E/Bw5HD;A+Bt5HC;;EAEE,uBAAA;E/Bw5HH;A+Bj6HD;EAaI,gBAAA;E/Bu5HH;A+B94HD;EALI;;IAEE,oBAAA;I/Bs5HH;EACF;A+B54HD;EACE,oBAAA;EACA,cAAA;EACA,oBAAA;EACA,mBAAA;EC9LA,iBAAA;EACA,oBAAA;ED+LA,+BAAA;EACA,wBAAA;EACA,+BAAA;EACA,oBAAA;E/B+4HD;A+B34HC;EACE,YAAA;E/B64HH;A+B35HD;EAmBI,gBAAA;EACA,aAAA;EACA,aAAA;EACA,oBAAA;E/B24HH;A+Bj6HD;EAyBI,iBAAA;E/B24HH;A+Br4HD;EAAA;IAFI,eAAA;I/B24HD;EACF;A+Bl4HD;EACE,qBAAA;E/Bo4HD;A+Br4HD;EAII,mBAAA;EACA,sBAAA;EACA,mBAAA;E/Bo4HH;A+Bx2HC;EAAA;IAtBI,kBAAA;IACA,aAAA;IACA,aAAA;IACA,eAAA;IACA,+BAAA;IACA,WAAA;IACA,0BAAA;YAAA,kBAAA;I/Bk4HH;E+Bl3HD;;IAbM,4BAAA;I/Bm4HL;E+Bt3HD;IAVM,mBAAA;I/Bm4HL;E+Bl4HK;;IAEE,wBAAA;I/Bo4HP;EACF;A+Bl3HD;EAAA;IAXI,aAAA;IACA,WAAA;I/Bi4HD;E+Bv3HH;IAPM,aAAA;I/Bi4HH;E+B13HH;IALQ,mBAAA;IACA,sBAAA;I/Bk4HL;EACF;A+Bv3HD;EACE,oBAAA;EACA,qBAAA;EACA,oBAAA;EACA,mCAAA;EACA,sCAAA;E1B9NA,8FAAA;EACQ,sFAAA;E2B/DR,iBAAA;EACA,oBAAA;EhCwpID;AkBvqHD;EAAA;IA9DM,uBAAA;IACA,kBAAA;IACA,wBAAA;IlByuHH;EkB7qHH;IAvDM,uBAAA;IACA,aAAA;IACA,wBAAA;IlBuuHH;EkBlrHH;IAhDM,uBAAA;IlBquHH;EkBrrHH;IA5CM,uBAAA;IACA,wBAAA;IlBouHH;EkBzrHH;;;IAtCQ,aAAA;IlBouHL;EkB9rHH;IAhCM,aAAA;IlBiuHH;EkBjsHH;IA5BM,kBAAA;IACA,wBAAA;IlBguHH;EkBrsHH;;IApBM,uBAAA;IACA,eAAA;IACA,kBAAA;IACA,wBAAA;IlB6tHH;EkB5sHH;;IAdQ,iBAAA;IlB8tHL;EkBhtHH;;IATM,oBAAA;IACA,gBAAA;IlB6tHH;EkBrtHH;IAHM,QAAA;IlB2tHH;EACF;A+Bh6HC;EAAA;IANI,oBAAA;I/B06HH;E+Bx6HG;IACE,kBAAA;I/B06HL;EACF;A+Bz5HD;EAAA;IARI,aAAA;IACA,WAAA;IACA,gBAAA;IACA,iBAAA;IACA,gBAAA;IACA,mBAAA;I1BzPF,0BAAA;IACQ,kBAAA;IL+pIP;EACF;A+B/5HD;EACE,eAAA;EHpUA,4BAAA;EACC,2BAAA;E5BsuIF;A+B/5HD;EACE,kBAAA;EHzUA,8BAAA;EACC,6BAAA;EAOD,+BAAA;EACC,8BAAA;E5BquIF;A+B35HD;EChVE,iBAAA;EACA,oBAAA;EhC8uID;A+B55HC;ECnVA,kBAAA;EACA,qBAAA;EhCkvID;A+B75HC;ECtVA,kBAAA;EACA,qBAAA;EhCsvID;A+Bv5HD;EChWE,kBAAA;EACA,qBAAA;EhC0vID;A+Bn5HD;EAAA;IAJI,aAAA;IACA,mBAAA;IACA,oBAAA;I/B25HD;EACF;A+B93HD;EAhBE;IExWA,wBAAA;IjC0vIC;E+Bj5HD;IE5WA,yBAAA;IF8WE,qBAAA;I/Bm5HD;E+Br5HD;IAKI,iBAAA;I/Bm5HH;EACF;A+B14HD;EACE,2BAAA;EACA,uBAAA;E/B44HD;A+B94HD;EAKI,gBAAA;E/B44HH;A+B34HG;;EAEE,gBAAA;EACA,+BAAA;E/B64HL;A+Bt5HD;EAcI,gBAAA;E/B24HH;A+Bz5HD;EAmBM,gBAAA;E/By4HL;A+Bv4HK;;EAEE,gBAAA;EACA,+BAAA;E/By4HP;A+Br4HK;;;EAGE,gBAAA;EACA,2BAAA;E/Bu4HP;A+Bn4HK;;;EAGE,gBAAA;EACA,+BAAA;E/Bq4HP;A+B76HD;EA8CI,uBAAA;E/Bk4HH;A+Bj4HG;;EAEE,2BAAA;E/Bm4HL;A+Bp7HD;EAoDM,2BAAA;E/Bm4HL;A+Bv7HD;;EA0DI,uBAAA;E/Bi4HH;A+B13HK;;;EAGE,2BAAA;EACA,gBAAA;E/B43HP;A+B31HC;EAAA;IAzBQ,gBAAA;I/Bw3HP;E+Bv3HO;;IAEE,gBAAA;IACA,+BAAA;I/By3HT;E+Br3HO;;;IAGE,gBAAA;IACA,2BAAA;I/Bu3HT;E+Bn3HO;;;IAGE,gBAAA;IACA,+BAAA;I/Bq3HT;EACF;A+Bv9HD;EA8GI,gBAAA;E/B42HH;A+B32HG;EACE,gBAAA;E/B62HL;A+B79HD;EAqHI,gBAAA;E/B22HH;A+B12HG;;EAEE,gBAAA;E/B42HL;A+Bx2HK;;;;EAEE,gBAAA;E/B42HP;A+Bp2HD;EACE,2BAAA;EACA,uBAAA;E/Bs2HD;A+Bx2HD;EAKI,gBAAA;E/Bs2HH;A+Br2HG;;EAEE,gBAAA;EACA,+BAAA;E/Bu2HL;A+Bh3HD;EAcI,gBAAA;E/Bq2HH;A+Bn3HD;EAmBM,gBAAA;E/Bm2HL;A+Bj2HK;;EAEE,gBAAA;EACA,+BAAA;E/Bm2HP;A+B/1HK;;;EAGE,gBAAA;EACA,2BAAA;E/Bi2HP;A+B71HK;;;EAGE,gBAAA;EACA,+BAAA;E/B+1HP;A+Bv4HD;EA+CI,uBAAA;E/B21HH;A+B11HG;;EAEE,2BAAA;E/B41HL;A+B94HD;EAqDM,2BAAA;E/B41HL;A+Bj5HD;;EA2DI,uBAAA;E/B01HH;A+Bp1HK;;;EAGE,2BAAA;EACA,gBAAA;E/Bs1HP;A+B/yHC;EAAA;IA/BQ,uBAAA;I/Bk1HP;E+BnzHD;IA5BQ,2BAAA;I/Bk1HP;E+BtzHD;IAzBQ,gBAAA;I/Bk1HP;E+Bj1HO;;IAEE,gBAAA;IACA,+BAAA;I/Bm1HT;E+B/0HO;;;IAGE,gBAAA;IACA,2BAAA;I/Bi1HT;E+B70HO;;;IAGE,gBAAA;IACA,+BAAA;I/B+0HT;EACF;A+Bv7HD;EA+GI,gBAAA;E/B20HH;A+B10HG;EACE,gBAAA;E/B40HL;A+B77HD;EAsHI,gBAAA;E/B00HH;A+Bz0HG;;EAEE,gBAAA;E/B20HL;A+Bv0HK;;;;EAEE,gBAAA;E/B20HP;AkCr9ID;EACE,mBAAA;EACA,qBAAA;EACA,kBAAA;EACA,2BAAA;EACA,oBAAA;ElCu9ID;AkC59ID;EAQI,uBAAA;ElCu9IH;AkC/9ID;EAWM,mBAAA;EACA,gBAAA;EACA,gBAAA;ElCu9IL;AkCp+ID;EAkBI,gBAAA;ElCq9IH;AmCz+ID;EACE,uBAAA;EACA,iBAAA;EACA,gBAAA;EACA,oBAAA;EnC2+ID;AmC/+ID;EAOI,iBAAA;EnC2+IH;AmCl/ID;;EAUM,oBAAA;EACA,aAAA;EACA,mBAAA;EACA,yBAAA;EACA,uBAAA;EACA,gBAAA;EACA,2BAAA;EACA,2BAAA;EACA,mBAAA;EnC4+IL;AmC1+IG;;EAGI,gBAAA;EPXN,gCAAA;EACG,6BAAA;E5Bu/IJ;AmCz+IG;;EPvBF,iCAAA;EACG,8BAAA;E5BogJJ;AmCp+IG;;;;EAEE,gBAAA;EACA,2BAAA;EACA,uBAAA;EnCw+IL;AmCl+IG;;;;;;EAGE,YAAA;EACA,gBAAA;EACA,2BAAA;EACA,uBAAA;EACA,iBAAA;EnCu+IL;AmC7hJD;;;;;;EAiEM,gBAAA;EACA,2BAAA;EACA,uBAAA;EACA,qBAAA;EnCo+IL;AmC39ID;;EC1EM,oBAAA;EACA,iBAAA;EpCyiJL;AoCviJG;;ERMF,gCAAA;EACG,6BAAA;E5BqiJJ;AoCtiJG;;ERRF,iCAAA;EACG,8BAAA;E5BkjJJ;AmCr+ID;;EC/EM,mBAAA;EACA,iBAAA;EpCwjJL;AoCtjJG;;ERMF,gCAAA;EACG,6BAAA;E5BojJJ;AoCrjJG;;ERRF,iCAAA;EACG,8BAAA;E5BikJJ;AqCpkJD;EACE,iBAAA;EACA,gBAAA;EACA,kBAAA;EACA,oBAAA;ErCskJD;AqC1kJD;EAOI,iBAAA;ErCskJH;AqC7kJD;;EAUM,uBAAA;EACA,mBAAA;EACA,2BAAA;EACA,2BAAA;EACA,qBAAA;ErCukJL;AqCrlJD;;EAmBM,uBAAA;EACA,2BAAA;ErCskJL;AqC1lJD;;EA2BM,cAAA;ErCmkJL;AqC9lJD;;EAkCM,aAAA;ErCgkJL;AqClmJD;;;;EA2CM,gBAAA;EACA,2BAAA;EACA,qBAAA;ErC6jJL;AsC3mJD;EACE,iBAAA;EACA,yBAAA;EACA,gBAAA;EACA,mBAAA;EACA,gBAAA;EACA,gBAAA;EACA,oBAAA;EACA,qBAAA;EACA,0BAAA;EACA,sBAAA;EtC6mJD;AsCzmJG;;EAEE,gBAAA;EACA,uBAAA;EACA,iBAAA;EtC2mJL;AsCtmJC;EACE,eAAA;EtCwmJH;AsCpmJC;EACE,oBAAA;EACA,WAAA;EtCsmJH;AsC/lJD;ECtCE,2BAAA;EvCwoJD;AuCroJG;;EAEE,2BAAA;EvCuoJL;AsClmJD;EC1CE,2BAAA;EvC+oJD;AuC5oJG;;EAEE,2BAAA;EvC8oJL;AsCrmJD;EC9CE,2BAAA;EvCspJD;AuCnpJG;;EAEE,2BAAA;EvCqpJL;AsCxmJD;EClDE,2BAAA;EvC6pJD;AuC1pJG;;EAEE,2BAAA;EvC4pJL;AsC3mJD;ECtDE,2BAAA;EvCoqJD;AuCjqJG;;EAEE,2BAAA;EvCmqJL;AsC9mJD;EC1DE,2BAAA;EvC2qJD;AuCxqJG;;EAEE,2BAAA;EvC0qJL;AwC5qJD;EACE,uBAAA;EACA,iBAAA;EACA,kBAAA;EACA,iBAAA;EACA,mBAAA;EACA,gBAAA;EACA,gBAAA;EACA,0BAAA;EACA,qBAAA;EACA,oBAAA;EACA,2BAAA;EACA,qBAAA;ExC8qJD;AwC3qJC;EACE,eAAA;ExC6qJH;AwCzqJC;EACE,oBAAA;EACA,WAAA;ExC2qJH;AwCxqJC;;EAEE,QAAA;EACA,kBAAA;ExC0qJH;AwCrqJG;;EAEE,gBAAA;EACA,uBAAA;EACA,iBAAA;ExCuqJL;AwClqJC;;EAEE,gBAAA;EACA,2BAAA;ExCoqJH;AwCjqJC;EACE,cAAA;ExCmqJH;AwChqJC;EACE,mBAAA;ExCkqJH;AwC/pJC;EACE,kBAAA;ExCiqJH;AyC3tJD;EACE,oBAAA;EACA,qBAAA;EACA,gBAAA;EACA,2BAAA;EzC6tJD;AyCjuJD;;EAQI,gBAAA;EzC6tJH;AyCruJD;EAYI,qBAAA;EACA,iBAAA;EACA,kBAAA;EzC4tJH;AyC1uJD;EAkBI,2BAAA;EzC2tJH;AyCxtJC;;EAEE,oBAAA;EzC0tJH;AyCjvJD;EA2BI,iBAAA;EzCytJH;AyCxsJD;EAAA;IAbI,iBAAA;IzCytJD;EyCvtJC;;IAEE,oBAAA;IACA,qBAAA;IzCytJH;EyCjtJH;;IAHM,iBAAA;IzCwtJH;EACF;A0CjwJD;EACE,gBAAA;EACA,cAAA;EACA,qBAAA;EACA,yBAAA;EACA,2BAAA;EACA,2BAAA;EACA,oBAAA;ErCiLA,6CAAA;EACK,wCAAA;EACG,qCAAA;ELmlJT;A0C7wJD;;EAaI,mBAAA;EACA,oBAAA;E1CowJH;A0ChwJC;;;EAGE,uBAAA;E1CkwJH;A0CvxJD;EA0BI,cAAA;EACA,gBAAA;E1CgwJH;A2CzxJD;EACE,eAAA;EACA,qBAAA;EACA,+BAAA;EACA,oBAAA;E3C2xJD;A2C/xJD;EAQI,eAAA;EAEA,gBAAA;E3CyxJH;A2CnyJD;EAeI,mBAAA;E3CuxJH;A2CtyJD;;EAqBI,kBAAA;E3CqxJH;A2C1yJD;EAyBI,iBAAA;E3CoxJH;A2C5wJD;;EAEE,qBAAA;E3C8wJD;A2ChxJD;;EAMI,oBAAA;EACA,WAAA;EACA,cAAA;EACA,gBAAA;E3C8wJH;A2CtwJD;ECvDE,2BAAA;EACA,uBAAA;EACA,gBAAA;E5Cg0JD;A2C3wJD;EClDI,2BAAA;E5Cg0JH;A2C9wJD;EC/CI,gBAAA;E5Cg0JH;A2C7wJD;EC3DE,2BAAA;EACA,uBAAA;EACA,gBAAA;E5C20JD;A2ClxJD;ECtDI,2BAAA;E5C20JH;A2CrxJD;ECnDI,gBAAA;E5C20JH;A2CpxJD;EC/DE,2BAAA;EACA,uBAAA;EACA,gBAAA;E5Cs1JD;A2CzxJD;EC1DI,2BAAA;E5Cs1JH;A2C5xJD;ECvDI,gBAAA;E5Cs1JH;A2C3xJD;ECnEE,2BAAA;EACA,uBAAA;EACA,gBAAA;E5Ci2JD;A2ChyJD;EC9DI,2BAAA;E5Ci2JH;A2CnyJD;EC3DI,gBAAA;E5Ci2JH;A6Cn2JD;EACE;IAAQ,6BAAA;I7Cs2JP;E6Cr2JD;IAAQ,0BAAA;I7Cw2JP;EACF;A6Cr2JD;EACE;IAAQ,6BAAA;I7Cw2JP;E6Cv2JD;IAAQ,0BAAA;I7C02JP;EACF;A6C72JD;EACE;IAAQ,6BAAA;I7Cw2JP;E6Cv2JD;IAAQ,0BAAA;I7C02JP;EACF;A6Cn2JD;EACE,kBAAA;EACA,cAAA;EACA,qBAAA;EACA,2BAAA;EACA,oBAAA;ExCsCA,wDAAA;EACQ,gDAAA;ELg0JT;A6Cl2JD;EACE,aAAA;EACA,WAAA;EACA,cAAA;EACA,iBAAA;EACA,mBAAA;EACA,gBAAA;EACA,oBAAA;EACA,2BAAA;ExCyBA,wDAAA;EACQ,gDAAA;EAyHR,qCAAA;EACK,gCAAA;EACG,6BAAA;ELotJT;A6C/1JD;;ECCI,+MAAA;EACA,0MAAA;EACA,uMAAA;EDAF,oCAAA;UAAA,4BAAA;E7Cm2JD;A6C51JD;;ExC5CE,4DAAA;EACK,uDAAA;EACG,oDAAA;EL44JT;A6Cz1JD;EErEE,2BAAA;E/Ci6JD;A+C95JC;EDgDE,+MAAA;EACA,0MAAA;EACA,uMAAA;E9Ci3JH;A6C71JD;EEzEE,2BAAA;E/Cy6JD;A+Ct6JC;EDgDE,+MAAA;EACA,0MAAA;EACA,uMAAA;E9Cy3JH;A6Cj2JD;EE7EE,2BAAA;E/Ci7JD;A+C96JC;EDgDE,+MAAA;EACA,0MAAA;EACA,uMAAA;E9Ci4JH;A6Cr2JD;EEjFE,2BAAA;E/Cy7JD;A+Ct7JC;EDgDE,+MAAA;EACA,0MAAA;EACA,uMAAA;E9Cy4JH;AgDj8JD;EAEE,kBAAA;EhDk8JD;AgDh8JC;EACE,eAAA;EhDk8JH;AgD97JD;;EAEE,SAAA;EACA,kBAAA;EhDg8JD;AgD77JD;EACE,gBAAA;EhD+7JD;AgD57JD;EACE,gBAAA;EhD87JD;AgD37JD;;EAEE,oBAAA;EhD67JD;AgD17JD;;EAEE,qBAAA;EhD47JD;AgDz7JD;;;EAGE,qBAAA;EACA,qBAAA;EhD27JD;AgDx7JD;EACE,wBAAA;EhD07JD;AgDv7JD;EACE,wBAAA;EhDy7JD;AgDr7JD;EACE,eAAA;EACA,oBAAA;EhDu7JD;AgDj7JD;EACE,iBAAA;EACA,kBAAA;EhDm7JD;AiDr+JD;EAEE,qBAAA;EACA,iBAAA;EjDs+JD;AiD99JD;EACE,oBAAA;EACA,gBAAA;EACA,oBAAA;EAEA,qBAAA;EACA,2BAAA;EACA,2BAAA;EjD+9JD;AiD59JC;ErB3BA,8BAAA;EACC,6BAAA;E5B0/JF;AiD79JC;EACE,kBAAA;ErBvBF,iCAAA;EACC,gCAAA;E5Bu/JF;AiDt9JD;EACE,gBAAA;EjDw9JD;AiDz9JD;EAII,gBAAA;EjDw9JH;AiDp9JC;;EAEE,uBAAA;EACA,gBAAA;EACA,2BAAA;EjDs9JH;AiDh9JC;;;EAGE,2BAAA;EACA,gBAAA;EACA,qBAAA;EjDk9JH;AiDv9JC;;;EASI,gBAAA;EjDm9JL;AiD59JC;;;EAYI,gBAAA;EjDq9JL;AiDh9JC;;;EAGE,YAAA;EACA,gBAAA;EACA,2BAAA;EACA,uBAAA;EjDk9JH;AiDx9JC;;;;;;;;;EAYI,gBAAA;EjDu9JL;AiDn+JC;;;EAeI,gBAAA;EjDy9JL;AkDrjKC;EACE,gBAAA;EACA,2BAAA;ElDujKH;AkDrjKG;EACE,gBAAA;ElDujKL;AkDxjKG;EAII,gBAAA;ElDujKP;AkDpjKK;;EAEE,gBAAA;EACA,2BAAA;ElDsjKP;AkDpjKK;;;EAGE,aAAA;EACA,2BAAA;EACA,uBAAA;ElDsjKP;AkD3kKC;EACE,gBAAA;EACA,2BAAA;ElD6kKH;AkD3kKG;EACE,gBAAA;ElD6kKL;AkD9kKG;EAII,gBAAA;ElD6kKP;AkD1kKK;;EAEE,gBAAA;EACA,2BAAA;ElD4kKP;AkD1kKK;;;EAGE,aAAA;EACA,2BAAA;EACA,uBAAA;ElD4kKP;AkDjmKC;EACE,gBAAA;EACA,2BAAA;ElDmmKH;AkDjmKG;EACE,gBAAA;ElDmmKL;AkDpmKG;EAII,gBAAA;ElDmmKP;AkDhmKK;;EAEE,gBAAA;EACA,2BAAA;ElDkmKP;AkDhmKK;;;EAGE,aAAA;EACA,2BAAA;EACA,uBAAA;ElDkmKP;AkDvnKC;EACE,gBAAA;EACA,2BAAA;ElDynKH;AkDvnKG;EACE,gBAAA;ElDynKL;AkD1nKG;EAII,gBAAA;ElDynKP;AkDtnKK;;EAEE,gBAAA;EACA,2BAAA;ElDwnKP;AkDtnKK;;;EAGE,aAAA;EACA,2BAAA;EACA,uBAAA;ElDwnKP;AiD5hKD;EACE,eAAA;EACA,oBAAA;EjD8hKD;AiD5hKD;EACE,kBAAA;EACA,kBAAA;EjD8hKD;AmDlpKD;EACE,qBAAA;EACA,2BAAA;EACA,+BAAA;EACA,oBAAA;E9C0DA,mDAAA;EACQ,2CAAA;EL2lKT;AmDjpKD;EACE,eAAA;EnDmpKD;AmD9oKD;EACE,oBAAA;EACA,sCAAA;EvBpBA,8BAAA;EACC,6BAAA;E5BqqKF;AmDppKD;EAMI,gBAAA;EnDipKH;AmD5oKD;EACE,eAAA;EACA,kBAAA;EACA,iBAAA;EACA,gBAAA;EnD8oKD;AmDlpKD;;;;;EAWI,gBAAA;EnD8oKH;AmDzoKD;EACE,oBAAA;EACA,2BAAA;EACA,+BAAA;EvBxCA,iCAAA;EACC,gCAAA;E5BorKF;AmDnoKD;;EAGI,kBAAA;EnDooKH;AmDvoKD;;EAMM,qBAAA;EACA,kBAAA;EnDqoKL;AmDjoKG;;EAEI,eAAA;EvBvEN,8BAAA;EACC,6BAAA;E5B2sKF;AmDhoKG;;EAEI,kBAAA;EvBtEN,iCAAA;EACC,gCAAA;E5BysKF;AmD7nKD;EAEI,qBAAA;EnD8nKH;AmD3nKD;EACE,qBAAA;EnD6nKD;AmDrnKD;;;EAII,kBAAA;EnDsnKH;AmD1nKD;;;EAOM,oBAAA;EACA,qBAAA;EnDwnKL;AmDhoKD;;EvBnGE,8BAAA;EACC,6BAAA;E5BuuKF;AmDroKD;;;;EAmBQ,6BAAA;EACA,8BAAA;EnDwnKP;AmD5oKD;;;;;;;;EAwBU,6BAAA;EnD8nKT;AmDtpKD;;;;;;;;EA4BU,8BAAA;EnDooKT;AmDhqKD;;EvB3FE,iCAAA;EACC,gCAAA;E5B+vKF;AmDrqKD;;;;EAyCQ,gCAAA;EACA,iCAAA;EnDkoKP;AmD5qKD;;;;;;;;EA8CU,gCAAA;EnDwoKT;AmDtrKD;;;;;;;;EAkDU,iCAAA;EnD8oKT;AmDhsKD;;;;EA2DI,+BAAA;EnD2oKH;AmDtsKD;;EA+DI,eAAA;EnD2oKH;AmD1sKD;;EAmEI,WAAA;EnD2oKH;AmD9sKD;;;;;;;;;;;;EA0EU,gBAAA;EnDkpKT;AmD5tKD;;;;;;;;;;;;EA8EU,iBAAA;EnD4pKT;AmD1uKD;;;;;;;;EAuFU,kBAAA;EnD6pKT;AmDpvKD;;;;;;;;EAgGU,kBAAA;EnD8pKT;AmD9vKD;EAsGI,WAAA;EACA,kBAAA;EnD2pKH;AmDjpKD;EACE,qBAAA;EnDmpKD;AmDppKD;EAKI,kBAAA;EACA,oBAAA;EnDkpKH;AmDxpKD;EASM,iBAAA;EnDkpKL;AmD3pKD;EAcI,kBAAA;EnDgpKH;AmD9pKD;;EAkBM,+BAAA;EnDgpKL;AmDlqKD;EAuBI,eAAA;EnD8oKH;AmDrqKD;EAyBM,kCAAA;EnD+oKL;AmDxoKD;ECpPE,uBAAA;EpD+3KD;AoD73KC;EACE,gBAAA;EACA,2BAAA;EACA,uBAAA;EpD+3KH;AoDl4KC;EAMI,2BAAA;EpD+3KL;AoDr4KC;EASI,gBAAA;EACA,2BAAA;EpD+3KL;AoD53KC;EAEI,8BAAA;EpD63KL;AmDvpKD;ECvPE,uBAAA;EpDi5KD;AoD/4KC;EACE,gBAAA;EACA,2BAAA;EACA,uBAAA;EpDi5KH;AoDp5KC;EAMI,2BAAA;EpDi5KL;AoDv5KC;EASI,gBAAA;EACA,2BAAA;EpDi5KL;AoD94KC;EAEI,8BAAA;EpD+4KL;AmDtqKD;EC1PE,uBAAA;EpDm6KD;AoDj6KC;EACE,gBAAA;EACA,2BAAA;EACA,uBAAA;EpDm6KH;AoDt6KC;EAMI,2BAAA;EpDm6KL;AoDz6KC;EASI,gBAAA;EACA,2BAAA;EpDm6KL;AoDh6KC;EAEI,8BAAA;EpDi6KL;AmDrrKD;EC7PE,uBAAA;EpDq7KD;AoDn7KC;EACE,gBAAA;EACA,2BAAA;EACA,uBAAA;EpDq7KH;AoDx7KC;EAMI,2BAAA;EpDq7KL;AoD37KC;EASI,gBAAA;EACA,2BAAA;EpDq7KL;AoDl7KC;EAEI,8BAAA;EpDm7KL;AmDpsKD;EChQE,uBAAA;EpDu8KD;AoDr8KC;EACE,gBAAA;EACA,2BAAA;EACA,uBAAA;EpDu8KH;AoD18KC;EAMI,2BAAA;EpDu8KL;AoD78KC;EASI,gBAAA;EACA,2BAAA;EpDu8KL;AoDp8KC;EAEI,8BAAA;EpDq8KL;AmDntKD;ECnQE,uBAAA;EpDy9KD;AoDv9KC;EACE,gBAAA;EACA,2BAAA;EACA,uBAAA;EpDy9KH;AoD59KC;EAMI,2BAAA;EpDy9KL;AoD/9KC;EASI,gBAAA;EACA,2BAAA;EpDy9KL;AoDt9KC;EAEI,8BAAA;EpDu9KL;AqDv+KD;EACE,oBAAA;EACA,gBAAA;EACA,WAAA;EACA,YAAA;EACA,kBAAA;ErDy+KD;AqD9+KD;;;;;EAYI,oBAAA;EACA,QAAA;EACA,SAAA;EACA,WAAA;EACA,cAAA;EACA,aAAA;EACA,WAAA;ErDy+KH;AqDp+KD;EACE,wBAAA;ErDs+KD;AqDl+KD;EACE,qBAAA;ErDo+KD;AsD//KD;EACE,kBAAA;EACA,eAAA;EACA,qBAAA;EACA,2BAAA;EACA,2BAAA;EACA,oBAAA;EjDwDA,yDAAA;EACQ,iDAAA;EL08KT;AsDzgLD;EASI,oBAAA;EACA,mCAAA;EtDmgLH;AsD9/KD;EACE,eAAA;EACA,oBAAA;EtDggLD;AsD9/KD;EACE,cAAA;EACA,oBAAA;EtDggLD;AuDthLD;EACE,cAAA;EACA,iBAAA;EACA,mBAAA;EACA,gBAAA;EACA,gBAAA;EACA,8BAAA;EjCRA,cAAA;EAGA,2BAAA;EtB+hLD;AuDvhLC;;EAEE,gBAAA;EACA,uBAAA;EACA,iBAAA;EjCfF,cAAA;EAGA,2BAAA;EtBuiLD;AuDnhLC;EACE,YAAA;EACA,iBAAA;EACA,yBAAA;EACA,WAAA;EACA,0BAAA;EvDqhLH;AwD1iLD;EACE,kBAAA;ExD4iLD;AwDxiLD;EACE,eAAA;EACA,kBAAA;EACA,iBAAA;EACA,QAAA;EACA,UAAA;EACA,WAAA;EACA,SAAA;EACA,eAAA;EACA,mCAAA;EAIA,YAAA;ExDuiLD;AwDpiLC;EnD+GA,uCAAA;EACI,mCAAA;EACC,kCAAA;EACG,+BAAA;EAkER,qDAAA;EAEK,2CAAA;EACG,qCAAA;ELu3KT;AwD1iLC;EnD2GA,oCAAA;EACI,gCAAA;EACC,+BAAA;EACG,4BAAA;ELk8KT;AwD9iLD;EACE,oBAAA;EACA,kBAAA;ExDgjLD;AwD5iLD;EACE,oBAAA;EACA,aAAA;EACA,cAAA;ExD8iLD;AwD1iLD;EACE,oBAAA;EACA,2BAAA;EACA,2BAAA;EACA,sCAAA;EACA,oBAAA;EnDaA,kDAAA;EACQ,0CAAA;EmDZR,sCAAA;UAAA,8BAAA;EAEA,YAAA;ExD4iLD;AwDxiLD;EACE,iBAAA;EACA,QAAA;EACA,UAAA;EACA,WAAA;EACA,SAAA;EACA,eAAA;EACA,2BAAA;ExD0iLD;AwDxiLC;ElCrEA,YAAA;EAGA,0BAAA;EtB8mLD;AwD3iLC;ElCtEA,cAAA;EAGA,2BAAA;EtBknLD;AwD1iLD;EACE,eAAA;EACA,kCAAA;EACA,2BAAA;ExD4iLD;AwDziLD;EACE,kBAAA;ExD2iLD;AwDviLD;EACE,WAAA;EACA,yBAAA;ExDyiLD;AwDpiLD;EACE,oBAAA;EACA,eAAA;ExDsiLD;AwDliLD;EACE,eAAA;EACA,mBAAA;EACA,+BAAA;ExDoiLD;AwDviLD;EAQI,kBAAA;EACA,kBAAA;ExDkiLH;AwD3iLD;EAaI,mBAAA;ExDiiLH;AwD9iLD;EAiBI,gBAAA;ExDgiLH;AwD3hLD;EACE,oBAAA;EACA,cAAA;EACA,aAAA;EACA,cAAA;EACA,kBAAA;ExD6hLD;AwD3gLD;EAZE;IACE,cAAA;IACA,mBAAA;IxD0hLD;EwDxhLD;InDvEA,mDAAA;IACQ,2CAAA;ILkmLP;EwDvhLD;IAAY,cAAA;IxD0hLX;EACF;AwDrhLD;EAFE;IAAY,cAAA;IxD2hLX;EACF;AyD1qLD;EACE,oBAAA;EACA,eAAA;EACA,gBAAA;EAEA,6DAAA;EACA,iBAAA;EACA,qBAAA;EACA,kBAAA;EnCXA,YAAA;EAGA,0BAAA;EtBqrLD;AyD1qLC;EnCdA,cAAA;EAGA,2BAAA;EtByrLD;AyD7qLC;EAAW,kBAAA;EAAmB,gBAAA;EzDirL/B;AyDhrLC;EAAW,kBAAA;EAAmB,gBAAA;EzDorL/B;AyDnrLC;EAAW,iBAAA;EAAmB,gBAAA;EzDurL/B;AyDtrLC;EAAW,mBAAA;EAAmB,gBAAA;EzD0rL/B;AyDtrLD;EACE,kBAAA;EACA,kBAAA;EACA,gBAAA;EACA,oBAAA;EACA,uBAAA;EACA,2BAAA;EACA,oBAAA;EzDwrLD;AyDprLD;EACE,oBAAA;EACA,UAAA;EACA,WAAA;EACA,2BAAA;EACA,qBAAA;EzDsrLD;AyDlrLC;EACE,WAAA;EACA,WAAA;EACA,mBAAA;EACA,yBAAA;EACA,2BAAA;EzDorLH;AyDlrLC;EACE,WAAA;EACA,YAAA;EACA,qBAAA;EACA,yBAAA;EACA,2BAAA;EzDorLH;AyDlrLC;EACE,WAAA;EACA,WAAA;EACA,qBAAA;EACA,yBAAA;EACA,2BAAA;EzDorLH;AyDlrLC;EACE,UAAA;EACA,SAAA;EACA,kBAAA;EACA,6BAAA;EACA,6BAAA;EzDorLH;AyDlrLC;EACE,UAAA;EACA,UAAA;EACA,kBAAA;EACA,6BAAA;EACA,4BAAA;EzDorLH;AyDlrLC;EACE,QAAA;EACA,WAAA;EACA,mBAAA;EACA,yBAAA;EACA,8BAAA;EzDorLH;AyDlrLC;EACE,QAAA;EACA,YAAA;EACA,kBAAA;EACA,yBAAA;EACA,8BAAA;EzDorLH;AyDlrLC;EACE,QAAA;EACA,WAAA;EACA,kBAAA;EACA,yBAAA;EACA,8BAAA;EzDorLH;A0DlxLD;EACE,oBAAA;EACA,QAAA;EACA,SAAA;EACA,eAAA;EACA,eAAA;EACA,kBAAA;EACA,cAAA;EAEA,6DAAA;EACA,iBAAA;EACA,qBAAA;EACA,yBAAA;EACA,kBAAA;EACA,2BAAA;EACA,sCAAA;UAAA,8BAAA;EACA,2BAAA;EACA,sCAAA;EACA,oBAAA;ErD6CA,mDAAA;EACQ,2CAAA;EqD1CR,qBAAA;E1DkxLD;A0D/wLC;EAAY,mBAAA;E1DkxLb;A0DjxLC;EAAY,mBAAA;E1DoxLb;A0DnxLC;EAAY,kBAAA;E1DsxLb;A0DrxLC;EAAY,oBAAA;E1DwxLb;A0DrxLD;EACE,WAAA;EACA,mBAAA;EACA,iBAAA;EACA,2BAAA;EACA,kCAAA;EACA,4BAAA;E1DuxLD;A0DpxLD;EACE,mBAAA;E1DsxLD;A0D9wLC;;EAEE,oBAAA;EACA,gBAAA;EACA,UAAA;EACA,WAAA;EACA,2BAAA;EACA,qBAAA;E1DgxLH;A0D7wLD;EACE,oBAAA;E1D+wLD;A0D7wLD;EACE,oBAAA;EACA,aAAA;E1D+wLD;A0D3wLC;EACE,WAAA;EACA,oBAAA;EACA,wBAAA;EACA,2BAAA;EACA,uCAAA;EACA,eAAA;E1D6wLH;A0D5wLG;EACE,cAAA;EACA,aAAA;EACA,oBAAA;EACA,wBAAA;EACA,2BAAA;E1D8wLL;A0D3wLC;EACE,UAAA;EACA,aAAA;EACA,mBAAA;EACA,sBAAA;EACA,6BAAA;EACA,yCAAA;E1D6wLH;A0D5wLG;EACE,cAAA;EACA,WAAA;EACA,eAAA;EACA,sBAAA;EACA,6BAAA;E1D8wLL;A0D3wLC;EACE,WAAA;EACA,oBAAA;EACA,qBAAA;EACA,8BAAA;EACA,0CAAA;EACA,YAAA;E1D6wLH;A0D5wLG;EACE,cAAA;EACA,UAAA;EACA,oBAAA;EACA,qBAAA;EACA,8BAAA;E1D8wLL;A0D1wLC;EACE,UAAA;EACA,cAAA;EACA,mBAAA;EACA,uBAAA;EACA,4BAAA;EACA,wCAAA;E1D4wLH;A0D3wLG;EACE,cAAA;EACA,YAAA;EACA,uBAAA;EACA,4BAAA;EACA,eAAA;E1D6wLL;A2D14LD;EACE,oBAAA;E3D44LD;A2Dz4LD;EACE,oBAAA;EACA,kBAAA;EACA,aAAA;E3D24LD;A2D94LD;EAMI,eAAA;EACA,oBAAA;EtD6KF,2CAAA;EACK,sCAAA;EACG,mCAAA;EL+tLT;A2Dr5LD;;EAcM,gBAAA;E3D24LL;A2Dj3LC;EAAA;ItDiKA,wDAAA;IAEK,8CAAA;IACG,wCAAA;IA7JR,qCAAA;IAEQ,6BAAA;IA+GR,2BAAA;IAEQ,mBAAA;ILowLP;E2D/4LG;;ItDmHJ,4CAAA;IACQ,oCAAA;IsDjHF,SAAA;I3Dk5LL;E2Dh5LG;;ItD8GJ,6CAAA;IACQ,qCAAA;IsD5GF,SAAA;I3Dm5LL;E2Dj5LG;;;ItDyGJ,yCAAA;IACQ,iCAAA;IsDtGF,SAAA;I3Do5LL;EACF;A2D17LD;;;EA6CI,gBAAA;E3Dk5LH;A2D/7LD;EAiDI,SAAA;E3Di5LH;A2Dl8LD;;EAsDI,oBAAA;EACA,QAAA;EACA,aAAA;E3Dg5LH;A2Dx8LD;EA4DI,YAAA;E3D+4LH;A2D38LD;EA+DI,aAAA;E3D+4LH;A2D98LD;;EAmEI,SAAA;E3D+4LH;A2Dl9LD;EAuEI,aAAA;E3D84LH;A2Dr9LD;EA0EI,YAAA;E3D84LH;A2Dt4LD;EACE,oBAAA;EACA,QAAA;EACA,SAAA;EACA,WAAA;EACA,YAAA;ErC9FA,cAAA;EAGA,2BAAA;EqC6FA,iBAAA;EACA,gBAAA;EACA,oBAAA;EACA,2CAAA;E3Dy4LD;A2Dp4LC;EblGE,oGAAA;EACA,+FAAA;EACA,sHAAA;EAAA,gGAAA;EACA,6BAAA;EACA,wHAAA;E9Cy+LH;A2Dx4LC;EACE,YAAA;EACA,UAAA;EbvGA,oGAAA;EACA,+FAAA;EACA,sHAAA;EAAA,gGAAA;EACA,6BAAA;EACA,wHAAA;E9Ck/LH;A2D14LC;;EAEE,YAAA;EACA,gBAAA;EACA,uBAAA;ErCtHF,cAAA;EAGA,2BAAA;EtBigMD;A2D36LD;;;;EAsCI,oBAAA;EACA,UAAA;EACA,YAAA;EACA,uBAAA;E3D24LH;A2Dp7LD;;EA6CI,WAAA;EACA,oBAAA;E3D24LH;A2Dz7LD;;EAkDI,YAAA;EACA,qBAAA;E3D24LH;A2D97LD;;EAuDI,aAAA;EACA,cAAA;EACA,mBAAA;EACA,gBAAA;EACA,oBAAA;E3D24LH;A2Dt4LG;EACE,kBAAA;E3Dw4LL;A2Dp4LG;EACE,kBAAA;E3Ds4LL;A2D53LD;EACE,oBAAA;EACA,cAAA;EACA,WAAA;EACA,aAAA;EACA,YAAA;EACA,mBAAA;EACA,iBAAA;EACA,kBAAA;EACA,oBAAA;E3D83LD;A2Dv4LD;EAYI,uBAAA;EACA,aAAA;EACA,cAAA;EACA,aAAA;EACA,qBAAA;EACA,2BAAA;EACA,qBAAA;EACA,iBAAA;EAWA,2BAAA;EACA,oCAAA;E3Do3LH;A2Dn5LD;EAkCI,WAAA;EACA,aAAA;EACA,cAAA;EACA,2BAAA;E3Do3LH;A2D72LD;EACE,oBAAA;EACA,WAAA;EACA,YAAA;EACA,cAAA;EACA,aAAA;EACA,mBAAA;EACA,sBAAA;EACA,gBAAA;EACA,oBAAA;EACA,2CAAA;E3D+2LD;A2D92LC;EACE,mBAAA;E3Dg3LH;A2Dv0LD;EAhCE;;;;IAKI,aAAA;IACA,cAAA;IACA,mBAAA;IACA,iBAAA;I3Dy2LH;E2Dj3LD;;IAYI,oBAAA;I3Dy2LH;E2Dr3LD;;IAgBI,qBAAA;I3Dy2LH;E2Dp2LD;IACE,WAAA;IACA,YAAA;IACA,sBAAA;I3Ds2LD;E2Dl2LD;IACE,cAAA;I3Do2LD;EACF;A4DlmMC;;;;;;;;;;;;;;;;;;;;;;;;;;;;;;EAEE,cAAA;EACA,gBAAA;E5DgoMH;A4D9nMC;;;;;;;;;;;;;;;EACE,aAAA;E5D8oMH;AiCtpMD;E4BRE,gBAAA;EACA,mBAAA;EACA,oBAAA;E7DiqMD;AiCxpMD;EACE,yBAAA;EjC0pMD;AiCxpMD;EACE,wBAAA;EjC0pMD;AiClpMD;EACE,0BAAA;EjCopMD;AiClpMD;EACE,2BAAA;EjCopMD;AiClpMD;EACE,oBAAA;EjCopMD;AiClpMD;E6BzBE,aAAA;EACA,oBAAA;EACA,mBAAA;EACA,+BAAA;EACA,WAAA;E9D8qMD;AiChpMD;EACE,0BAAA;EjCkpMD;AiC3oMD;EACE,iBAAA;EjC6oMD;A+D9qMD;EACE,qBAAA;E/DgrMD;A+D1qMD;;;;ECdE,0BAAA;EhE8rMD;A+DzqMD;;;;;;;;;;;;EAYE,0BAAA;E/D2qMD;A+DpqMD;EAAA;IChDE,2BAAA;IhEwtMC;EgEvtMD;IAAU,gBAAA;IhE0tMT;EgEztMD;IAAU,+BAAA;IhE4tMT;EgE3tMD;;IACU,gCAAA;IhE8tMT;EACF;A+D9qMD;EAAA;IAFI,2BAAA;I/DorMD;EACF;A+D9qMD;EAAA;IAFI,4BAAA;I/DorMD;EACF;A+D9qMD;EAAA;IAFI,kCAAA;I/DorMD;EACF;A+D7qMD;EAAA;ICrEE,2BAAA;IhEsvMC;EgErvMD;IAAU,gBAAA;IhEwvMT;EgEvvMD;IAAU,+BAAA;IhE0vMT;EgEzvMD;;IACU,gCAAA;IhE4vMT;EACF;A+DvrMD;EAAA;IAFI,2BAAA;I/D6rMD;EACF;A+DvrMD;EAAA;IAFI,4BAAA;I/D6rMD;EACF;A+DvrMD;EAAA;IAFI,kCAAA;I/D6rMD;EACF;A+DtrMD;EAAA;IC1FE,2BAAA;IhEoxMC;EgEnxMD;IAAU,gBAAA;IhEsxMT;EgErxMD;IAAU,+BAAA;IhEwxMT;EgEvxMD;;IACU,gCAAA;IhE0xMT;EACF;A+DhsMD;EAAA;IAFI,2BAAA;I/DssMD;EACF;A+DhsMD;EAAA;IAFI,4BAAA;I/DssMD;EACF;A+DhsMD;EAAA;IAFI,kCAAA;I/DssMD;EACF;A+D/rMD;EAAA;IC/GE,2BAAA;IhEkzMC;EgEjzMD;IAAU,gBAAA;IhEozMT;EgEnzMD;IAAU,+BAAA;IhEszMT;EgErzMD;;IACU,gCAAA;IhEwzMT;EACF;A+DzsMD;EAAA;IAFI,2BAAA;I/D+sMD;EACF;A+DzsMD;EAAA;IAFI,4BAAA;I/D+sMD;EACF;A+DzsMD;EAAA;IAFI,kCAAA;I/D+sMD;EACF;A+DxsMD;EAAA;IC5HE,0BAAA;IhEw0MC;EACF;A+DxsMD;EAAA;ICjIE,0BAAA;IhE60MC;EACF;A+DxsMD;EAAA;ICtIE,0BAAA;IhEk1MC;EACF;A+DxsMD;EAAA;IC3IE,0BAAA;IhEu1MC;EACF;A+DrsMD;ECnJE,0BAAA;EhE21MD;A+DlsMD;EAAA;ICjKE,2BAAA;IhEu2MC;EgEt2MD;IAAU,gBAAA;IhEy2MT;EgEx2MD;IAAU,+BAAA;IhE22MT;EgE12MD;;IACU,gCAAA;IhE62MT;EACF;A+DhtMD;EACE,0BAAA;E/DktMD;A+D7sMD;EAAA;IAFI,2BAAA;I/DmtMD;EACF;A+DjtMD;EACE,0BAAA;E/DmtMD;A+D9sMD;EAAA;IAFI,4BAAA;I/DotMD;EACF;A+DltMD;EACE,0BAAA;E/DotMD;A+D/sMD;EAAA;IAFI,kCAAA;I/DqtMD;EACF;A+D9sMD;EAAA;ICpLE,0BAAA;IhEs4MC;EACF","file":"bootstrap.css","sourcesContent":["/*! normalize.css v3.0.2 | MIT License | git.io/normalize */\nhtml {\n font-family: sans-serif;\n -ms-text-size-adjust: 100%;\n -webkit-text-size-adjust: 100%;\n}\nbody {\n margin: 0;\n}\narticle,\naside,\ndetails,\nfigcaption,\nfigure,\nfooter,\nheader,\nhgroup,\nmain,\nmenu,\nnav,\nsection,\nsummary {\n display: block;\n}\naudio,\ncanvas,\nprogress,\nvideo {\n display: inline-block;\n vertical-align: baseline;\n}\naudio:not([controls]) {\n display: none;\n height: 0;\n}\n[hidden],\ntemplate {\n display: none;\n}\na {\n background-color: transparent;\n}\na:active,\na:hover {\n outline: 0;\n}\nabbr[title] {\n border-bottom: 1px dotted;\n}\nb,\nstrong {\n font-weight: bold;\n}\ndfn {\n font-style: italic;\n}\nh1 {\n font-size: 2em;\n margin: 0.67em 0;\n}\nmark {\n background: #ff0;\n color: #000;\n}\nsmall {\n font-size: 80%;\n}\nsub,\nsup {\n font-size: 75%;\n line-height: 0;\n position: relative;\n vertical-align: baseline;\n}\nsup {\n top: -0.5em;\n}\nsub {\n bottom: -0.25em;\n}\nimg {\n border: 0;\n}\nsvg:not(:root) {\n overflow: hidden;\n}\nfigure {\n margin: 1em 40px;\n}\nhr {\n -moz-box-sizing: content-box;\n box-sizing: content-box;\n height: 0;\n}\npre {\n overflow: auto;\n}\ncode,\nkbd,\npre,\nsamp {\n font-family: monospace, monospace;\n font-size: 1em;\n}\nbutton,\ninput,\noptgroup,\nselect,\ntextarea {\n color: inherit;\n font: inherit;\n margin: 0;\n}\nbutton {\n overflow: visible;\n}\nbutton,\nselect {\n text-transform: none;\n}\nbutton,\nhtml input[type=\"button\"],\ninput[type=\"reset\"],\ninput[type=\"submit\"] {\n -webkit-appearance: button;\n cursor: pointer;\n}\nbutton[disabled],\nhtml input[disabled] {\n cursor: default;\n}\nbutton::-moz-focus-inner,\ninput::-moz-focus-inner {\n border: 0;\n padding: 0;\n}\ninput {\n line-height: normal;\n}\ninput[type=\"checkbox\"],\ninput[type=\"radio\"] {\n box-sizing: border-box;\n padding: 0;\n}\ninput[type=\"number\"]::-webkit-inner-spin-button,\ninput[type=\"number\"]::-webkit-outer-spin-button {\n height: auto;\n}\ninput[type=\"search\"] {\n -webkit-appearance: textfield;\n -moz-box-sizing: content-box;\n -webkit-box-sizing: content-box;\n box-sizing: content-box;\n}\ninput[type=\"search\"]::-webkit-search-cancel-button,\ninput[type=\"search\"]::-webkit-search-decoration {\n -webkit-appearance: none;\n}\nfieldset {\n border: 1px solid #c0c0c0;\n margin: 0 2px;\n padding: 0.35em 0.625em 0.75em;\n}\nlegend {\n border: 0;\n padding: 0;\n}\ntextarea {\n overflow: auto;\n}\noptgroup {\n font-weight: bold;\n}\ntable {\n border-collapse: collapse;\n border-spacing: 0;\n}\ntd,\nth {\n padding: 0;\n}\n/*! Source: https://github.com/h5bp/html5-boilerplate/blob/master/src/css/main.css */\n@media print {\n *,\n *:before,\n *:after {\n background: transparent !important;\n color: #000 !important;\n box-shadow: none !important;\n text-shadow: none !important;\n }\n a,\n a:visited {\n text-decoration: underline;\n }\n a[href]:after {\n content: \" (\" attr(href) \")\";\n }\n abbr[title]:after {\n content: \" (\" attr(title) \")\";\n }\n a[href^=\"#\"]:after,\n a[href^=\"javascript:\"]:after {\n content: \"\";\n }\n pre,\n blockquote {\n border: 1px solid #999;\n page-break-inside: avoid;\n }\n thead {\n display: table-header-group;\n }\n tr,\n img {\n page-break-inside: avoid;\n }\n img {\n max-width: 100% !important;\n }\n p,\n h2,\n h3 {\n orphans: 3;\n widows: 3;\n }\n h2,\n h3 {\n page-break-after: avoid;\n }\n select {\n background: #fff !important;\n }\n .navbar {\n display: none;\n }\n .btn > .caret,\n .dropup > .btn > .caret {\n border-top-color: #000 !important;\n }\n .label {\n border: 1px solid #000;\n }\n .table {\n border-collapse: collapse !important;\n }\n .table td,\n .table th {\n background-color: #fff !important;\n }\n .table-bordered th,\n .table-bordered td {\n border: 1px solid #ddd !important;\n }\n}\n@font-face {\n font-family: 'Glyphicons Halflings';\n src: url('../fonts/glyphicons-halflings-regular.eot');\n src: url('../fonts/glyphicons-halflings-regular.eot?#iefix') format('embedded-opentype'), url('../fonts/glyphicons-halflings-regular.woff2') format('woff2'), url('../fonts/glyphicons-halflings-regular.woff') format('woff'), url('../fonts/glyphicons-halflings-regular.ttf') format('truetype'), url('../fonts/glyphicons-halflings-regular.svg#glyphicons_halflingsregular') format('svg');\n}\n.glyphicon {\n position: relative;\n top: 1px;\n display: inline-block;\n font-family: 'Glyphicons Halflings';\n font-style: normal;\n font-weight: normal;\n line-height: 1;\n -webkit-font-smoothing: antialiased;\n -moz-osx-font-smoothing: grayscale;\n}\n.glyphicon-asterisk:before {\n content: \"\\2a\";\n}\n.glyphicon-plus:before {\n content: \"\\2b\";\n}\n.glyphicon-euro:before,\n.glyphicon-eur:before {\n content: \"\\20ac\";\n}\n.glyphicon-minus:before {\n content: \"\\2212\";\n}\n.glyphicon-cloud:before {\n content: \"\\2601\";\n}\n.glyphicon-envelope:before {\n content: \"\\2709\";\n}\n.glyphicon-pencil:before {\n content: \"\\270f\";\n}\n.glyphicon-glass:before {\n content: \"\\e001\";\n}\n.glyphicon-music:before {\n content: \"\\e002\";\n}\n.glyphicon-search:before {\n content: \"\\e003\";\n}\n.glyphicon-heart:before {\n content: \"\\e005\";\n}\n.glyphicon-star:before {\n content: \"\\e006\";\n}\n.glyphicon-star-empty:before {\n content: \"\\e007\";\n}\n.glyphicon-user:before {\n content: \"\\e008\";\n}\n.glyphicon-film:before {\n content: \"\\e009\";\n}\n.glyphicon-th-large:before {\n content: \"\\e010\";\n}\n.glyphicon-th:before {\n content: \"\\e011\";\n}\n.glyphicon-th-list:before {\n content: \"\\e012\";\n}\n.glyphicon-ok:before {\n content: \"\\e013\";\n}\n.glyphicon-remove:before {\n content: \"\\e014\";\n}\n.glyphicon-zoom-in:before {\n content: \"\\e015\";\n}\n.glyphicon-zoom-out:before {\n content: \"\\e016\";\n}\n.glyphicon-off:before {\n content: \"\\e017\";\n}\n.glyphicon-signal:before {\n content: \"\\e018\";\n}\n.glyphicon-cog:before {\n content: \"\\e019\";\n}\n.glyphicon-trash:before {\n content: \"\\e020\";\n}\n.glyphicon-home:before {\n content: \"\\e021\";\n}\n.glyphicon-file:before {\n content: \"\\e022\";\n}\n.glyphicon-time:before {\n content: \"\\e023\";\n}\n.glyphicon-road:before {\n content: \"\\e024\";\n}\n.glyphicon-download-alt:before {\n content: \"\\e025\";\n}\n.glyphicon-download:before {\n content: \"\\e026\";\n}\n.glyphicon-upload:before {\n content: \"\\e027\";\n}\n.glyphicon-inbox:before {\n content: \"\\e028\";\n}\n.glyphicon-play-circle:before {\n content: \"\\e029\";\n}\n.glyphicon-repeat:before {\n content: \"\\e030\";\n}\n.glyphicon-refresh:before {\n content: \"\\e031\";\n}\n.glyphicon-list-alt:before {\n content: \"\\e032\";\n}\n.glyphicon-lock:before {\n content: \"\\e033\";\n}\n.glyphicon-flag:before {\n content: \"\\e034\";\n}\n.glyphicon-headphones:before {\n content: \"\\e035\";\n}\n.glyphicon-volume-off:before {\n content: \"\\e036\";\n}\n.glyphicon-volume-down:before {\n content: \"\\e037\";\n}\n.glyphicon-volume-up:before {\n content: \"\\e038\";\n}\n.glyphicon-qrcode:before {\n content: \"\\e039\";\n}\n.glyphicon-barcode:before {\n content: \"\\e040\";\n}\n.glyphicon-tag:before {\n content: \"\\e041\";\n}\n.glyphicon-tags:before {\n content: \"\\e042\";\n}\n.glyphicon-book:before {\n content: \"\\e043\";\n}\n.glyphicon-bookmark:before {\n content: \"\\e044\";\n}\n.glyphicon-print:before {\n content: \"\\e045\";\n}\n.glyphicon-camera:before {\n content: \"\\e046\";\n}\n.glyphicon-font:before {\n content: \"\\e047\";\n}\n.glyphicon-bold:before {\n content: \"\\e048\";\n}\n.glyphicon-italic:before {\n content: \"\\e049\";\n}\n.glyphicon-text-height:before {\n content: \"\\e050\";\n}\n.glyphicon-text-width:before {\n content: \"\\e051\";\n}\n.glyphicon-align-left:before {\n content: \"\\e052\";\n}\n.glyphicon-align-center:before {\n content: \"\\e053\";\n}\n.glyphicon-align-right:before {\n content: \"\\e054\";\n}\n.glyphicon-align-justify:before {\n content: \"\\e055\";\n}\n.glyphicon-list:before {\n content: \"\\e056\";\n}\n.glyphicon-indent-left:before {\n content: \"\\e057\";\n}\n.glyphicon-indent-right:before {\n content: \"\\e058\";\n}\n.glyphicon-facetime-video:before {\n content: \"\\e059\";\n}\n.glyphicon-picture:before {\n content: \"\\e060\";\n}\n.glyphicon-map-marker:before {\n content: \"\\e062\";\n}\n.glyphicon-adjust:before {\n content: \"\\e063\";\n}\n.glyphicon-tint:before {\n content: \"\\e064\";\n}\n.glyphicon-edit:before {\n content: \"\\e065\";\n}\n.glyphicon-share:before {\n content: \"\\e066\";\n}\n.glyphicon-check:before {\n content: \"\\e067\";\n}\n.glyphicon-move:before {\n content: \"\\e068\";\n}\n.glyphicon-step-backward:before {\n content: \"\\e069\";\n}\n.glyphicon-fast-backward:before {\n content: \"\\e070\";\n}\n.glyphicon-backward:before {\n content: \"\\e071\";\n}\n.glyphicon-play:before {\n content: \"\\e072\";\n}\n.glyphicon-pause:before {\n content: \"\\e073\";\n}\n.glyphicon-stop:before {\n content: \"\\e074\";\n}\n.glyphicon-forward:before {\n content: \"\\e075\";\n}\n.glyphicon-fast-forward:before {\n content: \"\\e076\";\n}\n.glyphicon-step-forward:before {\n content: \"\\e077\";\n}\n.glyphicon-eject:before {\n content: \"\\e078\";\n}\n.glyphicon-chevron-left:before {\n content: \"\\e079\";\n}\n.glyphicon-chevron-right:before {\n content: \"\\e080\";\n}\n.glyphicon-plus-sign:before {\n content: \"\\e081\";\n}\n.glyphicon-minus-sign:before {\n content: \"\\e082\";\n}\n.glyphicon-remove-sign:before {\n content: \"\\e083\";\n}\n.glyphicon-ok-sign:before {\n content: \"\\e084\";\n}\n.glyphicon-question-sign:before {\n content: \"\\e085\";\n}\n.glyphicon-info-sign:before {\n content: \"\\e086\";\n}\n.glyphicon-screenshot:before {\n content: \"\\e087\";\n}\n.glyphicon-remove-circle:before {\n content: \"\\e088\";\n}\n.glyphicon-ok-circle:before {\n content: \"\\e089\";\n}\n.glyphicon-ban-circle:before {\n content: \"\\e090\";\n}\n.glyphicon-arrow-left:before {\n content: \"\\e091\";\n}\n.glyphicon-arrow-right:before {\n content: \"\\e092\";\n}\n.glyphicon-arrow-up:before {\n content: \"\\e093\";\n}\n.glyphicon-arrow-down:before {\n content: \"\\e094\";\n}\n.glyphicon-share-alt:before {\n content: \"\\e095\";\n}\n.glyphicon-resize-full:before {\n content: \"\\e096\";\n}\n.glyphicon-resize-small:before {\n content: \"\\e097\";\n}\n.glyphicon-exclamation-sign:before {\n content: \"\\e101\";\n}\n.glyphicon-gift:before {\n content: \"\\e102\";\n}\n.glyphicon-leaf:before {\n content: \"\\e103\";\n}\n.glyphicon-fire:before {\n content: \"\\e104\";\n}\n.glyphicon-eye-open:before {\n content: \"\\e105\";\n}\n.glyphicon-eye-close:before {\n content: \"\\e106\";\n}\n.glyphicon-warning-sign:before {\n content: \"\\e107\";\n}\n.glyphicon-plane:before {\n content: \"\\e108\";\n}\n.glyphicon-calendar:before {\n content: \"\\e109\";\n}\n.glyphicon-random:before {\n content: \"\\e110\";\n}\n.glyphicon-comment:before {\n content: \"\\e111\";\n}\n.glyphicon-magnet:before {\n content: \"\\e112\";\n}\n.glyphicon-chevron-up:before {\n content: \"\\e113\";\n}\n.glyphicon-chevron-down:before {\n content: \"\\e114\";\n}\n.glyphicon-retweet:before {\n content: \"\\e115\";\n}\n.glyphicon-shopping-cart:before {\n content: \"\\e116\";\n}\n.glyphicon-folder-close:before {\n content: \"\\e117\";\n}\n.glyphicon-folder-open:before {\n content: \"\\e118\";\n}\n.glyphicon-resize-vertical:before {\n content: \"\\e119\";\n}\n.glyphicon-resize-horizontal:before {\n content: \"\\e120\";\n}\n.glyphicon-hdd:before {\n content: \"\\e121\";\n}\n.glyphicon-bullhorn:before {\n content: \"\\e122\";\n}\n.glyphicon-bell:before {\n content: \"\\e123\";\n}\n.glyphicon-certificate:before {\n content: \"\\e124\";\n}\n.glyphicon-thumbs-up:before {\n content: \"\\e125\";\n}\n.glyphicon-thumbs-down:before {\n content: \"\\e126\";\n}\n.glyphicon-hand-right:before {\n content: \"\\e127\";\n}\n.glyphicon-hand-left:before {\n content: \"\\e128\";\n}\n.glyphicon-hand-up:before {\n content: \"\\e129\";\n}\n.glyphicon-hand-down:before {\n content: \"\\e130\";\n}\n.glyphicon-circle-arrow-right:before {\n content: \"\\e131\";\n}\n.glyphicon-circle-arrow-left:before {\n content: \"\\e132\";\n}\n.glyphicon-circle-arrow-up:before {\n content: \"\\e133\";\n}\n.glyphicon-circle-arrow-down:before {\n content: \"\\e134\";\n}\n.glyphicon-globe:before {\n content: \"\\e135\";\n}\n.glyphicon-wrench:before {\n content: \"\\e136\";\n}\n.glyphicon-tasks:before {\n content: \"\\e137\";\n}\n.glyphicon-filter:before {\n content: \"\\e138\";\n}\n.glyphicon-briefcase:before {\n content: \"\\e139\";\n}\n.glyphicon-fullscreen:before {\n content: \"\\e140\";\n}\n.glyphicon-dashboard:before {\n content: \"\\e141\";\n}\n.glyphicon-paperclip:before {\n content: \"\\e142\";\n}\n.glyphicon-heart-empty:before {\n content: \"\\e143\";\n}\n.glyphicon-link:before {\n content: \"\\e144\";\n}\n.glyphicon-phone:before {\n content: \"\\e145\";\n}\n.glyphicon-pushpin:before {\n content: \"\\e146\";\n}\n.glyphicon-usd:before {\n content: \"\\e148\";\n}\n.glyphicon-gbp:before {\n content: \"\\e149\";\n}\n.glyphicon-sort:before {\n content: \"\\e150\";\n}\n.glyphicon-sort-by-alphabet:before {\n content: \"\\e151\";\n}\n.glyphicon-sort-by-alphabet-alt:before {\n content: \"\\e152\";\n}\n.glyphicon-sort-by-order:before {\n content: \"\\e153\";\n}\n.glyphicon-sort-by-order-alt:before {\n content: \"\\e154\";\n}\n.glyphicon-sort-by-attributes:before {\n content: \"\\e155\";\n}\n.glyphicon-sort-by-attributes-alt:before {\n content: \"\\e156\";\n}\n.glyphicon-unchecked:before {\n content: \"\\e157\";\n}\n.glyphicon-expand:before {\n content: \"\\e158\";\n}\n.glyphicon-collapse-down:before {\n content: \"\\e159\";\n}\n.glyphicon-collapse-up:before {\n content: \"\\e160\";\n}\n.glyphicon-log-in:before {\n content: \"\\e161\";\n}\n.glyphicon-flash:before {\n content: \"\\e162\";\n}\n.glyphicon-log-out:before {\n content: \"\\e163\";\n}\n.glyphicon-new-window:before {\n content: \"\\e164\";\n}\n.glyphicon-record:before {\n content: \"\\e165\";\n}\n.glyphicon-save:before {\n content: \"\\e166\";\n}\n.glyphicon-open:before {\n content: \"\\e167\";\n}\n.glyphicon-saved:before {\n content: \"\\e168\";\n}\n.glyphicon-import:before {\n content: \"\\e169\";\n}\n.glyphicon-export:before {\n content: \"\\e170\";\n}\n.glyphicon-send:before {\n content: \"\\e171\";\n}\n.glyphicon-floppy-disk:before {\n content: \"\\e172\";\n}\n.glyphicon-floppy-saved:before {\n content: \"\\e173\";\n}\n.glyphicon-floppy-remove:before {\n content: \"\\e174\";\n}\n.glyphicon-floppy-save:before {\n content: \"\\e175\";\n}\n.glyphicon-floppy-open:before {\n content: \"\\e176\";\n}\n.glyphicon-credit-card:before {\n content: \"\\e177\";\n}\n.glyphicon-transfer:before {\n content: \"\\e178\";\n}\n.glyphicon-cutlery:before {\n content: \"\\e179\";\n}\n.glyphicon-header:before {\n content: \"\\e180\";\n}\n.glyphicon-compressed:before {\n content: \"\\e181\";\n}\n.glyphicon-earphone:before {\n content: \"\\e182\";\n}\n.glyphicon-phone-alt:before {\n content: \"\\e183\";\n}\n.glyphicon-tower:before {\n content: \"\\e184\";\n}\n.glyphicon-stats:before {\n content: \"\\e185\";\n}\n.glyphicon-sd-video:before {\n content: \"\\e186\";\n}\n.glyphicon-hd-video:before {\n content: \"\\e187\";\n}\n.glyphicon-subtitles:before {\n content: \"\\e188\";\n}\n.glyphicon-sound-stereo:before {\n content: \"\\e189\";\n}\n.glyphicon-sound-dolby:before {\n content: \"\\e190\";\n}\n.glyphicon-sound-5-1:before {\n content: \"\\e191\";\n}\n.glyphicon-sound-6-1:before {\n content: \"\\e192\";\n}\n.glyphicon-sound-7-1:before {\n content: \"\\e193\";\n}\n.glyphicon-copyright-mark:before {\n content: \"\\e194\";\n}\n.glyphicon-registration-mark:before {\n content: \"\\e195\";\n}\n.glyphicon-cloud-download:before {\n content: \"\\e197\";\n}\n.glyphicon-cloud-upload:before {\n content: \"\\e198\";\n}\n.glyphicon-tree-conifer:before {\n content: \"\\e199\";\n}\n.glyphicon-tree-deciduous:before {\n content: \"\\e200\";\n}\n.glyphicon-cd:before {\n content: \"\\e201\";\n}\n.glyphicon-save-file:before {\n content: \"\\e202\";\n}\n.glyphicon-open-file:before {\n content: \"\\e203\";\n}\n.glyphicon-level-up:before {\n content: \"\\e204\";\n}\n.glyphicon-copy:before {\n content: \"\\e205\";\n}\n.glyphicon-paste:before {\n content: \"\\e206\";\n}\n.glyphicon-alert:before {\n content: \"\\e209\";\n}\n.glyphicon-equalizer:before {\n content: \"\\e210\";\n}\n.glyphicon-king:before {\n content: \"\\e211\";\n}\n.glyphicon-queen:before {\n content: \"\\e212\";\n}\n.glyphicon-pawn:before {\n content: \"\\e213\";\n}\n.glyphicon-bishop:before {\n content: \"\\e214\";\n}\n.glyphicon-knight:before {\n content: \"\\e215\";\n}\n.glyphicon-baby-formula:before {\n content: \"\\e216\";\n}\n.glyphicon-tent:before {\n content: \"\\26fa\";\n}\n.glyphicon-blackboard:before {\n content: \"\\e218\";\n}\n.glyphicon-bed:before {\n content: \"\\e219\";\n}\n.glyphicon-apple:before {\n content: \"\\f8ff\";\n}\n.glyphicon-erase:before {\n content: \"\\e221\";\n}\n.glyphicon-hourglass:before {\n content: \"\\231b\";\n}\n.glyphicon-lamp:before {\n content: \"\\e223\";\n}\n.glyphicon-duplicate:before {\n content: \"\\e224\";\n}\n.glyphicon-piggy-bank:before {\n content: \"\\e225\";\n}\n.glyphicon-scissors:before {\n content: \"\\e226\";\n}\n.glyphicon-bitcoin:before {\n content: \"\\e227\";\n}\n.glyphicon-btc:before {\n content: \"\\e227\";\n}\n.glyphicon-xbt:before {\n content: \"\\e227\";\n}\n.glyphicon-yen:before {\n content: \"\\00a5\";\n}\n.glyphicon-jpy:before {\n content: \"\\00a5\";\n}\n.glyphicon-ruble:before {\n content: \"\\20bd\";\n}\n.glyphicon-rub:before {\n content: \"\\20bd\";\n}\n.glyphicon-scale:before {\n content: \"\\e230\";\n}\n.glyphicon-ice-lolly:before {\n content: \"\\e231\";\n}\n.glyphicon-ice-lolly-tasted:before {\n content: \"\\e232\";\n}\n.glyphicon-education:before {\n content: \"\\e233\";\n}\n.glyphicon-option-horizontal:before {\n content: \"\\e234\";\n}\n.glyphicon-option-vertical:before {\n content: \"\\e235\";\n}\n.glyphicon-menu-hamburger:before {\n content: \"\\e236\";\n}\n.glyphicon-modal-window:before {\n content: \"\\e237\";\n}\n.glyphicon-oil:before {\n content: \"\\e238\";\n}\n.glyphicon-grain:before {\n content: \"\\e239\";\n}\n.glyphicon-sunglasses:before {\n content: \"\\e240\";\n}\n.glyphicon-text-size:before {\n content: \"\\e241\";\n}\n.glyphicon-text-color:before {\n content: \"\\e242\";\n}\n.glyphicon-text-background:before {\n content: \"\\e243\";\n}\n.glyphicon-object-align-top:before {\n content: \"\\e244\";\n}\n.glyphicon-object-align-bottom:before {\n content: \"\\e245\";\n}\n.glyphicon-object-align-horizontal:before {\n content: \"\\e246\";\n}\n.glyphicon-object-align-left:before {\n content: \"\\e247\";\n}\n.glyphicon-object-align-vertical:before {\n content: \"\\e248\";\n}\n.glyphicon-object-align-right:before {\n content: \"\\e249\";\n}\n.glyphicon-triangle-right:before {\n content: \"\\e250\";\n}\n.glyphicon-triangle-left:before {\n content: \"\\e251\";\n}\n.glyphicon-triangle-bottom:before {\n content: \"\\e252\";\n}\n.glyphicon-triangle-top:before {\n content: \"\\e253\";\n}\n.glyphicon-console:before {\n content: \"\\e254\";\n}\n.glyphicon-superscript:before {\n content: \"\\e255\";\n}\n.glyphicon-subscript:before {\n content: \"\\e256\";\n}\n.glyphicon-menu-left:before {\n content: \"\\e257\";\n}\n.glyphicon-menu-right:before {\n content: \"\\e258\";\n}\n.glyphicon-menu-down:before {\n content: \"\\e259\";\n}\n.glyphicon-menu-up:before {\n content: \"\\e260\";\n}\n* {\n -webkit-box-sizing: border-box;\n -moz-box-sizing: border-box;\n box-sizing: border-box;\n}\n*:before,\n*:after {\n -webkit-box-sizing: border-box;\n -moz-box-sizing: border-box;\n box-sizing: border-box;\n}\nhtml {\n font-size: 10px;\n -webkit-tap-highlight-color: rgba(0, 0, 0, 0);\n}\nbody {\n font-family: \"Helvetica Neue\", Helvetica, Arial, sans-serif;\n font-size: 14px;\n line-height: 1.42857143;\n color: #333333;\n background-color: #ffffff;\n}\ninput,\nbutton,\nselect,\ntextarea {\n font-family: inherit;\n font-size: inherit;\n line-height: inherit;\n}\na {\n color: #337ab7;\n text-decoration: none;\n}\na:hover,\na:focus {\n color: #23527c;\n text-decoration: underline;\n}\na:focus {\n outline: thin dotted;\n outline: 5px auto -webkit-focus-ring-color;\n outline-offset: -2px;\n}\nfigure {\n margin: 0;\n}\nimg {\n vertical-align: middle;\n}\n.img-responsive,\n.thumbnail > img,\n.thumbnail a > img,\n.carousel-inner > .item > img,\n.carousel-inner > .item > a > img {\n display: block;\n max-width: 100%;\n height: auto;\n}\n.img-rounded {\n border-radius: 6px;\n}\n.img-thumbnail {\n padding: 4px;\n line-height: 1.42857143;\n background-color: #ffffff;\n border: 1px solid #dddddd;\n border-radius: 4px;\n -webkit-transition: all 0.2s ease-in-out;\n -o-transition: all 0.2s ease-in-out;\n transition: all 0.2s ease-in-out;\n display: inline-block;\n max-width: 100%;\n height: auto;\n}\n.img-circle {\n border-radius: 50%;\n}\nhr {\n margin-top: 20px;\n margin-bottom: 20px;\n border: 0;\n border-top: 1px solid #eeeeee;\n}\n.sr-only {\n position: absolute;\n width: 1px;\n height: 1px;\n margin: -1px;\n padding: 0;\n overflow: hidden;\n clip: rect(0, 0, 0, 0);\n border: 0;\n}\n.sr-only-focusable:active,\n.sr-only-focusable:focus {\n position: static;\n width: auto;\n height: auto;\n margin: 0;\n overflow: visible;\n clip: auto;\n}\n[role=\"button\"] {\n cursor: pointer;\n}\nh1,\nh2,\nh3,\nh4,\nh5,\nh6,\n.h1,\n.h2,\n.h3,\n.h4,\n.h5,\n.h6 {\n font-family: inherit;\n font-weight: 500;\n line-height: 1.1;\n color: inherit;\n}\nh1 small,\nh2 small,\nh3 small,\nh4 small,\nh5 small,\nh6 small,\n.h1 small,\n.h2 small,\n.h3 small,\n.h4 small,\n.h5 small,\n.h6 small,\nh1 .small,\nh2 .small,\nh3 .small,\nh4 .small,\nh5 .small,\nh6 .small,\n.h1 .small,\n.h2 .small,\n.h3 .small,\n.h4 .small,\n.h5 .small,\n.h6 .small {\n font-weight: normal;\n line-height: 1;\n color: #777777;\n}\nh1,\n.h1,\nh2,\n.h2,\nh3,\n.h3 {\n margin-top: 20px;\n margin-bottom: 10px;\n}\nh1 small,\n.h1 small,\nh2 small,\n.h2 small,\nh3 small,\n.h3 small,\nh1 .small,\n.h1 .small,\nh2 .small,\n.h2 .small,\nh3 .small,\n.h3 .small {\n font-size: 65%;\n}\nh4,\n.h4,\nh5,\n.h5,\nh6,\n.h6 {\n margin-top: 10px;\n margin-bottom: 10px;\n}\nh4 small,\n.h4 small,\nh5 small,\n.h5 small,\nh6 small,\n.h6 small,\nh4 .small,\n.h4 .small,\nh5 .small,\n.h5 .small,\nh6 .small,\n.h6 .small {\n font-size: 75%;\n}\nh1,\n.h1 {\n font-size: 36px;\n}\nh2,\n.h2 {\n font-size: 30px;\n}\nh3,\n.h3 {\n font-size: 24px;\n}\nh4,\n.h4 {\n font-size: 18px;\n}\nh5,\n.h5 {\n font-size: 14px;\n}\nh6,\n.h6 {\n font-size: 12px;\n}\np {\n margin: 0 0 10px;\n}\n.lead {\n margin-bottom: 20px;\n font-size: 16px;\n font-weight: 300;\n line-height: 1.4;\n}\n@media (min-width: 768px) {\n .lead {\n font-size: 21px;\n }\n}\nsmall,\n.small {\n font-size: 85%;\n}\nmark,\n.mark {\n background-color: #fcf8e3;\n padding: .2em;\n}\n.text-left {\n text-align: left;\n}\n.text-right {\n text-align: right;\n}\n.text-center {\n text-align: center;\n}\n.text-justify {\n text-align: justify;\n}\n.text-nowrap {\n white-space: nowrap;\n}\n.text-lowercase {\n text-transform: lowercase;\n}\n.text-uppercase {\n text-transform: uppercase;\n}\n.text-capitalize {\n text-transform: capitalize;\n}\n.text-muted {\n color: #777777;\n}\n.text-primary {\n color: #337ab7;\n}\na.text-primary:hover {\n color: #286090;\n}\n.text-success {\n color: #3c763d;\n}\na.text-success:hover {\n color: #2b542c;\n}\n.text-info {\n color: #31708f;\n}\na.text-info:hover {\n color: #245269;\n}\n.text-warning {\n color: #8a6d3b;\n}\na.text-warning:hover {\n color: #66512c;\n}\n.text-danger {\n color: #a94442;\n}\na.text-danger:hover {\n color: #843534;\n}\n.bg-primary {\n color: #fff;\n background-color: #337ab7;\n}\na.bg-primary:hover {\n background-color: #286090;\n}\n.bg-success {\n background-color: #dff0d8;\n}\na.bg-success:hover {\n background-color: #c1e2b3;\n}\n.bg-info {\n background-color: #d9edf7;\n}\na.bg-info:hover {\n background-color: #afd9ee;\n}\n.bg-warning {\n background-color: #fcf8e3;\n}\na.bg-warning:hover {\n background-color: #f7ecb5;\n}\n.bg-danger {\n background-color: #f2dede;\n}\na.bg-danger:hover {\n background-color: #e4b9b9;\n}\n.page-header {\n padding-bottom: 9px;\n margin: 40px 0 20px;\n border-bottom: 1px solid #eeeeee;\n}\nul,\nol {\n margin-top: 0;\n margin-bottom: 10px;\n}\nul ul,\nol ul,\nul ol,\nol ol {\n margin-bottom: 0;\n}\n.list-unstyled {\n padding-left: 0;\n list-style: none;\n}\n.list-inline {\n padding-left: 0;\n list-style: none;\n margin-left: -5px;\n}\n.list-inline > li {\n display: inline-block;\n padding-left: 5px;\n padding-right: 5px;\n}\ndl {\n margin-top: 0;\n margin-bottom: 20px;\n}\ndt,\ndd {\n line-height: 1.42857143;\n}\ndt {\n font-weight: bold;\n}\ndd {\n margin-left: 0;\n}\n@media (min-width: 768px) {\n .dl-horizontal dt {\n float: left;\n width: 160px;\n clear: left;\n text-align: right;\n overflow: hidden;\n text-overflow: ellipsis;\n white-space: nowrap;\n }\n .dl-horizontal dd {\n margin-left: 180px;\n }\n}\nabbr[title],\nabbr[data-original-title] {\n cursor: help;\n border-bottom: 1px dotted #777777;\n}\n.initialism {\n font-size: 90%;\n text-transform: uppercase;\n}\nblockquote {\n padding: 10px 20px;\n margin: 0 0 20px;\n font-size: 17.5px;\n border-left: 5px solid #eeeeee;\n}\nblockquote p:last-child,\nblockquote ul:last-child,\nblockquote ol:last-child {\n margin-bottom: 0;\n}\nblockquote footer,\nblockquote small,\nblockquote .small {\n display: block;\n font-size: 80%;\n line-height: 1.42857143;\n color: #777777;\n}\nblockquote footer:before,\nblockquote small:before,\nblockquote .small:before {\n content: '\\2014 \\00A0';\n}\n.blockquote-reverse,\nblockquote.pull-right {\n padding-right: 15px;\n padding-left: 0;\n border-right: 5px solid #eeeeee;\n border-left: 0;\n text-align: right;\n}\n.blockquote-reverse footer:before,\nblockquote.pull-right footer:before,\n.blockquote-reverse small:before,\nblockquote.pull-right small:before,\n.blockquote-reverse .small:before,\nblockquote.pull-right .small:before {\n content: '';\n}\n.blockquote-reverse footer:after,\nblockquote.pull-right footer:after,\n.blockquote-reverse small:after,\nblockquote.pull-right small:after,\n.blockquote-reverse .small:after,\nblockquote.pull-right .small:after {\n content: '\\00A0 \\2014';\n}\naddress {\n margin-bottom: 20px;\n font-style: normal;\n line-height: 1.42857143;\n}\ncode,\nkbd,\npre,\nsamp {\n font-family: Menlo, Monaco, Consolas, \"Courier New\", monospace;\n}\ncode {\n padding: 2px 4px;\n font-size: 90%;\n color: #c7254e;\n background-color: #f9f2f4;\n border-radius: 4px;\n}\nkbd {\n padding: 2px 4px;\n font-size: 90%;\n color: #ffffff;\n background-color: #333333;\n border-radius: 3px;\n box-shadow: inset 0 -1px 0 rgba(0, 0, 0, 0.25);\n}\nkbd kbd {\n padding: 0;\n font-size: 100%;\n font-weight: bold;\n box-shadow: none;\n}\npre {\n display: block;\n padding: 9.5px;\n margin: 0 0 10px;\n font-size: 13px;\n line-height: 1.42857143;\n word-break: break-all;\n word-wrap: break-word;\n color: #333333;\n background-color: #f5f5f5;\n border: 1px solid #cccccc;\n border-radius: 4px;\n}\npre code {\n padding: 0;\n font-size: inherit;\n color: inherit;\n white-space: pre-wrap;\n background-color: transparent;\n border-radius: 0;\n}\n.pre-scrollable {\n max-height: 340px;\n overflow-y: scroll;\n}\n.container {\n margin-right: auto;\n margin-left: auto;\n padding-left: 15px;\n padding-right: 15px;\n}\n@media (min-width: 768px) {\n .container {\n width: 750px;\n }\n}\n@media (min-width: 992px) {\n .container {\n width: 970px;\n }\n}\n@media (min-width: 1200px) {\n .container {\n width: 1170px;\n }\n}\n.container-fluid {\n margin-right: auto;\n margin-left: auto;\n padding-left: 15px;\n padding-right: 15px;\n}\n.row {\n margin-left: -15px;\n margin-right: -15px;\n}\n.col-xs-1, .col-sm-1, .col-md-1, .col-lg-1, .col-xs-2, .col-sm-2, .col-md-2, .col-lg-2, .col-xs-3, .col-sm-3, .col-md-3, .col-lg-3, .col-xs-4, .col-sm-4, .col-md-4, .col-lg-4, .col-xs-5, .col-sm-5, .col-md-5, .col-lg-5, .col-xs-6, .col-sm-6, .col-md-6, .col-lg-6, .col-xs-7, .col-sm-7, .col-md-7, .col-lg-7, .col-xs-8, .col-sm-8, .col-md-8, .col-lg-8, .col-xs-9, .col-sm-9, .col-md-9, .col-lg-9, .col-xs-10, .col-sm-10, .col-md-10, .col-lg-10, .col-xs-11, .col-sm-11, .col-md-11, .col-lg-11, .col-xs-12, .col-sm-12, .col-md-12, .col-lg-12 {\n position: relative;\n min-height: 1px;\n padding-left: 15px;\n padding-right: 15px;\n}\n.col-xs-1, .col-xs-2, .col-xs-3, .col-xs-4, .col-xs-5, .col-xs-6, .col-xs-7, .col-xs-8, .col-xs-9, .col-xs-10, .col-xs-11, .col-xs-12 {\n float: left;\n}\n.col-xs-12 {\n width: 100%;\n}\n.col-xs-11 {\n width: 91.66666667%;\n}\n.col-xs-10 {\n width: 83.33333333%;\n}\n.col-xs-9 {\n width: 75%;\n}\n.col-xs-8 {\n width: 66.66666667%;\n}\n.col-xs-7 {\n width: 58.33333333%;\n}\n.col-xs-6 {\n width: 50%;\n}\n.col-xs-5 {\n width: 41.66666667%;\n}\n.col-xs-4 {\n width: 33.33333333%;\n}\n.col-xs-3 {\n width: 25%;\n}\n.col-xs-2 {\n width: 16.66666667%;\n}\n.col-xs-1 {\n width: 8.33333333%;\n}\n.col-xs-pull-12 {\n right: 100%;\n}\n.col-xs-pull-11 {\n right: 91.66666667%;\n}\n.col-xs-pull-10 {\n right: 83.33333333%;\n}\n.col-xs-pull-9 {\n right: 75%;\n}\n.col-xs-pull-8 {\n right: 66.66666667%;\n}\n.col-xs-pull-7 {\n right: 58.33333333%;\n}\n.col-xs-pull-6 {\n right: 50%;\n}\n.col-xs-pull-5 {\n right: 41.66666667%;\n}\n.col-xs-pull-4 {\n right: 33.33333333%;\n}\n.col-xs-pull-3 {\n right: 25%;\n}\n.col-xs-pull-2 {\n right: 16.66666667%;\n}\n.col-xs-pull-1 {\n right: 8.33333333%;\n}\n.col-xs-pull-0 {\n right: auto;\n}\n.col-xs-push-12 {\n left: 100%;\n}\n.col-xs-push-11 {\n left: 91.66666667%;\n}\n.col-xs-push-10 {\n left: 83.33333333%;\n}\n.col-xs-push-9 {\n left: 75%;\n}\n.col-xs-push-8 {\n left: 66.66666667%;\n}\n.col-xs-push-7 {\n left: 58.33333333%;\n}\n.col-xs-push-6 {\n left: 50%;\n}\n.col-xs-push-5 {\n left: 41.66666667%;\n}\n.col-xs-push-4 {\n left: 33.33333333%;\n}\n.col-xs-push-3 {\n left: 25%;\n}\n.col-xs-push-2 {\n left: 16.66666667%;\n}\n.col-xs-push-1 {\n left: 8.33333333%;\n}\n.col-xs-push-0 {\n left: auto;\n}\n.col-xs-offset-12 {\n margin-left: 100%;\n}\n.col-xs-offset-11 {\n margin-left: 91.66666667%;\n}\n.col-xs-offset-10 {\n margin-left: 83.33333333%;\n}\n.col-xs-offset-9 {\n margin-left: 75%;\n}\n.col-xs-offset-8 {\n margin-left: 66.66666667%;\n}\n.col-xs-offset-7 {\n margin-left: 58.33333333%;\n}\n.col-xs-offset-6 {\n margin-left: 50%;\n}\n.col-xs-offset-5 {\n margin-left: 41.66666667%;\n}\n.col-xs-offset-4 {\n margin-left: 33.33333333%;\n}\n.col-xs-offset-3 {\n margin-left: 25%;\n}\n.col-xs-offset-2 {\n margin-left: 16.66666667%;\n}\n.col-xs-offset-1 {\n margin-left: 8.33333333%;\n}\n.col-xs-offset-0 {\n margin-left: 0%;\n}\n@media (min-width: 768px) {\n .col-sm-1, .col-sm-2, .col-sm-3, .col-sm-4, .col-sm-5, .col-sm-6, .col-sm-7, .col-sm-8, .col-sm-9, .col-sm-10, .col-sm-11, .col-sm-12 {\n float: left;\n }\n .col-sm-12 {\n width: 100%;\n }\n .col-sm-11 {\n width: 91.66666667%;\n }\n .col-sm-10 {\n width: 83.33333333%;\n }\n .col-sm-9 {\n width: 75%;\n }\n .col-sm-8 {\n width: 66.66666667%;\n }\n .col-sm-7 {\n width: 58.33333333%;\n }\n .col-sm-6 {\n width: 50%;\n }\n .col-sm-5 {\n width: 41.66666667%;\n }\n .col-sm-4 {\n width: 33.33333333%;\n }\n .col-sm-3 {\n width: 25%;\n }\n .col-sm-2 {\n width: 16.66666667%;\n }\n .col-sm-1 {\n width: 8.33333333%;\n }\n .col-sm-pull-12 {\n right: 100%;\n }\n .col-sm-pull-11 {\n right: 91.66666667%;\n }\n .col-sm-pull-10 {\n right: 83.33333333%;\n }\n .col-sm-pull-9 {\n right: 75%;\n }\n .col-sm-pull-8 {\n right: 66.66666667%;\n }\n .col-sm-pull-7 {\n right: 58.33333333%;\n }\n .col-sm-pull-6 {\n right: 50%;\n }\n .col-sm-pull-5 {\n right: 41.66666667%;\n }\n .col-sm-pull-4 {\n right: 33.33333333%;\n }\n .col-sm-pull-3 {\n right: 25%;\n }\n .col-sm-pull-2 {\n right: 16.66666667%;\n }\n .col-sm-pull-1 {\n right: 8.33333333%;\n }\n .col-sm-pull-0 {\n right: auto;\n }\n .col-sm-push-12 {\n left: 100%;\n }\n .col-sm-push-11 {\n left: 91.66666667%;\n }\n .col-sm-push-10 {\n left: 83.33333333%;\n }\n .col-sm-push-9 {\n left: 75%;\n }\n .col-sm-push-8 {\n left: 66.66666667%;\n }\n .col-sm-push-7 {\n left: 58.33333333%;\n }\n .col-sm-push-6 {\n left: 50%;\n }\n .col-sm-push-5 {\n left: 41.66666667%;\n }\n .col-sm-push-4 {\n left: 33.33333333%;\n }\n .col-sm-push-3 {\n left: 25%;\n }\n .col-sm-push-2 {\n left: 16.66666667%;\n }\n .col-sm-push-1 {\n left: 8.33333333%;\n }\n .col-sm-push-0 {\n left: auto;\n }\n .col-sm-offset-12 {\n margin-left: 100%;\n }\n .col-sm-offset-11 {\n margin-left: 91.66666667%;\n }\n .col-sm-offset-10 {\n margin-left: 83.33333333%;\n }\n .col-sm-offset-9 {\n margin-left: 75%;\n }\n .col-sm-offset-8 {\n margin-left: 66.66666667%;\n }\n .col-sm-offset-7 {\n margin-left: 58.33333333%;\n }\n .col-sm-offset-6 {\n margin-left: 50%;\n }\n .col-sm-offset-5 {\n margin-left: 41.66666667%;\n }\n .col-sm-offset-4 {\n margin-left: 33.33333333%;\n }\n .col-sm-offset-3 {\n margin-left: 25%;\n }\n .col-sm-offset-2 {\n margin-left: 16.66666667%;\n }\n .col-sm-offset-1 {\n margin-left: 8.33333333%;\n }\n .col-sm-offset-0 {\n margin-left: 0%;\n }\n}\n@media (min-width: 992px) {\n .col-md-1, .col-md-2, .col-md-3, .col-md-4, .col-md-5, .col-md-6, .col-md-7, .col-md-8, .col-md-9, .col-md-10, .col-md-11, .col-md-12 {\n float: left;\n }\n .col-md-12 {\n width: 100%;\n }\n .col-md-11 {\n width: 91.66666667%;\n }\n .col-md-10 {\n width: 83.33333333%;\n }\n .col-md-9 {\n width: 75%;\n }\n .col-md-8 {\n width: 66.66666667%;\n }\n .col-md-7 {\n width: 58.33333333%;\n }\n .col-md-6 {\n width: 50%;\n }\n .col-md-5 {\n width: 41.66666667%;\n }\n .col-md-4 {\n width: 33.33333333%;\n }\n .col-md-3 {\n width: 25%;\n }\n .col-md-2 {\n width: 16.66666667%;\n }\n .col-md-1 {\n width: 8.33333333%;\n }\n .col-md-pull-12 {\n right: 100%;\n }\n .col-md-pull-11 {\n right: 91.66666667%;\n }\n .col-md-pull-10 {\n right: 83.33333333%;\n }\n .col-md-pull-9 {\n right: 75%;\n }\n .col-md-pull-8 {\n right: 66.66666667%;\n }\n .col-md-pull-7 {\n right: 58.33333333%;\n }\n .col-md-pull-6 {\n right: 50%;\n }\n .col-md-pull-5 {\n right: 41.66666667%;\n }\n .col-md-pull-4 {\n right: 33.33333333%;\n }\n .col-md-pull-3 {\n right: 25%;\n }\n .col-md-pull-2 {\n right: 16.66666667%;\n }\n .col-md-pull-1 {\n right: 8.33333333%;\n }\n .col-md-pull-0 {\n right: auto;\n }\n .col-md-push-12 {\n left: 100%;\n }\n .col-md-push-11 {\n left: 91.66666667%;\n }\n .col-md-push-10 {\n left: 83.33333333%;\n }\n .col-md-push-9 {\n left: 75%;\n }\n .col-md-push-8 {\n left: 66.66666667%;\n }\n .col-md-push-7 {\n left: 58.33333333%;\n }\n .col-md-push-6 {\n left: 50%;\n }\n .col-md-push-5 {\n left: 41.66666667%;\n }\n .col-md-push-4 {\n left: 33.33333333%;\n }\n .col-md-push-3 {\n left: 25%;\n }\n .col-md-push-2 {\n left: 16.66666667%;\n }\n .col-md-push-1 {\n left: 8.33333333%;\n }\n .col-md-push-0 {\n left: auto;\n }\n .col-md-offset-12 {\n margin-left: 100%;\n }\n .col-md-offset-11 {\n margin-left: 91.66666667%;\n }\n .col-md-offset-10 {\n margin-left: 83.33333333%;\n }\n .col-md-offset-9 {\n margin-left: 75%;\n }\n .col-md-offset-8 {\n margin-left: 66.66666667%;\n }\n .col-md-offset-7 {\n margin-left: 58.33333333%;\n }\n .col-md-offset-6 {\n margin-left: 50%;\n }\n .col-md-offset-5 {\n margin-left: 41.66666667%;\n }\n .col-md-offset-4 {\n margin-left: 33.33333333%;\n }\n .col-md-offset-3 {\n margin-left: 25%;\n }\n .col-md-offset-2 {\n margin-left: 16.66666667%;\n }\n .col-md-offset-1 {\n margin-left: 8.33333333%;\n }\n .col-md-offset-0 {\n margin-left: 0%;\n }\n}\n@media (min-width: 1200px) {\n .col-lg-1, .col-lg-2, .col-lg-3, .col-lg-4, .col-lg-5, .col-lg-6, .col-lg-7, .col-lg-8, .col-lg-9, .col-lg-10, .col-lg-11, .col-lg-12 {\n float: left;\n }\n .col-lg-12 {\n width: 100%;\n }\n .col-lg-11 {\n width: 91.66666667%;\n }\n .col-lg-10 {\n width: 83.33333333%;\n }\n .col-lg-9 {\n width: 75%;\n }\n .col-lg-8 {\n width: 66.66666667%;\n }\n .col-lg-7 {\n width: 58.33333333%;\n }\n .col-lg-6 {\n width: 50%;\n }\n .col-lg-5 {\n width: 41.66666667%;\n }\n .col-lg-4 {\n width: 33.33333333%;\n }\n .col-lg-3 {\n width: 25%;\n }\n .col-lg-2 {\n width: 16.66666667%;\n }\n .col-lg-1 {\n width: 8.33333333%;\n }\n .col-lg-pull-12 {\n right: 100%;\n }\n .col-lg-pull-11 {\n right: 91.66666667%;\n }\n .col-lg-pull-10 {\n right: 83.33333333%;\n }\n .col-lg-pull-9 {\n right: 75%;\n }\n .col-lg-pull-8 {\n right: 66.66666667%;\n }\n .col-lg-pull-7 {\n right: 58.33333333%;\n }\n .col-lg-pull-6 {\n right: 50%;\n }\n .col-lg-pull-5 {\n right: 41.66666667%;\n }\n .col-lg-pull-4 {\n right: 33.33333333%;\n }\n .col-lg-pull-3 {\n right: 25%;\n }\n .col-lg-pull-2 {\n right: 16.66666667%;\n }\n .col-lg-pull-1 {\n right: 8.33333333%;\n }\n .col-lg-pull-0 {\n right: auto;\n }\n .col-lg-push-12 {\n left: 100%;\n }\n .col-lg-push-11 {\n left: 91.66666667%;\n }\n .col-lg-push-10 {\n left: 83.33333333%;\n }\n .col-lg-push-9 {\n left: 75%;\n }\n .col-lg-push-8 {\n left: 66.66666667%;\n }\n .col-lg-push-7 {\n left: 58.33333333%;\n }\n .col-lg-push-6 {\n left: 50%;\n }\n .col-lg-push-5 {\n left: 41.66666667%;\n }\n .col-lg-push-4 {\n left: 33.33333333%;\n }\n .col-lg-push-3 {\n left: 25%;\n }\n .col-lg-push-2 {\n left: 16.66666667%;\n }\n .col-lg-push-1 {\n left: 8.33333333%;\n }\n .col-lg-push-0 {\n left: auto;\n }\n .col-lg-offset-12 {\n margin-left: 100%;\n }\n .col-lg-offset-11 {\n margin-left: 91.66666667%;\n }\n .col-lg-offset-10 {\n margin-left: 83.33333333%;\n }\n .col-lg-offset-9 {\n margin-left: 75%;\n }\n .col-lg-offset-8 {\n margin-left: 66.66666667%;\n }\n .col-lg-offset-7 {\n margin-left: 58.33333333%;\n }\n .col-lg-offset-6 {\n margin-left: 50%;\n }\n .col-lg-offset-5 {\n margin-left: 41.66666667%;\n }\n .col-lg-offset-4 {\n margin-left: 33.33333333%;\n }\n .col-lg-offset-3 {\n margin-left: 25%;\n }\n .col-lg-offset-2 {\n margin-left: 16.66666667%;\n }\n .col-lg-offset-1 {\n margin-left: 8.33333333%;\n }\n .col-lg-offset-0 {\n margin-left: 0%;\n }\n}\ntable {\n background-color: transparent;\n}\ncaption {\n padding-top: 8px;\n padding-bottom: 8px;\n color: #777777;\n text-align: left;\n}\nth {\n text-align: left;\n}\n.table {\n width: 100%;\n max-width: 100%;\n margin-bottom: 20px;\n}\n.table > thead > tr > th,\n.table > tbody > tr > th,\n.table > tfoot > tr > th,\n.table > thead > tr > td,\n.table > tbody > tr > td,\n.table > tfoot > tr > td {\n padding: 8px;\n line-height: 1.42857143;\n vertical-align: top;\n border-top: 1px solid #dddddd;\n}\n.table > thead > tr > th {\n vertical-align: bottom;\n border-bottom: 2px solid #dddddd;\n}\n.table > caption + thead > tr:first-child > th,\n.table > colgroup + thead > tr:first-child > th,\n.table > thead:first-child > tr:first-child > th,\n.table > caption + thead > tr:first-child > td,\n.table > colgroup + thead > tr:first-child > td,\n.table > thead:first-child > tr:first-child > td {\n border-top: 0;\n}\n.table > tbody + tbody {\n border-top: 2px solid #dddddd;\n}\n.table .table {\n background-color: #ffffff;\n}\n.table-condensed > thead > tr > th,\n.table-condensed > tbody > tr > th,\n.table-condensed > tfoot > tr > th,\n.table-condensed > thead > tr > td,\n.table-condensed > tbody > tr > td,\n.table-condensed > tfoot > tr > td {\n padding: 5px;\n}\n.table-bordered {\n border: 1px solid #dddddd;\n}\n.table-bordered > thead > tr > th,\n.table-bordered > tbody > tr > th,\n.table-bordered > tfoot > tr > th,\n.table-bordered > thead > tr > td,\n.table-bordered > tbody > tr > td,\n.table-bordered > tfoot > tr > td {\n border: 1px solid #dddddd;\n}\n.table-bordered > thead > tr > th,\n.table-bordered > thead > tr > td {\n border-bottom-width: 2px;\n}\n.table-striped > tbody > tr:nth-of-type(odd) {\n background-color: #f9f9f9;\n}\n.table-hover > tbody > tr:hover {\n background-color: #f5f5f5;\n}\ntable col[class*=\"col-\"] {\n position: static;\n float: none;\n display: table-column;\n}\ntable td[class*=\"col-\"],\ntable th[class*=\"col-\"] {\n position: static;\n float: none;\n display: table-cell;\n}\n.table > thead > tr > td.active,\n.table > tbody > tr > td.active,\n.table > tfoot > tr > td.active,\n.table > thead > tr > th.active,\n.table > tbody > tr > th.active,\n.table > tfoot > tr > th.active,\n.table > thead > tr.active > td,\n.table > tbody > tr.active > td,\n.table > tfoot > tr.active > td,\n.table > thead > tr.active > th,\n.table > tbody > tr.active > th,\n.table > tfoot > tr.active > th {\n background-color: #f5f5f5;\n}\n.table-hover > tbody > tr > td.active:hover,\n.table-hover > tbody > tr > th.active:hover,\n.table-hover > tbody > tr.active:hover > td,\n.table-hover > tbody > tr:hover > .active,\n.table-hover > tbody > tr.active:hover > th {\n background-color: #e8e8e8;\n}\n.table > thead > tr > td.success,\n.table > tbody > tr > td.success,\n.table > tfoot > tr > td.success,\n.table > thead > tr > th.success,\n.table > tbody > tr > th.success,\n.table > tfoot > tr > th.success,\n.table > thead > tr.success > td,\n.table > tbody > tr.success > td,\n.table > tfoot > tr.success > td,\n.table > thead > tr.success > th,\n.table > tbody > tr.success > th,\n.table > tfoot > tr.success > th {\n background-color: #dff0d8;\n}\n.table-hover > tbody > tr > td.success:hover,\n.table-hover > tbody > tr > th.success:hover,\n.table-hover > tbody > tr.success:hover > td,\n.table-hover > tbody > tr:hover > .success,\n.table-hover > tbody > tr.success:hover > th {\n background-color: #d0e9c6;\n}\n.table > thead > tr > td.info,\n.table > tbody > tr > td.info,\n.table > tfoot > tr > td.info,\n.table > thead > tr > th.info,\n.table > tbody > tr > th.info,\n.table > tfoot > tr > th.info,\n.table > thead > tr.info > td,\n.table > tbody > tr.info > td,\n.table > tfoot > tr.info > td,\n.table > thead > tr.info > th,\n.table > tbody > tr.info > th,\n.table > tfoot > tr.info > th {\n background-color: #d9edf7;\n}\n.table-hover > tbody > tr > td.info:hover,\n.table-hover > tbody > tr > th.info:hover,\n.table-hover > tbody > tr.info:hover > td,\n.table-hover > tbody > tr:hover > .info,\n.table-hover > tbody > tr.info:hover > th {\n background-color: #c4e3f3;\n}\n.table > thead > tr > td.warning,\n.table > tbody > tr > td.warning,\n.table > tfoot > tr > td.warning,\n.table > thead > tr > th.warning,\n.table > tbody > tr > th.warning,\n.table > tfoot > tr > th.warning,\n.table > thead > tr.warning > td,\n.table > tbody > tr.warning > td,\n.table > tfoot > tr.warning > td,\n.table > thead > tr.warning > th,\n.table > tbody > tr.warning > th,\n.table > tfoot > tr.warning > th {\n background-color: #fcf8e3;\n}\n.table-hover > tbody > tr > td.warning:hover,\n.table-hover > tbody > tr > th.warning:hover,\n.table-hover > tbody > tr.warning:hover > td,\n.table-hover > tbody > tr:hover > .warning,\n.table-hover > tbody > tr.warning:hover > th {\n background-color: #faf2cc;\n}\n.table > thead > tr > td.danger,\n.table > tbody > tr > td.danger,\n.table > tfoot > tr > td.danger,\n.table > thead > tr > th.danger,\n.table > tbody > tr > th.danger,\n.table > tfoot > tr > th.danger,\n.table > thead > tr.danger > td,\n.table > tbody > tr.danger > td,\n.table > tfoot > tr.danger > td,\n.table > thead > tr.danger > th,\n.table > tbody > tr.danger > th,\n.table > tfoot > tr.danger > th {\n background-color: #f2dede;\n}\n.table-hover > tbody > tr > td.danger:hover,\n.table-hover > tbody > tr > th.danger:hover,\n.table-hover > tbody > tr.danger:hover > td,\n.table-hover > tbody > tr:hover > .danger,\n.table-hover > tbody > tr.danger:hover > th {\n background-color: #ebcccc;\n}\n.table-responsive {\n overflow-x: auto;\n min-height: 0.01%;\n}\n@media screen and (max-width: 767px) {\n .table-responsive {\n width: 100%;\n margin-bottom: 15px;\n overflow-y: hidden;\n -ms-overflow-style: -ms-autohiding-scrollbar;\n border: 1px solid #dddddd;\n }\n .table-responsive > .table {\n margin-bottom: 0;\n }\n .table-responsive > .table > thead > tr > th,\n .table-responsive > .table > tbody > tr > th,\n .table-responsive > .table > tfoot > tr > th,\n .table-responsive > .table > thead > tr > td,\n .table-responsive > .table > tbody > tr > td,\n .table-responsive > .table > tfoot > tr > td {\n white-space: nowrap;\n }\n .table-responsive > .table-bordered {\n border: 0;\n }\n .table-responsive > .table-bordered > thead > tr > th:first-child,\n .table-responsive > .table-bordered > tbody > tr > th:first-child,\n .table-responsive > .table-bordered > tfoot > tr > th:first-child,\n .table-responsive > .table-bordered > thead > tr > td:first-child,\n .table-responsive > .table-bordered > tbody > tr > td:first-child,\n .table-responsive > .table-bordered > tfoot > tr > td:first-child {\n border-left: 0;\n }\n .table-responsive > .table-bordered > thead > tr > th:last-child,\n .table-responsive > .table-bordered > tbody > tr > th:last-child,\n .table-responsive > .table-bordered > tfoot > tr > th:last-child,\n .table-responsive > .table-bordered > thead > tr > td:last-child,\n .table-responsive > .table-bordered > tbody > tr > td:last-child,\n .table-responsive > .table-bordered > tfoot > tr > td:last-child {\n border-right: 0;\n }\n .table-responsive > .table-bordered > tbody > tr:last-child > th,\n .table-responsive > .table-bordered > tfoot > tr:last-child > th,\n .table-responsive > .table-bordered > tbody > tr:last-child > td,\n .table-responsive > .table-bordered > tfoot > tr:last-child > td {\n border-bottom: 0;\n }\n}\nfieldset {\n padding: 0;\n margin: 0;\n border: 0;\n min-width: 0;\n}\nlegend {\n display: block;\n width: 100%;\n padding: 0;\n margin-bottom: 20px;\n font-size: 21px;\n line-height: inherit;\n color: #333333;\n border: 0;\n border-bottom: 1px solid #e5e5e5;\n}\nlabel {\n display: inline-block;\n max-width: 100%;\n margin-bottom: 5px;\n font-weight: bold;\n}\ninput[type=\"search\"] {\n -webkit-box-sizing: border-box;\n -moz-box-sizing: border-box;\n box-sizing: border-box;\n}\ninput[type=\"radio\"],\ninput[type=\"checkbox\"] {\n margin: 4px 0 0;\n margin-top: 1px \\9;\n line-height: normal;\n}\ninput[type=\"file\"] {\n display: block;\n}\ninput[type=\"range\"] {\n display: block;\n width: 100%;\n}\nselect[multiple],\nselect[size] {\n height: auto;\n}\ninput[type=\"file\"]:focus,\ninput[type=\"radio\"]:focus,\ninput[type=\"checkbox\"]:focus {\n outline: thin dotted;\n outline: 5px auto -webkit-focus-ring-color;\n outline-offset: -2px;\n}\noutput {\n display: block;\n padding-top: 7px;\n font-size: 14px;\n line-height: 1.42857143;\n color: #555555;\n}\n.form-control {\n display: block;\n width: 100%;\n height: 34px;\n padding: 6px 12px;\n font-size: 14px;\n line-height: 1.42857143;\n color: #555555;\n background-color: #ffffff;\n background-image: none;\n border: 1px solid #cccccc;\n border-radius: 4px;\n -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);\n box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);\n -webkit-transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;\n -o-transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;\n transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;\n}\n.form-control:focus {\n border-color: #66afe9;\n outline: 0;\n -webkit-box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, 0.6);\n box-shadow: inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(102, 175, 233, 0.6);\n}\n.form-control::-moz-placeholder {\n color: #999999;\n opacity: 1;\n}\n.form-control:-ms-input-placeholder {\n color: #999999;\n}\n.form-control::-webkit-input-placeholder {\n color: #999999;\n}\n.form-control[disabled],\n.form-control[readonly],\nfieldset[disabled] .form-control {\n background-color: #eeeeee;\n opacity: 1;\n}\n.form-control[disabled],\nfieldset[disabled] .form-control {\n cursor: not-allowed;\n}\ntextarea.form-control {\n height: auto;\n}\ninput[type=\"search\"] {\n -webkit-appearance: none;\n}\n@media screen and (-webkit-min-device-pixel-ratio: 0) {\n input[type=\"date\"],\n input[type=\"time\"],\n input[type=\"datetime-local\"],\n input[type=\"month\"] {\n line-height: 34px;\n }\n input[type=\"date\"].input-sm,\n input[type=\"time\"].input-sm,\n input[type=\"datetime-local\"].input-sm,\n input[type=\"month\"].input-sm,\n .input-group-sm input[type=\"date\"],\n .input-group-sm input[type=\"time\"],\n .input-group-sm input[type=\"datetime-local\"],\n .input-group-sm input[type=\"month\"] {\n line-height: 30px;\n }\n input[type=\"date\"].input-lg,\n input[type=\"time\"].input-lg,\n input[type=\"datetime-local\"].input-lg,\n input[type=\"month\"].input-lg,\n .input-group-lg input[type=\"date\"],\n .input-group-lg input[type=\"time\"],\n .input-group-lg input[type=\"datetime-local\"],\n .input-group-lg input[type=\"month\"] {\n line-height: 46px;\n }\n}\n.form-group {\n margin-bottom: 15px;\n}\n.radio,\n.checkbox {\n position: relative;\n display: block;\n margin-top: 10px;\n margin-bottom: 10px;\n}\n.radio label,\n.checkbox label {\n min-height: 20px;\n padding-left: 20px;\n margin-bottom: 0;\n font-weight: normal;\n cursor: pointer;\n}\n.radio input[type=\"radio\"],\n.radio-inline input[type=\"radio\"],\n.checkbox input[type=\"checkbox\"],\n.checkbox-inline input[type=\"checkbox\"] {\n position: absolute;\n margin-left: -20px;\n margin-top: 4px \\9;\n}\n.radio + .radio,\n.checkbox + .checkbox {\n margin-top: -5px;\n}\n.radio-inline,\n.checkbox-inline {\n position: relative;\n display: inline-block;\n padding-left: 20px;\n margin-bottom: 0;\n vertical-align: middle;\n font-weight: normal;\n cursor: pointer;\n}\n.radio-inline + .radio-inline,\n.checkbox-inline + .checkbox-inline {\n margin-top: 0;\n margin-left: 10px;\n}\ninput[type=\"radio\"][disabled],\ninput[type=\"checkbox\"][disabled],\ninput[type=\"radio\"].disabled,\ninput[type=\"checkbox\"].disabled,\nfieldset[disabled] input[type=\"radio\"],\nfieldset[disabled] input[type=\"checkbox\"] {\n cursor: not-allowed;\n}\n.radio-inline.disabled,\n.checkbox-inline.disabled,\nfieldset[disabled] .radio-inline,\nfieldset[disabled] .checkbox-inline {\n cursor: not-allowed;\n}\n.radio.disabled label,\n.checkbox.disabled label,\nfieldset[disabled] .radio label,\nfieldset[disabled] .checkbox label {\n cursor: not-allowed;\n}\n.form-control-static {\n padding-top: 7px;\n padding-bottom: 7px;\n margin-bottom: 0;\n min-height: 34px;\n}\n.form-control-static.input-lg,\n.form-control-static.input-sm {\n padding-left: 0;\n padding-right: 0;\n}\n.input-sm {\n height: 30px;\n padding: 5px 10px;\n font-size: 12px;\n line-height: 1.5;\n border-radius: 3px;\n}\nselect.input-sm {\n height: 30px;\n line-height: 30px;\n}\ntextarea.input-sm,\nselect[multiple].input-sm {\n height: auto;\n}\n.form-group-sm .form-control {\n height: 30px;\n padding: 5px 10px;\n font-size: 12px;\n line-height: 1.5;\n border-radius: 3px;\n}\nselect.form-group-sm .form-control {\n height: 30px;\n line-height: 30px;\n}\ntextarea.form-group-sm .form-control,\nselect[multiple].form-group-sm .form-control {\n height: auto;\n}\n.form-group-sm .form-control-static {\n height: 30px;\n padding: 5px 10px;\n font-size: 12px;\n line-height: 1.5;\n min-height: 32px;\n}\n.input-lg {\n height: 46px;\n padding: 10px 16px;\n font-size: 18px;\n line-height: 1.3333333;\n border-radius: 6px;\n}\nselect.input-lg {\n height: 46px;\n line-height: 46px;\n}\ntextarea.input-lg,\nselect[multiple].input-lg {\n height: auto;\n}\n.form-group-lg .form-control {\n height: 46px;\n padding: 10px 16px;\n font-size: 18px;\n line-height: 1.3333333;\n border-radius: 6px;\n}\nselect.form-group-lg .form-control {\n height: 46px;\n line-height: 46px;\n}\ntextarea.form-group-lg .form-control,\nselect[multiple].form-group-lg .form-control {\n height: auto;\n}\n.form-group-lg .form-control-static {\n height: 46px;\n padding: 10px 16px;\n font-size: 18px;\n line-height: 1.3333333;\n min-height: 38px;\n}\n.has-feedback {\n position: relative;\n}\n.has-feedback .form-control {\n padding-right: 42.5px;\n}\n.form-control-feedback {\n position: absolute;\n top: 0;\n right: 0;\n z-index: 2;\n display: block;\n width: 34px;\n height: 34px;\n line-height: 34px;\n text-align: center;\n pointer-events: none;\n}\n.input-lg + .form-control-feedback {\n width: 46px;\n height: 46px;\n line-height: 46px;\n}\n.input-sm + .form-control-feedback {\n width: 30px;\n height: 30px;\n line-height: 30px;\n}\n.has-success .help-block,\n.has-success .control-label,\n.has-success .radio,\n.has-success .checkbox,\n.has-success .radio-inline,\n.has-success .checkbox-inline,\n.has-success.radio label,\n.has-success.checkbox label,\n.has-success.radio-inline label,\n.has-success.checkbox-inline label {\n color: #3c763d;\n}\n.has-success .form-control {\n border-color: #3c763d;\n -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);\n box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);\n}\n.has-success .form-control:focus {\n border-color: #2b542c;\n -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #67b168;\n box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #67b168;\n}\n.has-success .input-group-addon {\n color: #3c763d;\n border-color: #3c763d;\n background-color: #dff0d8;\n}\n.has-success .form-control-feedback {\n color: #3c763d;\n}\n.has-warning .help-block,\n.has-warning .control-label,\n.has-warning .radio,\n.has-warning .checkbox,\n.has-warning .radio-inline,\n.has-warning .checkbox-inline,\n.has-warning.radio label,\n.has-warning.checkbox label,\n.has-warning.radio-inline label,\n.has-warning.checkbox-inline label {\n color: #8a6d3b;\n}\n.has-warning .form-control {\n border-color: #8a6d3b;\n -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);\n box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);\n}\n.has-warning .form-control:focus {\n border-color: #66512c;\n -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #c0a16b;\n box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #c0a16b;\n}\n.has-warning .input-group-addon {\n color: #8a6d3b;\n border-color: #8a6d3b;\n background-color: #fcf8e3;\n}\n.has-warning .form-control-feedback {\n color: #8a6d3b;\n}\n.has-error .help-block,\n.has-error .control-label,\n.has-error .radio,\n.has-error .checkbox,\n.has-error .radio-inline,\n.has-error .checkbox-inline,\n.has-error.radio label,\n.has-error.checkbox label,\n.has-error.radio-inline label,\n.has-error.checkbox-inline label {\n color: #a94442;\n}\n.has-error .form-control {\n border-color: #a94442;\n -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);\n box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075);\n}\n.has-error .form-control:focus {\n border-color: #843534;\n -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #ce8483;\n box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #ce8483;\n}\n.has-error .input-group-addon {\n color: #a94442;\n border-color: #a94442;\n background-color: #f2dede;\n}\n.has-error .form-control-feedback {\n color: #a94442;\n}\n.has-feedback label ~ .form-control-feedback {\n top: 25px;\n}\n.has-feedback label.sr-only ~ .form-control-feedback {\n top: 0;\n}\n.help-block {\n display: block;\n margin-top: 5px;\n margin-bottom: 10px;\n color: #737373;\n}\n@media (min-width: 768px) {\n .form-inline .form-group {\n display: inline-block;\n margin-bottom: 0;\n vertical-align: middle;\n }\n .form-inline .form-control {\n display: inline-block;\n width: auto;\n vertical-align: middle;\n }\n .form-inline .form-control-static {\n display: inline-block;\n }\n .form-inline .input-group {\n display: inline-table;\n vertical-align: middle;\n }\n .form-inline .input-group .input-group-addon,\n .form-inline .input-group .input-group-btn,\n .form-inline .input-group .form-control {\n width: auto;\n }\n .form-inline .input-group > .form-control {\n width: 100%;\n }\n .form-inline .control-label {\n margin-bottom: 0;\n vertical-align: middle;\n }\n .form-inline .radio,\n .form-inline .checkbox {\n display: inline-block;\n margin-top: 0;\n margin-bottom: 0;\n vertical-align: middle;\n }\n .form-inline .radio label,\n .form-inline .checkbox label {\n padding-left: 0;\n }\n .form-inline .radio input[type=\"radio\"],\n .form-inline .checkbox input[type=\"checkbox\"] {\n position: relative;\n margin-left: 0;\n }\n .form-inline .has-feedback .form-control-feedback {\n top: 0;\n }\n}\n.form-horizontal .radio,\n.form-horizontal .checkbox,\n.form-horizontal .radio-inline,\n.form-horizontal .checkbox-inline {\n margin-top: 0;\n margin-bottom: 0;\n padding-top: 7px;\n}\n.form-horizontal .radio,\n.form-horizontal .checkbox {\n min-height: 27px;\n}\n.form-horizontal .form-group {\n margin-left: -15px;\n margin-right: -15px;\n}\n@media (min-width: 768px) {\n .form-horizontal .control-label {\n text-align: right;\n margin-bottom: 0;\n padding-top: 7px;\n }\n}\n.form-horizontal .has-feedback .form-control-feedback {\n right: 15px;\n}\n@media (min-width: 768px) {\n .form-horizontal .form-group-lg .control-label {\n padding-top: 14.333333px;\n }\n}\n@media (min-width: 768px) {\n .form-horizontal .form-group-sm .control-label {\n padding-top: 6px;\n }\n}\n.btn {\n display: inline-block;\n margin-bottom: 0;\n font-weight: normal;\n text-align: center;\n vertical-align: middle;\n touch-action: manipulation;\n cursor: pointer;\n background-image: none;\n border: 1px solid transparent;\n white-space: nowrap;\n padding: 6px 12px;\n font-size: 14px;\n line-height: 1.42857143;\n border-radius: 4px;\n -webkit-user-select: none;\n -moz-user-select: none;\n -ms-user-select: none;\n user-select: none;\n}\n.btn:focus,\n.btn:active:focus,\n.btn.active:focus,\n.btn.focus,\n.btn:active.focus,\n.btn.active.focus {\n outline: thin dotted;\n outline: 5px auto -webkit-focus-ring-color;\n outline-offset: -2px;\n}\n.btn:hover,\n.btn:focus,\n.btn.focus {\n color: #333333;\n text-decoration: none;\n}\n.btn:active,\n.btn.active {\n outline: 0;\n background-image: none;\n -webkit-box-shadow: inset 0 3px 5px rgba(0, 0, 0, 0.125);\n box-shadow: inset 0 3px 5px rgba(0, 0, 0, 0.125);\n}\n.btn.disabled,\n.btn[disabled],\nfieldset[disabled] .btn {\n cursor: not-allowed;\n pointer-events: none;\n opacity: 0.65;\n filter: alpha(opacity=65);\n -webkit-box-shadow: none;\n box-shadow: none;\n}\n.btn-default {\n color: #333333;\n background-color: #ffffff;\n border-color: #cccccc;\n}\n.btn-default:hover,\n.btn-default:focus,\n.btn-default.focus,\n.btn-default:active,\n.btn-default.active,\n.open > .dropdown-toggle.btn-default {\n color: #333333;\n background-color: #e6e6e6;\n border-color: #adadad;\n}\n.btn-default:active,\n.btn-default.active,\n.open > .dropdown-toggle.btn-default {\n background-image: none;\n}\n.btn-default.disabled,\n.btn-default[disabled],\nfieldset[disabled] .btn-default,\n.btn-default.disabled:hover,\n.btn-default[disabled]:hover,\nfieldset[disabled] .btn-default:hover,\n.btn-default.disabled:focus,\n.btn-default[disabled]:focus,\nfieldset[disabled] .btn-default:focus,\n.btn-default.disabled.focus,\n.btn-default[disabled].focus,\nfieldset[disabled] .btn-default.focus,\n.btn-default.disabled:active,\n.btn-default[disabled]:active,\nfieldset[disabled] .btn-default:active,\n.btn-default.disabled.active,\n.btn-default[disabled].active,\nfieldset[disabled] .btn-default.active {\n background-color: #ffffff;\n border-color: #cccccc;\n}\n.btn-default .badge {\n color: #ffffff;\n background-color: #333333;\n}\n.btn-primary {\n color: #ffffff;\n background-color: #337ab7;\n border-color: #2e6da4;\n}\n.btn-primary:hover,\n.btn-primary:focus,\n.btn-primary.focus,\n.btn-primary:active,\n.btn-primary.active,\n.open > .dropdown-toggle.btn-primary {\n color: #ffffff;\n background-color: #286090;\n border-color: #204d74;\n}\n.btn-primary:active,\n.btn-primary.active,\n.open > .dropdown-toggle.btn-primary {\n background-image: none;\n}\n.btn-primary.disabled,\n.btn-primary[disabled],\nfieldset[disabled] .btn-primary,\n.btn-primary.disabled:hover,\n.btn-primary[disabled]:hover,\nfieldset[disabled] .btn-primary:hover,\n.btn-primary.disabled:focus,\n.btn-primary[disabled]:focus,\nfieldset[disabled] .btn-primary:focus,\n.btn-primary.disabled.focus,\n.btn-primary[disabled].focus,\nfieldset[disabled] .btn-primary.focus,\n.btn-primary.disabled:active,\n.btn-primary[disabled]:active,\nfieldset[disabled] .btn-primary:active,\n.btn-primary.disabled.active,\n.btn-primary[disabled].active,\nfieldset[disabled] .btn-primary.active {\n background-color: #337ab7;\n border-color: #2e6da4;\n}\n.btn-primary .badge {\n color: #337ab7;\n background-color: #ffffff;\n}\n.btn-success {\n color: #ffffff;\n background-color: #5cb85c;\n border-color: #4cae4c;\n}\n.btn-success:hover,\n.btn-success:focus,\n.btn-success.focus,\n.btn-success:active,\n.btn-success.active,\n.open > .dropdown-toggle.btn-success {\n color: #ffffff;\n background-color: #449d44;\n border-color: #398439;\n}\n.btn-success:active,\n.btn-success.active,\n.open > .dropdown-toggle.btn-success {\n background-image: none;\n}\n.btn-success.disabled,\n.btn-success[disabled],\nfieldset[disabled] .btn-success,\n.btn-success.disabled:hover,\n.btn-success[disabled]:hover,\nfieldset[disabled] .btn-success:hover,\n.btn-success.disabled:focus,\n.btn-success[disabled]:focus,\nfieldset[disabled] .btn-success:focus,\n.btn-success.disabled.focus,\n.btn-success[disabled].focus,\nfieldset[disabled] .btn-success.focus,\n.btn-success.disabled:active,\n.btn-success[disabled]:active,\nfieldset[disabled] .btn-success:active,\n.btn-success.disabled.active,\n.btn-success[disabled].active,\nfieldset[disabled] .btn-success.active {\n background-color: #5cb85c;\n border-color: #4cae4c;\n}\n.btn-success .badge {\n color: #5cb85c;\n background-color: #ffffff;\n}\n.btn-info {\n color: #ffffff;\n background-color: #5bc0de;\n border-color: #46b8da;\n}\n.btn-info:hover,\n.btn-info:focus,\n.btn-info.focus,\n.btn-info:active,\n.btn-info.active,\n.open > .dropdown-toggle.btn-info {\n color: #ffffff;\n background-color: #31b0d5;\n border-color: #269abc;\n}\n.btn-info:active,\n.btn-info.active,\n.open > .dropdown-toggle.btn-info {\n background-image: none;\n}\n.btn-info.disabled,\n.btn-info[disabled],\nfieldset[disabled] .btn-info,\n.btn-info.disabled:hover,\n.btn-info[disabled]:hover,\nfieldset[disabled] .btn-info:hover,\n.btn-info.disabled:focus,\n.btn-info[disabled]:focus,\nfieldset[disabled] .btn-info:focus,\n.btn-info.disabled.focus,\n.btn-info[disabled].focus,\nfieldset[disabled] .btn-info.focus,\n.btn-info.disabled:active,\n.btn-info[disabled]:active,\nfieldset[disabled] .btn-info:active,\n.btn-info.disabled.active,\n.btn-info[disabled].active,\nfieldset[disabled] .btn-info.active {\n background-color: #5bc0de;\n border-color: #46b8da;\n}\n.btn-info .badge {\n color: #5bc0de;\n background-color: #ffffff;\n}\n.btn-warning {\n color: #ffffff;\n background-color: #f0ad4e;\n border-color: #eea236;\n}\n.btn-warning:hover,\n.btn-warning:focus,\n.btn-warning.focus,\n.btn-warning:active,\n.btn-warning.active,\n.open > .dropdown-toggle.btn-warning {\n color: #ffffff;\n background-color: #ec971f;\n border-color: #d58512;\n}\n.btn-warning:active,\n.btn-warning.active,\n.open > .dropdown-toggle.btn-warning {\n background-image: none;\n}\n.btn-warning.disabled,\n.btn-warning[disabled],\nfieldset[disabled] .btn-warning,\n.btn-warning.disabled:hover,\n.btn-warning[disabled]:hover,\nfieldset[disabled] .btn-warning:hover,\n.btn-warning.disabled:focus,\n.btn-warning[disabled]:focus,\nfieldset[disabled] .btn-warning:focus,\n.btn-warning.disabled.focus,\n.btn-warning[disabled].focus,\nfieldset[disabled] .btn-warning.focus,\n.btn-warning.disabled:active,\n.btn-warning[disabled]:active,\nfieldset[disabled] .btn-warning:active,\n.btn-warning.disabled.active,\n.btn-warning[disabled].active,\nfieldset[disabled] .btn-warning.active {\n background-color: #f0ad4e;\n border-color: #eea236;\n}\n.btn-warning .badge {\n color: #f0ad4e;\n background-color: #ffffff;\n}\n.btn-danger {\n color: #ffffff;\n background-color: #d9534f;\n border-color: #d43f3a;\n}\n.btn-danger:hover,\n.btn-danger:focus,\n.btn-danger.focus,\n.btn-danger:active,\n.btn-danger.active,\n.open > .dropdown-toggle.btn-danger {\n color: #ffffff;\n background-color: #c9302c;\n border-color: #ac2925;\n}\n.btn-danger:active,\n.btn-danger.active,\n.open > .dropdown-toggle.btn-danger {\n background-image: none;\n}\n.btn-danger.disabled,\n.btn-danger[disabled],\nfieldset[disabled] .btn-danger,\n.btn-danger.disabled:hover,\n.btn-danger[disabled]:hover,\nfieldset[disabled] .btn-danger:hover,\n.btn-danger.disabled:focus,\n.btn-danger[disabled]:focus,\nfieldset[disabled] .btn-danger:focus,\n.btn-danger.disabled.focus,\n.btn-danger[disabled].focus,\nfieldset[disabled] .btn-danger.focus,\n.btn-danger.disabled:active,\n.btn-danger[disabled]:active,\nfieldset[disabled] .btn-danger:active,\n.btn-danger.disabled.active,\n.btn-danger[disabled].active,\nfieldset[disabled] .btn-danger.active {\n background-color: #d9534f;\n border-color: #d43f3a;\n}\n.btn-danger .badge {\n color: #d9534f;\n background-color: #ffffff;\n}\n.btn-link {\n color: #337ab7;\n font-weight: normal;\n border-radius: 0;\n}\n.btn-link,\n.btn-link:active,\n.btn-link.active,\n.btn-link[disabled],\nfieldset[disabled] .btn-link {\n background-color: transparent;\n -webkit-box-shadow: none;\n box-shadow: none;\n}\n.btn-link,\n.btn-link:hover,\n.btn-link:focus,\n.btn-link:active {\n border-color: transparent;\n}\n.btn-link:hover,\n.btn-link:focus {\n color: #23527c;\n text-decoration: underline;\n background-color: transparent;\n}\n.btn-link[disabled]:hover,\nfieldset[disabled] .btn-link:hover,\n.btn-link[disabled]:focus,\nfieldset[disabled] .btn-link:focus {\n color: #777777;\n text-decoration: none;\n}\n.btn-lg,\n.btn-group-lg > .btn {\n padding: 10px 16px;\n font-size: 18px;\n line-height: 1.3333333;\n border-radius: 6px;\n}\n.btn-sm,\n.btn-group-sm > .btn {\n padding: 5px 10px;\n font-size: 12px;\n line-height: 1.5;\n border-radius: 3px;\n}\n.btn-xs,\n.btn-group-xs > .btn {\n padding: 1px 5px;\n font-size: 12px;\n line-height: 1.5;\n border-radius: 3px;\n}\n.btn-block {\n display: block;\n width: 100%;\n}\n.btn-block + .btn-block {\n margin-top: 5px;\n}\ninput[type=\"submit\"].btn-block,\ninput[type=\"reset\"].btn-block,\ninput[type=\"button\"].btn-block {\n width: 100%;\n}\n.fade {\n opacity: 0;\n -webkit-transition: opacity 0.15s linear;\n -o-transition: opacity 0.15s linear;\n transition: opacity 0.15s linear;\n}\n.fade.in {\n opacity: 1;\n}\n.collapse {\n display: none;\n}\n.collapse.in {\n display: block;\n}\ntr.collapse.in {\n display: table-row;\n}\ntbody.collapse.in {\n display: table-row-group;\n}\n.collapsing {\n position: relative;\n height: 0;\n overflow: hidden;\n -webkit-transition-property: height, visibility;\n transition-property: height, visibility;\n -webkit-transition-duration: 0.35s;\n transition-duration: 0.35s;\n -webkit-transition-timing-function: ease;\n transition-timing-function: ease;\n}\n.caret {\n display: inline-block;\n width: 0;\n height: 0;\n margin-left: 2px;\n vertical-align: middle;\n border-top: 4px dashed;\n border-right: 4px solid transparent;\n border-left: 4px solid transparent;\n}\n.dropup,\n.dropdown {\n position: relative;\n}\n.dropdown-toggle:focus {\n outline: 0;\n}\n.dropdown-menu {\n position: absolute;\n top: 100%;\n left: 0;\n z-index: 1000;\n display: none;\n float: left;\n min-width: 160px;\n padding: 5px 0;\n margin: 2px 0 0;\n list-style: none;\n font-size: 14px;\n text-align: left;\n background-color: #ffffff;\n border: 1px solid #cccccc;\n border: 1px solid rgba(0, 0, 0, 0.15);\n border-radius: 4px;\n -webkit-box-shadow: 0 6px 12px rgba(0, 0, 0, 0.175);\n box-shadow: 0 6px 12px rgba(0, 0, 0, 0.175);\n background-clip: padding-box;\n}\n.dropdown-menu.pull-right {\n right: 0;\n left: auto;\n}\n.dropdown-menu .divider {\n height: 1px;\n margin: 9px 0;\n overflow: hidden;\n background-color: #e5e5e5;\n}\n.dropdown-menu > li > a {\n display: block;\n padding: 3px 20px;\n clear: both;\n font-weight: normal;\n line-height: 1.42857143;\n color: #333333;\n white-space: nowrap;\n}\n.dropdown-menu > li > a:hover,\n.dropdown-menu > li > a:focus {\n text-decoration: none;\n color: #262626;\n background-color: #f5f5f5;\n}\n.dropdown-menu > .active > a,\n.dropdown-menu > .active > a:hover,\n.dropdown-menu > .active > a:focus {\n color: #ffffff;\n text-decoration: none;\n outline: 0;\n background-color: #337ab7;\n}\n.dropdown-menu > .disabled > a,\n.dropdown-menu > .disabled > a:hover,\n.dropdown-menu > .disabled > a:focus {\n color: #777777;\n}\n.dropdown-menu > .disabled > a:hover,\n.dropdown-menu > .disabled > a:focus {\n text-decoration: none;\n background-color: transparent;\n background-image: none;\n filter: progid:DXImageTransform.Microsoft.gradient(enabled = false);\n cursor: not-allowed;\n}\n.open > .dropdown-menu {\n display: block;\n}\n.open > a {\n outline: 0;\n}\n.dropdown-menu-right {\n left: auto;\n right: 0;\n}\n.dropdown-menu-left {\n left: 0;\n right: auto;\n}\n.dropdown-header {\n display: block;\n padding: 3px 20px;\n font-size: 12px;\n line-height: 1.42857143;\n color: #777777;\n white-space: nowrap;\n}\n.dropdown-backdrop {\n position: fixed;\n left: 0;\n right: 0;\n bottom: 0;\n top: 0;\n z-index: 990;\n}\n.pull-right > .dropdown-menu {\n right: 0;\n left: auto;\n}\n.dropup .caret,\n.navbar-fixed-bottom .dropdown .caret {\n border-top: 0;\n border-bottom: 4px solid;\n content: \"\";\n}\n.dropup .dropdown-menu,\n.navbar-fixed-bottom .dropdown .dropdown-menu {\n top: auto;\n bottom: 100%;\n margin-bottom: 2px;\n}\n@media (min-width: 768px) {\n .navbar-right .dropdown-menu {\n left: auto;\n right: 0;\n }\n .navbar-right .dropdown-menu-left {\n left: 0;\n right: auto;\n }\n}\n.btn-group,\n.btn-group-vertical {\n position: relative;\n display: inline-block;\n vertical-align: middle;\n}\n.btn-group > .btn,\n.btn-group-vertical > .btn {\n position: relative;\n float: left;\n}\n.btn-group > .btn:hover,\n.btn-group-vertical > .btn:hover,\n.btn-group > .btn:focus,\n.btn-group-vertical > .btn:focus,\n.btn-group > .btn:active,\n.btn-group-vertical > .btn:active,\n.btn-group > .btn.active,\n.btn-group-vertical > .btn.active {\n z-index: 2;\n}\n.btn-group .btn + .btn,\n.btn-group .btn + .btn-group,\n.btn-group .btn-group + .btn,\n.btn-group .btn-group + .btn-group {\n margin-left: -1px;\n}\n.btn-toolbar {\n margin-left: -5px;\n}\n.btn-toolbar .btn-group,\n.btn-toolbar .input-group {\n float: left;\n}\n.btn-toolbar > .btn,\n.btn-toolbar > .btn-group,\n.btn-toolbar > .input-group {\n margin-left: 5px;\n}\n.btn-group > .btn:not(:first-child):not(:last-child):not(.dropdown-toggle) {\n border-radius: 0;\n}\n.btn-group > .btn:first-child {\n margin-left: 0;\n}\n.btn-group > .btn:first-child:not(:last-child):not(.dropdown-toggle) {\n border-bottom-right-radius: 0;\n border-top-right-radius: 0;\n}\n.btn-group > .btn:last-child:not(:first-child),\n.btn-group > .dropdown-toggle:not(:first-child) {\n border-bottom-left-radius: 0;\n border-top-left-radius: 0;\n}\n.btn-group > .btn-group {\n float: left;\n}\n.btn-group > .btn-group:not(:first-child):not(:last-child) > .btn {\n border-radius: 0;\n}\n.btn-group > .btn-group:first-child:not(:last-child) > .btn:last-child,\n.btn-group > .btn-group:first-child:not(:last-child) > .dropdown-toggle {\n border-bottom-right-radius: 0;\n border-top-right-radius: 0;\n}\n.btn-group > .btn-group:last-child:not(:first-child) > .btn:first-child {\n border-bottom-left-radius: 0;\n border-top-left-radius: 0;\n}\n.btn-group .dropdown-toggle:active,\n.btn-group.open .dropdown-toggle {\n outline: 0;\n}\n.btn-group > .btn + .dropdown-toggle {\n padding-left: 8px;\n padding-right: 8px;\n}\n.btn-group > .btn-lg + .dropdown-toggle {\n padding-left: 12px;\n padding-right: 12px;\n}\n.btn-group.open .dropdown-toggle {\n -webkit-box-shadow: inset 0 3px 5px rgba(0, 0, 0, 0.125);\n box-shadow: inset 0 3px 5px rgba(0, 0, 0, 0.125);\n}\n.btn-group.open .dropdown-toggle.btn-link {\n -webkit-box-shadow: none;\n box-shadow: none;\n}\n.btn .caret {\n margin-left: 0;\n}\n.btn-lg .caret {\n border-width: 5px 5px 0;\n border-bottom-width: 0;\n}\n.dropup .btn-lg .caret {\n border-width: 0 5px 5px;\n}\n.btn-group-vertical > .btn,\n.btn-group-vertical > .btn-group,\n.btn-group-vertical > .btn-group > .btn {\n display: block;\n float: none;\n width: 100%;\n max-width: 100%;\n}\n.btn-group-vertical > .btn-group > .btn {\n float: none;\n}\n.btn-group-vertical > .btn + .btn,\n.btn-group-vertical > .btn + .btn-group,\n.btn-group-vertical > .btn-group + .btn,\n.btn-group-vertical > .btn-group + .btn-group {\n margin-top: -1px;\n margin-left: 0;\n}\n.btn-group-vertical > .btn:not(:first-child):not(:last-child) {\n border-radius: 0;\n}\n.btn-group-vertical > .btn:first-child:not(:last-child) {\n border-top-right-radius: 4px;\n border-bottom-right-radius: 0;\n border-bottom-left-radius: 0;\n}\n.btn-group-vertical > .btn:last-child:not(:first-child) {\n border-bottom-left-radius: 4px;\n border-top-right-radius: 0;\n border-top-left-radius: 0;\n}\n.btn-group-vertical > .btn-group:not(:first-child):not(:last-child) > .btn {\n border-radius: 0;\n}\n.btn-group-vertical > .btn-group:first-child:not(:last-child) > .btn:last-child,\n.btn-group-vertical > .btn-group:first-child:not(:last-child) > .dropdown-toggle {\n border-bottom-right-radius: 0;\n border-bottom-left-radius: 0;\n}\n.btn-group-vertical > .btn-group:last-child:not(:first-child) > .btn:first-child {\n border-top-right-radius: 0;\n border-top-left-radius: 0;\n}\n.btn-group-justified {\n display: table;\n width: 100%;\n table-layout: fixed;\n border-collapse: separate;\n}\n.btn-group-justified > .btn,\n.btn-group-justified > .btn-group {\n float: none;\n display: table-cell;\n width: 1%;\n}\n.btn-group-justified > .btn-group .btn {\n width: 100%;\n}\n.btn-group-justified > .btn-group .dropdown-menu {\n left: auto;\n}\n[data-toggle=\"buttons\"] > .btn input[type=\"radio\"],\n[data-toggle=\"buttons\"] > .btn-group > .btn input[type=\"radio\"],\n[data-toggle=\"buttons\"] > .btn input[type=\"checkbox\"],\n[data-toggle=\"buttons\"] > .btn-group > .btn input[type=\"checkbox\"] {\n position: absolute;\n clip: rect(0, 0, 0, 0);\n pointer-events: none;\n}\n.input-group {\n position: relative;\n display: table;\n border-collapse: separate;\n}\n.input-group[class*=\"col-\"] {\n float: none;\n padding-left: 0;\n padding-right: 0;\n}\n.input-group .form-control {\n position: relative;\n z-index: 2;\n float: left;\n width: 100%;\n margin-bottom: 0;\n}\n.input-group-lg > .form-control,\n.input-group-lg > .input-group-addon,\n.input-group-lg > .input-group-btn > .btn {\n height: 46px;\n padding: 10px 16px;\n font-size: 18px;\n line-height: 1.3333333;\n border-radius: 6px;\n}\nselect.input-group-lg > .form-control,\nselect.input-group-lg > .input-group-addon,\nselect.input-group-lg > .input-group-btn > .btn {\n height: 46px;\n line-height: 46px;\n}\ntextarea.input-group-lg > .form-control,\ntextarea.input-group-lg > .input-group-addon,\ntextarea.input-group-lg > .input-group-btn > .btn,\nselect[multiple].input-group-lg > .form-control,\nselect[multiple].input-group-lg > .input-group-addon,\nselect[multiple].input-group-lg > .input-group-btn > .btn {\n height: auto;\n}\n.input-group-sm > .form-control,\n.input-group-sm > .input-group-addon,\n.input-group-sm > .input-group-btn > .btn {\n height: 30px;\n padding: 5px 10px;\n font-size: 12px;\n line-height: 1.5;\n border-radius: 3px;\n}\nselect.input-group-sm > .form-control,\nselect.input-group-sm > .input-group-addon,\nselect.input-group-sm > .input-group-btn > .btn {\n height: 30px;\n line-height: 30px;\n}\ntextarea.input-group-sm > .form-control,\ntextarea.input-group-sm > .input-group-addon,\ntextarea.input-group-sm > .input-group-btn > .btn,\nselect[multiple].input-group-sm > .form-control,\nselect[multiple].input-group-sm > .input-group-addon,\nselect[multiple].input-group-sm > .input-group-btn > .btn {\n height: auto;\n}\n.input-group-addon,\n.input-group-btn,\n.input-group .form-control {\n display: table-cell;\n}\n.input-group-addon:not(:first-child):not(:last-child),\n.input-group-btn:not(:first-child):not(:last-child),\n.input-group .form-control:not(:first-child):not(:last-child) {\n border-radius: 0;\n}\n.input-group-addon,\n.input-group-btn {\n width: 1%;\n white-space: nowrap;\n vertical-align: middle;\n}\n.input-group-addon {\n padding: 6px 12px;\n font-size: 14px;\n font-weight: normal;\n line-height: 1;\n color: #555555;\n text-align: center;\n background-color: #eeeeee;\n border: 1px solid #cccccc;\n border-radius: 4px;\n}\n.input-group-addon.input-sm {\n padding: 5px 10px;\n font-size: 12px;\n border-radius: 3px;\n}\n.input-group-addon.input-lg {\n padding: 10px 16px;\n font-size: 18px;\n border-radius: 6px;\n}\n.input-group-addon input[type=\"radio\"],\n.input-group-addon input[type=\"checkbox\"] {\n margin-top: 0;\n}\n.input-group .form-control:first-child,\n.input-group-addon:first-child,\n.input-group-btn:first-child > .btn,\n.input-group-btn:first-child > .btn-group > .btn,\n.input-group-btn:first-child > .dropdown-toggle,\n.input-group-btn:last-child > .btn:not(:last-child):not(.dropdown-toggle),\n.input-group-btn:last-child > .btn-group:not(:last-child) > .btn {\n border-bottom-right-radius: 0;\n border-top-right-radius: 0;\n}\n.input-group-addon:first-child {\n border-right: 0;\n}\n.input-group .form-control:last-child,\n.input-group-addon:last-child,\n.input-group-btn:last-child > .btn,\n.input-group-btn:last-child > .btn-group > .btn,\n.input-group-btn:last-child > .dropdown-toggle,\n.input-group-btn:first-child > .btn:not(:first-child),\n.input-group-btn:first-child > .btn-group:not(:first-child) > .btn {\n border-bottom-left-radius: 0;\n border-top-left-radius: 0;\n}\n.input-group-addon:last-child {\n border-left: 0;\n}\n.input-group-btn {\n position: relative;\n font-size: 0;\n white-space: nowrap;\n}\n.input-group-btn > .btn {\n position: relative;\n}\n.input-group-btn > .btn + .btn {\n margin-left: -1px;\n}\n.input-group-btn > .btn:hover,\n.input-group-btn > .btn:focus,\n.input-group-btn > .btn:active {\n z-index: 2;\n}\n.input-group-btn:first-child > .btn,\n.input-group-btn:first-child > .btn-group {\n margin-right: -1px;\n}\n.input-group-btn:last-child > .btn,\n.input-group-btn:last-child > .btn-group {\n margin-left: -1px;\n}\n.nav {\n margin-bottom: 0;\n padding-left: 0;\n list-style: none;\n}\n.nav > li {\n position: relative;\n display: block;\n}\n.nav > li > a {\n position: relative;\n display: block;\n padding: 10px 15px;\n}\n.nav > li > a:hover,\n.nav > li > a:focus {\n text-decoration: none;\n background-color: #eeeeee;\n}\n.nav > li.disabled > a {\n color: #777777;\n}\n.nav > li.disabled > a:hover,\n.nav > li.disabled > a:focus {\n color: #777777;\n text-decoration: none;\n background-color: transparent;\n cursor: not-allowed;\n}\n.nav .open > a,\n.nav .open > a:hover,\n.nav .open > a:focus {\n background-color: #eeeeee;\n border-color: #337ab7;\n}\n.nav .nav-divider {\n height: 1px;\n margin: 9px 0;\n overflow: hidden;\n background-color: #e5e5e5;\n}\n.nav > li > a > img {\n max-width: none;\n}\n.nav-tabs {\n border-bottom: 1px solid #dddddd;\n}\n.nav-tabs > li {\n float: left;\n margin-bottom: -1px;\n}\n.nav-tabs > li > a {\n margin-right: 2px;\n line-height: 1.42857143;\n border: 1px solid transparent;\n border-radius: 4px 4px 0 0;\n}\n.nav-tabs > li > a:hover {\n border-color: #eeeeee #eeeeee #dddddd;\n}\n.nav-tabs > li.active > a,\n.nav-tabs > li.active > a:hover,\n.nav-tabs > li.active > a:focus {\n color: #555555;\n background-color: #ffffff;\n border: 1px solid #dddddd;\n border-bottom-color: transparent;\n cursor: default;\n}\n.nav-tabs.nav-justified {\n width: 100%;\n border-bottom: 0;\n}\n.nav-tabs.nav-justified > li {\n float: none;\n}\n.nav-tabs.nav-justified > li > a {\n text-align: center;\n margin-bottom: 5px;\n}\n.nav-tabs.nav-justified > .dropdown .dropdown-menu {\n top: auto;\n left: auto;\n}\n@media (min-width: 768px) {\n .nav-tabs.nav-justified > li {\n display: table-cell;\n width: 1%;\n }\n .nav-tabs.nav-justified > li > a {\n margin-bottom: 0;\n }\n}\n.nav-tabs.nav-justified > li > a {\n margin-right: 0;\n border-radius: 4px;\n}\n.nav-tabs.nav-justified > .active > a,\n.nav-tabs.nav-justified > .active > a:hover,\n.nav-tabs.nav-justified > .active > a:focus {\n border: 1px solid #dddddd;\n}\n@media (min-width: 768px) {\n .nav-tabs.nav-justified > li > a {\n border-bottom: 1px solid #dddddd;\n border-radius: 4px 4px 0 0;\n }\n .nav-tabs.nav-justified > .active > a,\n .nav-tabs.nav-justified > .active > a:hover,\n .nav-tabs.nav-justified > .active > a:focus {\n border-bottom-color: #ffffff;\n }\n}\n.nav-pills > li {\n float: left;\n}\n.nav-pills > li > a {\n border-radius: 4px;\n}\n.nav-pills > li + li {\n margin-left: 2px;\n}\n.nav-pills > li.active > a,\n.nav-pills > li.active > a:hover,\n.nav-pills > li.active > a:focus {\n color: #ffffff;\n background-color: #337ab7;\n}\n.nav-stacked > li {\n float: none;\n}\n.nav-stacked > li + li {\n margin-top: 2px;\n margin-left: 0;\n}\n.nav-justified {\n width: 100%;\n}\n.nav-justified > li {\n float: none;\n}\n.nav-justified > li > a {\n text-align: center;\n margin-bottom: 5px;\n}\n.nav-justified > .dropdown .dropdown-menu {\n top: auto;\n left: auto;\n}\n@media (min-width: 768px) {\n .nav-justified > li {\n display: table-cell;\n width: 1%;\n }\n .nav-justified > li > a {\n margin-bottom: 0;\n }\n}\n.nav-tabs-justified {\n border-bottom: 0;\n}\n.nav-tabs-justified > li > a {\n margin-right: 0;\n border-radius: 4px;\n}\n.nav-tabs-justified > .active > a,\n.nav-tabs-justified > .active > a:hover,\n.nav-tabs-justified > .active > a:focus {\n border: 1px solid #dddddd;\n}\n@media (min-width: 768px) {\n .nav-tabs-justified > li > a {\n border-bottom: 1px solid #dddddd;\n border-radius: 4px 4px 0 0;\n }\n .nav-tabs-justified > .active > a,\n .nav-tabs-justified > .active > a:hover,\n .nav-tabs-justified > .active > a:focus {\n border-bottom-color: #ffffff;\n }\n}\n.tab-content > .tab-pane {\n display: none;\n}\n.tab-content > .active {\n display: block;\n}\n.nav-tabs .dropdown-menu {\n margin-top: -1px;\n border-top-right-radius: 0;\n border-top-left-radius: 0;\n}\n.navbar {\n position: relative;\n min-height: 50px;\n margin-bottom: 20px;\n border: 1px solid transparent;\n}\n@media (min-width: 768px) {\n .navbar {\n border-radius: 4px;\n }\n}\n@media (min-width: 768px) {\n .navbar-header {\n float: left;\n }\n}\n.navbar-collapse {\n overflow-x: visible;\n padding-right: 15px;\n padding-left: 15px;\n border-top: 1px solid transparent;\n box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.1);\n -webkit-overflow-scrolling: touch;\n}\n.navbar-collapse.in {\n overflow-y: auto;\n}\n@media (min-width: 768px) {\n .navbar-collapse {\n width: auto;\n border-top: 0;\n box-shadow: none;\n }\n .navbar-collapse.collapse {\n display: block !important;\n height: auto !important;\n padding-bottom: 0;\n overflow: visible !important;\n }\n .navbar-collapse.in {\n overflow-y: visible;\n }\n .navbar-fixed-top .navbar-collapse,\n .navbar-static-top .navbar-collapse,\n .navbar-fixed-bottom .navbar-collapse {\n padding-left: 0;\n padding-right: 0;\n }\n}\n.navbar-fixed-top .navbar-collapse,\n.navbar-fixed-bottom .navbar-collapse {\n max-height: 340px;\n}\n@media (max-device-width: 480px) and (orientation: landscape) {\n .navbar-fixed-top .navbar-collapse,\n .navbar-fixed-bottom .navbar-collapse {\n max-height: 200px;\n }\n}\n.container > .navbar-header,\n.container-fluid > .navbar-header,\n.container > .navbar-collapse,\n.container-fluid > .navbar-collapse {\n margin-right: -15px;\n margin-left: -15px;\n}\n@media (min-width: 768px) {\n .container > .navbar-header,\n .container-fluid > .navbar-header,\n .container > .navbar-collapse,\n .container-fluid > .navbar-collapse {\n margin-right: 0;\n margin-left: 0;\n }\n}\n.navbar-static-top {\n z-index: 1000;\n border-width: 0 0 1px;\n}\n@media (min-width: 768px) {\n .navbar-static-top {\n border-radius: 0;\n }\n}\n.navbar-fixed-top,\n.navbar-fixed-bottom {\n position: fixed;\n right: 0;\n left: 0;\n z-index: 1030;\n}\n@media (min-width: 768px) {\n .navbar-fixed-top,\n .navbar-fixed-bottom {\n border-radius: 0;\n }\n}\n.navbar-fixed-top {\n top: 0;\n border-width: 0 0 1px;\n}\n.navbar-fixed-bottom {\n bottom: 0;\n margin-bottom: 0;\n border-width: 1px 0 0;\n}\n.navbar-brand {\n float: left;\n padding: 15px 15px;\n font-size: 18px;\n line-height: 20px;\n height: 50px;\n}\n.navbar-brand:hover,\n.navbar-brand:focus {\n text-decoration: none;\n}\n.navbar-brand > img {\n display: block;\n}\n@media (min-width: 768px) {\n .navbar > .container .navbar-brand,\n .navbar > .container-fluid .navbar-brand {\n margin-left: -15px;\n }\n}\n.navbar-toggle {\n position: relative;\n float: right;\n margin-right: 15px;\n padding: 9px 10px;\n margin-top: 8px;\n margin-bottom: 8px;\n background-color: transparent;\n background-image: none;\n border: 1px solid transparent;\n border-radius: 4px;\n}\n.navbar-toggle:focus {\n outline: 0;\n}\n.navbar-toggle .icon-bar {\n display: block;\n width: 22px;\n height: 2px;\n border-radius: 1px;\n}\n.navbar-toggle .icon-bar + .icon-bar {\n margin-top: 4px;\n}\n@media (min-width: 768px) {\n .navbar-toggle {\n display: none;\n }\n}\n.navbar-nav {\n margin: 7.5px -15px;\n}\n.navbar-nav > li > a {\n padding-top: 10px;\n padding-bottom: 10px;\n line-height: 20px;\n}\n@media (max-width: 767px) {\n .navbar-nav .open .dropdown-menu {\n position: static;\n float: none;\n width: auto;\n margin-top: 0;\n background-color: transparent;\n border: 0;\n box-shadow: none;\n }\n .navbar-nav .open .dropdown-menu > li > a,\n .navbar-nav .open .dropdown-menu .dropdown-header {\n padding: 5px 15px 5px 25px;\n }\n .navbar-nav .open .dropdown-menu > li > a {\n line-height: 20px;\n }\n .navbar-nav .open .dropdown-menu > li > a:hover,\n .navbar-nav .open .dropdown-menu > li > a:focus {\n background-image: none;\n }\n}\n@media (min-width: 768px) {\n .navbar-nav {\n float: left;\n margin: 0;\n }\n .navbar-nav > li {\n float: left;\n }\n .navbar-nav > li > a {\n padding-top: 15px;\n padding-bottom: 15px;\n }\n}\n.navbar-form {\n margin-left: -15px;\n margin-right: -15px;\n padding: 10px 15px;\n border-top: 1px solid transparent;\n border-bottom: 1px solid transparent;\n -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.1), 0 1px 0 rgba(255, 255, 255, 0.1);\n box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.1), 0 1px 0 rgba(255, 255, 255, 0.1);\n margin-top: 8px;\n margin-bottom: 8px;\n}\n@media (min-width: 768px) {\n .navbar-form .form-group {\n display: inline-block;\n margin-bottom: 0;\n vertical-align: middle;\n }\n .navbar-form .form-control {\n display: inline-block;\n width: auto;\n vertical-align: middle;\n }\n .navbar-form .form-control-static {\n display: inline-block;\n }\n .navbar-form .input-group {\n display: inline-table;\n vertical-align: middle;\n }\n .navbar-form .input-group .input-group-addon,\n .navbar-form .input-group .input-group-btn,\n .navbar-form .input-group .form-control {\n width: auto;\n }\n .navbar-form .input-group > .form-control {\n width: 100%;\n }\n .navbar-form .control-label {\n margin-bottom: 0;\n vertical-align: middle;\n }\n .navbar-form .radio,\n .navbar-form .checkbox {\n display: inline-block;\n margin-top: 0;\n margin-bottom: 0;\n vertical-align: middle;\n }\n .navbar-form .radio label,\n .navbar-form .checkbox label {\n padding-left: 0;\n }\n .navbar-form .radio input[type=\"radio\"],\n .navbar-form .checkbox input[type=\"checkbox\"] {\n position: relative;\n margin-left: 0;\n }\n .navbar-form .has-feedback .form-control-feedback {\n top: 0;\n }\n}\n@media (max-width: 767px) {\n .navbar-form .form-group {\n margin-bottom: 5px;\n }\n .navbar-form .form-group:last-child {\n margin-bottom: 0;\n }\n}\n@media (min-width: 768px) {\n .navbar-form {\n width: auto;\n border: 0;\n margin-left: 0;\n margin-right: 0;\n padding-top: 0;\n padding-bottom: 0;\n -webkit-box-shadow: none;\n box-shadow: none;\n }\n}\n.navbar-nav > li > .dropdown-menu {\n margin-top: 0;\n border-top-right-radius: 0;\n border-top-left-radius: 0;\n}\n.navbar-fixed-bottom .navbar-nav > li > .dropdown-menu {\n margin-bottom: 0;\n border-top-right-radius: 4px;\n border-top-left-radius: 4px;\n border-bottom-right-radius: 0;\n border-bottom-left-radius: 0;\n}\n.navbar-btn {\n margin-top: 8px;\n margin-bottom: 8px;\n}\n.navbar-btn.btn-sm {\n margin-top: 10px;\n margin-bottom: 10px;\n}\n.navbar-btn.btn-xs {\n margin-top: 14px;\n margin-bottom: 14px;\n}\n.navbar-text {\n margin-top: 15px;\n margin-bottom: 15px;\n}\n@media (min-width: 768px) {\n .navbar-text {\n float: left;\n margin-left: 15px;\n margin-right: 15px;\n }\n}\n@media (min-width: 768px) {\n .navbar-left {\n float: left !important;\n }\n .navbar-right {\n float: right !important;\n margin-right: -15px;\n }\n .navbar-right ~ .navbar-right {\n margin-right: 0;\n }\n}\n.navbar-default {\n background-color: #f8f8f8;\n border-color: #e7e7e7;\n}\n.navbar-default .navbar-brand {\n color: #777777;\n}\n.navbar-default .navbar-brand:hover,\n.navbar-default .navbar-brand:focus {\n color: #5e5e5e;\n background-color: transparent;\n}\n.navbar-default .navbar-text {\n color: #777777;\n}\n.navbar-default .navbar-nav > li > a {\n color: #777777;\n}\n.navbar-default .navbar-nav > li > a:hover,\n.navbar-default .navbar-nav > li > a:focus {\n color: #333333;\n background-color: transparent;\n}\n.navbar-default .navbar-nav > .active > a,\n.navbar-default .navbar-nav > .active > a:hover,\n.navbar-default .navbar-nav > .active > a:focus {\n color: #555555;\n background-color: #e7e7e7;\n}\n.navbar-default .navbar-nav > .disabled > a,\n.navbar-default .navbar-nav > .disabled > a:hover,\n.navbar-default .navbar-nav > .disabled > a:focus {\n color: #cccccc;\n background-color: transparent;\n}\n.navbar-default .navbar-toggle {\n border-color: #dddddd;\n}\n.navbar-default .navbar-toggle:hover,\n.navbar-default .navbar-toggle:focus {\n background-color: #dddddd;\n}\n.navbar-default .navbar-toggle .icon-bar {\n background-color: #888888;\n}\n.navbar-default .navbar-collapse,\n.navbar-default .navbar-form {\n border-color: #e7e7e7;\n}\n.navbar-default .navbar-nav > .open > a,\n.navbar-default .navbar-nav > .open > a:hover,\n.navbar-default .navbar-nav > .open > a:focus {\n background-color: #e7e7e7;\n color: #555555;\n}\n@media (max-width: 767px) {\n .navbar-default .navbar-nav .open .dropdown-menu > li > a {\n color: #777777;\n }\n .navbar-default .navbar-nav .open .dropdown-menu > li > a:hover,\n .navbar-default .navbar-nav .open .dropdown-menu > li > a:focus {\n color: #333333;\n background-color: transparent;\n }\n .navbar-default .navbar-nav .open .dropdown-menu > .active > a,\n .navbar-default .navbar-nav .open .dropdown-menu > .active > a:hover,\n .navbar-default .navbar-nav .open .dropdown-menu > .active > a:focus {\n color: #555555;\n background-color: #e7e7e7;\n }\n .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a,\n .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a:hover,\n .navbar-default .navbar-nav .open .dropdown-menu > .disabled > a:focus {\n color: #cccccc;\n background-color: transparent;\n }\n}\n.navbar-default .navbar-link {\n color: #777777;\n}\n.navbar-default .navbar-link:hover {\n color: #333333;\n}\n.navbar-default .btn-link {\n color: #777777;\n}\n.navbar-default .btn-link:hover,\n.navbar-default .btn-link:focus {\n color: #333333;\n}\n.navbar-default .btn-link[disabled]:hover,\nfieldset[disabled] .navbar-default .btn-link:hover,\n.navbar-default .btn-link[disabled]:focus,\nfieldset[disabled] .navbar-default .btn-link:focus {\n color: #cccccc;\n}\n.navbar-inverse {\n background-color: #222222;\n border-color: #080808;\n}\n.navbar-inverse .navbar-brand {\n color: #9d9d9d;\n}\n.navbar-inverse .navbar-brand:hover,\n.navbar-inverse .navbar-brand:focus {\n color: #ffffff;\n background-color: transparent;\n}\n.navbar-inverse .navbar-text {\n color: #9d9d9d;\n}\n.navbar-inverse .navbar-nav > li > a {\n color: #9d9d9d;\n}\n.navbar-inverse .navbar-nav > li > a:hover,\n.navbar-inverse .navbar-nav > li > a:focus {\n color: #ffffff;\n background-color: transparent;\n}\n.navbar-inverse .navbar-nav > .active > a,\n.navbar-inverse .navbar-nav > .active > a:hover,\n.navbar-inverse .navbar-nav > .active > a:focus {\n color: #ffffff;\n background-color: #080808;\n}\n.navbar-inverse .navbar-nav > .disabled > a,\n.navbar-inverse .navbar-nav > .disabled > a:hover,\n.navbar-inverse .navbar-nav > .disabled > a:focus {\n color: #444444;\n background-color: transparent;\n}\n.navbar-inverse .navbar-toggle {\n border-color: #333333;\n}\n.navbar-inverse .navbar-toggle:hover,\n.navbar-inverse .navbar-toggle:focus {\n background-color: #333333;\n}\n.navbar-inverse .navbar-toggle .icon-bar {\n background-color: #ffffff;\n}\n.navbar-inverse .navbar-collapse,\n.navbar-inverse .navbar-form {\n border-color: #101010;\n}\n.navbar-inverse .navbar-nav > .open > a,\n.navbar-inverse .navbar-nav > .open > a:hover,\n.navbar-inverse .navbar-nav > .open > a:focus {\n background-color: #080808;\n color: #ffffff;\n}\n@media (max-width: 767px) {\n .navbar-inverse .navbar-nav .open .dropdown-menu > .dropdown-header {\n border-color: #080808;\n }\n .navbar-inverse .navbar-nav .open .dropdown-menu .divider {\n background-color: #080808;\n }\n .navbar-inverse .navbar-nav .open .dropdown-menu > li > a {\n color: #9d9d9d;\n }\n .navbar-inverse .navbar-nav .open .dropdown-menu > li > a:hover,\n .navbar-inverse .navbar-nav .open .dropdown-menu > li > a:focus {\n color: #ffffff;\n background-color: transparent;\n }\n .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a,\n .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a:hover,\n .navbar-inverse .navbar-nav .open .dropdown-menu > .active > a:focus {\n color: #ffffff;\n background-color: #080808;\n }\n .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a,\n .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a:hover,\n .navbar-inverse .navbar-nav .open .dropdown-menu > .disabled > a:focus {\n color: #444444;\n background-color: transparent;\n }\n}\n.navbar-inverse .navbar-link {\n color: #9d9d9d;\n}\n.navbar-inverse .navbar-link:hover {\n color: #ffffff;\n}\n.navbar-inverse .btn-link {\n color: #9d9d9d;\n}\n.navbar-inverse .btn-link:hover,\n.navbar-inverse .btn-link:focus {\n color: #ffffff;\n}\n.navbar-inverse .btn-link[disabled]:hover,\nfieldset[disabled] .navbar-inverse .btn-link:hover,\n.navbar-inverse .btn-link[disabled]:focus,\nfieldset[disabled] .navbar-inverse .btn-link:focus {\n color: #444444;\n}\n.breadcrumb {\n padding: 8px 15px;\n margin-bottom: 20px;\n list-style: none;\n background-color: #f5f5f5;\n border-radius: 4px;\n}\n.breadcrumb > li {\n display: inline-block;\n}\n.breadcrumb > li + li:before {\n content: \"/\\00a0\";\n padding: 0 5px;\n color: #cccccc;\n}\n.breadcrumb > .active {\n color: #777777;\n}\n.pagination {\n display: inline-block;\n padding-left: 0;\n margin: 20px 0;\n border-radius: 4px;\n}\n.pagination > li {\n display: inline;\n}\n.pagination > li > a,\n.pagination > li > span {\n position: relative;\n float: left;\n padding: 6px 12px;\n line-height: 1.42857143;\n text-decoration: none;\n color: #337ab7;\n background-color: #ffffff;\n border: 1px solid #dddddd;\n margin-left: -1px;\n}\n.pagination > li:first-child > a,\n.pagination > li:first-child > span {\n margin-left: 0;\n border-bottom-left-radius: 4px;\n border-top-left-radius: 4px;\n}\n.pagination > li:last-child > a,\n.pagination > li:last-child > span {\n border-bottom-right-radius: 4px;\n border-top-right-radius: 4px;\n}\n.pagination > li > a:hover,\n.pagination > li > span:hover,\n.pagination > li > a:focus,\n.pagination > li > span:focus {\n color: #23527c;\n background-color: #eeeeee;\n border-color: #dddddd;\n}\n.pagination > .active > a,\n.pagination > .active > span,\n.pagination > .active > a:hover,\n.pagination > .active > span:hover,\n.pagination > .active > a:focus,\n.pagination > .active > span:focus {\n z-index: 2;\n color: #ffffff;\n background-color: #337ab7;\n border-color: #337ab7;\n cursor: default;\n}\n.pagination > .disabled > span,\n.pagination > .disabled > span:hover,\n.pagination > .disabled > span:focus,\n.pagination > .disabled > a,\n.pagination > .disabled > a:hover,\n.pagination > .disabled > a:focus {\n color: #777777;\n background-color: #ffffff;\n border-color: #dddddd;\n cursor: not-allowed;\n}\n.pagination-lg > li > a,\n.pagination-lg > li > span {\n padding: 10px 16px;\n font-size: 18px;\n}\n.pagination-lg > li:first-child > a,\n.pagination-lg > li:first-child > span {\n border-bottom-left-radius: 6px;\n border-top-left-radius: 6px;\n}\n.pagination-lg > li:last-child > a,\n.pagination-lg > li:last-child > span {\n border-bottom-right-radius: 6px;\n border-top-right-radius: 6px;\n}\n.pagination-sm > li > a,\n.pagination-sm > li > span {\n padding: 5px 10px;\n font-size: 12px;\n}\n.pagination-sm > li:first-child > a,\n.pagination-sm > li:first-child > span {\n border-bottom-left-radius: 3px;\n border-top-left-radius: 3px;\n}\n.pagination-sm > li:last-child > a,\n.pagination-sm > li:last-child > span {\n border-bottom-right-radius: 3px;\n border-top-right-radius: 3px;\n}\n.pager {\n padding-left: 0;\n margin: 20px 0;\n list-style: none;\n text-align: center;\n}\n.pager li {\n display: inline;\n}\n.pager li > a,\n.pager li > span {\n display: inline-block;\n padding: 5px 14px;\n background-color: #ffffff;\n border: 1px solid #dddddd;\n border-radius: 15px;\n}\n.pager li > a:hover,\n.pager li > a:focus {\n text-decoration: none;\n background-color: #eeeeee;\n}\n.pager .next > a,\n.pager .next > span {\n float: right;\n}\n.pager .previous > a,\n.pager .previous > span {\n float: left;\n}\n.pager .disabled > a,\n.pager .disabled > a:hover,\n.pager .disabled > a:focus,\n.pager .disabled > span {\n color: #777777;\n background-color: #ffffff;\n cursor: not-allowed;\n}\n.label {\n display: inline;\n padding: .2em .6em .3em;\n font-size: 75%;\n font-weight: bold;\n line-height: 1;\n color: #ffffff;\n text-align: center;\n white-space: nowrap;\n vertical-align: baseline;\n border-radius: .25em;\n}\na.label:hover,\na.label:focus {\n color: #ffffff;\n text-decoration: none;\n cursor: pointer;\n}\n.label:empty {\n display: none;\n}\n.btn .label {\n position: relative;\n top: -1px;\n}\n.label-default {\n background-color: #777777;\n}\n.label-default[href]:hover,\n.label-default[href]:focus {\n background-color: #5e5e5e;\n}\n.label-primary {\n background-color: #337ab7;\n}\n.label-primary[href]:hover,\n.label-primary[href]:focus {\n background-color: #286090;\n}\n.label-success {\n background-color: #5cb85c;\n}\n.label-success[href]:hover,\n.label-success[href]:focus {\n background-color: #449d44;\n}\n.label-info {\n background-color: #5bc0de;\n}\n.label-info[href]:hover,\n.label-info[href]:focus {\n background-color: #31b0d5;\n}\n.label-warning {\n background-color: #f0ad4e;\n}\n.label-warning[href]:hover,\n.label-warning[href]:focus {\n background-color: #ec971f;\n}\n.label-danger {\n background-color: #d9534f;\n}\n.label-danger[href]:hover,\n.label-danger[href]:focus {\n background-color: #c9302c;\n}\n.badge {\n display: inline-block;\n min-width: 10px;\n padding: 3px 7px;\n font-size: 12px;\n font-weight: bold;\n color: #ffffff;\n line-height: 1;\n vertical-align: baseline;\n white-space: nowrap;\n text-align: center;\n background-color: #777777;\n border-radius: 10px;\n}\n.badge:empty {\n display: none;\n}\n.btn .badge {\n position: relative;\n top: -1px;\n}\n.btn-xs .badge,\n.btn-group-xs > .btn .badge {\n top: 0;\n padding: 1px 5px;\n}\na.badge:hover,\na.badge:focus {\n color: #ffffff;\n text-decoration: none;\n cursor: pointer;\n}\n.list-group-item.active > .badge,\n.nav-pills > .active > a > .badge {\n color: #337ab7;\n background-color: #ffffff;\n}\n.list-group-item > .badge {\n float: right;\n}\n.list-group-item > .badge + .badge {\n margin-right: 5px;\n}\n.nav-pills > li > a > .badge {\n margin-left: 3px;\n}\n.jumbotron {\n padding: 30px 15px;\n margin-bottom: 30px;\n color: inherit;\n background-color: #eeeeee;\n}\n.jumbotron h1,\n.jumbotron .h1 {\n color: inherit;\n}\n.jumbotron p {\n margin-bottom: 15px;\n font-size: 21px;\n font-weight: 200;\n}\n.jumbotron > hr {\n border-top-color: #d5d5d5;\n}\n.container .jumbotron,\n.container-fluid .jumbotron {\n border-radius: 6px;\n}\n.jumbotron .container {\n max-width: 100%;\n}\n@media screen and (min-width: 768px) {\n .jumbotron {\n padding: 48px 0;\n }\n .container .jumbotron,\n .container-fluid .jumbotron {\n padding-left: 60px;\n padding-right: 60px;\n }\n .jumbotron h1,\n .jumbotron .h1 {\n font-size: 63px;\n }\n}\n.thumbnail {\n display: block;\n padding: 4px;\n margin-bottom: 20px;\n line-height: 1.42857143;\n background-color: #ffffff;\n border: 1px solid #dddddd;\n border-radius: 4px;\n -webkit-transition: border 0.2s ease-in-out;\n -o-transition: border 0.2s ease-in-out;\n transition: border 0.2s ease-in-out;\n}\n.thumbnail > img,\n.thumbnail a > img {\n margin-left: auto;\n margin-right: auto;\n}\na.thumbnail:hover,\na.thumbnail:focus,\na.thumbnail.active {\n border-color: #337ab7;\n}\n.thumbnail .caption {\n padding: 9px;\n color: #333333;\n}\n.alert {\n padding: 15px;\n margin-bottom: 20px;\n border: 1px solid transparent;\n border-radius: 4px;\n}\n.alert h4 {\n margin-top: 0;\n color: inherit;\n}\n.alert .alert-link {\n font-weight: bold;\n}\n.alert > p,\n.alert > ul {\n margin-bottom: 0;\n}\n.alert > p + p {\n margin-top: 5px;\n}\n.alert-dismissable,\n.alert-dismissible {\n padding-right: 35px;\n}\n.alert-dismissable .close,\n.alert-dismissible .close {\n position: relative;\n top: -2px;\n right: -21px;\n color: inherit;\n}\n.alert-success {\n background-color: #dff0d8;\n border-color: #d6e9c6;\n color: #3c763d;\n}\n.alert-success hr {\n border-top-color: #c9e2b3;\n}\n.alert-success .alert-link {\n color: #2b542c;\n}\n.alert-info {\n background-color: #d9edf7;\n border-color: #bce8f1;\n color: #31708f;\n}\n.alert-info hr {\n border-top-color: #a6e1ec;\n}\n.alert-info .alert-link {\n color: #245269;\n}\n.alert-warning {\n background-color: #fcf8e3;\n border-color: #faebcc;\n color: #8a6d3b;\n}\n.alert-warning hr {\n border-top-color: #f7e1b5;\n}\n.alert-warning .alert-link {\n color: #66512c;\n}\n.alert-danger {\n background-color: #f2dede;\n border-color: #ebccd1;\n color: #a94442;\n}\n.alert-danger hr {\n border-top-color: #e4b9c0;\n}\n.alert-danger .alert-link {\n color: #843534;\n}\n@-webkit-keyframes progress-bar-stripes {\n from {\n background-position: 40px 0;\n }\n to {\n background-position: 0 0;\n }\n}\n@keyframes progress-bar-stripes {\n from {\n background-position: 40px 0;\n }\n to {\n background-position: 0 0;\n }\n}\n.progress {\n overflow: hidden;\n height: 20px;\n margin-bottom: 20px;\n background-color: #f5f5f5;\n border-radius: 4px;\n -webkit-box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.1);\n box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.1);\n}\n.progress-bar {\n float: left;\n width: 0%;\n height: 100%;\n font-size: 12px;\n line-height: 20px;\n color: #ffffff;\n text-align: center;\n background-color: #337ab7;\n -webkit-box-shadow: inset 0 -1px 0 rgba(0, 0, 0, 0.15);\n box-shadow: inset 0 -1px 0 rgba(0, 0, 0, 0.15);\n -webkit-transition: width 0.6s ease;\n -o-transition: width 0.6s ease;\n transition: width 0.6s ease;\n}\n.progress-striped .progress-bar,\n.progress-bar-striped {\n background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-size: 40px 40px;\n}\n.progress.active .progress-bar,\n.progress-bar.active {\n -webkit-animation: progress-bar-stripes 2s linear infinite;\n -o-animation: progress-bar-stripes 2s linear infinite;\n animation: progress-bar-stripes 2s linear infinite;\n}\n.progress-bar-success {\n background-color: #5cb85c;\n}\n.progress-striped .progress-bar-success {\n background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n}\n.progress-bar-info {\n background-color: #5bc0de;\n}\n.progress-striped .progress-bar-info {\n background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n}\n.progress-bar-warning {\n background-color: #f0ad4e;\n}\n.progress-striped .progress-bar-warning {\n background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n}\n.progress-bar-danger {\n background-color: #d9534f;\n}\n.progress-striped .progress-bar-danger {\n background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n background-image: linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);\n}\n.media {\n margin-top: 15px;\n}\n.media:first-child {\n margin-top: 0;\n}\n.media,\n.media-body {\n zoom: 1;\n overflow: hidden;\n}\n.media-body {\n width: 10000px;\n}\n.media-object {\n display: block;\n}\n.media-right,\n.media > .pull-right {\n padding-left: 10px;\n}\n.media-left,\n.media > .pull-left {\n padding-right: 10px;\n}\n.media-left,\n.media-right,\n.media-body {\n display: table-cell;\n vertical-align: top;\n}\n.media-middle {\n vertical-align: middle;\n}\n.media-bottom {\n vertical-align: bottom;\n}\n.media-heading {\n margin-top: 0;\n margin-bottom: 5px;\n}\n.media-list {\n padding-left: 0;\n list-style: none;\n}\n.list-group {\n margin-bottom: 20px;\n padding-left: 0;\n}\n.list-group-item {\n position: relative;\n display: block;\n padding: 10px 15px;\n margin-bottom: -1px;\n background-color: #ffffff;\n border: 1px solid #dddddd;\n}\n.list-group-item:first-child {\n border-top-right-radius: 4px;\n border-top-left-radius: 4px;\n}\n.list-group-item:last-child {\n margin-bottom: 0;\n border-bottom-right-radius: 4px;\n border-bottom-left-radius: 4px;\n}\na.list-group-item {\n color: #555555;\n}\na.list-group-item .list-group-item-heading {\n color: #333333;\n}\na.list-group-item:hover,\na.list-group-item:focus {\n text-decoration: none;\n color: #555555;\n background-color: #f5f5f5;\n}\n.list-group-item.disabled,\n.list-group-item.disabled:hover,\n.list-group-item.disabled:focus {\n background-color: #eeeeee;\n color: #777777;\n cursor: not-allowed;\n}\n.list-group-item.disabled .list-group-item-heading,\n.list-group-item.disabled:hover .list-group-item-heading,\n.list-group-item.disabled:focus .list-group-item-heading {\n color: inherit;\n}\n.list-group-item.disabled .list-group-item-text,\n.list-group-item.disabled:hover .list-group-item-text,\n.list-group-item.disabled:focus .list-group-item-text {\n color: #777777;\n}\n.list-group-item.active,\n.list-group-item.active:hover,\n.list-group-item.active:focus {\n z-index: 2;\n color: #ffffff;\n background-color: #337ab7;\n border-color: #337ab7;\n}\n.list-group-item.active .list-group-item-heading,\n.list-group-item.active:hover .list-group-item-heading,\n.list-group-item.active:focus .list-group-item-heading,\n.list-group-item.active .list-group-item-heading > small,\n.list-group-item.active:hover .list-group-item-heading > small,\n.list-group-item.active:focus .list-group-item-heading > small,\n.list-group-item.active .list-group-item-heading > .small,\n.list-group-item.active:hover .list-group-item-heading > .small,\n.list-group-item.active:focus .list-group-item-heading > .small {\n color: inherit;\n}\n.list-group-item.active .list-group-item-text,\n.list-group-item.active:hover .list-group-item-text,\n.list-group-item.active:focus .list-group-item-text {\n color: #c7ddef;\n}\n.list-group-item-success {\n color: #3c763d;\n background-color: #dff0d8;\n}\na.list-group-item-success {\n color: #3c763d;\n}\na.list-group-item-success .list-group-item-heading {\n color: inherit;\n}\na.list-group-item-success:hover,\na.list-group-item-success:focus {\n color: #3c763d;\n background-color: #d0e9c6;\n}\na.list-group-item-success.active,\na.list-group-item-success.active:hover,\na.list-group-item-success.active:focus {\n color: #fff;\n background-color: #3c763d;\n border-color: #3c763d;\n}\n.list-group-item-info {\n color: #31708f;\n background-color: #d9edf7;\n}\na.list-group-item-info {\n color: #31708f;\n}\na.list-group-item-info .list-group-item-heading {\n color: inherit;\n}\na.list-group-item-info:hover,\na.list-group-item-info:focus {\n color: #31708f;\n background-color: #c4e3f3;\n}\na.list-group-item-info.active,\na.list-group-item-info.active:hover,\na.list-group-item-info.active:focus {\n color: #fff;\n background-color: #31708f;\n border-color: #31708f;\n}\n.list-group-item-warning {\n color: #8a6d3b;\n background-color: #fcf8e3;\n}\na.list-group-item-warning {\n color: #8a6d3b;\n}\na.list-group-item-warning .list-group-item-heading {\n color: inherit;\n}\na.list-group-item-warning:hover,\na.list-group-item-warning:focus {\n color: #8a6d3b;\n background-color: #faf2cc;\n}\na.list-group-item-warning.active,\na.list-group-item-warning.active:hover,\na.list-group-item-warning.active:focus {\n color: #fff;\n background-color: #8a6d3b;\n border-color: #8a6d3b;\n}\n.list-group-item-danger {\n color: #a94442;\n background-color: #f2dede;\n}\na.list-group-item-danger {\n color: #a94442;\n}\na.list-group-item-danger .list-group-item-heading {\n color: inherit;\n}\na.list-group-item-danger:hover,\na.list-group-item-danger:focus {\n color: #a94442;\n background-color: #ebcccc;\n}\na.list-group-item-danger.active,\na.list-group-item-danger.active:hover,\na.list-group-item-danger.active:focus {\n color: #fff;\n background-color: #a94442;\n border-color: #a94442;\n}\n.list-group-item-heading {\n margin-top: 0;\n margin-bottom: 5px;\n}\n.list-group-item-text {\n margin-bottom: 0;\n line-height: 1.3;\n}\n.panel {\n margin-bottom: 20px;\n background-color: #ffffff;\n border: 1px solid transparent;\n border-radius: 4px;\n -webkit-box-shadow: 0 1px 1px rgba(0, 0, 0, 0.05);\n box-shadow: 0 1px 1px rgba(0, 0, 0, 0.05);\n}\n.panel-body {\n padding: 15px;\n}\n.panel-heading {\n padding: 10px 15px;\n border-bottom: 1px solid transparent;\n border-top-right-radius: 3px;\n border-top-left-radius: 3px;\n}\n.panel-heading > .dropdown .dropdown-toggle {\n color: inherit;\n}\n.panel-title {\n margin-top: 0;\n margin-bottom: 0;\n font-size: 16px;\n color: inherit;\n}\n.panel-title > a,\n.panel-title > small,\n.panel-title > .small,\n.panel-title > small > a,\n.panel-title > .small > a {\n color: inherit;\n}\n.panel-footer {\n padding: 10px 15px;\n background-color: #f5f5f5;\n border-top: 1px solid #dddddd;\n border-bottom-right-radius: 3px;\n border-bottom-left-radius: 3px;\n}\n.panel > .list-group,\n.panel > .panel-collapse > .list-group {\n margin-bottom: 0;\n}\n.panel > .list-group .list-group-item,\n.panel > .panel-collapse > .list-group .list-group-item {\n border-width: 1px 0;\n border-radius: 0;\n}\n.panel > .list-group:first-child .list-group-item:first-child,\n.panel > .panel-collapse > .list-group:first-child .list-group-item:first-child {\n border-top: 0;\n border-top-right-radius: 3px;\n border-top-left-radius: 3px;\n}\n.panel > .list-group:last-child .list-group-item:last-child,\n.panel > .panel-collapse > .list-group:last-child .list-group-item:last-child {\n border-bottom: 0;\n border-bottom-right-radius: 3px;\n border-bottom-left-radius: 3px;\n}\n.panel-heading + .list-group .list-group-item:first-child {\n border-top-width: 0;\n}\n.list-group + .panel-footer {\n border-top-width: 0;\n}\n.panel > .table,\n.panel > .table-responsive > .table,\n.panel > .panel-collapse > .table {\n margin-bottom: 0;\n}\n.panel > .table caption,\n.panel > .table-responsive > .table caption,\n.panel > .panel-collapse > .table caption {\n padding-left: 15px;\n padding-right: 15px;\n}\n.panel > .table:first-child,\n.panel > .table-responsive:first-child > .table:first-child {\n border-top-right-radius: 3px;\n border-top-left-radius: 3px;\n}\n.panel > .table:first-child > thead:first-child > tr:first-child,\n.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child,\n.panel > .table:first-child > tbody:first-child > tr:first-child,\n.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child {\n border-top-left-radius: 3px;\n border-top-right-radius: 3px;\n}\n.panel > .table:first-child > thead:first-child > tr:first-child td:first-child,\n.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child td:first-child,\n.panel > .table:first-child > tbody:first-child > tr:first-child td:first-child,\n.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child td:first-child,\n.panel > .table:first-child > thead:first-child > tr:first-child th:first-child,\n.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child th:first-child,\n.panel > .table:first-child > tbody:first-child > tr:first-child th:first-child,\n.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child th:first-child {\n border-top-left-radius: 3px;\n}\n.panel > .table:first-child > thead:first-child > tr:first-child td:last-child,\n.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child td:last-child,\n.panel > .table:first-child > tbody:first-child > tr:first-child td:last-child,\n.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child td:last-child,\n.panel > .table:first-child > thead:first-child > tr:first-child th:last-child,\n.panel > .table-responsive:first-child > .table:first-child > thead:first-child > tr:first-child th:last-child,\n.panel > .table:first-child > tbody:first-child > tr:first-child th:last-child,\n.panel > .table-responsive:first-child > .table:first-child > tbody:first-child > tr:first-child th:last-child {\n border-top-right-radius: 3px;\n}\n.panel > .table:last-child,\n.panel > .table-responsive:last-child > .table:last-child {\n border-bottom-right-radius: 3px;\n border-bottom-left-radius: 3px;\n}\n.panel > .table:last-child > tbody:last-child > tr:last-child,\n.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child,\n.panel > .table:last-child > tfoot:last-child > tr:last-child,\n.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child {\n border-bottom-left-radius: 3px;\n border-bottom-right-radius: 3px;\n}\n.panel > .table:last-child > tbody:last-child > tr:last-child td:first-child,\n.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child td:first-child,\n.panel > .table:last-child > tfoot:last-child > tr:last-child td:first-child,\n.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child td:first-child,\n.panel > .table:last-child > tbody:last-child > tr:last-child th:first-child,\n.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child th:first-child,\n.panel > .table:last-child > tfoot:last-child > tr:last-child th:first-child,\n.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child th:first-child {\n border-bottom-left-radius: 3px;\n}\n.panel > .table:last-child > tbody:last-child > tr:last-child td:last-child,\n.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child td:last-child,\n.panel > .table:last-child > tfoot:last-child > tr:last-child td:last-child,\n.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child td:last-child,\n.panel > .table:last-child > tbody:last-child > tr:last-child th:last-child,\n.panel > .table-responsive:last-child > .table:last-child > tbody:last-child > tr:last-child th:last-child,\n.panel > .table:last-child > tfoot:last-child > tr:last-child th:last-child,\n.panel > .table-responsive:last-child > .table:last-child > tfoot:last-child > tr:last-child th:last-child {\n border-bottom-right-radius: 3px;\n}\n.panel > .panel-body + .table,\n.panel > .panel-body + .table-responsive,\n.panel > .table + .panel-body,\n.panel > .table-responsive + .panel-body {\n border-top: 1px solid #dddddd;\n}\n.panel > .table > tbody:first-child > tr:first-child th,\n.panel > .table > tbody:first-child > tr:first-child td {\n border-top: 0;\n}\n.panel > .table-bordered,\n.panel > .table-responsive > .table-bordered {\n border: 0;\n}\n.panel > .table-bordered > thead > tr > th:first-child,\n.panel > .table-responsive > .table-bordered > thead > tr > th:first-child,\n.panel > .table-bordered > tbody > tr > th:first-child,\n.panel > .table-responsive > .table-bordered > tbody > tr > th:first-child,\n.panel > .table-bordered > tfoot > tr > th:first-child,\n.panel > .table-responsive > .table-bordered > tfoot > tr > th:first-child,\n.panel > .table-bordered > thead > tr > td:first-child,\n.panel > .table-responsive > .table-bordered > thead > tr > td:first-child,\n.panel > .table-bordered > tbody > tr > td:first-child,\n.panel > .table-responsive > .table-bordered > tbody > tr > td:first-child,\n.panel > .table-bordered > tfoot > tr > td:first-child,\n.panel > .table-responsive > .table-bordered > tfoot > tr > td:first-child {\n border-left: 0;\n}\n.panel > .table-bordered > thead > tr > th:last-child,\n.panel > .table-responsive > .table-bordered > thead > tr > th:last-child,\n.panel > .table-bordered > tbody > tr > th:last-child,\n.panel > .table-responsive > .table-bordered > tbody > tr > th:last-child,\n.panel > .table-bordered > tfoot > tr > th:last-child,\n.panel > .table-responsive > .table-bordered > tfoot > tr > th:last-child,\n.panel > .table-bordered > thead > tr > td:last-child,\n.panel > .table-responsive > .table-bordered > thead > tr > td:last-child,\n.panel > .table-bordered > tbody > tr > td:last-child,\n.panel > .table-responsive > .table-bordered > tbody > tr > td:last-child,\n.panel > .table-bordered > tfoot > tr > td:last-child,\n.panel > .table-responsive > .table-bordered > tfoot > tr > td:last-child {\n border-right: 0;\n}\n.panel > .table-bordered > thead > tr:first-child > td,\n.panel > .table-responsive > .table-bordered > thead > tr:first-child > td,\n.panel > .table-bordered > tbody > tr:first-child > td,\n.panel > .table-responsive > .table-bordered > tbody > tr:first-child > td,\n.panel > .table-bordered > thead > tr:first-child > th,\n.panel > .table-responsive > .table-bordered > thead > tr:first-child > th,\n.panel > .table-bordered > tbody > tr:first-child > th,\n.panel > .table-responsive > .table-bordered > tbody > tr:first-child > th {\n border-bottom: 0;\n}\n.panel > .table-bordered > tbody > tr:last-child > td,\n.panel > .table-responsive > .table-bordered > tbody > tr:last-child > td,\n.panel > .table-bordered > tfoot > tr:last-child > td,\n.panel > .table-responsive > .table-bordered > tfoot > tr:last-child > td,\n.panel > .table-bordered > tbody > tr:last-child > th,\n.panel > .table-responsive > .table-bordered > tbody > tr:last-child > th,\n.panel > .table-bordered > tfoot > tr:last-child > th,\n.panel > .table-responsive > .table-bordered > tfoot > tr:last-child > th {\n border-bottom: 0;\n}\n.panel > .table-responsive {\n border: 0;\n margin-bottom: 0;\n}\n.panel-group {\n margin-bottom: 20px;\n}\n.panel-group .panel {\n margin-bottom: 0;\n border-radius: 4px;\n}\n.panel-group .panel + .panel {\n margin-top: 5px;\n}\n.panel-group .panel-heading {\n border-bottom: 0;\n}\n.panel-group .panel-heading + .panel-collapse > .panel-body,\n.panel-group .panel-heading + .panel-collapse > .list-group {\n border-top: 1px solid #dddddd;\n}\n.panel-group .panel-footer {\n border-top: 0;\n}\n.panel-group .panel-footer + .panel-collapse .panel-body {\n border-bottom: 1px solid #dddddd;\n}\n.panel-default {\n border-color: #dddddd;\n}\n.panel-default > .panel-heading {\n color: #333333;\n background-color: #f5f5f5;\n border-color: #dddddd;\n}\n.panel-default > .panel-heading + .panel-collapse > .panel-body {\n border-top-color: #dddddd;\n}\n.panel-default > .panel-heading .badge {\n color: #f5f5f5;\n background-color: #333333;\n}\n.panel-default > .panel-footer + .panel-collapse > .panel-body {\n border-bottom-color: #dddddd;\n}\n.panel-primary {\n border-color: #337ab7;\n}\n.panel-primary > .panel-heading {\n color: #ffffff;\n background-color: #337ab7;\n border-color: #337ab7;\n}\n.panel-primary > .panel-heading + .panel-collapse > .panel-body {\n border-top-color: #337ab7;\n}\n.panel-primary > .panel-heading .badge {\n color: #337ab7;\n background-color: #ffffff;\n}\n.panel-primary > .panel-footer + .panel-collapse > .panel-body {\n border-bottom-color: #337ab7;\n}\n.panel-success {\n border-color: #d6e9c6;\n}\n.panel-success > .panel-heading {\n color: #3c763d;\n background-color: #dff0d8;\n border-color: #d6e9c6;\n}\n.panel-success > .panel-heading + .panel-collapse > .panel-body {\n border-top-color: #d6e9c6;\n}\n.panel-success > .panel-heading .badge {\n color: #dff0d8;\n background-color: #3c763d;\n}\n.panel-success > .panel-footer + .panel-collapse > .panel-body {\n border-bottom-color: #d6e9c6;\n}\n.panel-info {\n border-color: #bce8f1;\n}\n.panel-info > .panel-heading {\n color: #31708f;\n background-color: #d9edf7;\n border-color: #bce8f1;\n}\n.panel-info > .panel-heading + .panel-collapse > .panel-body {\n border-top-color: #bce8f1;\n}\n.panel-info > .panel-heading .badge {\n color: #d9edf7;\n background-color: #31708f;\n}\n.panel-info > .panel-footer + .panel-collapse > .panel-body {\n border-bottom-color: #bce8f1;\n}\n.panel-warning {\n border-color: #faebcc;\n}\n.panel-warning > .panel-heading {\n color: #8a6d3b;\n background-color: #fcf8e3;\n border-color: #faebcc;\n}\n.panel-warning > .panel-heading + .panel-collapse > .panel-body {\n border-top-color: #faebcc;\n}\n.panel-warning > .panel-heading .badge {\n color: #fcf8e3;\n background-color: #8a6d3b;\n}\n.panel-warning > .panel-footer + .panel-collapse > .panel-body {\n border-bottom-color: #faebcc;\n}\n.panel-danger {\n border-color: #ebccd1;\n}\n.panel-danger > .panel-heading {\n color: #a94442;\n background-color: #f2dede;\n border-color: #ebccd1;\n}\n.panel-danger > .panel-heading + .panel-collapse > .panel-body {\n border-top-color: #ebccd1;\n}\n.panel-danger > .panel-heading .badge {\n color: #f2dede;\n background-color: #a94442;\n}\n.panel-danger > .panel-footer + .panel-collapse > .panel-body {\n border-bottom-color: #ebccd1;\n}\n.embed-responsive {\n position: relative;\n display: block;\n height: 0;\n padding: 0;\n overflow: hidden;\n}\n.embed-responsive .embed-responsive-item,\n.embed-responsive iframe,\n.embed-responsive embed,\n.embed-responsive object,\n.embed-responsive video {\n position: absolute;\n top: 0;\n left: 0;\n bottom: 0;\n height: 100%;\n width: 100%;\n border: 0;\n}\n.embed-responsive-16by9 {\n padding-bottom: 56.25%;\n}\n.embed-responsive-4by3 {\n padding-bottom: 75%;\n}\n.well {\n min-height: 20px;\n padding: 19px;\n margin-bottom: 20px;\n background-color: #f5f5f5;\n border: 1px solid #e3e3e3;\n border-radius: 4px;\n -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.05);\n box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.05);\n}\n.well blockquote {\n border-color: #ddd;\n border-color: rgba(0, 0, 0, 0.15);\n}\n.well-lg {\n padding: 24px;\n border-radius: 6px;\n}\n.well-sm {\n padding: 9px;\n border-radius: 3px;\n}\n.close {\n float: right;\n font-size: 21px;\n font-weight: bold;\n line-height: 1;\n color: #000000;\n text-shadow: 0 1px 0 #ffffff;\n opacity: 0.2;\n filter: alpha(opacity=20);\n}\n.close:hover,\n.close:focus {\n color: #000000;\n text-decoration: none;\n cursor: pointer;\n opacity: 0.5;\n filter: alpha(opacity=50);\n}\nbutton.close {\n padding: 0;\n cursor: pointer;\n background: transparent;\n border: 0;\n -webkit-appearance: none;\n}\n.modal-open {\n overflow: hidden;\n}\n.modal {\n display: none;\n overflow: hidden;\n position: fixed;\n top: 0;\n right: 0;\n bottom: 0;\n left: 0;\n z-index: 1050;\n -webkit-overflow-scrolling: touch;\n outline: 0;\n}\n.modal.fade .modal-dialog {\n -webkit-transform: translate(0, -25%);\n -ms-transform: translate(0, -25%);\n -o-transform: translate(0, -25%);\n transform: translate(0, -25%);\n -webkit-transition: -webkit-transform 0.3s ease-out;\n -moz-transition: -moz-transform 0.3s ease-out;\n -o-transition: -o-transform 0.3s ease-out;\n transition: transform 0.3s ease-out;\n}\n.modal.in .modal-dialog {\n -webkit-transform: translate(0, 0);\n -ms-transform: translate(0, 0);\n -o-transform: translate(0, 0);\n transform: translate(0, 0);\n}\n.modal-open .modal {\n overflow-x: hidden;\n overflow-y: auto;\n}\n.modal-dialog {\n position: relative;\n width: auto;\n margin: 10px;\n}\n.modal-content {\n position: relative;\n background-color: #ffffff;\n border: 1px solid #999999;\n border: 1px solid rgba(0, 0, 0, 0.2);\n border-radius: 6px;\n -webkit-box-shadow: 0 3px 9px rgba(0, 0, 0, 0.5);\n box-shadow: 0 3px 9px rgba(0, 0, 0, 0.5);\n background-clip: padding-box;\n outline: 0;\n}\n.modal-backdrop {\n position: fixed;\n top: 0;\n right: 0;\n bottom: 0;\n left: 0;\n z-index: 1040;\n background-color: #000000;\n}\n.modal-backdrop.fade {\n opacity: 0;\n filter: alpha(opacity=0);\n}\n.modal-backdrop.in {\n opacity: 0.5;\n filter: alpha(opacity=50);\n}\n.modal-header {\n padding: 15px;\n border-bottom: 1px solid #e5e5e5;\n min-height: 16.42857143px;\n}\n.modal-header .close {\n margin-top: -2px;\n}\n.modal-title {\n margin: 0;\n line-height: 1.42857143;\n}\n.modal-body {\n position: relative;\n padding: 15px;\n}\n.modal-footer {\n padding: 15px;\n text-align: right;\n border-top: 1px solid #e5e5e5;\n}\n.modal-footer .btn + .btn {\n margin-left: 5px;\n margin-bottom: 0;\n}\n.modal-footer .btn-group .btn + .btn {\n margin-left: -1px;\n}\n.modal-footer .btn-block + .btn-block {\n margin-left: 0;\n}\n.modal-scrollbar-measure {\n position: absolute;\n top: -9999px;\n width: 50px;\n height: 50px;\n overflow: scroll;\n}\n@media (min-width: 768px) {\n .modal-dialog {\n width: 600px;\n margin: 30px auto;\n }\n .modal-content {\n -webkit-box-shadow: 0 5px 15px rgba(0, 0, 0, 0.5);\n box-shadow: 0 5px 15px rgba(0, 0, 0, 0.5);\n }\n .modal-sm {\n width: 300px;\n }\n}\n@media (min-width: 992px) {\n .modal-lg {\n width: 900px;\n }\n}\n.tooltip {\n position: absolute;\n z-index: 1070;\n display: block;\n font-family: \"Helvetica Neue\", Helvetica, Arial, sans-serif;\n font-size: 12px;\n font-weight: normal;\n line-height: 1.4;\n opacity: 0;\n filter: alpha(opacity=0);\n}\n.tooltip.in {\n opacity: 0.9;\n filter: alpha(opacity=90);\n}\n.tooltip.top {\n margin-top: -3px;\n padding: 5px 0;\n}\n.tooltip.right {\n margin-left: 3px;\n padding: 0 5px;\n}\n.tooltip.bottom {\n margin-top: 3px;\n padding: 5px 0;\n}\n.tooltip.left {\n margin-left: -3px;\n padding: 0 5px;\n}\n.tooltip-inner {\n max-width: 200px;\n padding: 3px 8px;\n color: #ffffff;\n text-align: center;\n text-decoration: none;\n background-color: #000000;\n border-radius: 4px;\n}\n.tooltip-arrow {\n position: absolute;\n width: 0;\n height: 0;\n border-color: transparent;\n border-style: solid;\n}\n.tooltip.top .tooltip-arrow {\n bottom: 0;\n left: 50%;\n margin-left: -5px;\n border-width: 5px 5px 0;\n border-top-color: #000000;\n}\n.tooltip.top-left .tooltip-arrow {\n bottom: 0;\n right: 5px;\n margin-bottom: -5px;\n border-width: 5px 5px 0;\n border-top-color: #000000;\n}\n.tooltip.top-right .tooltip-arrow {\n bottom: 0;\n left: 5px;\n margin-bottom: -5px;\n border-width: 5px 5px 0;\n border-top-color: #000000;\n}\n.tooltip.right .tooltip-arrow {\n top: 50%;\n left: 0;\n margin-top: -5px;\n border-width: 5px 5px 5px 0;\n border-right-color: #000000;\n}\n.tooltip.left .tooltip-arrow {\n top: 50%;\n right: 0;\n margin-top: -5px;\n border-width: 5px 0 5px 5px;\n border-left-color: #000000;\n}\n.tooltip.bottom .tooltip-arrow {\n top: 0;\n left: 50%;\n margin-left: -5px;\n border-width: 0 5px 5px;\n border-bottom-color: #000000;\n}\n.tooltip.bottom-left .tooltip-arrow {\n top: 0;\n right: 5px;\n margin-top: -5px;\n border-width: 0 5px 5px;\n border-bottom-color: #000000;\n}\n.tooltip.bottom-right .tooltip-arrow {\n top: 0;\n left: 5px;\n margin-top: -5px;\n border-width: 0 5px 5px;\n border-bottom-color: #000000;\n}\n.popover {\n position: absolute;\n top: 0;\n left: 0;\n z-index: 1060;\n display: none;\n max-width: 276px;\n padding: 1px;\n font-family: \"Helvetica Neue\", Helvetica, Arial, sans-serif;\n font-size: 14px;\n font-weight: normal;\n line-height: 1.42857143;\n text-align: left;\n background-color: #ffffff;\n background-clip: padding-box;\n border: 1px solid #cccccc;\n border: 1px solid rgba(0, 0, 0, 0.2);\n border-radius: 6px;\n -webkit-box-shadow: 0 5px 10px rgba(0, 0, 0, 0.2);\n box-shadow: 0 5px 10px rgba(0, 0, 0, 0.2);\n white-space: normal;\n}\n.popover.top {\n margin-top: -10px;\n}\n.popover.right {\n margin-left: 10px;\n}\n.popover.bottom {\n margin-top: 10px;\n}\n.popover.left {\n margin-left: -10px;\n}\n.popover-title {\n margin: 0;\n padding: 8px 14px;\n font-size: 14px;\n background-color: #f7f7f7;\n border-bottom: 1px solid #ebebeb;\n border-radius: 5px 5px 0 0;\n}\n.popover-content {\n padding: 9px 14px;\n}\n.popover > .arrow,\n.popover > .arrow:after {\n position: absolute;\n display: block;\n width: 0;\n height: 0;\n border-color: transparent;\n border-style: solid;\n}\n.popover > .arrow {\n border-width: 11px;\n}\n.popover > .arrow:after {\n border-width: 10px;\n content: \"\";\n}\n.popover.top > .arrow {\n left: 50%;\n margin-left: -11px;\n border-bottom-width: 0;\n border-top-color: #999999;\n border-top-color: rgba(0, 0, 0, 0.25);\n bottom: -11px;\n}\n.popover.top > .arrow:after {\n content: \" \";\n bottom: 1px;\n margin-left: -10px;\n border-bottom-width: 0;\n border-top-color: #ffffff;\n}\n.popover.right > .arrow {\n top: 50%;\n left: -11px;\n margin-top: -11px;\n border-left-width: 0;\n border-right-color: #999999;\n border-right-color: rgba(0, 0, 0, 0.25);\n}\n.popover.right > .arrow:after {\n content: \" \";\n left: 1px;\n bottom: -10px;\n border-left-width: 0;\n border-right-color: #ffffff;\n}\n.popover.bottom > .arrow {\n left: 50%;\n margin-left: -11px;\n border-top-width: 0;\n border-bottom-color: #999999;\n border-bottom-color: rgba(0, 0, 0, 0.25);\n top: -11px;\n}\n.popover.bottom > .arrow:after {\n content: \" \";\n top: 1px;\n margin-left: -10px;\n border-top-width: 0;\n border-bottom-color: #ffffff;\n}\n.popover.left > .arrow {\n top: 50%;\n right: -11px;\n margin-top: -11px;\n border-right-width: 0;\n border-left-color: #999999;\n border-left-color: rgba(0, 0, 0, 0.25);\n}\n.popover.left > .arrow:after {\n content: \" \";\n right: 1px;\n border-right-width: 0;\n border-left-color: #ffffff;\n bottom: -10px;\n}\n.carousel {\n position: relative;\n}\n.carousel-inner {\n position: relative;\n overflow: hidden;\n width: 100%;\n}\n.carousel-inner > .item {\n display: none;\n position: relative;\n -webkit-transition: 0.6s ease-in-out left;\n -o-transition: 0.6s ease-in-out left;\n transition: 0.6s ease-in-out left;\n}\n.carousel-inner > .item > img,\n.carousel-inner > .item > a > img {\n line-height: 1;\n}\n@media all and (transform-3d), (-webkit-transform-3d) {\n .carousel-inner > .item {\n -webkit-transition: -webkit-transform 0.6s ease-in-out;\n -moz-transition: -moz-transform 0.6s ease-in-out;\n -o-transition: -o-transform 0.6s ease-in-out;\n transition: transform 0.6s ease-in-out;\n -webkit-backface-visibility: hidden;\n -moz-backface-visibility: hidden;\n backface-visibility: hidden;\n -webkit-perspective: 1000;\n -moz-perspective: 1000;\n perspective: 1000;\n }\n .carousel-inner > .item.next,\n .carousel-inner > .item.active.right {\n -webkit-transform: translate3d(100%, 0, 0);\n transform: translate3d(100%, 0, 0);\n left: 0;\n }\n .carousel-inner > .item.prev,\n .carousel-inner > .item.active.left {\n -webkit-transform: translate3d(-100%, 0, 0);\n transform: translate3d(-100%, 0, 0);\n left: 0;\n }\n .carousel-inner > .item.next.left,\n .carousel-inner > .item.prev.right,\n .carousel-inner > .item.active {\n -webkit-transform: translate3d(0, 0, 0);\n transform: translate3d(0, 0, 0);\n left: 0;\n }\n}\n.carousel-inner > .active,\n.carousel-inner > .next,\n.carousel-inner > .prev {\n display: block;\n}\n.carousel-inner > .active {\n left: 0;\n}\n.carousel-inner > .next,\n.carousel-inner > .prev {\n position: absolute;\n top: 0;\n width: 100%;\n}\n.carousel-inner > .next {\n left: 100%;\n}\n.carousel-inner > .prev {\n left: -100%;\n}\n.carousel-inner > .next.left,\n.carousel-inner > .prev.right {\n left: 0;\n}\n.carousel-inner > .active.left {\n left: -100%;\n}\n.carousel-inner > .active.right {\n left: 100%;\n}\n.carousel-control {\n position: absolute;\n top: 0;\n left: 0;\n bottom: 0;\n width: 15%;\n opacity: 0.5;\n filter: alpha(opacity=50);\n font-size: 20px;\n color: #ffffff;\n text-align: center;\n text-shadow: 0 1px 2px rgba(0, 0, 0, 0.6);\n}\n.carousel-control.left {\n background-image: -webkit-linear-gradient(left, rgba(0, 0, 0, 0.5) 0%, rgba(0, 0, 0, 0.0001) 100%);\n background-image: -o-linear-gradient(left, rgba(0, 0, 0, 0.5) 0%, rgba(0, 0, 0, 0.0001) 100%);\n background-image: linear-gradient(to right, rgba(0, 0, 0, 0.5) 0%, rgba(0, 0, 0, 0.0001) 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#80000000', endColorstr='#00000000', GradientType=1);\n}\n.carousel-control.right {\n left: auto;\n right: 0;\n background-image: -webkit-linear-gradient(left, rgba(0, 0, 0, 0.0001) 0%, rgba(0, 0, 0, 0.5) 100%);\n background-image: -o-linear-gradient(left, rgba(0, 0, 0, 0.0001) 0%, rgba(0, 0, 0, 0.5) 100%);\n background-image: linear-gradient(to right, rgba(0, 0, 0, 0.0001) 0%, rgba(0, 0, 0, 0.5) 100%);\n background-repeat: repeat-x;\n filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#00000000', endColorstr='#80000000', GradientType=1);\n}\n.carousel-control:hover,\n.carousel-control:focus {\n outline: 0;\n color: #ffffff;\n text-decoration: none;\n opacity: 0.9;\n filter: alpha(opacity=90);\n}\n.carousel-control .icon-prev,\n.carousel-control .icon-next,\n.carousel-control .glyphicon-chevron-left,\n.carousel-control .glyphicon-chevron-right {\n position: absolute;\n top: 50%;\n z-index: 5;\n display: inline-block;\n}\n.carousel-control .icon-prev,\n.carousel-control .glyphicon-chevron-left {\n left: 50%;\n margin-left: -10px;\n}\n.carousel-control .icon-next,\n.carousel-control .glyphicon-chevron-right {\n right: 50%;\n margin-right: -10px;\n}\n.carousel-control .icon-prev,\n.carousel-control .icon-next {\n width: 20px;\n height: 20px;\n margin-top: -10px;\n line-height: 1;\n font-family: serif;\n}\n.carousel-control .icon-prev:before {\n content: '\\2039';\n}\n.carousel-control .icon-next:before {\n content: '\\203a';\n}\n.carousel-indicators {\n position: absolute;\n bottom: 10px;\n left: 50%;\n z-index: 15;\n width: 60%;\n margin-left: -30%;\n padding-left: 0;\n list-style: none;\n text-align: center;\n}\n.carousel-indicators li {\n display: inline-block;\n width: 10px;\n height: 10px;\n margin: 1px;\n text-indent: -999px;\n border: 1px solid #ffffff;\n border-radius: 10px;\n cursor: pointer;\n background-color: #000 \\9;\n background-color: rgba(0, 0, 0, 0);\n}\n.carousel-indicators .active {\n margin: 0;\n width: 12px;\n height: 12px;\n background-color: #ffffff;\n}\n.carousel-caption {\n position: absolute;\n left: 15%;\n right: 15%;\n bottom: 20px;\n z-index: 10;\n padding-top: 20px;\n padding-bottom: 20px;\n color: #ffffff;\n text-align: center;\n text-shadow: 0 1px 2px rgba(0, 0, 0, 0.6);\n}\n.carousel-caption .btn {\n text-shadow: none;\n}\n@media screen and (min-width: 768px) {\n .carousel-control .glyphicon-chevron-left,\n .carousel-control .glyphicon-chevron-right,\n .carousel-control .icon-prev,\n .carousel-control .icon-next {\n width: 30px;\n height: 30px;\n margin-top: -15px;\n font-size: 30px;\n }\n .carousel-control .glyphicon-chevron-left,\n .carousel-control .icon-prev {\n margin-left: -15px;\n }\n .carousel-control .glyphicon-chevron-right,\n .carousel-control .icon-next {\n margin-right: -15px;\n }\n .carousel-caption {\n left: 20%;\n right: 20%;\n padding-bottom: 30px;\n }\n .carousel-indicators {\n bottom: 20px;\n }\n}\n.clearfix:before,\n.clearfix:after,\n.dl-horizontal dd:before,\n.dl-horizontal dd:after,\n.container:before,\n.container:after,\n.container-fluid:before,\n.container-fluid:after,\n.row:before,\n.row:after,\n.form-horizontal .form-group:before,\n.form-horizontal .form-group:after,\n.btn-toolbar:before,\n.btn-toolbar:after,\n.btn-group-vertical > .btn-group:before,\n.btn-group-vertical > .btn-group:after,\n.nav:before,\n.nav:after,\n.navbar:before,\n.navbar:after,\n.navbar-header:before,\n.navbar-header:after,\n.navbar-collapse:before,\n.navbar-collapse:after,\n.pager:before,\n.pager:after,\n.panel-body:before,\n.panel-body:after,\n.modal-footer:before,\n.modal-footer:after {\n content: \" \";\n display: table;\n}\n.clearfix:after,\n.dl-horizontal dd:after,\n.container:after,\n.container-fluid:after,\n.row:after,\n.form-horizontal .form-group:after,\n.btn-toolbar:after,\n.btn-group-vertical > .btn-group:after,\n.nav:after,\n.navbar:after,\n.navbar-header:after,\n.navbar-collapse:after,\n.pager:after,\n.panel-body:after,\n.modal-footer:after {\n clear: both;\n}\n.center-block {\n display: block;\n margin-left: auto;\n margin-right: auto;\n}\n.pull-right {\n float: right !important;\n}\n.pull-left {\n float: left !important;\n}\n.hide {\n display: none !important;\n}\n.show {\n display: block !important;\n}\n.invisible {\n visibility: hidden;\n}\n.text-hide {\n font: 0/0 a;\n color: transparent;\n text-shadow: none;\n background-color: transparent;\n border: 0;\n}\n.hidden {\n display: none !important;\n}\n.affix {\n position: fixed;\n}\n@-ms-viewport {\n width: device-width;\n}\n.visible-xs,\n.visible-sm,\n.visible-md,\n.visible-lg {\n display: none !important;\n}\n.visible-xs-block,\n.visible-xs-inline,\n.visible-xs-inline-block,\n.visible-sm-block,\n.visible-sm-inline,\n.visible-sm-inline-block,\n.visible-md-block,\n.visible-md-inline,\n.visible-md-inline-block,\n.visible-lg-block,\n.visible-lg-inline,\n.visible-lg-inline-block {\n display: none !important;\n}\n@media (max-width: 767px) {\n .visible-xs {\n display: block !important;\n }\n table.visible-xs {\n display: table;\n }\n tr.visible-xs {\n display: table-row !important;\n }\n th.visible-xs,\n td.visible-xs {\n display: table-cell !important;\n }\n}\n@media (max-width: 767px) {\n .visible-xs-block {\n display: block !important;\n }\n}\n@media (max-width: 767px) {\n .visible-xs-inline {\n display: inline !important;\n }\n}\n@media (max-width: 767px) {\n .visible-xs-inline-block {\n display: inline-block !important;\n }\n}\n@media (min-width: 768px) and (max-width: 991px) {\n .visible-sm {\n display: block !important;\n }\n table.visible-sm {\n display: table;\n }\n tr.visible-sm {\n display: table-row !important;\n }\n th.visible-sm,\n td.visible-sm {\n display: table-cell !important;\n }\n}\n@media (min-width: 768px) and (max-width: 991px) {\n .visible-sm-block {\n display: block !important;\n }\n}\n@media (min-width: 768px) and (max-width: 991px) {\n .visible-sm-inline {\n display: inline !important;\n }\n}\n@media (min-width: 768px) and (max-width: 991px) {\n .visible-sm-inline-block {\n display: inline-block !important;\n }\n}\n@media (min-width: 992px) and (max-width: 1199px) {\n .visible-md {\n display: block !important;\n }\n table.visible-md {\n display: table;\n }\n tr.visible-md {\n display: table-row !important;\n }\n th.visible-md,\n td.visible-md {\n display: table-cell !important;\n }\n}\n@media (min-width: 992px) and (max-width: 1199px) {\n .visible-md-block {\n display: block !important;\n }\n}\n@media (min-width: 992px) and (max-width: 1199px) {\n .visible-md-inline {\n display: inline !important;\n }\n}\n@media (min-width: 992px) and (max-width: 1199px) {\n .visible-md-inline-block {\n display: inline-block !important;\n }\n}\n@media (min-width: 1200px) {\n .visible-lg {\n display: block !important;\n }\n table.visible-lg {\n display: table;\n }\n tr.visible-lg {\n display: table-row !important;\n }\n th.visible-lg,\n td.visible-lg {\n display: table-cell !important;\n }\n}\n@media (min-width: 1200px) {\n .visible-lg-block {\n display: block !important;\n }\n}\n@media (min-width: 1200px) {\n .visible-lg-inline {\n display: inline !important;\n }\n}\n@media (min-width: 1200px) {\n .visible-lg-inline-block {\n display: inline-block !important;\n }\n}\n@media (max-width: 767px) {\n .hidden-xs {\n display: none !important;\n }\n}\n@media (min-width: 768px) and (max-width: 991px) {\n .hidden-sm {\n display: none !important;\n }\n}\n@media (min-width: 992px) and (max-width: 1199px) {\n .hidden-md {\n display: none !important;\n }\n}\n@media (min-width: 1200px) {\n .hidden-lg {\n display: none !important;\n }\n}\n.visible-print {\n display: none !important;\n}\n@media print {\n .visible-print {\n display: block !important;\n }\n table.visible-print {\n display: table;\n }\n tr.visible-print {\n display: table-row !important;\n }\n th.visible-print,\n td.visible-print {\n display: table-cell !important;\n }\n}\n.visible-print-block {\n display: none !important;\n}\n@media print {\n .visible-print-block {\n display: block !important;\n }\n}\n.visible-print-inline {\n display: none !important;\n}\n@media print {\n .visible-print-inline {\n display: inline !important;\n }\n}\n.visible-print-inline-block {\n display: none !important;\n}\n@media print {\n .visible-print-inline-block {\n display: inline-block !important;\n }\n}\n@media print {\n .hidden-print {\n display: none !important;\n }\n}\n/*# sourceMappingURL=bootstrap.css.map */","/*! normalize.css v3.0.2 | MIT License | git.io/normalize */\n\n//\n// 1. Set default font family to sans-serif.\n// 2. Prevent iOS text size adjust after orientation change, without disabling\n// user zoom.\n//\n\nhtml {\n font-family: sans-serif; // 1\n -ms-text-size-adjust: 100%; // 2\n -webkit-text-size-adjust: 100%; // 2\n}\n\n//\n// Remove default margin.\n//\n\nbody {\n margin: 0;\n}\n\n// HTML5 display definitions\n// ==========================================================================\n\n//\n// Correct `block` display not defined for any HTML5 element in IE 8/9.\n// Correct `block` display not defined for `details` or `summary` in IE 10/11\n// and Firefox.\n// Correct `block` display not defined for `main` in IE 11.\n//\n\narticle,\naside,\ndetails,\nfigcaption,\nfigure,\nfooter,\nheader,\nhgroup,\nmain,\nmenu,\nnav,\nsection,\nsummary {\n display: block;\n}\n\n//\n// 1. Correct `inline-block` display not defined in IE 8/9.\n// 2. Normalize vertical alignment of `progress` in Chrome, Firefox, and Opera.\n//\n\naudio,\ncanvas,\nprogress,\nvideo {\n display: inline-block; // 1\n vertical-align: baseline; // 2\n}\n\n//\n// Prevent modern browsers from displaying `audio` without controls.\n// Remove excess height in iOS 5 devices.\n//\n\naudio:not([controls]) {\n display: none;\n height: 0;\n}\n\n//\n// Address `[hidden]` styling not present in IE 8/9/10.\n// Hide the `template` element in IE 8/9/11, Safari, and Firefox < 22.\n//\n\n[hidden],\ntemplate {\n display: none;\n}\n\n// Links\n// ==========================================================================\n\n//\n// Remove the gray background color from active links in IE 10.\n//\n\na {\n background-color: transparent;\n}\n\n//\n// Improve readability when focused and also mouse hovered in all browsers.\n//\n\na:active,\na:hover {\n outline: 0;\n}\n\n// Text-level semantics\n// ==========================================================================\n\n//\n// Address styling not present in IE 8/9/10/11, Safari, and Chrome.\n//\n\nabbr[title] {\n border-bottom: 1px dotted;\n}\n\n//\n// Address style set to `bolder` in Firefox 4+, Safari, and Chrome.\n//\n\nb,\nstrong {\n font-weight: bold;\n}\n\n//\n// Address styling not present in Safari and Chrome.\n//\n\ndfn {\n font-style: italic;\n}\n\n//\n// Address variable `h1` font-size and margin within `section` and `article`\n// contexts in Firefox 4+, Safari, and Chrome.\n//\n\nh1 {\n font-size: 2em;\n margin: 0.67em 0;\n}\n\n//\n// Address styling not present in IE 8/9.\n//\n\nmark {\n background: #ff0;\n color: #000;\n}\n\n//\n// Address inconsistent and variable font size in all browsers.\n//\n\nsmall {\n font-size: 80%;\n}\n\n//\n// Prevent `sub` and `sup` affecting `line-height` in all browsers.\n//\n\nsub,\nsup {\n font-size: 75%;\n line-height: 0;\n position: relative;\n vertical-align: baseline;\n}\n\nsup {\n top: -0.5em;\n}\n\nsub {\n bottom: -0.25em;\n}\n\n// Embedded content\n// ==========================================================================\n\n//\n// Remove border when inside `a` element in IE 8/9/10.\n//\n\nimg {\n border: 0;\n}\n\n//\n// Correct overflow not hidden in IE 9/10/11.\n//\n\nsvg:not(:root) {\n overflow: hidden;\n}\n\n// Grouping content\n// ==========================================================================\n\n//\n// Address margin not present in IE 8/9 and Safari.\n//\n\nfigure {\n margin: 1em 40px;\n}\n\n//\n// Address differences between Firefox and other browsers.\n//\n\nhr {\n -moz-box-sizing: content-box;\n box-sizing: content-box;\n height: 0;\n}\n\n//\n// Contain overflow in all browsers.\n//\n\npre {\n overflow: auto;\n}\n\n//\n// Address odd `em`-unit font size rendering in all browsers.\n//\n\ncode,\nkbd,\npre,\nsamp {\n font-family: monospace, monospace;\n font-size: 1em;\n}\n\n// Forms\n// ==========================================================================\n\n//\n// Known limitation: by default, Chrome and Safari on OS X allow very limited\n// styling of `select`, unless a `border` property is set.\n//\n\n//\n// 1. Correct color not being inherited.\n// Known issue: affects color of disabled elements.\n// 2. Correct font properties not being inherited.\n// 3. Address margins set differently in Firefox 4+, Safari, and Chrome.\n//\n\nbutton,\ninput,\noptgroup,\nselect,\ntextarea {\n color: inherit; // 1\n font: inherit; // 2\n margin: 0; // 3\n}\n\n//\n// Address `overflow` set to `hidden` in IE 8/9/10/11.\n//\n\nbutton {\n overflow: visible;\n}\n\n//\n// Address inconsistent `text-transform` inheritance for `button` and `select`.\n// All other form control elements do not inherit `text-transform` values.\n// Correct `button` style inheritance in Firefox, IE 8/9/10/11, and Opera.\n// Correct `select` style inheritance in Firefox.\n//\n\nbutton,\nselect {\n text-transform: none;\n}\n\n//\n// 1. Avoid the WebKit bug in Android 4.0.* where (2) destroys native `audio`\n// and `video` controls.\n// 2. Correct inability to style clickable `input` types in iOS.\n// 3. Improve usability and consistency of cursor style between image-type\n// `input` and others.\n//\n\nbutton,\nhtml input[type=\"button\"], // 1\ninput[type=\"reset\"],\ninput[type=\"submit\"] {\n -webkit-appearance: button; // 2\n cursor: pointer; // 3\n}\n\n//\n// Re-set default cursor for disabled elements.\n//\n\nbutton[disabled],\nhtml input[disabled] {\n cursor: default;\n}\n\n//\n// Remove inner padding and border in Firefox 4+.\n//\n\nbutton::-moz-focus-inner,\ninput::-moz-focus-inner {\n border: 0;\n padding: 0;\n}\n\n//\n// Address Firefox 4+ setting `line-height` on `input` using `!important` in\n// the UA stylesheet.\n//\n\ninput {\n line-height: normal;\n}\n\n//\n// It's recommended that you don't attempt to style these elements.\n// Firefox's implementation doesn't respect box-sizing, padding, or width.\n//\n// 1. Address box sizing set to `content-box` in IE 8/9/10.\n// 2. Remove excess padding in IE 8/9/10.\n//\n\ninput[type=\"checkbox\"],\ninput[type=\"radio\"] {\n box-sizing: border-box; // 1\n padding: 0; // 2\n}\n\n//\n// Fix the cursor style for Chrome's increment/decrement buttons. For certain\n// `font-size` values of the `input`, it causes the cursor style of the\n// decrement button to change from `default` to `text`.\n//\n\ninput[type=\"number\"]::-webkit-inner-spin-button,\ninput[type=\"number\"]::-webkit-outer-spin-button {\n height: auto;\n}\n\n//\n// 1. Address `appearance` set to `searchfield` in Safari and Chrome.\n// 2. Address `box-sizing` set to `border-box` in Safari and Chrome\n// (include `-moz` to future-proof).\n//\n\ninput[type=\"search\"] {\n -webkit-appearance: textfield; // 1\n -moz-box-sizing: content-box;\n -webkit-box-sizing: content-box; // 2\n box-sizing: content-box;\n}\n\n//\n// Remove inner padding and search cancel button in Safari and Chrome on OS X.\n// Safari (but not Chrome) clips the cancel button when the search input has\n// padding (and `textfield` appearance).\n//\n\ninput[type=\"search\"]::-webkit-search-cancel-button,\ninput[type=\"search\"]::-webkit-search-decoration {\n -webkit-appearance: none;\n}\n\n//\n// Define consistent border, margin, and padding.\n//\n\nfieldset {\n border: 1px solid #c0c0c0;\n margin: 0 2px;\n padding: 0.35em 0.625em 0.75em;\n}\n\n//\n// 1. Correct `color` not being inherited in IE 8/9/10/11.\n// 2. Remove padding so people aren't caught out if they zero out fieldsets.\n//\n\nlegend {\n border: 0; // 1\n padding: 0; // 2\n}\n\n//\n// Remove default vertical scrollbar in IE 8/9/10/11.\n//\n\ntextarea {\n overflow: auto;\n}\n\n//\n// Don't inherit the `font-weight` (applied by a rule above).\n// NOTE: the default cannot safely be changed in Chrome and Safari on OS X.\n//\n\noptgroup {\n font-weight: bold;\n}\n\n// Tables\n// ==========================================================================\n\n//\n// Remove most spacing between table cells.\n//\n\ntable {\n border-collapse: collapse;\n border-spacing: 0;\n}\n\ntd,\nth {\n padding: 0;\n}\n","/*! Source: https://github.com/h5bp/html5-boilerplate/blob/master/src/css/main.css */\n\n// ==========================================================================\n// Print styles.\n// Inlined to avoid the additional HTTP request: h5bp.com/r\n// ==========================================================================\n\n@media print {\n *,\n *:before,\n *:after {\n background: transparent !important;\n color: #000 !important; // Black prints faster: h5bp.com/s\n box-shadow: none !important;\n text-shadow: none !important;\n }\n\n a,\n a:visited {\n text-decoration: underline;\n }\n\n a[href]:after {\n content: \" (\" attr(href) \")\";\n }\n\n abbr[title]:after {\n content: \" (\" attr(title) \")\";\n }\n\n // Don't show links that are fragment identifiers,\n // or use the `javascript:` pseudo protocol\n a[href^=\"#\"]:after,\n a[href^=\"javascript:\"]:after {\n content: \"\";\n }\n\n pre,\n blockquote {\n border: 1px solid #999;\n page-break-inside: avoid;\n }\n\n thead {\n display: table-header-group; // h5bp.com/t\n }\n\n tr,\n img {\n page-break-inside: avoid;\n }\n\n img {\n max-width: 100% !important;\n }\n\n p,\n h2,\n h3 {\n orphans: 3;\n widows: 3;\n }\n\n h2,\n h3 {\n page-break-after: avoid;\n }\n\n // Bootstrap specific changes start\n //\n // Chrome (OSX) fix for https://github.com/twbs/bootstrap/issues/11245\n // Once fixed, we can just straight up remove this.\n select {\n background: #fff !important;\n }\n\n // Bootstrap components\n .navbar {\n display: none;\n }\n .btn,\n .dropup > .btn {\n > .caret {\n border-top-color: #000 !important;\n }\n }\n .label {\n border: 1px solid #000;\n }\n\n .table {\n border-collapse: collapse !important;\n\n td,\n th {\n background-color: #fff !important;\n }\n }\n .table-bordered {\n th,\n td {\n border: 1px solid #ddd !important;\n }\n }\n\n // Bootstrap specific changes end\n}\n","//\n// Glyphicons for Bootstrap\n//\n// Since icons are fonts, they can be placed anywhere text is placed and are\n// thus automatically sized to match the surrounding child. To use, create an\n// inline element with the appropriate classes, like so:\n//\n// Star\n\n// Import the fonts\n@font-face {\n font-family: 'Glyphicons Halflings';\n src: url('@{icon-font-path}@{icon-font-name}.eot');\n src: url('@{icon-font-path}@{icon-font-name}.eot?#iefix') format('embedded-opentype'),\n url('@{icon-font-path}@{icon-font-name}.woff2') format('woff2'),\n url('@{icon-font-path}@{icon-font-name}.woff') format('woff'),\n url('@{icon-font-path}@{icon-font-name}.ttf') format('truetype'),\n url('@{icon-font-path}@{icon-font-name}.svg#@{icon-font-svg-id}') format('svg');\n}\n\n// Catchall baseclass\n.glyphicon {\n position: relative;\n top: 1px;\n display: inline-block;\n font-family: 'Glyphicons Halflings';\n font-style: normal;\n font-weight: normal;\n line-height: 1;\n -webkit-font-smoothing: antialiased;\n -moz-osx-font-smoothing: grayscale;\n}\n\n// Individual icons\n.glyphicon-asterisk { &:before { content: \"\\2a\"; } }\n.glyphicon-plus { &:before { content: \"\\2b\"; } }\n.glyphicon-euro,\n.glyphicon-eur { &:before { content: \"\\20ac\"; } }\n.glyphicon-minus { &:before { content: \"\\2212\"; } }\n.glyphicon-cloud { &:before { content: \"\\2601\"; } }\n.glyphicon-envelope { &:before { content: \"\\2709\"; } }\n.glyphicon-pencil { &:before { content: \"\\270f\"; } }\n.glyphicon-glass { &:before { content: \"\\e001\"; } }\n.glyphicon-music { &:before { content: \"\\e002\"; } }\n.glyphicon-search { &:before { content: \"\\e003\"; } }\n.glyphicon-heart { &:before { content: \"\\e005\"; } }\n.glyphicon-star { &:before { content: \"\\e006\"; } }\n.glyphicon-star-empty { &:before { content: \"\\e007\"; } }\n.glyphicon-user { &:before { content: \"\\e008\"; } }\n.glyphicon-film { &:before { content: \"\\e009\"; } }\n.glyphicon-th-large { &:before { content: \"\\e010\"; } }\n.glyphicon-th { &:before { content: \"\\e011\"; } }\n.glyphicon-th-list { &:before { content: \"\\e012\"; } }\n.glyphicon-ok { &:before { content: \"\\e013\"; } }\n.glyphicon-remove { &:before { content: \"\\e014\"; } }\n.glyphicon-zoom-in { &:before { content: \"\\e015\"; } }\n.glyphicon-zoom-out { &:before { content: \"\\e016\"; } }\n.glyphicon-off { &:before { content: \"\\e017\"; } }\n.glyphicon-signal { &:before { content: \"\\e018\"; } }\n.glyphicon-cog { &:before { content: \"\\e019\"; } }\n.glyphicon-trash { &:before { content: \"\\e020\"; } }\n.glyphicon-home { &:before { content: \"\\e021\"; } }\n.glyphicon-file { &:before { content: \"\\e022\"; } }\n.glyphicon-time { &:before { content: \"\\e023\"; } }\n.glyphicon-road { &:before { content: \"\\e024\"; } }\n.glyphicon-download-alt { &:before { content: \"\\e025\"; } }\n.glyphicon-download { &:before { content: \"\\e026\"; } }\n.glyphicon-upload { &:before { content: \"\\e027\"; } }\n.glyphicon-inbox { &:before { content: \"\\e028\"; } }\n.glyphicon-play-circle { &:before { content: \"\\e029\"; } }\n.glyphicon-repeat { &:before { content: \"\\e030\"; } }\n.glyphicon-refresh { &:before { content: \"\\e031\"; } }\n.glyphicon-list-alt { &:before { content: \"\\e032\"; } }\n.glyphicon-lock { &:before { content: \"\\e033\"; } }\n.glyphicon-flag { &:before { content: \"\\e034\"; } }\n.glyphicon-headphones { &:before { content: \"\\e035\"; } }\n.glyphicon-volume-off { &:before { content: \"\\e036\"; } }\n.glyphicon-volume-down { &:before { content: \"\\e037\"; } }\n.glyphicon-volume-up { &:before { content: \"\\e038\"; } }\n.glyphicon-qrcode { &:before { content: \"\\e039\"; } }\n.glyphicon-barcode { &:before { content: \"\\e040\"; } }\n.glyphicon-tag { &:before { content: \"\\e041\"; } }\n.glyphicon-tags { &:before { content: \"\\e042\"; } }\n.glyphicon-book { &:before { content: \"\\e043\"; } }\n.glyphicon-bookmark { &:before { content: \"\\e044\"; } }\n.glyphicon-print { &:before { content: \"\\e045\"; } }\n.glyphicon-camera { &:before { content: \"\\e046\"; } }\n.glyphicon-font { &:before { content: \"\\e047\"; } }\n.glyphicon-bold { &:before { content: \"\\e048\"; } }\n.glyphicon-italic { &:before { content: \"\\e049\"; } }\n.glyphicon-text-height { &:before { content: \"\\e050\"; } }\n.glyphicon-text-width { &:before { content: \"\\e051\"; } }\n.glyphicon-align-left { &:before { content: \"\\e052\"; } }\n.glyphicon-align-center { &:before { content: \"\\e053\"; } }\n.glyphicon-align-right { &:before { content: \"\\e054\"; } }\n.glyphicon-align-justify { &:before { content: \"\\e055\"; } }\n.glyphicon-list { &:before { content: \"\\e056\"; } }\n.glyphicon-indent-left { &:before { content: \"\\e057\"; } }\n.glyphicon-indent-right { &:before { content: \"\\e058\"; } }\n.glyphicon-facetime-video { &:before { content: \"\\e059\"; } }\n.glyphicon-picture { &:before { content: \"\\e060\"; } }\n.glyphicon-map-marker { &:before { content: \"\\e062\"; } }\n.glyphicon-adjust { &:before { content: \"\\e063\"; } }\n.glyphicon-tint { &:before { content: \"\\e064\"; } }\n.glyphicon-edit { &:before { content: \"\\e065\"; } }\n.glyphicon-share { &:before { content: \"\\e066\"; } }\n.glyphicon-check { &:before { content: \"\\e067\"; } }\n.glyphicon-move { &:before { content: \"\\e068\"; } }\n.glyphicon-step-backward { &:before { content: \"\\e069\"; } }\n.glyphicon-fast-backward { &:before { content: \"\\e070\"; } }\n.glyphicon-backward { &:before { content: \"\\e071\"; } }\n.glyphicon-play { &:before { content: \"\\e072\"; } }\n.glyphicon-pause { &:before { content: \"\\e073\"; } }\n.glyphicon-stop { &:before { content: \"\\e074\"; } }\n.glyphicon-forward { &:before { content: \"\\e075\"; } }\n.glyphicon-fast-forward { &:before { content: \"\\e076\"; } }\n.glyphicon-step-forward { &:before { content: \"\\e077\"; } }\n.glyphicon-eject { &:before { content: \"\\e078\"; } }\n.glyphicon-chevron-left { &:before { content: \"\\e079\"; } }\n.glyphicon-chevron-right { &:before { content: \"\\e080\"; } }\n.glyphicon-plus-sign { &:before { content: \"\\e081\"; } }\n.glyphicon-minus-sign { &:before { content: \"\\e082\"; } }\n.glyphicon-remove-sign { &:before { content: \"\\e083\"; } }\n.glyphicon-ok-sign { &:before { content: \"\\e084\"; } }\n.glyphicon-question-sign { &:before { content: \"\\e085\"; } }\n.glyphicon-info-sign { &:before { content: \"\\e086\"; } }\n.glyphicon-screenshot { &:before { content: \"\\e087\"; } }\n.glyphicon-remove-circle { &:before { content: \"\\e088\"; } }\n.glyphicon-ok-circle { &:before { content: \"\\e089\"; } }\n.glyphicon-ban-circle { &:before { content: \"\\e090\"; } }\n.glyphicon-arrow-left { &:before { content: \"\\e091\"; } }\n.glyphicon-arrow-right { &:before { content: \"\\e092\"; } }\n.glyphicon-arrow-up { &:before { content: \"\\e093\"; } }\n.glyphicon-arrow-down { &:before { content: \"\\e094\"; } }\n.glyphicon-share-alt { &:before { content: \"\\e095\"; } }\n.glyphicon-resize-full { &:before { content: \"\\e096\"; } }\n.glyphicon-resize-small { &:before { content: \"\\e097\"; } }\n.glyphicon-exclamation-sign { &:before { content: \"\\e101\"; } }\n.glyphicon-gift { &:before { content: \"\\e102\"; } }\n.glyphicon-leaf { &:before { content: \"\\e103\"; } }\n.glyphicon-fire { &:before { content: \"\\e104\"; } }\n.glyphicon-eye-open { &:before { content: \"\\e105\"; } }\n.glyphicon-eye-close { &:before { content: \"\\e106\"; } }\n.glyphicon-warning-sign { &:before { content: \"\\e107\"; } }\n.glyphicon-plane { &:before { content: \"\\e108\"; } }\n.glyphicon-calendar { &:before { content: \"\\e109\"; } }\n.glyphicon-random { &:before { content: \"\\e110\"; } }\n.glyphicon-comment { &:before { content: \"\\e111\"; } }\n.glyphicon-magnet { &:before { content: \"\\e112\"; } }\n.glyphicon-chevron-up { &:before { content: \"\\e113\"; } }\n.glyphicon-chevron-down { &:before { content: \"\\e114\"; } }\n.glyphicon-retweet { &:before { content: \"\\e115\"; } }\n.glyphicon-shopping-cart { &:before { content: \"\\e116\"; } }\n.glyphicon-folder-close { &:before { content: \"\\e117\"; } }\n.glyphicon-folder-open { &:before { content: \"\\e118\"; } }\n.glyphicon-resize-vertical { &:before { content: \"\\e119\"; } }\n.glyphicon-resize-horizontal { &:before { content: \"\\e120\"; } }\n.glyphicon-hdd { &:before { content: \"\\e121\"; } }\n.glyphicon-bullhorn { &:before { content: \"\\e122\"; } }\n.glyphicon-bell { &:before { content: \"\\e123\"; } }\n.glyphicon-certificate { &:before { content: \"\\e124\"; } }\n.glyphicon-thumbs-up { &:before { content: \"\\e125\"; } }\n.glyphicon-thumbs-down { &:before { content: \"\\e126\"; } }\n.glyphicon-hand-right { &:before { content: \"\\e127\"; } }\n.glyphicon-hand-left { &:before { content: \"\\e128\"; } }\n.glyphicon-hand-up { &:before { content: \"\\e129\"; } }\n.glyphicon-hand-down { &:before { content: \"\\e130\"; } }\n.glyphicon-circle-arrow-right { &:before { content: \"\\e131\"; } }\n.glyphicon-circle-arrow-left { &:before { content: \"\\e132\"; } }\n.glyphicon-circle-arrow-up { &:before { content: \"\\e133\"; } }\n.glyphicon-circle-arrow-down { &:before { content: \"\\e134\"; } }\n.glyphicon-globe { &:before { content: \"\\e135\"; } }\n.glyphicon-wrench { &:before { content: \"\\e136\"; } }\n.glyphicon-tasks { &:before { content: \"\\e137\"; } }\n.glyphicon-filter { &:before { content: \"\\e138\"; } }\n.glyphicon-briefcase { &:before { content: \"\\e139\"; } }\n.glyphicon-fullscreen { &:before { content: \"\\e140\"; } }\n.glyphicon-dashboard { &:before { content: \"\\e141\"; } }\n.glyphicon-paperclip { &:before { content: \"\\e142\"; } }\n.glyphicon-heart-empty { &:before { content: \"\\e143\"; } }\n.glyphicon-link { &:before { content: \"\\e144\"; } }\n.glyphicon-phone { &:before { content: \"\\e145\"; } }\n.glyphicon-pushpin { &:before { content: \"\\e146\"; } }\n.glyphicon-usd { &:before { content: \"\\e148\"; } }\n.glyphicon-gbp { &:before { content: \"\\e149\"; } }\n.glyphicon-sort { &:before { content: \"\\e150\"; } }\n.glyphicon-sort-by-alphabet { &:before { content: \"\\e151\"; } }\n.glyphicon-sort-by-alphabet-alt { &:before { content: \"\\e152\"; } }\n.glyphicon-sort-by-order { &:before { content: \"\\e153\"; } }\n.glyphicon-sort-by-order-alt { &:before { content: \"\\e154\"; } }\n.glyphicon-sort-by-attributes { &:before { content: \"\\e155\"; } }\n.glyphicon-sort-by-attributes-alt { &:before { content: \"\\e156\"; } }\n.glyphicon-unchecked { &:before { content: \"\\e157\"; } }\n.glyphicon-expand { &:before { content: \"\\e158\"; } }\n.glyphicon-collapse-down { &:before { content: \"\\e159\"; } }\n.glyphicon-collapse-up { &:before { content: \"\\e160\"; } }\n.glyphicon-log-in { &:before { content: \"\\e161\"; } }\n.glyphicon-flash { &:before { content: \"\\e162\"; } }\n.glyphicon-log-out { &:before { content: \"\\e163\"; } }\n.glyphicon-new-window { &:before { content: \"\\e164\"; } }\n.glyphicon-record { &:before { content: \"\\e165\"; } }\n.glyphicon-save { &:before { content: \"\\e166\"; } }\n.glyphicon-open { &:before { content: \"\\e167\"; } }\n.glyphicon-saved { &:before { content: \"\\e168\"; } }\n.glyphicon-import { &:before { content: \"\\e169\"; } }\n.glyphicon-export { &:before { content: \"\\e170\"; } }\n.glyphicon-send { &:before { content: \"\\e171\"; } }\n.glyphicon-floppy-disk { &:before { content: \"\\e172\"; } }\n.glyphicon-floppy-saved { &:before { content: \"\\e173\"; } }\n.glyphicon-floppy-remove { &:before { content: \"\\e174\"; } }\n.glyphicon-floppy-save { &:before { content: \"\\e175\"; } }\n.glyphicon-floppy-open { &:before { content: \"\\e176\"; } }\n.glyphicon-credit-card { &:before { content: \"\\e177\"; } }\n.glyphicon-transfer { &:before { content: \"\\e178\"; } }\n.glyphicon-cutlery { &:before { content: \"\\e179\"; } }\n.glyphicon-header { &:before { content: \"\\e180\"; } }\n.glyphicon-compressed { &:before { content: \"\\e181\"; } }\n.glyphicon-earphone { &:before { content: \"\\e182\"; } }\n.glyphicon-phone-alt { &:before { content: \"\\e183\"; } }\n.glyphicon-tower { &:before { content: \"\\e184\"; } }\n.glyphicon-stats { &:before { content: \"\\e185\"; } }\n.glyphicon-sd-video { &:before { content: \"\\e186\"; } }\n.glyphicon-hd-video { &:before { content: \"\\e187\"; } }\n.glyphicon-subtitles { &:before { content: \"\\e188\"; } }\n.glyphicon-sound-stereo { &:before { content: \"\\e189\"; } }\n.glyphicon-sound-dolby { &:before { content: \"\\e190\"; } }\n.glyphicon-sound-5-1 { &:before { content: \"\\e191\"; } }\n.glyphicon-sound-6-1 { &:before { content: \"\\e192\"; } }\n.glyphicon-sound-7-1 { &:before { content: \"\\e193\"; } }\n.glyphicon-copyright-mark { &:before { content: \"\\e194\"; } }\n.glyphicon-registration-mark { &:before { content: \"\\e195\"; } }\n.glyphicon-cloud-download { &:before { content: \"\\e197\"; } }\n.glyphicon-cloud-upload { &:before { content: \"\\e198\"; } }\n.glyphicon-tree-conifer { &:before { content: \"\\e199\"; } }\n.glyphicon-tree-deciduous { &:before { content: \"\\e200\"; } }\n.glyphicon-cd { &:before { content: \"\\e201\"; } }\n.glyphicon-save-file { &:before { content: \"\\e202\"; } }\n.glyphicon-open-file { &:before { content: \"\\e203\"; } }\n.glyphicon-level-up { &:before { content: \"\\e204\"; } }\n.glyphicon-copy { &:before { content: \"\\e205\"; } }\n.glyphicon-paste { &:before { content: \"\\e206\"; } }\n// The following 2 Glyphicons are omitted for the time being because\n// they currently use Unicode codepoints that are outside the\n// Basic Multilingual Plane (BMP). Older buggy versions of WebKit can't handle\n// non-BMP codepoints in CSS string escapes, and thus can't display these two icons.\n// Notably, the bug affects some older versions of the Android Browser.\n// More info: https://github.com/twbs/bootstrap/issues/10106\n// .glyphicon-door { &:before { content: \"\\1f6aa\"; } }\n// .glyphicon-key { &:before { content: \"\\1f511\"; } }\n.glyphicon-alert { &:before { content: \"\\e209\"; } }\n.glyphicon-equalizer { &:before { content: \"\\e210\"; } }\n.glyphicon-king { &:before { content: \"\\e211\"; } }\n.glyphicon-queen { &:before { content: \"\\e212\"; } }\n.glyphicon-pawn { &:before { content: \"\\e213\"; } }\n.glyphicon-bishop { &:before { content: \"\\e214\"; } }\n.glyphicon-knight { &:before { content: \"\\e215\"; } }\n.glyphicon-baby-formula { &:before { content: \"\\e216\"; } }\n.glyphicon-tent { &:before { content: \"\\26fa\"; } }\n.glyphicon-blackboard { &:before { content: \"\\e218\"; } }\n.glyphicon-bed { &:before { content: \"\\e219\"; } }\n.glyphicon-apple { &:before { content: \"\\f8ff\"; } }\n.glyphicon-erase { &:before { content: \"\\e221\"; } }\n.glyphicon-hourglass { &:before { content: \"\\231b\"; } }\n.glyphicon-lamp { &:before { content: \"\\e223\"; } }\n.glyphicon-duplicate { &:before { content: \"\\e224\"; } }\n.glyphicon-piggy-bank { &:before { content: \"\\e225\"; } }\n.glyphicon-scissors { &:before { content: \"\\e226\"; } }\n.glyphicon-bitcoin { &:before { content: \"\\e227\"; } }\n.glyphicon-btc { &:before { content: \"\\e227\"; } }\n.glyphicon-xbt { &:before { content: \"\\e227\"; } }\n.glyphicon-yen { &:before { content: \"\\00a5\"; } }\n.glyphicon-jpy { &:before { content: \"\\00a5\"; } }\n.glyphicon-ruble { &:before { content: \"\\20bd\"; } }\n.glyphicon-rub { &:before { content: \"\\20bd\"; } }\n.glyphicon-scale { &:before { content: \"\\e230\"; } }\n.glyphicon-ice-lolly { &:before { content: \"\\e231\"; } }\n.glyphicon-ice-lolly-tasted { &:before { content: \"\\e232\"; } }\n.glyphicon-education { &:before { content: \"\\e233\"; } }\n.glyphicon-option-horizontal { &:before { content: \"\\e234\"; } }\n.glyphicon-option-vertical { &:before { content: \"\\e235\"; } }\n.glyphicon-menu-hamburger { &:before { content: \"\\e236\"; } }\n.glyphicon-modal-window { &:before { content: \"\\e237\"; } }\n.glyphicon-oil { &:before { content: \"\\e238\"; } }\n.glyphicon-grain { &:before { content: \"\\e239\"; } }\n.glyphicon-sunglasses { &:before { content: \"\\e240\"; } }\n.glyphicon-text-size { &:before { content: \"\\e241\"; } }\n.glyphicon-text-color { &:before { content: \"\\e242\"; } }\n.glyphicon-text-background { &:before { content: \"\\e243\"; } }\n.glyphicon-object-align-top { &:before { content: \"\\e244\"; } }\n.glyphicon-object-align-bottom { &:before { content: \"\\e245\"; } }\n.glyphicon-object-align-horizontal{ &:before { content: \"\\e246\"; } }\n.glyphicon-object-align-left { &:before { content: \"\\e247\"; } }\n.glyphicon-object-align-vertical { &:before { content: \"\\e248\"; } }\n.glyphicon-object-align-right { &:before { content: \"\\e249\"; } }\n.glyphicon-triangle-right { &:before { content: \"\\e250\"; } }\n.glyphicon-triangle-left { &:before { content: \"\\e251\"; } }\n.glyphicon-triangle-bottom { &:before { content: \"\\e252\"; } }\n.glyphicon-triangle-top { &:before { content: \"\\e253\"; } }\n.glyphicon-console { &:before { content: \"\\e254\"; } }\n.glyphicon-superscript { &:before { content: \"\\e255\"; } }\n.glyphicon-subscript { &:before { content: \"\\e256\"; } }\n.glyphicon-menu-left { &:before { content: \"\\e257\"; } }\n.glyphicon-menu-right { &:before { content: \"\\e258\"; } }\n.glyphicon-menu-down { &:before { content: \"\\e259\"; } }\n.glyphicon-menu-up { &:before { content: \"\\e260\"; } }\n","//\n// Scaffolding\n// --------------------------------------------------\n\n\n// Reset the box-sizing\n//\n// Heads up! This reset may cause conflicts with some third-party widgets.\n// For recommendations on resolving such conflicts, see\n// http://getbootstrap.com/getting-started/#third-box-sizing\n* {\n .box-sizing(border-box);\n}\n*:before,\n*:after {\n .box-sizing(border-box);\n}\n\n\n// Body reset\n\nhtml {\n font-size: 10px;\n -webkit-tap-highlight-color: rgba(0,0,0,0);\n}\n\nbody {\n font-family: @font-family-base;\n font-size: @font-size-base;\n line-height: @line-height-base;\n color: @text-color;\n background-color: @body-bg;\n}\n\n// Reset fonts for relevant elements\ninput,\nbutton,\nselect,\ntextarea {\n font-family: inherit;\n font-size: inherit;\n line-height: inherit;\n}\n\n\n// Links\n\na {\n color: @link-color;\n text-decoration: none;\n\n &:hover,\n &:focus {\n color: @link-hover-color;\n text-decoration: @link-hover-decoration;\n }\n\n &:focus {\n .tab-focus();\n }\n}\n\n\n// Figures\n//\n// We reset this here because previously Normalize had no `figure` margins. This\n// ensures we don't break anyone's use of the element.\n\nfigure {\n margin: 0;\n}\n\n\n// Images\n\nimg {\n vertical-align: middle;\n}\n\n// Responsive images (ensure images don't scale beyond their parents)\n.img-responsive {\n .img-responsive();\n}\n\n// Rounded corners\n.img-rounded {\n border-radius: @border-radius-large;\n}\n\n// Image thumbnails\n//\n// Heads up! This is mixin-ed into thumbnails.less for `.thumbnail`.\n.img-thumbnail {\n padding: @thumbnail-padding;\n line-height: @line-height-base;\n background-color: @thumbnail-bg;\n border: 1px solid @thumbnail-border;\n border-radius: @thumbnail-border-radius;\n .transition(all .2s ease-in-out);\n\n // Keep them at most 100% wide\n .img-responsive(inline-block);\n}\n\n// Perfect circle\n.img-circle {\n border-radius: 50%; // set radius in percents\n}\n\n\n// Horizontal rules\n\nhr {\n margin-top: @line-height-computed;\n margin-bottom: @line-height-computed;\n border: 0;\n border-top: 1px solid @hr-border;\n}\n\n\n// Only display content to screen readers\n//\n// See: http://a11yproject.com/posts/how-to-hide-content/\n\n.sr-only {\n position: absolute;\n width: 1px;\n height: 1px;\n margin: -1px;\n padding: 0;\n overflow: hidden;\n clip: rect(0,0,0,0);\n border: 0;\n}\n\n// Use in conjunction with .sr-only to only display content when it's focused.\n// Useful for \"Skip to main content\" links; see http://www.w3.org/TR/2013/NOTE-WCAG20-TECHS-20130905/G1\n// Credit: HTML5 Boilerplate\n\n.sr-only-focusable {\n &:active,\n &:focus {\n position: static;\n width: auto;\n height: auto;\n margin: 0;\n overflow: visible;\n clip: auto;\n }\n}\n\n\n// iOS \"clickable elements\" fix for role=\"button\"\n//\n// Fixes \"clickability\" issue (and more generally, the firing of events such as focus as well)\n// for traditionally non-focusable elements with role=\"button\"\n// see https://developer.mozilla.org/en-US/docs/Web/Events/click#Safari_Mobile\n// Upstream patch for normalize.css submitted: https://github.com/necolas/normalize.css/pull/379 - remove this fix once that is merged\n\n[role=\"button\"] {\n cursor: pointer;\n}","// Vendor Prefixes\n//\n// All vendor mixins are deprecated as of v3.2.0 due to the introduction of\n// Autoprefixer in our Gruntfile. They will be removed in v4.\n\n// - Animations\n// - Backface visibility\n// - Box shadow\n// - Box sizing\n// - Content columns\n// - Hyphens\n// - Placeholder text\n// - Transformations\n// - Transitions\n// - User Select\n\n\n// Animations\n.animation(@animation) {\n -webkit-animation: @animation;\n -o-animation: @animation;\n animation: @animation;\n}\n.animation-name(@name) {\n -webkit-animation-name: @name;\n animation-name: @name;\n}\n.animation-duration(@duration) {\n -webkit-animation-duration: @duration;\n animation-duration: @duration;\n}\n.animation-timing-function(@timing-function) {\n -webkit-animation-timing-function: @timing-function;\n animation-timing-function: @timing-function;\n}\n.animation-delay(@delay) {\n -webkit-animation-delay: @delay;\n animation-delay: @delay;\n}\n.animation-iteration-count(@iteration-count) {\n -webkit-animation-iteration-count: @iteration-count;\n animation-iteration-count: @iteration-count;\n}\n.animation-direction(@direction) {\n -webkit-animation-direction: @direction;\n animation-direction: @direction;\n}\n.animation-fill-mode(@fill-mode) {\n -webkit-animation-fill-mode: @fill-mode;\n animation-fill-mode: @fill-mode;\n}\n\n// Backface visibility\n// Prevent browsers from flickering when using CSS 3D transforms.\n// Default value is `visible`, but can be changed to `hidden`\n\n.backface-visibility(@visibility){\n -webkit-backface-visibility: @visibility;\n -moz-backface-visibility: @visibility;\n backface-visibility: @visibility;\n}\n\n// Drop shadows\n//\n// Note: Deprecated `.box-shadow()` as of v3.1.0 since all of Bootstrap's\n// supported browsers that have box shadow capabilities now support it.\n\n.box-shadow(@shadow) {\n -webkit-box-shadow: @shadow; // iOS <4.3 & Android <4.1\n box-shadow: @shadow;\n}\n\n// Box sizing\n.box-sizing(@boxmodel) {\n -webkit-box-sizing: @boxmodel;\n -moz-box-sizing: @boxmodel;\n box-sizing: @boxmodel;\n}\n\n// CSS3 Content Columns\n.content-columns(@column-count; @column-gap: @grid-gutter-width) {\n -webkit-column-count: @column-count;\n -moz-column-count: @column-count;\n column-count: @column-count;\n -webkit-column-gap: @column-gap;\n -moz-column-gap: @column-gap;\n column-gap: @column-gap;\n}\n\n// Optional hyphenation\n.hyphens(@mode: auto) {\n word-wrap: break-word;\n -webkit-hyphens: @mode;\n -moz-hyphens: @mode;\n -ms-hyphens: @mode; // IE10+\n -o-hyphens: @mode;\n hyphens: @mode;\n}\n\n// Placeholder text\n.placeholder(@color: @input-color-placeholder) {\n // Firefox\n &::-moz-placeholder {\n color: @color;\n opacity: 1; // Override Firefox's unusual default opacity; see https://github.com/twbs/bootstrap/pull/11526\n }\n &:-ms-input-placeholder { color: @color; } // Internet Explorer 10+\n &::-webkit-input-placeholder { color: @color; } // Safari and Chrome\n}\n\n// Transformations\n.scale(@ratio) {\n -webkit-transform: scale(@ratio);\n -ms-transform: scale(@ratio); // IE9 only\n -o-transform: scale(@ratio);\n transform: scale(@ratio);\n}\n.scale(@ratioX; @ratioY) {\n -webkit-transform: scale(@ratioX, @ratioY);\n -ms-transform: scale(@ratioX, @ratioY); // IE9 only\n -o-transform: scale(@ratioX, @ratioY);\n transform: scale(@ratioX, @ratioY);\n}\n.scaleX(@ratio) {\n -webkit-transform: scaleX(@ratio);\n -ms-transform: scaleX(@ratio); // IE9 only\n -o-transform: scaleX(@ratio);\n transform: scaleX(@ratio);\n}\n.scaleY(@ratio) {\n -webkit-transform: scaleY(@ratio);\n -ms-transform: scaleY(@ratio); // IE9 only\n -o-transform: scaleY(@ratio);\n transform: scaleY(@ratio);\n}\n.skew(@x; @y) {\n -webkit-transform: skewX(@x) skewY(@y);\n -ms-transform: skewX(@x) skewY(@y); // See https://github.com/twbs/bootstrap/issues/4885; IE9+\n -o-transform: skewX(@x) skewY(@y);\n transform: skewX(@x) skewY(@y);\n}\n.translate(@x; @y) {\n -webkit-transform: translate(@x, @y);\n -ms-transform: translate(@x, @y); // IE9 only\n -o-transform: translate(@x, @y);\n transform: translate(@x, @y);\n}\n.translate3d(@x; @y; @z) {\n -webkit-transform: translate3d(@x, @y, @z);\n transform: translate3d(@x, @y, @z);\n}\n.rotate(@degrees) {\n -webkit-transform: rotate(@degrees);\n -ms-transform: rotate(@degrees); // IE9 only\n -o-transform: rotate(@degrees);\n transform: rotate(@degrees);\n}\n.rotateX(@degrees) {\n -webkit-transform: rotateX(@degrees);\n -ms-transform: rotateX(@degrees); // IE9 only\n -o-transform: rotateX(@degrees);\n transform: rotateX(@degrees);\n}\n.rotateY(@degrees) {\n -webkit-transform: rotateY(@degrees);\n -ms-transform: rotateY(@degrees); // IE9 only\n -o-transform: rotateY(@degrees);\n transform: rotateY(@degrees);\n}\n.perspective(@perspective) {\n -webkit-perspective: @perspective;\n -moz-perspective: @perspective;\n perspective: @perspective;\n}\n.perspective-origin(@perspective) {\n -webkit-perspective-origin: @perspective;\n -moz-perspective-origin: @perspective;\n perspective-origin: @perspective;\n}\n.transform-origin(@origin) {\n -webkit-transform-origin: @origin;\n -moz-transform-origin: @origin;\n -ms-transform-origin: @origin; // IE9 only\n transform-origin: @origin;\n}\n\n\n// Transitions\n\n.transition(@transition) {\n -webkit-transition: @transition;\n -o-transition: @transition;\n transition: @transition;\n}\n.transition-property(@transition-property) {\n -webkit-transition-property: @transition-property;\n transition-property: @transition-property;\n}\n.transition-delay(@transition-delay) {\n -webkit-transition-delay: @transition-delay;\n transition-delay: @transition-delay;\n}\n.transition-duration(@transition-duration) {\n -webkit-transition-duration: @transition-duration;\n transition-duration: @transition-duration;\n}\n.transition-timing-function(@timing-function) {\n -webkit-transition-timing-function: @timing-function;\n transition-timing-function: @timing-function;\n}\n.transition-transform(@transition) {\n -webkit-transition: -webkit-transform @transition;\n -moz-transition: -moz-transform @transition;\n -o-transition: -o-transform @transition;\n transition: transform @transition;\n}\n\n\n// User select\n// For selecting text on the page\n\n.user-select(@select) {\n -webkit-user-select: @select;\n -moz-user-select: @select;\n -ms-user-select: @select; // IE10+\n user-select: @select;\n}\n","// WebKit-style focus\n\n.tab-focus() {\n // Default\n outline: thin dotted;\n // WebKit\n outline: 5px auto -webkit-focus-ring-color;\n outline-offset: -2px;\n}\n","// Image Mixins\n// - Responsive image\n// - Retina image\n\n\n// Responsive image\n//\n// Keep images from scaling beyond the width of their parents.\n.img-responsive(@display: block) {\n display: @display;\n max-width: 100%; // Part 1: Set a maximum relative to the parent\n height: auto; // Part 2: Scale the height according to the width, otherwise you get stretching\n}\n\n\n// Retina image\n//\n// Short retina mixin for setting background-image and -size. Note that the\n// spelling of `min--moz-device-pixel-ratio` is intentional.\n.img-retina(@file-1x; @file-2x; @width-1x; @height-1x) {\n background-image: url(\"@{file-1x}\");\n\n @media\n only screen and (-webkit-min-device-pixel-ratio: 2),\n only screen and ( min--moz-device-pixel-ratio: 2),\n only screen and ( -o-min-device-pixel-ratio: 2/1),\n only screen and ( min-device-pixel-ratio: 2),\n only screen and ( min-resolution: 192dpi),\n only screen and ( min-resolution: 2dppx) {\n background-image: url(\"@{file-2x}\");\n background-size: @width-1x @height-1x;\n }\n}\n","//\n// Typography\n// --------------------------------------------------\n\n\n// Headings\n// -------------------------\n\nh1, h2, h3, h4, h5, h6,\n.h1, .h2, .h3, .h4, .h5, .h6 {\n font-family: @headings-font-family;\n font-weight: @headings-font-weight;\n line-height: @headings-line-height;\n color: @headings-color;\n\n small,\n .small {\n font-weight: normal;\n line-height: 1;\n color: @headings-small-color;\n }\n}\n\nh1, .h1,\nh2, .h2,\nh3, .h3 {\n margin-top: @line-height-computed;\n margin-bottom: (@line-height-computed / 2);\n\n small,\n .small {\n font-size: 65%;\n }\n}\nh4, .h4,\nh5, .h5,\nh6, .h6 {\n margin-top: (@line-height-computed / 2);\n margin-bottom: (@line-height-computed / 2);\n\n small,\n .small {\n font-size: 75%;\n }\n}\n\nh1, .h1 { font-size: @font-size-h1; }\nh2, .h2 { font-size: @font-size-h2; }\nh3, .h3 { font-size: @font-size-h3; }\nh4, .h4 { font-size: @font-size-h4; }\nh5, .h5 { font-size: @font-size-h5; }\nh6, .h6 { font-size: @font-size-h6; }\n\n\n// Body text\n// -------------------------\n\np {\n margin: 0 0 (@line-height-computed / 2);\n}\n\n.lead {\n margin-bottom: @line-height-computed;\n font-size: floor((@font-size-base * 1.15));\n font-weight: 300;\n line-height: 1.4;\n\n @media (min-width: @screen-sm-min) {\n font-size: (@font-size-base * 1.5);\n }\n}\n\n\n// Emphasis & misc\n// -------------------------\n\n// Ex: (12px small font / 14px base font) * 100% = about 85%\nsmall,\n.small {\n font-size: floor((100% * @font-size-small / @font-size-base));\n}\n\nmark,\n.mark {\n background-color: @state-warning-bg;\n padding: .2em;\n}\n\n// Alignment\n.text-left { text-align: left; }\n.text-right { text-align: right; }\n.text-center { text-align: center; }\n.text-justify { text-align: justify; }\n.text-nowrap { white-space: nowrap; }\n\n// Transformation\n.text-lowercase { text-transform: lowercase; }\n.text-uppercase { text-transform: uppercase; }\n.text-capitalize { text-transform: capitalize; }\n\n// Contextual colors\n.text-muted {\n color: @text-muted;\n}\n.text-primary {\n .text-emphasis-variant(@brand-primary);\n}\n.text-success {\n .text-emphasis-variant(@state-success-text);\n}\n.text-info {\n .text-emphasis-variant(@state-info-text);\n}\n.text-warning {\n .text-emphasis-variant(@state-warning-text);\n}\n.text-danger {\n .text-emphasis-variant(@state-danger-text);\n}\n\n// Contextual backgrounds\n// For now we'll leave these alongside the text classes until v4 when we can\n// safely shift things around (per SemVer rules).\n.bg-primary {\n // Given the contrast here, this is the only class to have its color inverted\n // automatically.\n color: #fff;\n .bg-variant(@brand-primary);\n}\n.bg-success {\n .bg-variant(@state-success-bg);\n}\n.bg-info {\n .bg-variant(@state-info-bg);\n}\n.bg-warning {\n .bg-variant(@state-warning-bg);\n}\n.bg-danger {\n .bg-variant(@state-danger-bg);\n}\n\n\n// Page header\n// -------------------------\n\n.page-header {\n padding-bottom: ((@line-height-computed / 2) - 1);\n margin: (@line-height-computed * 2) 0 @line-height-computed;\n border-bottom: 1px solid @page-header-border-color;\n}\n\n\n// Lists\n// -------------------------\n\n// Unordered and Ordered lists\nul,\nol {\n margin-top: 0;\n margin-bottom: (@line-height-computed / 2);\n ul,\n ol {\n margin-bottom: 0;\n }\n}\n\n// List options\n\n// Unstyled keeps list items block level, just removes default browser padding and list-style\n.list-unstyled {\n padding-left: 0;\n list-style: none;\n}\n\n// Inline turns list items into inline-block\n.list-inline {\n .list-unstyled();\n margin-left: -5px;\n\n > li {\n display: inline-block;\n padding-left: 5px;\n padding-right: 5px;\n }\n}\n\n// Description Lists\ndl {\n margin-top: 0; // Remove browser default\n margin-bottom: @line-height-computed;\n}\ndt,\ndd {\n line-height: @line-height-base;\n}\ndt {\n font-weight: bold;\n}\ndd {\n margin-left: 0; // Undo browser default\n}\n\n// Horizontal description lists\n//\n// Defaults to being stacked without any of the below styles applied, until the\n// grid breakpoint is reached (default of ~768px).\n\n.dl-horizontal {\n dd {\n &:extend(.clearfix all); // Clear the floated `dt` if an empty `dd` is present\n }\n\n @media (min-width: @grid-float-breakpoint) {\n dt {\n float: left;\n width: (@dl-horizontal-offset - 20);\n clear: left;\n text-align: right;\n .text-overflow();\n }\n dd {\n margin-left: @dl-horizontal-offset;\n }\n }\n}\n\n\n// Misc\n// -------------------------\n\n// Abbreviations and acronyms\nabbr[title],\n// Add data-* attribute to help out our tooltip plugin, per https://github.com/twbs/bootstrap/issues/5257\nabbr[data-original-title] {\n cursor: help;\n border-bottom: 1px dotted @abbr-border-color;\n}\n.initialism {\n font-size: 90%;\n .text-uppercase();\n}\n\n// Blockquotes\nblockquote {\n padding: (@line-height-computed / 2) @line-height-computed;\n margin: 0 0 @line-height-computed;\n font-size: @blockquote-font-size;\n border-left: 5px solid @blockquote-border-color;\n\n p,\n ul,\n ol {\n &:last-child {\n margin-bottom: 0;\n }\n }\n\n // Note: Deprecated small and .small as of v3.1.0\n // Context: https://github.com/twbs/bootstrap/issues/11660\n footer,\n small,\n .small {\n display: block;\n font-size: 80%; // back to default font-size\n line-height: @line-height-base;\n color: @blockquote-small-color;\n\n &:before {\n content: '\\2014 \\00A0'; // em dash, nbsp\n }\n }\n}\n\n// Opposite alignment of blockquote\n//\n// Heads up: `blockquote.pull-right` has been deprecated as of v3.1.0.\n.blockquote-reverse,\nblockquote.pull-right {\n padding-right: 15px;\n padding-left: 0;\n border-right: 5px solid @blockquote-border-color;\n border-left: 0;\n text-align: right;\n\n // Account for citation\n footer,\n small,\n .small {\n &:before { content: ''; }\n &:after {\n content: '\\00A0 \\2014'; // nbsp, em dash\n }\n }\n}\n\n// Addresses\naddress {\n margin-bottom: @line-height-computed;\n font-style: normal;\n line-height: @line-height-base;\n}\n","// Typography\n\n.text-emphasis-variant(@color) {\n color: @color;\n a&:hover {\n color: darken(@color, 10%);\n }\n}\n","// Contextual backgrounds\n\n.bg-variant(@color) {\n background-color: @color;\n a&:hover {\n background-color: darken(@color, 10%);\n }\n}\n","// Text overflow\n// Requires inline-block or block for proper styling\n\n.text-overflow() {\n overflow: hidden;\n text-overflow: ellipsis;\n white-space: nowrap;\n}\n","//\n// Code (inline and block)\n// --------------------------------------------------\n\n\n// Inline and block code styles\ncode,\nkbd,\npre,\nsamp {\n font-family: @font-family-monospace;\n}\n\n// Inline code\ncode {\n padding: 2px 4px;\n font-size: 90%;\n color: @code-color;\n background-color: @code-bg;\n border-radius: @border-radius-base;\n}\n\n// User input typically entered via keyboard\nkbd {\n padding: 2px 4px;\n font-size: 90%;\n color: @kbd-color;\n background-color: @kbd-bg;\n border-radius: @border-radius-small;\n box-shadow: inset 0 -1px 0 rgba(0,0,0,.25);\n\n kbd {\n padding: 0;\n font-size: 100%;\n font-weight: bold;\n box-shadow: none;\n }\n}\n\n// Blocks of code\npre {\n display: block;\n padding: ((@line-height-computed - 1) / 2);\n margin: 0 0 (@line-height-computed / 2);\n font-size: (@font-size-base - 1); // 14px to 13px\n line-height: @line-height-base;\n word-break: break-all;\n word-wrap: break-word;\n color: @pre-color;\n background-color: @pre-bg;\n border: 1px solid @pre-border-color;\n border-radius: @border-radius-base;\n\n // Account for some code outputs that place code tags in pre tags\n code {\n padding: 0;\n font-size: inherit;\n color: inherit;\n white-space: pre-wrap;\n background-color: transparent;\n border-radius: 0;\n }\n}\n\n// Enable scrollable blocks of code\n.pre-scrollable {\n max-height: @pre-scrollable-max-height;\n overflow-y: scroll;\n}\n","//\n// Grid system\n// --------------------------------------------------\n\n\n// Container widths\n//\n// Set the container width, and override it for fixed navbars in media queries.\n\n.container {\n .container-fixed();\n\n @media (min-width: @screen-sm-min) {\n width: @container-sm;\n }\n @media (min-width: @screen-md-min) {\n width: @container-md;\n }\n @media (min-width: @screen-lg-min) {\n width: @container-lg;\n }\n}\n\n\n// Fluid container\n//\n// Utilizes the mixin meant for fixed width containers, but without any defined\n// width for fluid, full width layouts.\n\n.container-fluid {\n .container-fixed();\n}\n\n\n// Row\n//\n// Rows contain and clear the floats of your columns.\n\n.row {\n .make-row();\n}\n\n\n// Columns\n//\n// Common styles for small and large grid columns\n\n.make-grid-columns();\n\n\n// Extra small grid\n//\n// Columns, offsets, pushes, and pulls for extra small devices like\n// smartphones.\n\n.make-grid(xs);\n\n\n// Small grid\n//\n// Columns, offsets, pushes, and pulls for the small device range, from phones\n// to tablets.\n\n@media (min-width: @screen-sm-min) {\n .make-grid(sm);\n}\n\n\n// Medium grid\n//\n// Columns, offsets, pushes, and pulls for the desktop device range.\n\n@media (min-width: @screen-md-min) {\n .make-grid(md);\n}\n\n\n// Large grid\n//\n// Columns, offsets, pushes, and pulls for the large desktop device range.\n\n@media (min-width: @screen-lg-min) {\n .make-grid(lg);\n}\n","// Grid system\n//\n// Generate semantic grid columns with these mixins.\n\n// Centered container element\n.container-fixed(@gutter: @grid-gutter-width) {\n margin-right: auto;\n margin-left: auto;\n padding-left: (@gutter / 2);\n padding-right: (@gutter / 2);\n &:extend(.clearfix all);\n}\n\n// Creates a wrapper for a series of columns\n.make-row(@gutter: @grid-gutter-width) {\n margin-left: (@gutter / -2);\n margin-right: (@gutter / -2);\n &:extend(.clearfix all);\n}\n\n// Generate the extra small columns\n.make-xs-column(@columns; @gutter: @grid-gutter-width) {\n position: relative;\n float: left;\n width: percentage((@columns / @grid-columns));\n min-height: 1px;\n padding-left: (@gutter / 2);\n padding-right: (@gutter / 2);\n}\n.make-xs-column-offset(@columns) {\n margin-left: percentage((@columns / @grid-columns));\n}\n.make-xs-column-push(@columns) {\n left: percentage((@columns / @grid-columns));\n}\n.make-xs-column-pull(@columns) {\n right: percentage((@columns / @grid-columns));\n}\n\n// Generate the small columns\n.make-sm-column(@columns; @gutter: @grid-gutter-width) {\n position: relative;\n min-height: 1px;\n padding-left: (@gutter / 2);\n padding-right: (@gutter / 2);\n\n @media (min-width: @screen-sm-min) {\n float: left;\n width: percentage((@columns / @grid-columns));\n }\n}\n.make-sm-column-offset(@columns) {\n @media (min-width: @screen-sm-min) {\n margin-left: percentage((@columns / @grid-columns));\n }\n}\n.make-sm-column-push(@columns) {\n @media (min-width: @screen-sm-min) {\n left: percentage((@columns / @grid-columns));\n }\n}\n.make-sm-column-pull(@columns) {\n @media (min-width: @screen-sm-min) {\n right: percentage((@columns / @grid-columns));\n }\n}\n\n// Generate the medium columns\n.make-md-column(@columns; @gutter: @grid-gutter-width) {\n position: relative;\n min-height: 1px;\n padding-left: (@gutter / 2);\n padding-right: (@gutter / 2);\n\n @media (min-width: @screen-md-min) {\n float: left;\n width: percentage((@columns / @grid-columns));\n }\n}\n.make-md-column-offset(@columns) {\n @media (min-width: @screen-md-min) {\n margin-left: percentage((@columns / @grid-columns));\n }\n}\n.make-md-column-push(@columns) {\n @media (min-width: @screen-md-min) {\n left: percentage((@columns / @grid-columns));\n }\n}\n.make-md-column-pull(@columns) {\n @media (min-width: @screen-md-min) {\n right: percentage((@columns / @grid-columns));\n }\n}\n\n// Generate the large columns\n.make-lg-column(@columns; @gutter: @grid-gutter-width) {\n position: relative;\n min-height: 1px;\n padding-left: (@gutter / 2);\n padding-right: (@gutter / 2);\n\n @media (min-width: @screen-lg-min) {\n float: left;\n width: percentage((@columns / @grid-columns));\n }\n}\n.make-lg-column-offset(@columns) {\n @media (min-width: @screen-lg-min) {\n margin-left: percentage((@columns / @grid-columns));\n }\n}\n.make-lg-column-push(@columns) {\n @media (min-width: @screen-lg-min) {\n left: percentage((@columns / @grid-columns));\n }\n}\n.make-lg-column-pull(@columns) {\n @media (min-width: @screen-lg-min) {\n right: percentage((@columns / @grid-columns));\n }\n}\n","// Framework grid generation\n//\n// Used only by Bootstrap to generate the correct number of grid classes given\n// any value of `@grid-columns`.\n\n.make-grid-columns() {\n // Common styles for all sizes of grid columns, widths 1-12\n .col(@index) { // initial\n @item: ~\".col-xs-@{index}, .col-sm-@{index}, .col-md-@{index}, .col-lg-@{index}\";\n .col((@index + 1), @item);\n }\n .col(@index, @list) when (@index =< @grid-columns) { // general; \"=<\" isn't a typo\n @item: ~\".col-xs-@{index}, .col-sm-@{index}, .col-md-@{index}, .col-lg-@{index}\";\n .col((@index + 1), ~\"@{list}, @{item}\");\n }\n .col(@index, @list) when (@index > @grid-columns) { // terminal\n @{list} {\n position: relative;\n // Prevent columns from collapsing when empty\n min-height: 1px;\n // Inner gutter via padding\n padding-left: (@grid-gutter-width / 2);\n padding-right: (@grid-gutter-width / 2);\n }\n }\n .col(1); // kickstart it\n}\n\n.float-grid-columns(@class) {\n .col(@index) { // initial\n @item: ~\".col-@{class}-@{index}\";\n .col((@index + 1), @item);\n }\n .col(@index, @list) when (@index =< @grid-columns) { // general\n @item: ~\".col-@{class}-@{index}\";\n .col((@index + 1), ~\"@{list}, @{item}\");\n }\n .col(@index, @list) when (@index > @grid-columns) { // terminal\n @{list} {\n float: left;\n }\n }\n .col(1); // kickstart it\n}\n\n.calc-grid-column(@index, @class, @type) when (@type = width) and (@index > 0) {\n .col-@{class}-@{index} {\n width: percentage((@index / @grid-columns));\n }\n}\n.calc-grid-column(@index, @class, @type) when (@type = push) and (@index > 0) {\n .col-@{class}-push-@{index} {\n left: percentage((@index / @grid-columns));\n }\n}\n.calc-grid-column(@index, @class, @type) when (@type = push) and (@index = 0) {\n .col-@{class}-push-0 {\n left: auto;\n }\n}\n.calc-grid-column(@index, @class, @type) when (@type = pull) and (@index > 0) {\n .col-@{class}-pull-@{index} {\n right: percentage((@index / @grid-columns));\n }\n}\n.calc-grid-column(@index, @class, @type) when (@type = pull) and (@index = 0) {\n .col-@{class}-pull-0 {\n right: auto;\n }\n}\n.calc-grid-column(@index, @class, @type) when (@type = offset) {\n .col-@{class}-offset-@{index} {\n margin-left: percentage((@index / @grid-columns));\n }\n}\n\n// Basic looping in LESS\n.loop-grid-columns(@index, @class, @type) when (@index >= 0) {\n .calc-grid-column(@index, @class, @type);\n // next iteration\n .loop-grid-columns((@index - 1), @class, @type);\n}\n\n// Create grid for specific class\n.make-grid(@class) {\n .float-grid-columns(@class);\n .loop-grid-columns(@grid-columns, @class, width);\n .loop-grid-columns(@grid-columns, @class, pull);\n .loop-grid-columns(@grid-columns, @class, push);\n .loop-grid-columns(@grid-columns, @class, offset);\n}\n","//\n// Tables\n// --------------------------------------------------\n\n\ntable {\n background-color: @table-bg;\n}\ncaption {\n padding-top: @table-cell-padding;\n padding-bottom: @table-cell-padding;\n color: @text-muted;\n text-align: left;\n}\nth {\n text-align: left;\n}\n\n\n// Baseline styles\n\n.table {\n width: 100%;\n max-width: 100%;\n margin-bottom: @line-height-computed;\n // Cells\n > thead,\n > tbody,\n > tfoot {\n > tr {\n > th,\n > td {\n padding: @table-cell-padding;\n line-height: @line-height-base;\n vertical-align: top;\n border-top: 1px solid @table-border-color;\n }\n }\n }\n // Bottom align for column headings\n > thead > tr > th {\n vertical-align: bottom;\n border-bottom: 2px solid @table-border-color;\n }\n // Remove top border from thead by default\n > caption + thead,\n > colgroup + thead,\n > thead:first-child {\n > tr:first-child {\n > th,\n > td {\n border-top: 0;\n }\n }\n }\n // Account for multiple tbody instances\n > tbody + tbody {\n border-top: 2px solid @table-border-color;\n }\n\n // Nesting\n .table {\n background-color: @body-bg;\n }\n}\n\n\n// Condensed table w/ half padding\n\n.table-condensed {\n > thead,\n > tbody,\n > tfoot {\n > tr {\n > th,\n > td {\n padding: @table-condensed-cell-padding;\n }\n }\n }\n}\n\n\n// Bordered version\n//\n// Add borders all around the table and between all the columns.\n\n.table-bordered {\n border: 1px solid @table-border-color;\n > thead,\n > tbody,\n > tfoot {\n > tr {\n > th,\n > td {\n border: 1px solid @table-border-color;\n }\n }\n }\n > thead > tr {\n > th,\n > td {\n border-bottom-width: 2px;\n }\n }\n}\n\n\n// Zebra-striping\n//\n// Default zebra-stripe styles (alternating gray and transparent backgrounds)\n\n.table-striped {\n > tbody > tr:nth-of-type(odd) {\n background-color: @table-bg-accent;\n }\n}\n\n\n// Hover effect\n//\n// Placed here since it has to come after the potential zebra striping\n\n.table-hover {\n > tbody > tr:hover {\n background-color: @table-bg-hover;\n }\n}\n\n\n// Table cell sizing\n//\n// Reset default table behavior\n\ntable col[class*=\"col-\"] {\n position: static; // Prevent border hiding in Firefox and IE9-11 (see https://github.com/twbs/bootstrap/issues/11623)\n float: none;\n display: table-column;\n}\ntable {\n td,\n th {\n &[class*=\"col-\"] {\n position: static; // Prevent border hiding in Firefox and IE9-11 (see https://github.com/twbs/bootstrap/issues/11623)\n float: none;\n display: table-cell;\n }\n }\n}\n\n\n// Table backgrounds\n//\n// Exact selectors below required to override `.table-striped` and prevent\n// inheritance to nested tables.\n\n// Generate the contextual variants\n.table-row-variant(active; @table-bg-active);\n.table-row-variant(success; @state-success-bg);\n.table-row-variant(info; @state-info-bg);\n.table-row-variant(warning; @state-warning-bg);\n.table-row-variant(danger; @state-danger-bg);\n\n\n// Responsive tables\n//\n// Wrap your tables in `.table-responsive` and we'll make them mobile friendly\n// by enabling horizontal scrolling. Only applies <768px. Everything above that\n// will display normally.\n\n.table-responsive {\n overflow-x: auto;\n min-height: 0.01%; // Workaround for IE9 bug (see https://github.com/twbs/bootstrap/issues/14837)\n\n @media screen and (max-width: @screen-xs-max) {\n width: 100%;\n margin-bottom: (@line-height-computed * 0.75);\n overflow-y: hidden;\n -ms-overflow-style: -ms-autohiding-scrollbar;\n border: 1px solid @table-border-color;\n\n // Tighten up spacing\n > .table {\n margin-bottom: 0;\n\n // Ensure the content doesn't wrap\n > thead,\n > tbody,\n > tfoot {\n > tr {\n > th,\n > td {\n white-space: nowrap;\n }\n }\n }\n }\n\n // Special overrides for the bordered tables\n > .table-bordered {\n border: 0;\n\n // Nuke the appropriate borders so that the parent can handle them\n > thead,\n > tbody,\n > tfoot {\n > tr {\n > th:first-child,\n > td:first-child {\n border-left: 0;\n }\n > th:last-child,\n > td:last-child {\n border-right: 0;\n }\n }\n }\n\n // Only nuke the last row's bottom-border in `tbody` and `tfoot` since\n // chances are there will be only one `tr` in a `thead` and that would\n // remove the border altogether.\n > tbody,\n > tfoot {\n > tr:last-child {\n > th,\n > td {\n border-bottom: 0;\n }\n }\n }\n\n }\n }\n}\n","// Tables\n\n.table-row-variant(@state; @background) {\n // Exact selectors below required to override `.table-striped` and prevent\n // inheritance to nested tables.\n .table > thead > tr,\n .table > tbody > tr,\n .table > tfoot > tr {\n > td.@{state},\n > th.@{state},\n &.@{state} > td,\n &.@{state} > th {\n background-color: @background;\n }\n }\n\n // Hover states for `.table-hover`\n // Note: this is not available for cells or rows within `thead` or `tfoot`.\n .table-hover > tbody > tr {\n > td.@{state}:hover,\n > th.@{state}:hover,\n &.@{state}:hover > td,\n &:hover > .@{state},\n &.@{state}:hover > th {\n background-color: darken(@background, 5%);\n }\n }\n}\n","//\n// Forms\n// --------------------------------------------------\n\n\n// Normalize non-controls\n//\n// Restyle and baseline non-control form elements.\n\nfieldset {\n padding: 0;\n margin: 0;\n border: 0;\n // Chrome and Firefox set a `min-width: min-content;` on fieldsets,\n // so we reset that to ensure it behaves more like a standard block element.\n // See https://github.com/twbs/bootstrap/issues/12359.\n min-width: 0;\n}\n\nlegend {\n display: block;\n width: 100%;\n padding: 0;\n margin-bottom: @line-height-computed;\n font-size: (@font-size-base * 1.5);\n line-height: inherit;\n color: @legend-color;\n border: 0;\n border-bottom: 1px solid @legend-border-color;\n}\n\nlabel {\n display: inline-block;\n max-width: 100%; // Force IE8 to wrap long content (see https://github.com/twbs/bootstrap/issues/13141)\n margin-bottom: 5px;\n font-weight: bold;\n}\n\n\n// Normalize form controls\n//\n// While most of our form styles require extra classes, some basic normalization\n// is required to ensure optimum display with or without those classes to better\n// address browser inconsistencies.\n\n// Override content-box in Normalize (* isn't specific enough)\ninput[type=\"search\"] {\n .box-sizing(border-box);\n}\n\n// Position radios and checkboxes better\ninput[type=\"radio\"],\ninput[type=\"checkbox\"] {\n margin: 4px 0 0;\n margin-top: 1px \\9; // IE8-9\n line-height: normal;\n}\n\n// Set the height of file controls to match text inputs\ninput[type=\"file\"] {\n display: block;\n}\n\n// Make range inputs behave like textual form controls\ninput[type=\"range\"] {\n display: block;\n width: 100%;\n}\n\n// Make multiple select elements height not fixed\nselect[multiple],\nselect[size] {\n height: auto;\n}\n\n// Focus for file, radio, and checkbox\ninput[type=\"file\"]:focus,\ninput[type=\"radio\"]:focus,\ninput[type=\"checkbox\"]:focus {\n .tab-focus();\n}\n\n// Adjust output element\noutput {\n display: block;\n padding-top: (@padding-base-vertical + 1);\n font-size: @font-size-base;\n line-height: @line-height-base;\n color: @input-color;\n}\n\n\n// Common form controls\n//\n// Shared size and type resets for form controls. Apply `.form-control` to any\n// of the following form controls:\n//\n// select\n// textarea\n// input[type=\"text\"]\n// input[type=\"password\"]\n// input[type=\"datetime\"]\n// input[type=\"datetime-local\"]\n// input[type=\"date\"]\n// input[type=\"month\"]\n// input[type=\"time\"]\n// input[type=\"week\"]\n// input[type=\"number\"]\n// input[type=\"email\"]\n// input[type=\"url\"]\n// input[type=\"search\"]\n// input[type=\"tel\"]\n// input[type=\"color\"]\n\n.form-control {\n display: block;\n width: 100%;\n height: @input-height-base; // Make inputs at least the height of their button counterpart (base line-height + padding + border)\n padding: @padding-base-vertical @padding-base-horizontal;\n font-size: @font-size-base;\n line-height: @line-height-base;\n color: @input-color;\n background-color: @input-bg;\n background-image: none; // Reset unusual Firefox-on-Android default style; see https://github.com/necolas/normalize.css/issues/214\n border: 1px solid @input-border;\n border-radius: @input-border-radius; // Note: This has no effect on s in CSS.\n .box-shadow(inset 0 1px 1px rgba(0,0,0,.075));\n .transition(~\"border-color ease-in-out .15s, box-shadow ease-in-out .15s\");\n\n // Customize the `:focus` state to imitate native WebKit styles.\n .form-control-focus();\n\n // Placeholder\n .placeholder();\n\n // Disabled and read-only inputs\n //\n // HTML5 says that controls under a fieldset > legend:first-child won't be\n // disabled if the fieldset is disabled. Due to implementation difficulty, we\n // don't honor that edge case; we style them as disabled anyway.\n &[disabled],\n &[readonly],\n fieldset[disabled] & {\n background-color: @input-bg-disabled;\n opacity: 1; // iOS fix for unreadable disabled content; see https://github.com/twbs/bootstrap/issues/11655\n }\n\n &[disabled],\n fieldset[disabled] & {\n cursor: @cursor-disabled;\n }\n\n // Reset height for `textarea`s\n textarea& {\n height: auto;\n }\n}\n\n\n// Search inputs in iOS\n//\n// This overrides the extra rounded corners on search inputs in iOS so that our\n// `.form-control` class can properly style them. Note that this cannot simply\n// be added to `.form-control` as it's not specific enough. For details, see\n// https://github.com/twbs/bootstrap/issues/11586.\n\ninput[type=\"search\"] {\n -webkit-appearance: none;\n}\n\n\n// Special styles for iOS temporal inputs\n//\n// In Mobile Safari, setting `display: block` on temporal inputs causes the\n// text within the input to become vertically misaligned. As a workaround, we\n// set a pixel line-height that matches the given height of the input, but only\n// for Safari. See https://bugs.webkit.org/show_bug.cgi?id=139848\n\n@media screen and (-webkit-min-device-pixel-ratio: 0) {\n input[type=\"date\"],\n input[type=\"time\"],\n input[type=\"datetime-local\"],\n input[type=\"month\"] {\n line-height: @input-height-base;\n\n &.input-sm,\n .input-group-sm & {\n line-height: @input-height-small;\n }\n\n &.input-lg,\n .input-group-lg & {\n line-height: @input-height-large;\n }\n }\n}\n\n\n// Form groups\n//\n// Designed to help with the organization and spacing of vertical forms. For\n// horizontal forms, use the predefined grid classes.\n\n.form-group {\n margin-bottom: @form-group-margin-bottom;\n}\n\n\n// Checkboxes and radios\n//\n// Indent the labels to position radios/checkboxes as hanging controls.\n\n.radio,\n.checkbox {\n position: relative;\n display: block;\n margin-top: 10px;\n margin-bottom: 10px;\n\n label {\n min-height: @line-height-computed; // Ensure the input doesn't jump when there is no text\n padding-left: 20px;\n margin-bottom: 0;\n font-weight: normal;\n cursor: pointer;\n }\n}\n.radio input[type=\"radio\"],\n.radio-inline input[type=\"radio\"],\n.checkbox input[type=\"checkbox\"],\n.checkbox-inline input[type=\"checkbox\"] {\n position: absolute;\n margin-left: -20px;\n margin-top: 4px \\9;\n}\n\n.radio + .radio,\n.checkbox + .checkbox {\n margin-top: -5px; // Move up sibling radios or checkboxes for tighter spacing\n}\n\n// Radios and checkboxes on same line\n.radio-inline,\n.checkbox-inline {\n position: relative;\n display: inline-block;\n padding-left: 20px;\n margin-bottom: 0;\n vertical-align: middle;\n font-weight: normal;\n cursor: pointer;\n}\n.radio-inline + .radio-inline,\n.checkbox-inline + .checkbox-inline {\n margin-top: 0;\n margin-left: 10px; // space out consecutive inline controls\n}\n\n// Apply same disabled cursor tweak as for inputs\n// Some special care is needed because